diff options
Diffstat (limited to 'src/crimson/os')
155 files changed, 58332 insertions, 0 deletions
diff --git a/src/crimson/os/CMakeLists.txt b/src/crimson/os/CMakeLists.txt new file mode 100644 index 000000000..5054cabf4 --- /dev/null +++ b/src/crimson/os/CMakeLists.txt @@ -0,0 +1,16 @@ +add_library(crimson-os STATIC + futurized_store.cc + ${PROJECT_SOURCE_DIR}/src/os/Transaction.cc) +add_subdirectory(cyanstore) + +if(WITH_BLUESTORE) + add_subdirectory(alienstore) + set(alienstore_lib crimson-alienstore) +endif() + +add_subdirectory(seastore) +target_link_libraries(crimson-os + crimson-cyanstore + ${alienstore_lib} + crimson-seastore + crimson) diff --git a/src/crimson/os/alienstore/CMakeLists.txt b/src/crimson/os/alienstore/CMakeLists.txt new file mode 100644 index 000000000..c881f4fbc --- /dev/null +++ b/src/crimson/os/alienstore/CMakeLists.txt @@ -0,0 +1,86 @@ +include_directories(SYSTEM "${CMAKE_SOURCE_DIR}/src/rocksdb/include") + +add_library(alien::cflags INTERFACE IMPORTED) +set_target_properties(alien::cflags PROPERTIES + INTERFACE_COMPILE_DEFINITIONS "WITH_SEASTAR;WITH_ALIEN" + INTERFACE_INCLUDE_DIRECTORIES $<TARGET_PROPERTY:Seastar::seastar,INTERFACE_INCLUDE_DIRECTORIES>) + +set(crimson_alien_common_srcs + ${PROJECT_SOURCE_DIR}/src/common/admin_socket.cc + ${PROJECT_SOURCE_DIR}/src/common/url_escape.cc + ${PROJECT_SOURCE_DIR}/src/common/blkdev.cc + ${PROJECT_SOURCE_DIR}/src/common/ceph_context.cc + ${PROJECT_SOURCE_DIR}/src/common/ceph_crypto.cc + ${PROJECT_SOURCE_DIR}/src/common/Finisher.cc + ${PROJECT_SOURCE_DIR}/src/common/HeartbeatMap.cc + ${PROJECT_SOURCE_DIR}/src/common/PluginRegistry.cc + ${PROJECT_SOURCE_DIR}/src/common/perf_counters.cc + ${PROJECT_SOURCE_DIR}/src/common/perf_counters_collection.cc + ${PROJECT_SOURCE_DIR}/src/common/RefCountedObj.cc + ${PROJECT_SOURCE_DIR}/src/common/SubProcess.cc + ${PROJECT_SOURCE_DIR}/src/common/Throttle.cc + ${PROJECT_SOURCE_DIR}/src/common/Timer.cc + ${PROJECT_SOURCE_DIR}/src/common/TrackedOp.cc + ${PROJECT_SOURCE_DIR}/src/common/WorkQueue.cc + ${PROJECT_SOURCE_DIR}/src/common/util.cc + ${PROJECT_SOURCE_DIR}/src/crush/CrushLocation.cc + ${PROJECT_SOURCE_DIR}/src/global/global_context.cc + ${PROJECT_SOURCE_DIR}/src/log/Log.cc + $<TARGET_OBJECTS:compressor_objs> + $<TARGET_OBJECTS:common_prioritycache_obj>) +if(WITH_CEPH_DEBUG_MUTEX) + list(APPEND crimson_alien_common_srcs + ${PROJECT_SOURCE_DIR}/src/common/lockdep.cc + ${PROJECT_SOURCE_DIR}/src/common/mutex_debug.cc + ${PROJECT_SOURCE_DIR}/src/common/condition_variable_debug.cc + ${PROJECT_SOURCE_DIR}/src/common/shared_mutex_debug.cc) +endif() +add_library(crimson-alien-common STATIC + ${crimson_alien_common_srcs}) + +target_link_libraries(crimson-alien-common + crimson-common + alien::cflags) + +set(alien_store_srcs + alien_store.cc + thread_pool.cc + alien_log.cc + ${PROJECT_SOURCE_DIR}/src/os/ObjectStore.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/Allocator.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/AvlAllocator.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/BtreeAllocator.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/BitmapFreelistManager.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/BlueFS.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/bluefs_types.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/BlueRocksEnv.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/BlueStore.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/simple_bitmap.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/bluestore_types.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/fastbmap_allocator_impl.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/FreelistManager.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/HybridAllocator.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/StupidAllocator.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/BitmapAllocator.cc + ${PROJECT_SOURCE_DIR}/src/os/memstore/MemStore.cc) +if(WITH_ZBD) + list(APPEND alien_store_srcs + ${PROJECT_SOURCE_DIR}/src/os/bluestore/ZonedFreelistManager.cc + ${PROJECT_SOURCE_DIR}/src/os/bluestore/ZonedAllocator.cc) +endif() +add_library(crimson-alienstore STATIC + ${alien_store_srcs}) +if(WITH_LTTNG) + add_dependencies(crimson-alienstore bluestore-tp) +endif() +target_link_libraries(crimson-alienstore + PRIVATE + alien::cflags + fmt::fmt + kv + heap_profiler + crimson-alien-common + ${BLKID_LIBRARIES} + ${UDEV_LIBRARIES} + crimson + blk) diff --git a/src/crimson/os/alienstore/alien_collection.h b/src/crimson/os/alienstore/alien_collection.h new file mode 100644 index 000000000..17a930e77 --- /dev/null +++ b/src/crimson/os/alienstore/alien_collection.h @@ -0,0 +1,39 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "os/ObjectStore.h" + +#include "crimson/os/futurized_collection.h" +#include "crimson/os/futurized_store.h" +#include "alien_store.h" + +namespace crimson::os { + +class AlienCollection final : public FuturizedCollection { +public: + AlienCollection(ObjectStore::CollectionHandle ch) + : FuturizedCollection(ch->cid), + collection(ch) {} + + ~AlienCollection() {} + + template <typename Func, typename Result = std::invoke_result_t<Func>> + seastar::futurize_t<Result> with_lock(Func&& func) { + // newer versions of Seastar provide two variants of `with_lock` + // - generic, friendly towards throwing move constructors of Func, + // - specialized for `noexcept`. + // unfortunately, the former has a limitation: the return value + // of `Func` must be compatible with `current_exception_as_future()` + // which boils down to returning `seastar::future<void>`. + static_assert(std::is_nothrow_move_constructible_v<Func>); + return seastar::with_lock(mutex, std::forward<Func>(func)); + } + +private: + ObjectStore::CollectionHandle collection; + seastar::shared_mutex mutex; + friend AlienStore; +}; +} diff --git a/src/crimson/os/alienstore/alien_log.cc b/src/crimson/os/alienstore/alien_log.cc new file mode 100644 index 000000000..b371af897 --- /dev/null +++ b/src/crimson/os/alienstore/alien_log.cc @@ -0,0 +1,33 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "alien_log.h" +#include "log/SubsystemMap.h" +#include <seastar/core/alien.hh> +#include "crimson/common/log.h" + +namespace ceph::logging { +CnLog::CnLog(const SubsystemMap *s, seastar::alien::instance& inst, unsigned shard) + :Log(s) + ,inst(inst) + ,shard(shard) { +} + +CnLog::~CnLog() { +} + +void CnLog::_flush(EntryVector& q, bool crash) { + seastar::alien::submit_to(inst, shard, [&q] { + for (auto& it : q) { + crimson::get_logger(it.m_subsys).log( + crimson::to_log_level(it.m_prio), + "{}", + it.strv()); + } + return seastar::make_ready_future<>(); + }).wait(); + q.clear(); + return; +} + +} //namespace ceph::logging diff --git a/src/crimson/os/alienstore/alien_log.h b/src/crimson/os/alienstore/alien_log.h new file mode 100644 index 000000000..0f966d9ab --- /dev/null +++ b/src/crimson/os/alienstore/alien_log.h @@ -0,0 +1,31 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#ifndef ALIEN_LOG_H +#define ALIEN_LOG_H + +#include "log/Log.h" + +namespace ceph { +namespace logging { +class SubsystemMap; +} +} + +namespace seastar::alien { + class instance; +} +namespace ceph::logging +{ +class CnLog : public ceph::logging::Log +{ + seastar::alien::instance& inst; + unsigned shard; + void _flush(EntryVector& q, bool crash) override; +public: + CnLog(const SubsystemMap *s, seastar::alien::instance& inst, unsigned shard); + ~CnLog() override; +}; +} + +#endif diff --git a/src/crimson/os/alienstore/alien_store.cc b/src/crimson/os/alienstore/alien_store.cc new file mode 100644 index 000000000..61f23de97 --- /dev/null +++ b/src/crimson/os/alienstore/alien_store.cc @@ -0,0 +1,620 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "alien_collection.h" +#include "alien_store.h" +#include "alien_log.h" + +#include <algorithm> +#include <iterator> +#include <map> +#include <string_view> +#include <boost/algorithm/string/trim.hpp> +#include <boost/iterator/counting_iterator.hpp> +#include <fmt/format.h> +#include <fmt/ostream.h> + +#include <seastar/core/alien.hh> +#include <seastar/core/future-util.hh> +#include <seastar/core/reactor.hh> +#include <seastar/core/resource.hh> + +#include "common/ceph_context.h" +#include "global/global_context.h" +#include "include/Context.h" +#include "os/ObjectStore.h" +#include "os/Transaction.h" + +#include "crimson/common/config_proxy.h" +#include "crimson/common/log.h" +#include "crimson/os/futurized_store.h" + +using std::map; +using std::set; +using std::string; + +namespace { + +seastar::logger& logger() +{ + return crimson::get_logger(ceph_subsys_alienstore); +} + +class OnCommit final: public Context +{ + const int cpuid; + seastar::alien::instance &alien; + seastar::promise<> &alien_done; +public: + OnCommit( + int id, + seastar::promise<> &done, + seastar::alien::instance &alien, + ceph::os::Transaction& txn) + : cpuid(id), + alien(alien), + alien_done(done) { + } + + void finish(int) final { + return seastar::alien::submit_to(alien, cpuid, [this] { + alien_done.set_value(); + return seastar::make_ready_future<>(); + }).wait(); + } +}; +} + +namespace crimson::os { + +using crimson::common::get_conf; + +AlienStore::AlienStore(const std::string& type, + const std::string& path, + const ConfigValues& values) + : type(type), + path{path}, + values(values) +{ +} + +AlienStore::~AlienStore() +{ +} + +seastar::future<> AlienStore::start() +{ + cct = std::make_unique<CephContext>( + CEPH_ENTITY_TYPE_OSD, + CephContext::create_options { CODE_ENVIRONMENT_UTILITY, 0, + [](const ceph::logging::SubsystemMap* subsys_map) { + return new ceph::logging::CnLog(subsys_map, seastar::engine().alien(), seastar::this_shard_id()); + } + } + ); + g_ceph_context = cct.get(); + cct->_conf.set_config_values(values); + cct->_log->start(); + + store = ObjectStore::create(cct.get(), type, path); + if (!store) { + ceph_abort_msgf("unsupported objectstore type: %s", type.c_str()); + } + auto cpu_cores = seastar::resource::parse_cpuset( + get_conf<std::string>("crimson_alien_thread_cpu_cores")); + // cores except the first "N_CORES_FOR_SEASTAR" ones will + // be used for alien threads scheduling: + // [0, N_CORES_FOR_SEASTAR) are reserved for seastar reactors + // [N_CORES_FOR_SEASTAR, ..] are assigned to alien threads. + if (!cpu_cores.has_value()) { + seastar::resource::cpuset cpuset; + std::copy(boost::counting_iterator<unsigned>(N_CORES_FOR_SEASTAR), + boost::counting_iterator<unsigned>(sysconf(_SC_NPROCESSORS_ONLN)), + std::inserter(cpuset, cpuset.end())); + if (cpuset.empty()) { + logger().error("{}: unable to get nproc: {}", __func__, errno); + } else { + cpu_cores = cpuset; + } + } + const auto num_threads = + get_conf<uint64_t>("crimson_alien_op_num_threads"); + tp = std::make_unique<crimson::os::ThreadPool>(num_threads, 128, cpu_cores); + return tp->start(); +} + +seastar::future<> AlienStore::stop() +{ + if (!tp) { + // not really started yet + return seastar::now(); + } + return tp->submit([this] { + for (auto [cid, ch]: coll_map) { + static_cast<AlienCollection*>(ch.get())->collection.reset(); + } + store.reset(); + cct.reset(); + g_ceph_context = nullptr; + + }).then([this] { + return tp->stop(); + }); +} + +AlienStore::mount_ertr::future<> AlienStore::mount() +{ + logger().debug("{}", __func__); + assert(tp); + return tp->submit([this] { + return store->mount(); + }).then([] (const int r) -> mount_ertr::future<> { + if (r != 0) { + return crimson::stateful_ec{ + std::error_code(-r, std::generic_category()) }; + } else { + return mount_ertr::now(); + } + }); +} + +seastar::future<> AlienStore::umount() +{ + logger().info("{}", __func__); + if (!tp) { + // not really started yet + return seastar::now(); + } + return op_gate.close().then([this] { + return tp->submit([this] { + return store->umount(); + }); + }).then([] (int r) { + assert(r == 0); + return seastar::now(); + }); +} + +AlienStore::mkfs_ertr::future<> AlienStore::mkfs(uuid_d osd_fsid) +{ + logger().debug("{}", __func__); + store->set_fsid(osd_fsid); + assert(tp); + return tp->submit([this] { + return store->mkfs(); + }).then([] (int r) -> mkfs_ertr::future<> { + if (r != 0) { + return crimson::stateful_ec{ + std::error_code(-r, std::generic_category()) }; + } else { + return mkfs_ertr::now(); + } + }); +} + +seastar::future<std::tuple<std::vector<ghobject_t>, ghobject_t>> +AlienStore::list_objects(CollectionRef ch, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) const +{ + logger().debug("{}", __func__); + assert(tp); + return do_with_op_gate(std::vector<ghobject_t>(), ghobject_t(), + [=, this] (auto &objects, auto &next) { + objects.reserve(limit); + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), + [=, this, &objects, &next] { + auto c = static_cast<AlienCollection*>(ch.get()); + return store->collection_list(c->collection, start, end, + store->get_ideal_list_max(), + &objects, &next); + }).then([&objects, &next] (int r) { + assert(r == 0); + return seastar::make_ready_future< + std::tuple<std::vector<ghobject_t>, ghobject_t>>( + std::move(objects), std::move(next)); + }); + }); +} + +seastar::future<CollectionRef> AlienStore::create_new_collection(const coll_t& cid) +{ + logger().debug("{}", __func__); + assert(tp); + return tp->submit([this, cid] { + return store->create_new_collection(cid); + }).then([this, cid] (ObjectStore::CollectionHandle c) { + CollectionRef ch; + auto cp = coll_map.find(c->cid); + if (cp == coll_map.end()) { + ch = new AlienCollection(c); + coll_map[c->cid] = ch; + } else { + ch = cp->second; + auto ach = static_cast<AlienCollection*>(ch.get()); + if (ach->collection != c) { + ach->collection = c; + } + } + return seastar::make_ready_future<CollectionRef>(ch); + }); + +} + +seastar::future<CollectionRef> AlienStore::open_collection(const coll_t& cid) +{ + logger().debug("{}", __func__); + assert(tp); + return tp->submit([this, cid] { + return store->open_collection(cid); + }).then([this] (ObjectStore::CollectionHandle c) { + if (!c) { + return seastar::make_ready_future<CollectionRef>(); + } + CollectionRef ch; + auto cp = coll_map.find(c->cid); + if (cp == coll_map.end()){ + ch = new AlienCollection(c); + coll_map[c->cid] = ch; + } else { + ch = cp->second; + auto ach = static_cast<AlienCollection*>(ch.get()); + if (ach->collection != c){ + ach->collection = c; + } + } + return seastar::make_ready_future<CollectionRef>(ch); + }); +} + +seastar::future<std::vector<coll_core_t>> AlienStore::list_collections() +{ + logger().debug("{}", __func__); + assert(tp); + + return do_with_op_gate(std::vector<coll_t>{}, [this] (auto &ls) { + return tp->submit([this, &ls] { + return store->list_collections(ls); + }).then([&ls] (int r) -> seastar::future<std::vector<coll_core_t>> { + assert(r == 0); + std::vector<coll_core_t> ret; + ret.resize(ls.size()); + std::transform( + ls.begin(), ls.end(), ret.begin(), + [](auto p) { return std::make_pair(p, NULL_CORE); }); + return seastar::make_ready_future<std::vector<coll_core_t>>(std::move(ret)); + }); + }); +} + +AlienStore::read_errorator::future<ceph::bufferlist> +AlienStore::read(CollectionRef ch, + const ghobject_t& oid, + uint64_t offset, + size_t len, + uint32_t op_flags) +{ + logger().debug("{}", __func__); + assert(tp); + return do_with_op_gate(ceph::bufferlist{}, [=, this] (auto &bl) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), [=, this, &bl] { + auto c = static_cast<AlienCollection*>(ch.get()); + return store->read(c->collection, oid, offset, len, bl, op_flags); + }).then([&bl] (int r) -> read_errorator::future<ceph::bufferlist> { + if (r == -ENOENT) { + return crimson::ct_error::enoent::make(); + } else if (r == -EIO) { + return crimson::ct_error::input_output_error::make(); + } else { + return read_errorator::make_ready_future<ceph::bufferlist>( + std::move(bl)); + } + }); + }); +} + +AlienStore::read_errorator::future<ceph::bufferlist> +AlienStore::readv(CollectionRef ch, + const ghobject_t& oid, + interval_set<uint64_t>& m, + uint32_t op_flags) +{ + logger().debug("{}", __func__); + assert(tp); + return do_with_op_gate(ceph::bufferlist{}, + [this, ch, oid, &m, op_flags](auto& bl) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), + [this, ch, oid, &m, op_flags, &bl] { + auto c = static_cast<AlienCollection*>(ch.get()); + return store->readv(c->collection, oid, m, bl, op_flags); + }).then([&bl](int r) -> read_errorator::future<ceph::bufferlist> { + if (r == -ENOENT) { + return crimson::ct_error::enoent::make(); + } else if (r == -EIO) { + return crimson::ct_error::input_output_error::make(); + } else { + return read_errorator::make_ready_future<ceph::bufferlist>( + std::move(bl)); + } + }); + }); +} + +AlienStore::get_attr_errorator::future<ceph::bufferlist> +AlienStore::get_attr(CollectionRef ch, + const ghobject_t& oid, + std::string_view name) const +{ + logger().debug("{}", __func__); + assert(tp); + return do_with_op_gate(ceph::bufferlist{}, std::string{name}, + [=, this] (auto &value, const auto& name) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), [=, this, &value, &name] { + // XXX: `name` isn't a `std::string_view` anymore! it had to be converted + // to `std::string` for the sake of extending life-time not only of + // a _ptr-to-data_ but _data_ as well. Otherwise we would run into a use- + // after-free issue. + auto c = static_cast<AlienCollection*>(ch.get()); + return store->getattr(c->collection, oid, name.c_str(), value); + }).then([oid, &value](int r) -> get_attr_errorator::future<ceph::bufferlist> { + if (r == -ENOENT) { + return crimson::ct_error::enoent::make(); + } else if (r == -ENODATA) { + return crimson::ct_error::enodata::make(); + } else { + return get_attr_errorator::make_ready_future<ceph::bufferlist>( + std::move(value)); + } + }); + }); +} + +AlienStore::get_attrs_ertr::future<AlienStore::attrs_t> +AlienStore::get_attrs(CollectionRef ch, + const ghobject_t& oid) +{ + logger().debug("{}", __func__); + assert(tp); + return do_with_op_gate(attrs_t{}, [=, this] (auto &aset) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), [=, this, &aset] { + auto c = static_cast<AlienCollection*>(ch.get()); + const auto r = store->getattrs(c->collection, oid, aset); + return r; + }).then([&aset] (int r) -> get_attrs_ertr::future<attrs_t> { + if (r == -ENOENT) { + return crimson::ct_error::enoent::make(); + } else { + return get_attrs_ertr::make_ready_future<attrs_t>(std::move(aset)); + } + }); + }); +} + +auto AlienStore::omap_get_values(CollectionRef ch, + const ghobject_t& oid, + const set<string>& keys) + -> read_errorator::future<omap_values_t> +{ + logger().debug("{}", __func__); + assert(tp); + return do_with_op_gate(omap_values_t{}, [=, this] (auto &values) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), [=, this, &values] { + auto c = static_cast<AlienCollection*>(ch.get()); + return store->omap_get_values(c->collection, oid, keys, + reinterpret_cast<map<string, bufferlist>*>(&values)); + }).then([&values] (int r) -> read_errorator::future<omap_values_t> { + if (r == -ENOENT) { + return crimson::ct_error::enoent::make(); + } else { + assert(r == 0); + return read_errorator::make_ready_future<omap_values_t>( + std::move(values)); + } + }); + }); +} + +auto AlienStore::omap_get_values(CollectionRef ch, + const ghobject_t &oid, + const std::optional<string> &start) + -> read_errorator::future<std::tuple<bool, omap_values_t>> +{ + logger().debug("{} with_start", __func__); + assert(tp); + return do_with_op_gate(omap_values_t{}, [=, this] (auto &values) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), [=, this, &values] { + auto c = static_cast<AlienCollection*>(ch.get()); + return store->omap_get_values(c->collection, oid, start, + reinterpret_cast<map<string, bufferlist>*>(&values)); + }).then([&values] (int r) + -> read_errorator::future<std::tuple<bool, omap_values_t>> { + if (r == -ENOENT) { + return crimson::ct_error::enoent::make(); + } else if (r < 0){ + logger().error("omap_get_values(start): {}", r); + return crimson::ct_error::input_output_error::make(); + } else { + return read_errorator::make_ready_future<std::tuple<bool, omap_values_t>>( + true, std::move(values)); + } + }); + }); +} + +seastar::future<> AlienStore::do_transaction_no_callbacks( + CollectionRef ch, + ceph::os::Transaction&& txn) +{ + logger().debug("{}", __func__); + auto id = seastar::this_shard_id(); + auto done = seastar::promise<>(); + return do_with_op_gate( + std::move(txn), + std::move(done), + [this, ch, id] (auto &txn, auto &done) { + AlienCollection* alien_coll = static_cast<AlienCollection*>(ch.get()); + // moving the `ch` is crucial for buildability on newer S* versions. + return alien_coll->with_lock([this, ch=std::move(ch), id, &txn, &done] { + assert(tp); + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), + [this, ch, id, &txn, &done, &alien=seastar::engine().alien()] { + txn.register_on_commit(new OnCommit(id, done, alien, txn)); + auto c = static_cast<AlienCollection*>(ch.get()); + return store->queue_transaction(c->collection, std::move(txn)); + }); + }).then([&done] (int r) { + assert(r == 0); + return done.get_future(); + }); + }); +} + +seastar::future<> AlienStore::inject_data_error(const ghobject_t& o) +{ + logger().debug("{}", __func__); + assert(tp); + return seastar::with_gate(op_gate, [=, this] { + return tp->submit([o, this] { + return store->inject_data_error(o); + }); + }); +} + +seastar::future<> AlienStore::inject_mdata_error(const ghobject_t& o) +{ + logger().debug("{}", __func__); + assert(tp); + return seastar::with_gate(op_gate, [=, this] { + return tp->submit([=, this] { + return store->inject_mdata_error(o); + }); + }); +} + +seastar::future<> AlienStore::write_meta(const std::string& key, + const std::string& value) +{ + logger().debug("{}", __func__); + assert(tp); + return seastar::with_gate(op_gate, [=, this] { + return tp->submit([=, this] { + return store->write_meta(key, value); + }).then([] (int r) { + assert(r == 0); + return seastar::make_ready_future<>(); + }); + }); +} + +seastar::future<std::tuple<int, std::string>> +AlienStore::read_meta(const std::string& key) +{ + logger().debug("{}", __func__); + assert(tp); + return seastar::with_gate(op_gate, [this, key] { + return tp->submit([this, key] { + std::string value; + int r = store->read_meta(key, &value); + if (r > 0) { + value.resize(r); + boost::algorithm::trim_right_if(value, + [] (unsigned char c) {return isspace(c);}); + } else { + value.clear(); + } + return std::make_pair(r, value); + }).then([] (auto entry) { + return seastar::make_ready_future<std::tuple<int, std::string>>( + std::move(entry)); + }); + }); +} + +uuid_d AlienStore::get_fsid() const +{ + logger().debug("{}", __func__); + return store->get_fsid(); +} + +seastar::future<store_statfs_t> AlienStore::stat() const +{ + logger().info("{}", __func__); + assert(tp); + return do_with_op_gate(store_statfs_t{}, [this] (store_statfs_t &st) { + return tp->submit([this, &st] { + return store->statfs(&st, nullptr); + }).then([&st] (int r) { + assert(r == 0); + return seastar::make_ready_future<store_statfs_t>(std::move(st)); + }); + }); +} + +unsigned AlienStore::get_max_attr_name_length() const +{ + logger().info("{}", __func__); + return 256; +} + +seastar::future<struct stat> AlienStore::stat( + CollectionRef ch, + const ghobject_t& oid) +{ + assert(tp); + return do_with_op_gate((struct stat){}, [this, ch, oid](auto& st) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), [this, ch, oid, &st] { + auto c = static_cast<AlienCollection*>(ch.get()); + store->stat(c->collection, oid, &st); + return st; + }); + }); +} + +auto AlienStore::omap_get_header(CollectionRef ch, + const ghobject_t& oid) + -> get_attr_errorator::future<ceph::bufferlist> +{ + assert(tp); + return do_with_op_gate(ceph::bufferlist(), [=, this](auto& bl) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), [=, this, &bl] { + auto c = static_cast<AlienCollection*>(ch.get()); + return store->omap_get_header(c->collection, oid, &bl); + }).then([&bl](int r) -> get_attr_errorator::future<ceph::bufferlist> { + if (r == -ENOENT) { + return crimson::ct_error::enoent::make(); + } else if (r < 0) { + logger().error("omap_get_header: {}", r); + ceph_assert(0 == "impossible"); + } else { + return get_attr_errorator::make_ready_future<ceph::bufferlist>( + std::move(bl)); + } + }); + }); +} + +AlienStore::read_errorator::future<std::map<uint64_t, uint64_t>> AlienStore::fiemap( + CollectionRef ch, + const ghobject_t& oid, + uint64_t off, + uint64_t len) +{ + assert(tp); + return do_with_op_gate(std::map<uint64_t, uint64_t>(), [=, this](auto& destmap) { + return tp->submit(ch->get_cid().hash_to_shard(tp->size()), [=, this, &destmap] { + auto c = static_cast<AlienCollection*>(ch.get()); + return store->fiemap(c->collection, oid, off, len, destmap); + }).then([&destmap](int r) + -> read_errorator::future<std::map<uint64_t, uint64_t>> { + if (r == -ENOENT) { + return crimson::ct_error::enoent::make(); + } else { + return read_errorator::make_ready_future<std::map<uint64_t, uint64_t>>( + std::move(destmap)); + } + }); + }); +} + +} diff --git a/src/crimson/os/alienstore/alien_store.h b/src/crimson/os/alienstore/alien_store.h new file mode 100644 index 000000000..79c19b298 --- /dev/null +++ b/src/crimson/os/alienstore/alien_store.h @@ -0,0 +1,133 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/shared_mutex.hh> + +#include "common/ceph_context.h" +#include "os/ObjectStore.h" +#include "osd/osd_types.h" + +#include "crimson/os/alienstore/thread_pool.h" +#include "crimson/os/futurized_collection.h" +#include "crimson/os/futurized_store.h" + +namespace ceph::os { +class Transaction; +} + +namespace crimson::os { +using coll_core_t = FuturizedStore::coll_core_t; +class AlienStore final : public FuturizedStore, + public FuturizedStore::Shard { +public: + AlienStore(const std::string& type, + const std::string& path, + const ConfigValues& values); + ~AlienStore() final; + + seastar::future<> start() final; + seastar::future<> stop() final; + mount_ertr::future<> mount() final; + seastar::future<> umount() final; + + mkfs_ertr::future<> mkfs(uuid_d new_osd_fsid) final; + read_errorator::future<ceph::bufferlist> read(CollectionRef c, + const ghobject_t& oid, + uint64_t offset, + size_t len, + uint32_t op_flags = 0) final; + read_errorator::future<ceph::bufferlist> readv(CollectionRef c, + const ghobject_t& oid, + interval_set<uint64_t>& m, + uint32_t op_flags = 0) final; + + + get_attr_errorator::future<ceph::bufferlist> get_attr(CollectionRef c, + const ghobject_t& oid, + std::string_view name) const final; + get_attrs_ertr::future<attrs_t> get_attrs(CollectionRef c, + const ghobject_t& oid) final; + + read_errorator::future<omap_values_t> omap_get_values( + CollectionRef c, + const ghobject_t& oid, + const omap_keys_t& keys) final; + + /// Retrieves paged set of values > start (if present) + read_errorator::future<std::tuple<bool, omap_values_t>> omap_get_values( + CollectionRef c, ///< [in] collection + const ghobject_t &oid, ///< [in] oid + const std::optional<std::string> &start ///< [in] start, empty for begin + ) final; ///< @return <done, values> values.empty() iff done + + seastar::future<std::tuple<std::vector<ghobject_t>, ghobject_t>> list_objects( + CollectionRef c, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) const final; + + seastar::future<CollectionRef> create_new_collection(const coll_t& cid) final; + seastar::future<CollectionRef> open_collection(const coll_t& cid) final; + seastar::future<std::vector<coll_core_t>> list_collections() final; + + seastar::future<> do_transaction_no_callbacks( + CollectionRef c, + ceph::os::Transaction&& txn) final; + + // error injection + seastar::future<> inject_data_error(const ghobject_t& o) final; + seastar::future<> inject_mdata_error(const ghobject_t& o) final; + + seastar::future<> write_meta(const std::string& key, + const std::string& value) final; + seastar::future<std::tuple<int, std::string>> read_meta( + const std::string& key) final; + uuid_d get_fsid() const final; + seastar::future<store_statfs_t> stat() const final; + unsigned get_max_attr_name_length() const final; + seastar::future<struct stat> stat( + CollectionRef, + const ghobject_t&) final; + get_attr_errorator::future<ceph::bufferlist> omap_get_header( + CollectionRef, + const ghobject_t&) final; + read_errorator::future<std::map<uint64_t, uint64_t>> fiemap( + CollectionRef, + const ghobject_t&, + uint64_t off, + uint64_t len) final; + + FuturizedStore::Shard& get_sharded_store() final { + return *this; + } + +private: + template <class... Args> + auto do_with_op_gate(Args&&... args) const { + return seastar::with_gate(op_gate, + // perfect forwarding in lambda's closure isn't available in C++17 + // using tuple as workaround; see: https://stackoverflow.com/a/49902823 + [args = std::make_tuple(std::forward<Args>(args)...)] () mutable { + return std::apply([] (auto&&... args) { + return seastar::do_with(std::forward<decltype(args)>(args)...); + }, std::move(args)); + }); + } + + // number of cores that are PREVENTED from being scheduled + // to run alien store threads. + static constexpr int N_CORES_FOR_SEASTAR = 3; + mutable std::unique_ptr<crimson::os::ThreadPool> tp; + const std::string type; + const std::string path; + const ConfigValues values; + uint64_t used_bytes = 0; + std::unique_ptr<ObjectStore> store; + std::unique_ptr<CephContext> cct; + mutable seastar::gate op_gate; + std::unordered_map<coll_t, CollectionRef> coll_map; +}; +} diff --git a/src/crimson/os/alienstore/semaphore.h b/src/crimson/os/alienstore/semaphore.h new file mode 100644 index 000000000..8cba02ab1 --- /dev/null +++ b/src/crimson/os/alienstore/semaphore.h @@ -0,0 +1,90 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab +#pragma once + +#include <semaphore.h> +#include <ctime> +#include <cerrno> +#include <exception> +#include <chrono> + +namespace crimson { + +// an implementation of std::counting_semaphore<> in C++17 using the POSIX +// semaphore. +// +// LeastMaxValue is ignored, as we don't have different backends optimized +// for different LeastMaxValues +template<unsigned LeastMaxValue = 64> +class counting_semaphore { + using clock_t = std::chrono::system_clock; +public: + explicit counting_semaphore(unsigned count) noexcept { + sem_init(&sem, 0, count); + } + + counting_semaphore(const counting_semaphore&) = delete; + counting_semaphore& operator=(const counting_semaphore&) = delete; + + ~counting_semaphore() { + sem_destroy(&sem); + } + + void acquire() noexcept { + for (;;) { + int err = sem_wait(&sem); + if (err != 0) { + if (errno == EINTR) { + continue; + } else { + std::terminate(); + } + } else { + break; + } + } + } + + void release(unsigned update = 1) { + for (; update != 0; --update) { + int err = sem_post(&sem); + if (err != 0) { + std::terminate(); + } + } + } + + template<typename Clock, typename Duration> + bool try_acquire_until(const std::chrono::time_point<Clock, Duration>& abs_time) noexcept { + auto s = std::chrono::time_point_cast<std::chrono::seconds>(abs_time); + auto ns = std::chrono::duration_cast<std::chrono::nanoseconds>(abs_time - s); + struct timespec ts = { + static_cast<std::time_t>(s.time_since_epoch().count()), + static_cast<long>(ns.count()) + }; + for (;;) { + if (int err = sem_timedwait(&sem, &ts); err) { + if (errno == EINTR) { + continue; + } else if (errno == ETIMEDOUT || errno == EINVAL) { + return false; + } else { + std::terminate(); + } + } else { + break; + } + } + return true; + } + + template<typename Rep, typename Period> + bool try_acquire_for(const std::chrono::duration<Rep, Period>& rel_time) { + return try_acquire_until(clock_t::now() + rel_time); + } + +private: + sem_t sem; +}; + +} diff --git a/src/crimson/os/alienstore/thread_pool.cc b/src/crimson/os/alienstore/thread_pool.cc new file mode 100644 index 000000000..5cf9590e6 --- /dev/null +++ b/src/crimson/os/alienstore/thread_pool.cc @@ -0,0 +1,98 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#include "thread_pool.h" + +#include <chrono> +#include <pthread.h> + +#include "include/ceph_assert.h" +#include "crimson/common/config_proxy.h" + +using crimson::common::local_conf; + +namespace crimson::os { + +ThreadPool::ThreadPool(size_t n_threads, + size_t queue_sz, + const std::optional<seastar::resource::cpuset>& cpus) + : n_threads(n_threads), + queue_size{round_up_to(queue_sz, seastar::smp::count)}, + pending_queues(n_threads) +{ + auto queue_max_wait = std::chrono::seconds(local_conf()->threadpool_empty_queue_max_wait); + for (size_t i = 0; i < n_threads; i++) { + threads.emplace_back([this, cpus, queue_max_wait, i] { + if (cpus.has_value()) { + pin(*cpus); + } + block_sighup(); + (void) pthread_setname_np(pthread_self(), "alien-store-tp"); + loop(queue_max_wait, i); + }); + } +} + +ThreadPool::~ThreadPool() +{ + for (auto& thread : threads) { + thread.join(); + } +} + +void ThreadPool::pin(const seastar::resource::cpuset& cpus) +{ + cpu_set_t cs; + CPU_ZERO(&cs); + for (auto cpu : cpus) { + CPU_SET(cpu, &cs); + } + [[maybe_unused]] auto r = pthread_setaffinity_np(pthread_self(), + sizeof(cs), &cs); + ceph_assert(r == 0); +} + +void ThreadPool::block_sighup() +{ + sigset_t sigs; + sigemptyset(&sigs); + // alien threads must ignore the SIGHUP. It's necessary as in + // `crimson/osd/main.cc` we set a handler using the Seastar's + // signal handling infrastrucute which assumes the `_backend` + // of `seastar::engine()` is not null. Grep `reactor.cc` for + // `sigaction` or just visit `reactor::signals::handle_signal()`. + sigaddset(&sigs, SIGHUP); + pthread_sigmask(SIG_BLOCK, &sigs, nullptr); +} + +void ThreadPool::loop(std::chrono::milliseconds queue_max_wait, size_t shard) +{ + auto& pending = pending_queues[shard]; + for (;;) { + WorkItem* work_item = nullptr; + work_item = pending.pop_front(queue_max_wait); + if (work_item) { + work_item->process(); + } else if (is_stopping()) { + break; + } + } +} + +seastar::future<> ThreadPool::start() +{ + auto slots_per_shard = queue_size / seastar::smp::count; + return submit_queue.start(slots_per_shard); +} + +seastar::future<> ThreadPool::stop() +{ + return submit_queue.stop().then([this] { + stopping = true; + for (auto& q : pending_queues) { + q.stop(); + } + }); +} + +} // namespace crimson::os diff --git a/src/crimson/os/alienstore/thread_pool.h b/src/crimson/os/alienstore/thread_pool.h new file mode 100644 index 000000000..78e18219a --- /dev/null +++ b/src/crimson/os/alienstore/thread_pool.h @@ -0,0 +1,184 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab +#pragma once + +#include <atomic> +#include <condition_variable> +#include <tuple> +#include <type_traits> +#include <boost/lockfree/queue.hpp> +#include <boost/optional.hpp> +#include <seastar/core/future.hh> +#include <seastar/core/gate.hh> +#include <seastar/core/reactor.hh> +#include <seastar/core/resource.hh> +#include <seastar/core/semaphore.hh> +#include <seastar/core/sharded.hh> + +#if __cplusplus > 201703L +#include <semaphore> +namespace crimson { + using std::counting_semaphore; +} +#else +#include "semaphore.h" +#endif + +namespace crimson::os { + +struct WorkItem { + virtual ~WorkItem() {} + virtual void process() = 0; +}; + +template<typename Func> +struct Task final : WorkItem { + using T = std::invoke_result_t<Func>; + using future_stored_type_t = + std::conditional_t<std::is_void_v<T>, + seastar::internal::future_stored_type_t<>, + seastar::internal::future_stored_type_t<T>>; + using futurator_t = seastar::futurize<T>; +public: + explicit Task(Func&& f) + : func(std::move(f)) + {} + void process() override { + try { + if constexpr (std::is_void_v<T>) { + func(); + state.set(); + } else { + state.set(func()); + } + } catch (...) { + state.set_exception(std::current_exception()); + } + on_done.write_side().signal(1); + } + typename futurator_t::type get_future() { + return on_done.wait().then([this](size_t) { + if (state.failed()) { + return futurator_t::make_exception_future(state.get_exception()); + } else { + return futurator_t::from_tuple(state.get_value()); + } + }); + } +private: + Func func; + seastar::future_state<future_stored_type_t> state; + seastar::readable_eventfd on_done; +}; + +struct SubmitQueue { + seastar::semaphore free_slots; + seastar::gate pending_tasks; + explicit SubmitQueue(size_t num_free_slots) + : free_slots(num_free_slots) + {} + seastar::future<> stop() { + return pending_tasks.close(); + } +}; + +struct ShardedWorkQueue { +public: + WorkItem* pop_front(std::chrono::milliseconds& queue_max_wait) { + if (sem.try_acquire_for(queue_max_wait)) { + if (!is_stopping()) { + WorkItem* work_item = nullptr; + [[maybe_unused]] bool popped = pending.pop(work_item); + assert(popped); + return work_item; + } + } + return nullptr; + } + void stop() { + stopping = true; + sem.release(); + } + void push_back(WorkItem* work_item) { + [[maybe_unused]] bool pushed = pending.push(work_item); + assert(pushed); + sem.release(); + } +private: + bool is_stopping() const { + return stopping; + } + std::atomic<bool> stopping = false; + static constexpr unsigned QUEUE_SIZE = 128; + crimson::counting_semaphore<QUEUE_SIZE> sem{0}; + boost::lockfree::queue<WorkItem*> pending{QUEUE_SIZE}; +}; + +/// an engine for scheduling non-seastar tasks from seastar fibers +class ThreadPool { +public: + /** + * @param queue_sz the depth of pending queue. before a task is scheduled, + * it waits in this queue. we will round this number to + * multiple of the number of cores. + * @param n_threads the number of threads in this thread pool. + * @param cpu the CPU core to which this thread pool is assigned + * @note each @c Task has its own crimson::thread::Condition, which possesses + * an fd, so we should keep the size of queue under a reasonable limit. + */ + ThreadPool(size_t n_threads, size_t queue_sz, const std::optional<seastar::resource::cpuset>& cpus); + ~ThreadPool(); + seastar::future<> start(); + seastar::future<> stop(); + size_t size() { + return n_threads; + } + template<typename Func, typename...Args> + auto submit(int shard, Func&& func, Args&&... args) { + auto packaged = [func=std::move(func), + args=std::forward_as_tuple(args...)] { + return std::apply(std::move(func), std::move(args)); + }; + return seastar::with_gate(submit_queue.local().pending_tasks, + [packaged=std::move(packaged), shard, this] { + return local_free_slots().wait() + .then([packaged=std::move(packaged), shard, this] { + auto task = new Task{std::move(packaged)}; + auto fut = task->get_future(); + pending_queues[shard].push_back(task); + return fut.finally([task, this] { + local_free_slots().signal(); + delete task; + }); + }); + }); + } + + template<typename Func> + auto submit(Func&& func) { + return submit(::rand() % n_threads, std::forward<Func>(func)); + } + +private: + void loop(std::chrono::milliseconds queue_max_wait, size_t shard); + bool is_stopping() const { + return stopping.load(std::memory_order_relaxed); + } + static void pin(const seastar::resource::cpuset& cpus); + static void block_sighup(); + seastar::semaphore& local_free_slots() { + return submit_queue.local().free_slots; + } + ThreadPool(const ThreadPool&) = delete; + ThreadPool& operator=(const ThreadPool&) = delete; + +private: + size_t n_threads; + std::atomic<bool> stopping = false; + std::vector<std::thread> threads; + seastar::sharded<SubmitQueue> submit_queue; + const size_t queue_size; + std::vector<ShardedWorkQueue> pending_queues; +}; + +} // namespace crimson::os diff --git a/src/crimson/os/cyanstore/CMakeLists.txt b/src/crimson/os/cyanstore/CMakeLists.txt new file mode 100644 index 000000000..65f2b5498 --- /dev/null +++ b/src/crimson/os/cyanstore/CMakeLists.txt @@ -0,0 +1,7 @@ +add_library(crimson-cyanstore STATIC + cyan_store.cc + cyan_collection.cc + cyan_object.cc) +target_link_libraries(crimson-cyanstore + crimson + crimson-os) diff --git a/src/crimson/os/cyanstore/cyan_collection.cc b/src/crimson/os/cyanstore/cyan_collection.cc new file mode 100644 index 000000000..9a814f978 --- /dev/null +++ b/src/crimson/os/cyanstore/cyan_collection.cc @@ -0,0 +1,78 @@ +#include "cyan_collection.h" + +#include "cyan_object.h" + +using std::make_pair; + +namespace crimson::os +{ + +Collection::Collection(const coll_t& c) + : FuturizedCollection{c} +{} + +Collection::~Collection() = default; + +Collection::ObjectRef Collection::create_object() const +{ + return new crimson::os::Object; +} + +Collection::ObjectRef Collection::get_object(ghobject_t oid) +{ + auto o = object_hash.find(oid); + if (o == object_hash.end()) + return ObjectRef(); + return o->second; +} + +Collection::ObjectRef Collection::get_or_create_object(ghobject_t oid) +{ + auto result = object_hash.emplace(oid, ObjectRef{}); + if (result.second) + object_map[oid] = result.first->second = create_object(); + return result.first->second; +} + +uint64_t Collection::used_bytes() const +{ + uint64_t result = 0; + for (auto& obj : object_map) { + result += obj.second->get_size(); + } + return result; +} + +void Collection::encode(bufferlist& bl) const +{ + ENCODE_START(1, 1, bl); + encode(xattr, bl); + encode(use_page_set, bl); + uint32_t s = object_map.size(); + encode(s, bl); + for (auto& [oid, obj] : object_map) { + encode(oid, bl); + obj->encode(bl); + } + ENCODE_FINISH(bl); +} + +void Collection::decode(bufferlist::const_iterator& p) +{ + DECODE_START(1, p); + decode(xattr, p); + decode(use_page_set, p); + uint32_t s; + decode(s, p); + while (s--) { + ghobject_t k; + decode(k, p); + auto o = create_object(); + o->decode(p); + object_map.insert(make_pair(k, o)); + object_hash.insert(make_pair(k, o)); + } + DECODE_FINISH(p); +} + +} diff --git a/src/crimson/os/cyanstore/cyan_collection.h b/src/crimson/os/cyanstore/cyan_collection.h new file mode 100644 index 000000000..068e427d8 --- /dev/null +++ b/src/crimson/os/cyanstore/cyan_collection.h @@ -0,0 +1,51 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <string> +#include <unordered_map> +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include "include/buffer.h" +#include "osd/osd_types.h" + +#include "crimson/os/futurized_collection.h" + +namespace crimson::os { + +class Object; +/** + * a collection also orders transactions + * + * Any transactions queued under a given collection will be applied in + * sequence. Transactions queued under different collections may run + * in parallel. + * + * ObjectStore users may get collection handles with open_collection() (or, + * for bootstrapping a new collection, create_new_collection()). + */ +struct Collection final : public FuturizedCollection { + using ObjectRef = boost::intrusive_ptr<Object>; + int bits = 0; + // always use bufferlist object for testing + bool use_page_set = false; + std::unordered_map<ghobject_t, ObjectRef> object_hash; ///< for lookup + std::map<ghobject_t, ObjectRef> object_map; ///< for iteration + std::map<std::string,bufferptr> xattr; + bool exists = true; + + Collection(const coll_t& c); + ~Collection() final; + + ObjectRef create_object() const; + ObjectRef get_object(ghobject_t oid); + ObjectRef get_or_create_object(ghobject_t oid); + uint64_t used_bytes() const; + + void encode(bufferlist& bl) const; + void decode(bufferlist::const_iterator& p); +}; + +} diff --git a/src/crimson/os/cyanstore/cyan_object.cc b/src/crimson/os/cyanstore/cyan_object.cc new file mode 100644 index 000000000..34bc13b7f --- /dev/null +++ b/src/crimson/os/cyanstore/cyan_object.cc @@ -0,0 +1,89 @@ +#include "cyan_object.h" +#include "include/encoding.h" + +namespace crimson::os { + +size_t Object::get_size() const { + return data.length(); +} + +ceph::bufferlist Object::read(uint64_t offset, uint64_t len) +{ + bufferlist ret; + ret.substr_of(data, offset, len); + return ret; +} + +int Object::write(uint64_t offset, const bufferlist &src) +{ + unsigned len = src.length(); + // before + bufferlist newdata; + if (get_size() >= offset) { + newdata.substr_of(data, 0, offset); + } else { + if (get_size()) { + newdata.substr_of(data, 0, get_size()); + } + newdata.append_zero(offset - get_size()); + } + + newdata.append(src); + + // after + if (get_size() > offset + len) { + bufferlist tail; + tail.substr_of(data, offset + len, get_size() - (offset + len)); + newdata.append(tail); + } + + data = std::move(newdata); + return 0; +} + +int Object::clone(Object *src, uint64_t srcoff, uint64_t len, + uint64_t dstoff) +{ + bufferlist bl; + if (srcoff == dstoff && len == src->get_size()) { + data = src->data; + return 0; + } + bl.substr_of(src->data, srcoff, len); + return write(dstoff, bl); + +} + +int Object::truncate(uint64_t size) +{ + if (get_size() > size) { + bufferlist bl; + bl.substr_of(data, 0, size); + data = std::move(bl); + } else if (get_size() == size) { + // do nothing + } else { + data.append_zero(size - get_size()); + } + return 0; +} + +void Object::encode(bufferlist& bl) const { + ENCODE_START(1, 1, bl); + encode(data, bl); + encode(xattr, bl); + encode(omap_header, bl); + encode(omap, bl); + ENCODE_FINISH(bl); +} + +void Object::decode(bufferlist::const_iterator& p) { + DECODE_START(1, p); + decode(data, p); + decode(xattr, p); + decode(omap_header, p); + decode(omap, p); + DECODE_FINISH(p); +} + +} diff --git a/src/crimson/os/cyanstore/cyan_object.h b/src/crimson/os/cyanstore/cyan_object.h new file mode 100644 index 000000000..624f9513a --- /dev/null +++ b/src/crimson/os/cyanstore/cyan_object.h @@ -0,0 +1,45 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab +#pragma once + +#include <cstddef> +#include <map> +#include <string> +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include "include/buffer.h" + +namespace crimson::os { + +struct Object : public boost::intrusive_ref_counter< + Object, + boost::thread_unsafe_counter> +{ + using bufferlist = ceph::bufferlist; + + bufferlist data; + // use transparent comparator for better performance, see + // https://en.cppreference.com/w/cpp/utility/functional/less_void + std::map<std::string,bufferlist,std::less<>> xattr; + bufferlist omap_header; + std::map<std::string,bufferlist> omap; + + typedef boost::intrusive_ptr<Object> Ref; + + Object() = default; + + // interface for object data + size_t get_size() const; + ceph::bufferlist read(uint64_t offset, uint64_t len); + int write(uint64_t offset, const bufferlist &bl); + int clone(Object *src, uint64_t srcoff, uint64_t len, + uint64_t dstoff); + int truncate(uint64_t offset); + + void encode(bufferlist& bl) const; + void decode(bufferlist::const_iterator& p); +}; +using ObjectRef = boost::intrusive_ptr<Object>; + +} diff --git a/src/crimson/os/cyanstore/cyan_store.cc b/src/crimson/os/cyanstore/cyan_store.cc new file mode 100644 index 000000000..f2a6018e3 --- /dev/null +++ b/src/crimson/os/cyanstore/cyan_store.cc @@ -0,0 +1,952 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "cyan_store.h" + +#include <boost/algorithm/string/trim.hpp> +#include <fmt/format.h> +#include <fmt/ostream.h> + +#include "common/safe_io.h" +#include "os/Transaction.h" + +#include "crimson/common/buffer_io.h" +#include "crimson/common/config_proxy.h" +#include "cyan_collection.h" +#include "cyan_object.h" + +namespace { + seastar::logger& logger() { + return crimson::get_logger(ceph_subsys_cyanstore); + } +} + +using std::string; +using crimson::common::local_conf; + +namespace crimson::os { + +using ObjectRef = boost::intrusive_ptr<Object>; + +CyanStore::CyanStore(const std::string& path) + : path{path} +{} + +CyanStore::~CyanStore() = default; + +template <const char* MsgV> +struct singleton_ec : std::error_code { + singleton_ec() + : error_code(42, this_error_category{}) { + }; +private: + struct this_error_category : std::error_category { + const char* name() const noexcept final { + // XXX: we could concatenate with MsgV at compile-time but the burden + // isn't worth the benefit. + return "singleton_ec"; + } + std::string message([[maybe_unused]] const int ev) const final { + assert(ev == 42); + return MsgV; + } + }; +}; + +seastar::future<store_statfs_t> CyanStore::stat() const +{ + ceph_assert(seastar::this_shard_id() == primary_core); + logger().debug("{}", __func__); + return shard_stores.map_reduce0( + [](const CyanStore::Shard &local_store) { + return local_store.get_used_bytes(); + }, + (uint64_t)0, + std::plus<uint64_t>() + ).then([](uint64_t used_bytes) { + store_statfs_t st; + st.total = crimson::common::local_conf().get_val<Option::size_t>("memstore_device_bytes"); + st.available = st.total - used_bytes; + return seastar::make_ready_future<store_statfs_t>(std::move(st)); + }); +} + + +CyanStore::mkfs_ertr::future<> CyanStore::mkfs(uuid_d new_osd_fsid) +{ + ceph_assert(seastar::this_shard_id() == primary_core); + static const char read_meta_errmsg[]{"read_meta"}; + static const char parse_fsid_errmsg[]{"failed to parse fsid"}; + static const char match_ofsid_errmsg[]{"unmatched osd_fsid"}; + return read_meta("fsid").then([=, this](auto&& ret) -> mkfs_ertr::future<> { + auto& [r, fsid_str] = ret; + if (r == -ENOENT) { + if (new_osd_fsid.is_zero()) { + osd_fsid.generate_random(); + } else { + osd_fsid = new_osd_fsid; + } + return write_meta("fsid", fmt::format("{}", osd_fsid)); + } else if (r < 0) { + return crimson::stateful_ec{ singleton_ec<read_meta_errmsg>() }; + } else { + logger().info("mkfs already has fsid {}", fsid_str); + if (!osd_fsid.parse(fsid_str.c_str())) { + return crimson::stateful_ec{ singleton_ec<parse_fsid_errmsg>() }; + } else if (osd_fsid != new_osd_fsid) { + logger().error("on-disk fsid {} != provided {}", osd_fsid, new_osd_fsid); + return crimson::stateful_ec{ singleton_ec<match_ofsid_errmsg>() }; + } else { + return mkfs_ertr::now(); + } + } + }).safe_then([this]{ + return write_meta("type", "memstore"); + }).safe_then([this] { + return shard_stores.invoke_on_all( + [](auto &local_store) { + return local_store.mkfs(); + }); + }); +} + +seastar::future<> CyanStore::Shard::mkfs() +{ + std::string fn = + path + "/collections" + std::to_string(seastar::this_shard_id()); + ceph::bufferlist bl; + std::set<coll_t> collections; + ceph::encode(collections, bl); + return crimson::write_file(std::move(bl), fn); +} + +using coll_core_t = FuturizedStore::coll_core_t; +seastar::future<std::vector<coll_core_t>> +CyanStore::list_collections() +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return seastar::do_with(std::vector<coll_core_t>{}, [this](auto &collections) { + return shard_stores.map([](auto &local_store) { + return local_store.list_collections(); + }).then([&collections](std::vector<std::vector<coll_core_t>> results) { + for (auto& colls : results) { + collections.insert(collections.end(), colls.begin(), colls.end()); + } + return seastar::make_ready_future<std::vector<coll_core_t>>( + std::move(collections)); + }); + }); +} + +CyanStore::mount_ertr::future<> CyanStore::Shard::mount() +{ + static const char read_file_errmsg[]{"read_file"}; + ceph::bufferlist bl; + std::string fn = + path + "/collections" + std::to_string(seastar::this_shard_id()); + std::string err; + if (int r = bl.read_file(fn.c_str(), &err); r < 0) { + return crimson::stateful_ec{ singleton_ec<read_file_errmsg>() }; + } + + std::set<coll_t> collections; + auto p = bl.cbegin(); + ceph::decode(collections, p); + + for (auto& coll : collections) { + std::string fn = fmt::format("{}/{}{}", path, coll, + std::to_string(seastar::this_shard_id())); + ceph::bufferlist cbl; + if (int r = cbl.read_file(fn.c_str(), &err); r < 0) { + return crimson::stateful_ec{ singleton_ec<read_file_errmsg>() }; + } + boost::intrusive_ptr<Collection> c{new Collection{coll}}; + auto p = cbl.cbegin(); + c->decode(p); + coll_map[coll] = c; + used_bytes += c->used_bytes(); + } + return mount_ertr::now(); +} + +seastar::future<> CyanStore::Shard::umount() +{ + return seastar::do_with(std::set<coll_t>{}, [this](auto& collections) { + return seastar::do_for_each(coll_map, [&collections, this](auto& coll) { + auto& [col, ch] = coll; + collections.insert(col); + ceph::bufferlist bl; + ceph_assert(ch); + ch->encode(bl); + std::string fn = fmt::format("{}/{}{}", path, col, + std::to_string(seastar::this_shard_id())); + return crimson::write_file(std::move(bl), fn); + }).then([&collections, this] { + ceph::bufferlist bl; + ceph::encode(collections, bl); + std::string fn = fmt::format("{}/collections{}", + path, std::to_string(seastar::this_shard_id())); + return crimson::write_file(std::move(bl), fn); + }); + }); +} + +seastar::future<std::tuple<std::vector<ghobject_t>, ghobject_t>> +CyanStore::Shard::list_objects( + CollectionRef ch, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) const +{ + auto c = static_cast<Collection*>(ch.get()); + logger().debug("{} {} {} {} {}", + __func__, c->get_cid(), start, end, limit); + std::vector<ghobject_t> objects; + objects.reserve(limit); + ghobject_t next = ghobject_t::get_max(); + for (const auto& [oid, obj] : + boost::make_iterator_range(c->object_map.lower_bound(start), + c->object_map.end())) { + std::ignore = obj; + if (oid >= end || objects.size() >= limit) { + next = oid; + break; + } + objects.push_back(oid); + } + return seastar::make_ready_future<std::tuple<std::vector<ghobject_t>, ghobject_t>>( + std::make_tuple(std::move(objects), next)); +} + +seastar::future<CollectionRef> +CyanStore::Shard::create_new_collection(const coll_t& cid) +{ + auto c = new Collection{cid}; + new_coll_map[cid] = c; + return seastar::make_ready_future<CollectionRef>(c); +} + +seastar::future<CollectionRef> +CyanStore::Shard::open_collection(const coll_t& cid) +{ + return seastar::make_ready_future<CollectionRef>(_get_collection(cid)); +} + +seastar::future<std::vector<coll_core_t>> +CyanStore::Shard::list_collections() +{ + std::vector<coll_core_t> collections; + for (auto& coll : coll_map) { + collections.push_back(std::make_pair(coll.first, seastar::this_shard_id())); + } + return seastar::make_ready_future<std::vector<coll_core_t>>(std::move(collections)); +} + +CyanStore::Shard::read_errorator::future<ceph::bufferlist> +CyanStore::Shard::read( + CollectionRef ch, + const ghobject_t& oid, + uint64_t offset, + size_t len, + uint32_t op_flags) +{ + auto c = static_cast<Collection*>(ch.get()); + logger().debug("{} {} {} {}~{}", + __func__, c->get_cid(), oid, offset, len); + if (!c->exists) { + return crimson::ct_error::enoent::make(); + } + ObjectRef o = c->get_object(oid); + if (!o) { + return crimson::ct_error::enoent::make(); + } + if (offset >= o->get_size()) + return read_errorator::make_ready_future<ceph::bufferlist>(); + size_t l = len; + if (l == 0 && offset == 0) // note: len == 0 means read the entire object + l = o->get_size(); + else if (offset + l > o->get_size()) + l = o->get_size() - offset; + return read_errorator::make_ready_future<ceph::bufferlist>(o->read(offset, l)); +} + +CyanStore::Shard::read_errorator::future<ceph::bufferlist> +CyanStore::Shard::readv( + CollectionRef ch, + const ghobject_t& oid, + interval_set<uint64_t>& m, + uint32_t op_flags) +{ + return seastar::do_with(ceph::bufferlist{}, + [this, ch, oid, &m, op_flags](auto& bl) { + return crimson::do_for_each(m, + [this, ch, oid, op_flags, &bl](auto& p) { + return read(ch, oid, p.first, p.second, op_flags) + .safe_then([&bl](auto ret) { + bl.claim_append(ret); + }); + }).safe_then([&bl] { + return read_errorator::make_ready_future<ceph::bufferlist>(std::move(bl)); + }); + }); +} + +CyanStore::Shard::get_attr_errorator::future<ceph::bufferlist> +CyanStore::Shard::get_attr( + CollectionRef ch, + const ghobject_t& oid, + std::string_view name) const +{ + auto c = static_cast<Collection*>(ch.get()); + logger().debug("{} {} {}", + __func__, c->get_cid(), oid); + auto o = c->get_object(oid); + if (!o) { + return crimson::ct_error::enoent::make(); + } + if (auto found = o->xattr.find(name); found != o->xattr.end()) { + return get_attr_errorator::make_ready_future<ceph::bufferlist>(found->second); + } else { + return crimson::ct_error::enodata::make(); + } +} + +CyanStore::Shard::get_attrs_ertr::future<CyanStore::Shard::attrs_t> +CyanStore::Shard::get_attrs( + CollectionRef ch, + const ghobject_t& oid) +{ + auto c = static_cast<Collection*>(ch.get()); + logger().debug("{} {} {}", + __func__, c->get_cid(), oid); + auto o = c->get_object(oid); + if (!o) { + return crimson::ct_error::enoent::make(); + } + return get_attrs_ertr::make_ready_future<attrs_t>(o->xattr); +} + +auto CyanStore::Shard::omap_get_values( + CollectionRef ch, + const ghobject_t& oid, + const omap_keys_t& keys) + -> read_errorator::future<omap_values_t> +{ + auto c = static_cast<Collection*>(ch.get()); + logger().debug("{} {} {}", __func__, c->get_cid(), oid); + auto o = c->get_object(oid); + if (!o) { + return crimson::ct_error::enoent::make(); + } + omap_values_t values; + for (auto& key : keys) { + if (auto found = o->omap.find(key); found != o->omap.end()) { + values.insert(*found); + } + } + return seastar::make_ready_future<omap_values_t>(std::move(values)); +} + +auto CyanStore::Shard::omap_get_values( + CollectionRef ch, + const ghobject_t &oid, + const std::optional<string> &start) + -> CyanStore::Shard::read_errorator::future<std::tuple<bool, omap_values_t>> +{ + auto c = static_cast<Collection*>(ch.get()); + logger().debug("{} {} {}", __func__, c->get_cid(), oid); + auto o = c->get_object(oid); + if (!o) { + return crimson::ct_error::enoent::make(); + } + omap_values_t values; + for (auto i = start ? o->omap.upper_bound(*start) : o->omap.begin(); + i != o->omap.end(); + ++i) { + values.insert(*i); + } + return seastar::make_ready_future<std::tuple<bool, omap_values_t>>( + std::make_tuple(true, std::move(values))); +} + +auto CyanStore::Shard::omap_get_header( + CollectionRef ch, + const ghobject_t& oid) + -> CyanStore::Shard::get_attr_errorator::future<ceph::bufferlist> +{ + auto c = static_cast<Collection*>(ch.get()); + auto o = c->get_object(oid); + if (!o) { + return crimson::ct_error::enoent::make(); + } + + return get_attr_errorator::make_ready_future<ceph::bufferlist>( + o->omap_header); +} + +seastar::future<> CyanStore::Shard::do_transaction_no_callbacks( + CollectionRef ch, + ceph::os::Transaction&& t) +{ + using ceph::os::Transaction; + int r = 0; + try { + auto i = t.begin(); + while (i.have_op()) { + r = 0; + switch (auto op = i.decode_op(); op->op) { + case Transaction::OP_NOP: + break; + case Transaction::OP_REMOVE: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + r = _remove(cid, oid); + if (r == -ENOENT) { + r = 0; + } + } + break; + case Transaction::OP_TOUCH: + case Transaction::OP_CREATE: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + r = _touch(cid, oid); + } + break; + case Transaction::OP_WRITE: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + uint64_t off = op->off; + uint64_t len = op->len; + uint32_t fadvise_flags = i.get_fadvise_flags(); + ceph::bufferlist bl; + i.decode_bl(bl); + r = _write(cid, oid, off, len, bl, fadvise_flags); + } + break; + case Transaction::OP_ZERO: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + uint64_t off = op->off; + uint64_t len = op->len; + r = _zero(cid, oid, off, len); + } + break; + case Transaction::OP_TRUNCATE: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + uint64_t off = op->off; + r = _truncate(cid, oid, off); + } + break; + case Transaction::OP_CLONE: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + ghobject_t noid = i.get_oid(op->dest_oid); + r = _clone(cid, oid, noid); + } + break; + case Transaction::OP_SETATTR: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + std::string name = i.decode_string(); + ceph::bufferlist bl; + i.decode_bl(bl); + std::map<std::string, bufferlist> to_set; + to_set.emplace(name, std::move(bl)); + r = _setattrs(cid, oid, std::move(to_set)); + } + break; + case Transaction::OP_SETATTRS: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + std::map<std::string, bufferlist> aset; + i.decode_attrset(aset); + r = _setattrs(cid, oid, std::move(aset)); + } + break; + case Transaction::OP_RMATTR: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + std::string name = i.decode_string(); + r = _rm_attr(cid, oid, name); + } + break; + case Transaction::OP_RMATTRS: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + r = _rm_attrs(cid, oid); + } + break; + case Transaction::OP_MKCOLL: + { + coll_t cid = i.get_cid(op->cid); + r = _create_collection(cid, op->split_bits); + } + break; + case Transaction::OP_SETALLOCHINT: + { + r = 0; + } + break; + case Transaction::OP_OMAP_CLEAR: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + r = _omap_clear(cid, oid); + } + break; + case Transaction::OP_OMAP_SETKEYS: + { + coll_t cid = i.get_cid(op->cid); + ghobject_t oid = i.get_oid(op->oid); + std::map<std::string, ceph::bufferlist> aset; + i.decode_attrset(aset); + r = _omap_set_values(cid, oid, std::move(aset)); + } + break; + case Transaction::OP_OMAP_SETHEADER: + { + const coll_t &cid = i.get_cid(op->cid); + const ghobject_t &oid = i.get_oid(op->oid); + ceph::bufferlist bl; + i.decode_bl(bl); + r = _omap_set_header(cid, oid, bl); + } + break; + case Transaction::OP_OMAP_RMKEYS: + { + const coll_t &cid = i.get_cid(op->cid); + const ghobject_t &oid = i.get_oid(op->oid); + omap_keys_t keys; + i.decode_keyset(keys); + r = _omap_rmkeys(cid, oid, keys); + } + break; + case Transaction::OP_OMAP_RMKEYRANGE: + { + const coll_t &cid = i.get_cid(op->cid); + const ghobject_t &oid = i.get_oid(op->oid); + string first, last; + first = i.decode_string(); + last = i.decode_string(); + r = _omap_rmkeyrange(cid, oid, first, last); + } + break; + case Transaction::OP_COLL_HINT: + { + ceph::bufferlist hint; + i.decode_bl(hint); + // ignored + break; + } + default: + logger().error("bad op {}", static_cast<unsigned>(op->op)); + abort(); + } + if (r < 0) { + break; + } + } + } catch (std::exception &e) { + logger().error("{} got exception {}", __func__, e); + r = -EINVAL; + } + if (r < 0) { + logger().error(" transaction dump:\n"); + JSONFormatter f(true); + f.open_object_section("transaction"); + t.dump(&f); + f.close_section(); + std::stringstream str; + f.flush(str); + logger().error("{}", str.str()); + ceph_assert(r == 0); + } + return seastar::now(); +} + +int CyanStore::Shard::_remove(const coll_t& cid, const ghobject_t& oid) +{ + logger().debug("{} cid={} oid={}", + __func__, cid, oid); + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + auto i = c->object_hash.find(oid); + if (i == c->object_hash.end()) + return -ENOENT; + used_bytes -= i->second->get_size(); + c->object_hash.erase(i); + c->object_map.erase(oid); + return 0; +} + +int CyanStore::Shard::_touch(const coll_t& cid, const ghobject_t& oid) +{ + logger().debug("{} cid={} oid={}", + __func__, cid, oid); + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + c->get_or_create_object(oid); + return 0; +} + +int CyanStore::Shard::_write( + const coll_t& cid, + const ghobject_t& oid, + uint64_t offset, + size_t len, + const ceph::bufferlist& bl, + uint32_t fadvise_flags) +{ + logger().debug("{} {} {} {} ~ {}", + __func__, cid, oid, offset, len); + assert(len == bl.length()); + + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + ObjectRef o = c->get_or_create_object(oid); + if (len > 0 && !local_conf()->memstore_debug_omit_block_device_write) { + const ssize_t old_size = o->get_size(); + o->write(offset, bl); + used_bytes += (o->get_size() - old_size); + } + + return 0; +} + +int CyanStore::Shard::_zero( + const coll_t& cid, + const ghobject_t& oid, + uint64_t offset, + size_t len) +{ + logger().debug("{} {} {} {} ~ {}", + __func__, cid, oid, offset, len); + + ceph::buffer::list bl; + bl.append_zero(len); + return _write(cid, oid, offset, len, bl, 0); +} + +int CyanStore::Shard::_omap_clear( + const coll_t& cid, + const ghobject_t& oid) +{ + logger().debug("{} {} {}", __func__, cid, oid); + + auto c = _get_collection(cid); + if (!c) { + return -ENOENT; + } + ObjectRef o = c->get_object(oid); + if (!o) { + return -ENOENT; + } + o->omap.clear(); + o->omap_header.clear(); + return 0; +} + +int CyanStore::Shard::_omap_set_values( + const coll_t& cid, + const ghobject_t& oid, + std::map<std::string, ceph::bufferlist> &&aset) +{ + logger().debug( + "{} {} {} {} keys", + __func__, cid, oid, aset.size()); + + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + ObjectRef o = c->get_or_create_object(oid); + for (auto&& [key, val]: aset) { + o->omap.insert_or_assign(std::move(key), std::move(val)); + } + return 0; +} + +int CyanStore::Shard::_omap_set_header( + const coll_t& cid, + const ghobject_t& oid, + const ceph::bufferlist &header) +{ + logger().debug( + "{} {} {} {} bytes", + __func__, cid, oid, header.length()); + + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + ObjectRef o = c->get_or_create_object(oid); + o->omap_header = header; + return 0; +} + +int CyanStore::Shard::_omap_rmkeys( + const coll_t& cid, + const ghobject_t& oid, + const omap_keys_t& aset) +{ + logger().debug( + "{} {} {} {} keys", + __func__, cid, oid, aset.size()); + + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + ObjectRef o = c->get_or_create_object(oid); + for (auto &i: aset) { + o->omap.erase(i); + } + return 0; +} + +int CyanStore::Shard::_omap_rmkeyrange( + const coll_t& cid, + const ghobject_t& oid, + const std::string &first, + const std::string &last) +{ + logger().debug( + "{} {} {} first={} last={}", + __func__, cid, oid, first, last); + + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + ObjectRef o = c->get_or_create_object(oid); + for (auto i = o->omap.lower_bound(first); + i != o->omap.end() && i->first < last; + o->omap.erase(i++)); + return 0; +} + +int CyanStore::Shard::_truncate( + const coll_t& cid, + const ghobject_t& oid, + uint64_t size) +{ + logger().debug("{} cid={} oid={} size={}", + __func__, cid, oid, size); + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + ObjectRef o = c->get_object(oid); + if (!o) + return -ENOENT; + if (local_conf()->memstore_debug_omit_block_device_write) + return 0; + const ssize_t old_size = o->get_size(); + int r = o->truncate(size); + used_bytes += (o->get_size() - old_size); + return r; +} + +int CyanStore::Shard::_clone( + const coll_t& cid, + const ghobject_t& oid, + const ghobject_t& noid) +{ + logger().debug("{} cid={} oid={} noid={}", + __func__, cid, oid, noid); + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + ObjectRef oo = c->get_object(oid); + if (!oo) + return -ENOENT; + if (local_conf()->memstore_debug_omit_block_device_write) + return 0; + ObjectRef no = c->get_or_create_object(noid); + used_bytes += ((ssize_t)oo->get_size() - (ssize_t)no->get_size()); + no->clone(oo.get(), 0, oo->get_size(), 0); + + no->omap_header = oo->omap_header; + no->omap = oo->omap; + no->xattr = oo->xattr; + return 0; +} + +int CyanStore::Shard::_setattrs( + const coll_t& cid, + const ghobject_t& oid, + std::map<std::string,bufferlist>&& aset) +{ + logger().debug("{} cid={} oid={}", + __func__, cid, oid); + auto c = _get_collection(cid); + if (!c) + return -ENOENT; + + ObjectRef o = c->get_object(oid); + if (!o) + return -ENOENT; + for (auto&& [key, val]: aset) { + o->xattr.insert_or_assign(std::move(key), std::move(val)); + } + return 0; +} + +int CyanStore::Shard::_rm_attr( + const coll_t& cid, + const ghobject_t& oid, + std::string_view name) +{ + logger().debug("{} cid={} oid={} name={}", __func__, cid, oid, name); + auto c = _get_collection(cid); + if (!c) { + return -ENOENT; + } + ObjectRef o = c->get_object(oid); + if (!o) { + return -ENOENT; + } + auto i = o->xattr.find(name); + if (i == o->xattr.end()) { + return -ENODATA; + } + o->xattr.erase(i); + return 0; +} + +int CyanStore::Shard::_rm_attrs( + const coll_t& cid, + const ghobject_t& oid) +{ + logger().debug("{} cid={} oid={}", __func__, cid, oid); + auto c = _get_collection(cid); + if (!c) { + return -ENOENT; + } + ObjectRef o = c->get_object(oid); + if (!o) { + return -ENOENT; + } + o->xattr.clear(); + return 0; +} + +int CyanStore::Shard::_create_collection(const coll_t& cid, int bits) +{ + auto result = coll_map.try_emplace(cid); + if (!result.second) + return -EEXIST; + auto p = new_coll_map.find(cid); + assert(p != new_coll_map.end()); + result.first->second = p->second; + result.first->second->bits = bits; + new_coll_map.erase(p); + return 0; +} + +boost::intrusive_ptr<Collection> +CyanStore::Shard::_get_collection(const coll_t& cid) +{ + auto cp = coll_map.find(cid); + if (cp == coll_map.end()) + return {}; + return cp->second; +} + +seastar::future<> CyanStore::write_meta( + const std::string& key, + const std::string& value) +{ + ceph_assert(seastar::this_shard_id() == primary_core); + std::string v = value; + v += "\n"; + if (int r = safe_write_file(path.c_str(), key.c_str(), + v.c_str(), v.length(), 0600); + r < 0) { + throw std::runtime_error{fmt::format("unable to write_meta({})", key)}; + } + return seastar::make_ready_future<>(); +} + +seastar::future<std::tuple<int, std::string>> +CyanStore::read_meta(const std::string& key) +{ + ceph_assert(seastar::this_shard_id() == primary_core); + std::string fsid(4096, '\0'); + int r = safe_read_file(path.c_str(), key.c_str(), fsid.data(), fsid.size()); + if (r > 0) { + fsid.resize(r); + // drop trailing newlines + boost::algorithm::trim_right_if(fsid, + [](unsigned char c) {return isspace(c);}); + } else { + fsid.clear(); + } + return seastar::make_ready_future<std::tuple<int, std::string>>( + std::make_tuple(r, fsid)); +} + +uuid_d CyanStore::get_fsid() const +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return osd_fsid; +} + +unsigned CyanStore::Shard::get_max_attr_name_length() const +{ + // arbitrary limitation exactly like in the case of MemStore. + return 256; +} + +CyanStore::Shard::read_errorator::future<std::map<uint64_t, uint64_t>> +CyanStore::Shard::fiemap( + CollectionRef ch, + const ghobject_t& oid, + uint64_t off, + uint64_t len) +{ + auto c = static_cast<Collection*>(ch.get()); + + ObjectRef o = c->get_object(oid); + if (!o) { + throw std::runtime_error(fmt::format("object does not exist: {}", oid)); + } + std::map<uint64_t, uint64_t> m{{0, o->get_size()}}; + return seastar::make_ready_future<std::map<uint64_t, uint64_t>>(std::move(m)); +} + +seastar::future<struct stat> +CyanStore::Shard::stat( + CollectionRef ch, + const ghobject_t& oid) +{ + auto c = static_cast<Collection*>(ch.get()); + auto o = c->get_object(oid); + if (!o) { + throw std::runtime_error(fmt::format("object does not exist: {}", oid)); + } + struct stat st; + st.st_size = o->get_size(); + return seastar::make_ready_future<struct stat>(std::move(st)); +} + +} diff --git a/src/crimson/os/cyanstore/cyan_store.h b/src/crimson/os/cyanstore/cyan_store.h new file mode 100644 index 000000000..307f9ec32 --- /dev/null +++ b/src/crimson/os/cyanstore/cyan_store.h @@ -0,0 +1,219 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <string> +#include <unordered_map> +#include <map> +#include <typeinfo> +#include <vector> + +#include <optional> +#include <seastar/core/future.hh> +#include <seastar/core/future-util.hh> + +#include "osd/osd_types.h" +#include "include/uuid.h" + +#include "crimson/os/cyanstore/cyan_object.h" +#include "crimson/os/cyanstore/cyan_collection.h" +#include "crimson/os/futurized_store.h" + +namespace ceph::os { +class Transaction; +} + +namespace crimson::os { +class CyanStore final : public FuturizedStore { + class Shard : public FuturizedStore::Shard { + public: + Shard(std::string path) + :path(path){} + + seastar::future<struct stat> stat( + CollectionRef c, + const ghobject_t& oid) final; + + read_errorator::future<ceph::bufferlist> read( + CollectionRef c, + const ghobject_t& oid, + uint64_t offset, + size_t len, + uint32_t op_flags = 0) final; + + read_errorator::future<ceph::bufferlist> readv( + CollectionRef c, + const ghobject_t& oid, + interval_set<uint64_t>& m, + uint32_t op_flags = 0) final; + + get_attr_errorator::future<ceph::bufferlist> get_attr( + CollectionRef c, + const ghobject_t& oid, + std::string_view name) const final; + + get_attrs_ertr::future<attrs_t> get_attrs( + CollectionRef c, + const ghobject_t& oid) final; + + read_errorator::future<omap_values_t> omap_get_values( + CollectionRef c, + const ghobject_t& oid, + const omap_keys_t& keys) final; + + read_errorator::future<std::tuple<bool, omap_values_t>> omap_get_values( + CollectionRef c, ///< [in] collection + const ghobject_t &oid, ///< [in] oid + const std::optional<std::string> &start ///< [in] start, empty for begin + ) final; + + get_attr_errorator::future<ceph::bufferlist> omap_get_header( + CollectionRef c, + const ghobject_t& oid) final; + + seastar::future<std::tuple<std::vector<ghobject_t>, ghobject_t>> + list_objects( + CollectionRef c, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) const final; + + seastar::future<CollectionRef> create_new_collection(const coll_t& cid) final; + + seastar::future<CollectionRef> open_collection(const coll_t& cid) final; + + seastar::future<> do_transaction_no_callbacks( + CollectionRef ch, + ceph::os::Transaction&& txn) final; + + read_errorator::future<std::map<uint64_t, uint64_t>> + fiemap( + CollectionRef c, + const ghobject_t& oid, + uint64_t off, + uint64_t len) final; + + unsigned get_max_attr_name_length() const final; + + public: + // only exposed to CyanStore + mount_ertr::future<> mount(); + + seastar::future<> umount(); + + seastar::future<> mkfs(); + + mkfs_ertr::future<> mkcoll(uuid_d new_osd_fsid); + + using coll_core_t = FuturizedStore::coll_core_t; + seastar::future<std::vector<coll_core_t>> list_collections(); + + uint64_t get_used_bytes() const { return used_bytes; } + + private: + int _remove(const coll_t& cid, const ghobject_t& oid); + int _touch(const coll_t& cid, const ghobject_t& oid); + int _write(const coll_t& cid, const ghobject_t& oid, + uint64_t offset, size_t len, const ceph::bufferlist& bl, + uint32_t fadvise_flags); + int _zero(const coll_t& cid, const ghobject_t& oid, + uint64_t offset, size_t len); + int _omap_clear( + const coll_t& cid, + const ghobject_t& oid); + int _omap_set_values( + const coll_t& cid, + const ghobject_t& oid, + std::map<std::string, ceph::bufferlist> &&aset); + int _omap_set_header( + const coll_t& cid, + const ghobject_t& oid, + const ceph::bufferlist &header); + int _omap_rmkeys( + const coll_t& cid, + const ghobject_t& oid, + const omap_keys_t& aset); + int _omap_rmkeyrange( + const coll_t& cid, + const ghobject_t& oid, + const std::string &first, + const std::string &last); + int _truncate(const coll_t& cid, const ghobject_t& oid, uint64_t size); + int _clone(const coll_t& cid, const ghobject_t& oid, + const ghobject_t& noid); + int _setattrs(const coll_t& cid, const ghobject_t& oid, + std::map<std::string,bufferlist>&& aset); + int _rm_attr(const coll_t& cid, const ghobject_t& oid, + std::string_view name); + int _rm_attrs(const coll_t& cid, const ghobject_t& oid); + int _create_collection(const coll_t& cid, int bits); + boost::intrusive_ptr<Collection> _get_collection(const coll_t& cid); + + private: + uint64_t used_bytes = 0; + const std::string path; + std::unordered_map<coll_t, boost::intrusive_ptr<Collection>> coll_map; + std::map<coll_t, boost::intrusive_ptr<Collection>> new_coll_map; + }; + +public: + CyanStore(const std::string& path); + ~CyanStore() final; + + seastar::future<> start() final { + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.start(path); + } + + seastar::future<> stop() final { + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.stop(); + } + + mount_ertr::future<> mount() final { + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.invoke_on_all( + [](auto &local_store) { + return local_store.mount().handle_error( + crimson::stateful_ec::handle([](const auto& ec) { + crimson::get_logger(ceph_subsys_cyanstore).error( + "error mounting cyanstore: ({}) {}", + ec.value(), ec.message()); + std::exit(EXIT_FAILURE); + })); + }); + } + + seastar::future<> umount() final { + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.invoke_on_all( + [](auto &local_store) { + return local_store.umount(); + }); + } + + mkfs_ertr::future<> mkfs(uuid_d new_osd_fsid) final; + + seastar::future<store_statfs_t> stat() const final; + + uuid_d get_fsid() const final; + + seastar::future<> write_meta(const std::string& key, + const std::string& value) final; + + FuturizedStore::Shard& get_sharded_store() final{ + return shard_stores.local(); + } + + seastar::future<std::tuple<int, std::string>> + read_meta(const std::string& key) final; + + seastar::future<std::vector<coll_core_t>> list_collections() final; + +private: + seastar::sharded<CyanStore::Shard> shard_stores; + const std::string path; + uuid_d osd_fsid; +}; +} diff --git a/src/crimson/os/futurized_collection.h b/src/crimson/os/futurized_collection.h new file mode 100644 index 000000000..7b460dffe --- /dev/null +++ b/src/crimson/os/futurized_collection.h @@ -0,0 +1,37 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <seastar/core/future.hh> + +#include "osd/osd_types.h" + +namespace crimson::os { +class FuturizedStore; + +class FuturizedCollection + : public boost::intrusive_ref_counter<FuturizedCollection, + boost::thread_safe_counter> +{ +public: + FuturizedCollection(const coll_t& cid) + : cid{cid} {} + virtual ~FuturizedCollection() {} + virtual seastar::future<> flush() { + return seastar::make_ready_future<>(); + } + virtual seastar::future<bool> flush_commit() { + return seastar::make_ready_future<bool>(true); + } + const coll_t& get_cid() const { + return cid; + } +private: + const coll_t cid; +}; + +using CollectionRef = boost::intrusive_ptr<FuturizedCollection>; +} diff --git a/src/crimson/os/futurized_store.cc b/src/crimson/os/futurized_store.cc new file mode 100644 index 000000000..bc47ec78f --- /dev/null +++ b/src/crimson/os/futurized_store.cc @@ -0,0 +1,36 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "futurized_store.h" +#include "cyanstore/cyan_store.h" +#ifdef WITH_BLUESTORE +#include "alienstore/alien_store.h" +#endif +#include "seastore/seastore.h" + +namespace crimson::os { + +std::unique_ptr<FuturizedStore> +FuturizedStore::create(const std::string& type, + const std::string& data, + const ConfigValues& values) +{ + if (type == "cyanstore") { + using crimson::os::CyanStore; + return std::make_unique<CyanStore>(data); + } else if (type == "seastore") { + return crimson::os::seastore::make_seastore( + data); + } else { + using crimson::os::AlienStore; +#ifdef WITH_BLUESTORE + // use AlienStore as a fallback. It adapts e.g. BlueStore. + return std::make_unique<AlienStore>(type, data, values); +#else + ceph_abort_msgf("unsupported objectstore type: %s", type.c_str()); + return {}; +#endif + } +} + +} diff --git a/src/crimson/os/futurized_store.h b/src/crimson/os/futurized_store.h new file mode 100644 index 000000000..783cd7485 --- /dev/null +++ b/src/crimson/os/futurized_store.h @@ -0,0 +1,195 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <string> +#include <map> +#include <optional> +#include <vector> + +#include <seastar/core/future.hh> + +#include "os/Transaction.h" +#include "crimson/common/smp_helpers.h" +#include "crimson/common/smp_helpers.h" +#include "crimson/osd/exceptions.h" +#include "include/buffer_fwd.h" +#include "include/uuid.h" +#include "osd/osd_types.h" + +namespace ceph::os { +class Transaction; +} + +namespace crimson::os { +class FuturizedCollection; + +class FuturizedStore { +public: + class Shard { + public: + Shard() = default; + virtual ~Shard() = default; + // no copying + explicit Shard(const Shard& o) = delete; + const Shard& operator=(const Shard& o) = delete; + + using CollectionRef = boost::intrusive_ptr<FuturizedCollection>; + using read_errorator = crimson::errorator<crimson::ct_error::enoent, + crimson::ct_error::input_output_error>; + virtual read_errorator::future<ceph::bufferlist> read( + CollectionRef c, + const ghobject_t& oid, + uint64_t offset, + size_t len, + uint32_t op_flags = 0) = 0; + + virtual read_errorator::future<ceph::bufferlist> readv( + CollectionRef c, + const ghobject_t& oid, + interval_set<uint64_t>& m, + uint32_t op_flags = 0) = 0; + + using get_attr_errorator = crimson::errorator< + crimson::ct_error::enoent, + crimson::ct_error::enodata>; + virtual get_attr_errorator::future<ceph::bufferlist> get_attr( + CollectionRef c, + const ghobject_t& oid, + std::string_view name) const = 0; + + using get_attrs_ertr = crimson::errorator< + crimson::ct_error::enoent>; + using attrs_t = std::map<std::string, ceph::bufferlist, std::less<>>; + virtual get_attrs_ertr::future<attrs_t> get_attrs( + CollectionRef c, + const ghobject_t& oid) = 0; + + virtual seastar::future<struct stat> stat( + CollectionRef c, + const ghobject_t& oid) = 0; + + using omap_values_t = std::map<std::string, ceph::bufferlist, std::less<>>; + using omap_keys_t = std::set<std::string>; + virtual read_errorator::future<omap_values_t> omap_get_values( + CollectionRef c, + const ghobject_t& oid, + const omap_keys_t& keys) = 0; + + virtual read_errorator::future<std::tuple<bool, omap_values_t>> omap_get_values( + CollectionRef c, ///< [in] collection + const ghobject_t &oid, ///< [in] oid + const std::optional<std::string> &start ///< [in] start, empty for begin + ) = 0; ///< @return <done, values> values.empty() only if done + + virtual get_attr_errorator::future<bufferlist> omap_get_header( + CollectionRef c, + const ghobject_t& oid) = 0; + + virtual seastar::future<std::tuple<std::vector<ghobject_t>, ghobject_t>> list_objects( + CollectionRef c, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) const = 0; + + virtual seastar::future<CollectionRef> create_new_collection(const coll_t& cid) = 0; + + virtual seastar::future<CollectionRef> open_collection(const coll_t& cid) = 0; + + protected: + virtual seastar::future<> do_transaction_no_callbacks( + CollectionRef ch, + ceph::os::Transaction&& txn) = 0; + + public: + seastar::future<> do_transaction( + CollectionRef ch, + ceph::os::Transaction&& txn) { + std::unique_ptr<Context> on_commit( + ceph::os::Transaction::collect_all_contexts(txn)); + return do_transaction_no_callbacks( + std::move(ch), std::move(txn) + ).then([on_commit=std::move(on_commit)]() mutable { + auto c = on_commit.release(); + if (c) c->complete(0); + return seastar::now(); + }); + } + + + /** + * flush + * + * Flushes outstanding transactions on ch, returned future resolves + * after any previously submitted transactions on ch have committed. + * + * @param ch [in] collection on which to flush + */ + virtual seastar::future<> flush(CollectionRef ch) { + return do_transaction(ch, ceph::os::Transaction{}); + } + + // error injection + virtual seastar::future<> inject_data_error(const ghobject_t& o) { + return seastar::now(); + } + + virtual seastar::future<> inject_mdata_error(const ghobject_t& o) { + return seastar::now(); + } + + virtual read_errorator::future<std::map<uint64_t, uint64_t>> fiemap( + CollectionRef ch, + const ghobject_t& oid, + uint64_t off, + uint64_t len) = 0; + + virtual unsigned get_max_attr_name_length() const = 0; + }; + +public: + static std::unique_ptr<FuturizedStore> create(const std::string& type, + const std::string& data, + const ConfigValues& values); + FuturizedStore() + : primary_core(seastar::this_shard_id()) + {} + + virtual ~FuturizedStore() = default; + + // no copying + explicit FuturizedStore(const FuturizedStore& o) = delete; + const FuturizedStore& operator=(const FuturizedStore& o) = delete; + + virtual seastar::future<> start() = 0; + + virtual seastar::future<> stop() = 0; + + using mount_ertr = crimson::errorator<crimson::stateful_ec>; + virtual mount_ertr::future<> mount() = 0; + + virtual seastar::future<> umount() = 0; + + using mkfs_ertr = crimson::errorator<crimson::stateful_ec>; + virtual mkfs_ertr::future<> mkfs(uuid_d new_osd_fsid) = 0; + + virtual seastar::future<store_statfs_t> stat() const = 0; + + virtual uuid_d get_fsid() const = 0; + + virtual seastar::future<> write_meta(const std::string& key, + const std::string& value) = 0; + // called on the shard and get this FuturizedStore::shard; + virtual Shard& get_sharded_store() = 0; + + virtual seastar::future<std::tuple<int, std::string>> read_meta( + const std::string& key) = 0; + + using coll_core_t = std::pair<coll_t, core_id_t>; + virtual seastar::future<std::vector<coll_core_t>> list_collections() = 0; + +protected: + const core_id_t primary_core; +}; +} diff --git a/src/crimson/os/seastore/CMakeLists.txt b/src/crimson/os/seastore/CMakeLists.txt new file mode 100644 index 000000000..4bdbab8c4 --- /dev/null +++ b/src/crimson/os/seastore/CMakeLists.txt @@ -0,0 +1,79 @@ +set(crimson_seastore_srcs + cached_extent.cc + seastore_types.cc + segment_manager.cc + segment_manager/ephemeral.cc + segment_manager/block.cc + transaction_manager.cc + transaction.cc + cache.cc + root_block.cc + lba_manager.cc + async_cleaner.cc + backref_manager.cc + backref/backref_tree_node.cc + backref/btree_backref_manager.cc + lba_manager/btree/btree_lba_manager.cc + lba_manager/btree/lba_btree_node.cc + omap_manager.cc + omap_manager/btree/btree_omap_manager.cc + omap_manager/btree/omap_btree_node_impl.cc + btree/btree_range_pin.cc + btree/fixed_kv_node.cc + onode.cc + onode_manager/staged-fltree/node.cc + onode_manager/staged-fltree/node_extent_manager.cc + onode_manager/staged-fltree/node_extent_manager/seastore.cc + onode_manager/staged-fltree/node_impl.cc + onode_manager/staged-fltree/stages/item_iterator_stage.cc + onode_manager/staged-fltree/stages/key_layout.cc + onode_manager/staged-fltree/stages/node_stage_layout.cc + onode_manager/staged-fltree/stages/node_stage.cc + onode_manager/staged-fltree/stages/sub_items_stage.cc + onode_manager/staged-fltree/super.cc + onode_manager/staged-fltree/value.cc + onode_manager/staged-fltree/fltree_onode_manager.cc + collection_manager.cc + collection_manager/flat_collection_manager.cc + collection_manager/collection_flat_node.cc + extent_placement_manager.cc + object_data_handler.cc + seastore.cc + random_block_manager.cc + random_block_manager/block_rb_manager.cc + random_block_manager/rbm_device.cc + random_block_manager/nvme_block_device.cc + random_block_manager/avlallocator.cc + journal/segmented_journal.cc + journal/segment_allocator.cc + journal/record_submitter.cc + journal/circular_journal_space.cc + journal.cc + device.cc + segment_manager_group.cc + record_scanner.cc + journal/circular_bounded_journal.cc + ../../../test/crimson/seastore/test_block.cc + ${PROJECT_SOURCE_DIR}/src/os/Transaction.cc + ) + +CMAKE_DEPENDENT_OPTION(WITH_ZNS "enable Linux ZNS support" OFF + "CMAKE_SYSTEM_NAME STREQUAL Linux" OFF) +if(WITH_ZNS) + find_package(LinuxZNS REQUIRED) + list(APPEND crimson_seastore_srcs + segment_manager/zbd.cc) +endif() + +add_library(crimson-seastore STATIC + ${crimson_seastore_srcs}) + +target_link_libraries(crimson-seastore + crimson) +if(WITH_ZNS) + target_link_libraries(crimson-seastore + Linux::ZNS) +endif() + +set_target_properties(crimson-seastore PROPERTIES + JOB_POOL_COMPILE heavy_compile_job_pool) diff --git a/src/crimson/os/seastore/async_cleaner.cc b/src/crimson/os/seastore/async_cleaner.cc new file mode 100644 index 000000000..d7e398f5f --- /dev/null +++ b/src/crimson/os/seastore/async_cleaner.cc @@ -0,0 +1,1817 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <fmt/chrono.h> +#include <seastar/core/metrics.hh> + +#include "crimson/os/seastore/logging.h" + +#include "crimson/os/seastore/async_cleaner.h" +#include "crimson/os/seastore/backref_manager.h" +#include "crimson/os/seastore/transaction_manager.h" + +SET_SUBSYS(seastore_cleaner); + +namespace { + +enum class gc_formula_t { + GREEDY, + BENEFIT, + COST_BENEFIT, +}; +constexpr auto gc_formula = gc_formula_t::COST_BENEFIT; + +} + +namespace crimson::os::seastore { + +void segment_info_t::set_open( + segment_seq_t _seq, segment_type_t _type, + data_category_t _category, rewrite_gen_t _generation) +{ + ceph_assert(_seq != NULL_SEG_SEQ); + ceph_assert(_type != segment_type_t::NULL_SEG); + ceph_assert(_category != data_category_t::NUM); + ceph_assert(is_rewrite_generation(_generation)); + state = Segment::segment_state_t::OPEN; + seq = _seq; + type = _type; + category = _category; + generation = _generation; + written_to = 0; +} + +void segment_info_t::set_empty() +{ + state = Segment::segment_state_t::EMPTY; + seq = NULL_SEG_SEQ; + type = segment_type_t::NULL_SEG; + category = data_category_t::NUM; + generation = NULL_GENERATION; + modify_time = NULL_TIME; + num_extents = 0; + written_to = 0; +} + +void segment_info_t::set_closed() +{ + state = Segment::segment_state_t::CLOSED; + // the rest of information is unchanged +} + +void segment_info_t::init_closed( + segment_seq_t _seq, segment_type_t _type, + data_category_t _category, rewrite_gen_t _generation, + segment_off_t seg_size) +{ + ceph_assert(_seq != NULL_SEG_SEQ); + ceph_assert(_type != segment_type_t::NULL_SEG); + ceph_assert(_category != data_category_t::NUM); + ceph_assert(is_rewrite_generation(_generation)); + state = Segment::segment_state_t::CLOSED; + seq = _seq; + type = _type; + category = _category; + generation = _generation; + written_to = seg_size; +} + +std::ostream& operator<<(std::ostream &out, const segment_info_t &info) +{ + out << "seg_info_t(" + << "state=" << info.state + << ", " << info.id; + if (info.is_empty()) { + // pass + } else { // open or closed + out << " " << info.type + << " " << segment_seq_printer_t{info.seq} + << " " << info.category + << " " << rewrite_gen_printer_t{info.generation} + << ", modify_time=" << sea_time_point_printer_t{info.modify_time} + << ", num_extents=" << info.num_extents + << ", written_to=" << info.written_to; + } + return out << ")"; +} + +void segments_info_t::reset() +{ + segments.clear(); + + segment_size = 0; + + journal_segment_id = NULL_SEG_ID; + num_in_journal_open = 0; + num_type_journal = 0; + num_type_ool = 0; + + num_open = 0; + num_empty = 0; + num_closed = 0; + + count_open_journal = 0; + count_open_ool = 0; + count_release_journal = 0; + count_release_ool = 0; + count_close_journal = 0; + count_close_ool = 0; + + total_bytes = 0; + avail_bytes_in_open = 0; + + modify_times.clear(); +} + +void segments_info_t::add_segment_manager( + SegmentManager &segment_manager) +{ + LOG_PREFIX(segments_info_t::add_segment_manager); + device_id_t d_id = segment_manager.get_device_id(); + auto ssize = segment_manager.get_segment_size(); + auto nsegments = segment_manager.get_num_segments(); + auto sm_size = segment_manager.get_available_size(); + INFO("adding segment manager {}, size={}, ssize={}, segments={}", + device_id_printer_t{d_id}, sm_size, ssize, nsegments); + ceph_assert(ssize > 0); + ceph_assert(nsegments > 0); + ceph_assert(sm_size > 0); + + // also validate if the device is duplicated + segments.add_device(d_id, nsegments, segment_info_t{}); + + // assume all the segment managers share the same settings as follows. + if (segment_size == 0) { + ceph_assert(ssize > 0); + segment_size = ssize; + } else { + ceph_assert(segment_size == ssize); + } + + // NOTE: by default the segments are empty + num_empty += nsegments; + + total_bytes += sm_size; +} + +void segments_info_t::init_closed( + segment_id_t segment, segment_seq_t seq, segment_type_t type, + data_category_t category, rewrite_gen_t generation) +{ + LOG_PREFIX(segments_info_t::init_closed); + auto& segment_info = segments[segment]; + DEBUG("initiating {} {} {} {} {}, {}, " + "num_segments(empty={}, opened={}, closed={})", + segment, type, segment_seq_printer_t{seq}, + category, rewrite_gen_printer_t{generation}, + segment_info, num_empty, num_open, num_closed); + ceph_assert(segment_info.is_empty()); + ceph_assert(num_empty > 0); + --num_empty; + ++num_closed; + if (type == segment_type_t::JOURNAL) { + // init_closed won't initialize journal_segment_id + ceph_assert(get_submitted_journal_head() == JOURNAL_SEQ_NULL); + ++num_type_journal; + } else { + ++num_type_ool; + } + // do not increment count_close_*; + + if (segment_info.modify_time != NULL_TIME) { + modify_times.insert(segment_info.modify_time); + } else { + ceph_assert(segment_info.num_extents == 0); + } + + segment_info.init_closed( + seq, type, category, generation, get_segment_size()); +} + +void segments_info_t::mark_open( + segment_id_t segment, segment_seq_t seq, segment_type_t type, + data_category_t category, rewrite_gen_t generation) +{ + LOG_PREFIX(segments_info_t::mark_open); + auto& segment_info = segments[segment]; + INFO("opening {} {} {} {} {}, {}, " + "num_segments(empty={}, opened={}, closed={})", + segment, type, segment_seq_printer_t{seq}, + category, rewrite_gen_printer_t{generation}, + segment_info, num_empty, num_open, num_closed); + ceph_assert(segment_info.is_empty()); + ceph_assert(num_empty > 0); + --num_empty; + ++num_open; + if (type == segment_type_t::JOURNAL) { + if (journal_segment_id != NULL_SEG_ID) { + auto& last_journal_segment = segments[journal_segment_id]; + ceph_assert(last_journal_segment.is_closed()); + ceph_assert(last_journal_segment.type == segment_type_t::JOURNAL); + ceph_assert(last_journal_segment.seq + 1 == seq); + } + journal_segment_id = segment; + + ++num_in_journal_open; + ++num_type_journal; + ++count_open_journal; + } else { + ++num_type_ool; + ++count_open_ool; + } + avail_bytes_in_open += get_segment_size(); + + segment_info.set_open(seq, type, category, generation); +} + +void segments_info_t::mark_empty( + segment_id_t segment) +{ + LOG_PREFIX(segments_info_t::mark_empty); + auto& segment_info = segments[segment]; + INFO("releasing {}, {}, num_segments(empty={}, opened={}, closed={})", + segment, segment_info, + num_empty, num_open, num_closed); + ceph_assert(segment_info.is_closed()); + auto type = segment_info.type; + assert(type != segment_type_t::NULL_SEG); + ceph_assert(num_closed > 0); + --num_closed; + ++num_empty; + if (type == segment_type_t::JOURNAL) { + ceph_assert(num_type_journal > 0); + --num_type_journal; + ++count_release_journal; + } else { + ceph_assert(num_type_ool > 0); + --num_type_ool; + ++count_release_ool; + } + + if (segment_info.modify_time != NULL_TIME) { + auto to_erase = modify_times.find(segment_info.modify_time); + ceph_assert(to_erase != modify_times.end()); + modify_times.erase(to_erase); + } else { + ceph_assert(segment_info.num_extents == 0); + } + + segment_info.set_empty(); +} + +void segments_info_t::mark_closed( + segment_id_t segment) +{ + LOG_PREFIX(segments_info_t::mark_closed); + auto& segment_info = segments[segment]; + INFO("closing {}, {}, num_segments(empty={}, opened={}, closed={})", + segment, segment_info, + num_empty, num_open, num_closed); + ceph_assert(segment_info.is_open()); + ceph_assert(num_open > 0); + --num_open; + ++num_closed; + if (segment_info.type == segment_type_t::JOURNAL) { + ceph_assert(num_in_journal_open > 0); + --num_in_journal_open; + ++count_close_journal; + } else { + ++count_close_ool; + } + ceph_assert(get_segment_size() >= segment_info.written_to); + auto seg_avail_bytes = get_segment_size() - segment_info.written_to; + ceph_assert(avail_bytes_in_open >= (std::size_t)seg_avail_bytes); + avail_bytes_in_open -= seg_avail_bytes; + + if (segment_info.modify_time != NULL_TIME) { + modify_times.insert(segment_info.modify_time); + } else { + ceph_assert(segment_info.num_extents == 0); + } + + segment_info.set_closed(); +} + +void segments_info_t::update_written_to( + segment_type_t type, + paddr_t offset) +{ + LOG_PREFIX(segments_info_t::update_written_to); + auto& saddr = offset.as_seg_paddr(); + auto& segment_info = segments[saddr.get_segment_id()]; + if (!segment_info.is_open()) { + ERROR("segment is not open, not updating, type={}, offset={}, {}", + type, offset, segment_info); + ceph_abort(); + } + + auto new_written_to = saddr.get_segment_off(); + ceph_assert(new_written_to <= get_segment_size()); + if (segment_info.written_to > new_written_to) { + ERROR("written_to should not decrease! type={}, offset={}, {}", + type, offset, segment_info); + ceph_abort(); + } + + DEBUG("type={}, offset={}, {}", type, offset, segment_info); + ceph_assert(type == segment_info.type); + auto avail_deduction = new_written_to - segment_info.written_to; + ceph_assert(avail_bytes_in_open >= (std::size_t)avail_deduction); + avail_bytes_in_open -= avail_deduction; + segment_info.written_to = new_written_to; +} + +std::ostream &operator<<(std::ostream &os, const segments_info_t &infos) +{ + return os << "segments(" + << "empty=" << infos.get_num_empty() + << ", open=" << infos.get_num_open() + << ", closed=" << infos.get_num_closed() + << ", type_journal=" << infos.get_num_type_journal() + << ", type_ool=" << infos.get_num_type_ool() + << ", total=" << infos.get_total_bytes() << "B" + << ", available=" << infos.get_available_bytes() << "B" + << ", unavailable=" << infos.get_unavailable_bytes() << "B" + << ", available_ratio=" << infos.get_available_ratio() + << ", submitted_head=" << infos.get_submitted_journal_head() + << ", time_bound=" << sea_time_point_printer_t{infos.get_time_bound()} + << ")"; +} + +void JournalTrimmerImpl::config_t::validate() const +{ + ceph_assert(max_journal_bytes <= DEVICE_OFF_MAX); + ceph_assert(max_journal_bytes > target_journal_dirty_bytes); + ceph_assert(max_journal_bytes > target_journal_alloc_bytes); + ceph_assert(rewrite_dirty_bytes_per_cycle > 0); + ceph_assert(rewrite_backref_bytes_per_cycle > 0); +} + +JournalTrimmerImpl::config_t +JournalTrimmerImpl::config_t::get_default( + std::size_t roll_size, journal_type_t type) +{ + assert(roll_size); + std::size_t target_dirty_bytes = 0; + std::size_t target_alloc_bytes = 0; + std::size_t max_journal_bytes = 0; + if (type == journal_type_t::SEGMENTED) { + target_dirty_bytes = 12 * roll_size; + target_alloc_bytes = 2 * roll_size; + max_journal_bytes = 16 * roll_size; + } else { + assert(type == journal_type_t::RANDOM_BLOCK); + target_dirty_bytes = roll_size / 4; + target_alloc_bytes = roll_size / 4; + max_journal_bytes = roll_size / 2; + } + return config_t{ + target_dirty_bytes, + target_alloc_bytes, + max_journal_bytes, + 1<<17,// rewrite_dirty_bytes_per_cycle + 1<<24 // rewrite_backref_bytes_per_cycle + }; +} + +JournalTrimmerImpl::config_t +JournalTrimmerImpl::config_t::get_test( + std::size_t roll_size, journal_type_t type) +{ + assert(roll_size); + std::size_t target_dirty_bytes = 0; + std::size_t target_alloc_bytes = 0; + std::size_t max_journal_bytes = 0; + if (type == journal_type_t::SEGMENTED) { + target_dirty_bytes = 2 * roll_size; + target_alloc_bytes = 2 * roll_size; + max_journal_bytes = 4 * roll_size; + } else { + assert(type == journal_type_t::RANDOM_BLOCK); + target_dirty_bytes = roll_size / 4; + target_alloc_bytes = roll_size / 4; + max_journal_bytes = roll_size / 2; + } + return config_t{ + target_dirty_bytes, + target_alloc_bytes, + max_journal_bytes, + 1<<17,// rewrite_dirty_bytes_per_cycle + 1<<24 // rewrite_backref_bytes_per_cycle + }; +} + +JournalTrimmerImpl::JournalTrimmerImpl( + BackrefManager &backref_manager, + config_t config, + journal_type_t type, + device_off_t roll_start, + device_off_t roll_size) + : backref_manager(backref_manager), + config(config), + journal_type(type), + roll_start(roll_start), + roll_size(roll_size), + reserved_usage(0) +{ + config.validate(); + ceph_assert(roll_start >= 0); + ceph_assert(roll_size > 0); + register_metrics(); +} + +void JournalTrimmerImpl::set_journal_head(journal_seq_t head) +{ + LOG_PREFIX(JournalTrimmerImpl::set_journal_head); + + ceph_assert(head != JOURNAL_SEQ_NULL); + ceph_assert(journal_head == JOURNAL_SEQ_NULL || + head >= journal_head); + ceph_assert(journal_alloc_tail == JOURNAL_SEQ_NULL || + head >= journal_alloc_tail); + ceph_assert(journal_dirty_tail == JOURNAL_SEQ_NULL || + head >= journal_dirty_tail); + + std::swap(journal_head, head); + if (journal_head.segment_seq == head.segment_seq) { + DEBUG("journal_head {} => {}, {}", + head, journal_head, stat_printer_t{*this, false}); + } else { + INFO("journal_head {} => {}, {}", + head, journal_head, stat_printer_t{*this, false}); + } + background_callback->maybe_wake_background(); +} + +void JournalTrimmerImpl::update_journal_tails( + journal_seq_t dirty_tail, + journal_seq_t alloc_tail) +{ + LOG_PREFIX(JournalTrimmerImpl::update_journal_tails); + + if (dirty_tail != JOURNAL_SEQ_NULL) { + ceph_assert(journal_head == JOURNAL_SEQ_NULL || + journal_head >= dirty_tail); + if (journal_dirty_tail != JOURNAL_SEQ_NULL && + journal_dirty_tail > dirty_tail) { + ERROR("journal_dirty_tail {} => {} is backwards!", + journal_dirty_tail, dirty_tail); + ceph_abort(); + } + std::swap(journal_dirty_tail, dirty_tail); + if (journal_dirty_tail.segment_seq == dirty_tail.segment_seq) { + DEBUG("journal_dirty_tail {} => {}, {}", + dirty_tail, journal_dirty_tail, stat_printer_t{*this, false}); + } else { + INFO("journal_dirty_tail {} => {}, {}", + dirty_tail, journal_dirty_tail, stat_printer_t{*this, false}); + } + } + + if (alloc_tail != JOURNAL_SEQ_NULL) { + ceph_assert(journal_head == JOURNAL_SEQ_NULL || + journal_head >= alloc_tail); + if (journal_alloc_tail != JOURNAL_SEQ_NULL && + journal_alloc_tail > alloc_tail) { + ERROR("journal_alloc_tail {} => {} is backwards!", + journal_alloc_tail, alloc_tail); + ceph_abort(); + } + std::swap(journal_alloc_tail, alloc_tail); + if (journal_alloc_tail.segment_seq == alloc_tail.segment_seq) { + DEBUG("journal_alloc_tail {} => {}, {}", + alloc_tail, journal_alloc_tail, stat_printer_t{*this, false}); + } else { + INFO("journal_alloc_tail {} => {}, {}", + alloc_tail, journal_alloc_tail, stat_printer_t{*this, false}); + } + } + + background_callback->maybe_wake_background(); + background_callback->maybe_wake_blocked_io(); +} + +journal_seq_t JournalTrimmerImpl::get_tail_limit() const +{ + assert(background_callback->is_ready()); + auto ret = journal_head.add_offset( + journal_type, + -static_cast<device_off_t>(config.max_journal_bytes), + roll_start, + roll_size); + return ret; +} + +journal_seq_t JournalTrimmerImpl::get_dirty_tail_target() const +{ + assert(background_callback->is_ready()); + auto ret = journal_head.add_offset( + journal_type, + -static_cast<device_off_t>(config.target_journal_dirty_bytes), + roll_start, + roll_size); + return ret; +} + +journal_seq_t JournalTrimmerImpl::get_alloc_tail_target() const +{ + assert(background_callback->is_ready()); + auto ret = journal_head.add_offset( + journal_type, + -static_cast<device_off_t>(config.target_journal_alloc_bytes), + roll_start, + roll_size); + return ret; +} + +std::size_t JournalTrimmerImpl::get_dirty_journal_size() const +{ + if (!background_callback->is_ready()) { + return 0; + } + auto ret = journal_head.relative_to( + journal_type, + journal_dirty_tail, + roll_start, + roll_size); + ceph_assert(ret >= 0); + return static_cast<std::size_t>(ret); +} + +std::size_t JournalTrimmerImpl::get_alloc_journal_size() const +{ + if (!background_callback->is_ready()) { + return 0; + } + auto ret = journal_head.relative_to( + journal_type, + journal_alloc_tail, + roll_start, + roll_size); + ceph_assert(ret >= 0); + return static_cast<std::size_t>(ret); +} + +seastar::future<> JournalTrimmerImpl::trim() { + return seastar::when_all( + [this] { + if (should_trim_alloc()) { + return trim_alloc( + ).handle_error( + crimson::ct_error::assert_all{ + "encountered invalid error in trim_alloc" + } + ); + } else { + return seastar::now(); + } + }, + [this] { + if (should_trim_dirty()) { + return trim_dirty( + ).handle_error( + crimson::ct_error::assert_all{ + "encountered invalid error in trim_dirty" + } + ); + } else { + return seastar::now(); + } + } + ).discard_result(); +} + +JournalTrimmerImpl::trim_ertr::future<> +JournalTrimmerImpl::trim_alloc() +{ + LOG_PREFIX(JournalTrimmerImpl::trim_alloc); + assert(background_callback->is_ready()); + return repeat_eagain([this, FNAME] { + return extent_callback->with_transaction_intr( + Transaction::src_t::TRIM_ALLOC, + "trim_alloc", + [this, FNAME](auto &t) + { + auto target = get_alloc_tail_target(); + DEBUGT("start, alloc_tail={}, target={}", + t, journal_alloc_tail, target); + return backref_manager.merge_cached_backrefs( + t, + target, + config.rewrite_backref_bytes_per_cycle + ).si_then([this, FNAME, &t](auto trim_alloc_to) + -> ExtentCallbackInterface::submit_transaction_direct_iertr::future<> + { + DEBUGT("trim_alloc_to={}", t, trim_alloc_to); + if (trim_alloc_to != JOURNAL_SEQ_NULL) { + return extent_callback->submit_transaction_direct( + t, std::make_optional<journal_seq_t>(trim_alloc_to)); + } + return seastar::now(); + }); + }); + }).safe_then([this, FNAME] { + DEBUG("finish, alloc_tail={}", journal_alloc_tail); + }); +} + +JournalTrimmerImpl::trim_ertr::future<> +JournalTrimmerImpl::trim_dirty() +{ + LOG_PREFIX(JournalTrimmerImpl::trim_dirty); + assert(background_callback->is_ready()); + return repeat_eagain([this, FNAME] { + return extent_callback->with_transaction_intr( + Transaction::src_t::TRIM_DIRTY, + "trim_dirty", + [this, FNAME](auto &t) + { + auto target = get_dirty_tail_target(); + DEBUGT("start, dirty_tail={}, target={}", + t, journal_dirty_tail, target); + return extent_callback->get_next_dirty_extents( + t, + target, + config.rewrite_dirty_bytes_per_cycle + ).si_then([this, FNAME, &t](auto dirty_list) { + DEBUGT("rewrite {} dirty extents", t, dirty_list.size()); + return seastar::do_with( + std::move(dirty_list), + [this, &t](auto &dirty_list) + { + return trans_intr::do_for_each( + dirty_list, + [this, &t](auto &e) { + return extent_callback->rewrite_extent( + t, e, INIT_GENERATION, NULL_TIME); + }); + }); + }).si_then([this, &t] { + return extent_callback->submit_transaction_direct(t); + }); + }); + }).safe_then([this, FNAME] { + DEBUG("finish, dirty_tail={}", journal_dirty_tail); + }); +} + +void JournalTrimmerImpl::register_metrics() +{ + namespace sm = seastar::metrics; + metrics.add_group("journal_trimmer", { + sm::make_counter("dirty_journal_bytes", + [this] { return get_dirty_journal_size(); }, + sm::description("the size of the journal for dirty extents")), + sm::make_counter("alloc_journal_bytes", + [this] { return get_alloc_journal_size(); }, + sm::description("the size of the journal for alloc info")) + }); +} + +std::ostream &operator<<( + std::ostream &os, const JournalTrimmerImpl::stat_printer_t &stats) +{ + os << "JournalTrimmer("; + if (stats.trimmer.background_callback->is_ready()) { + os << "should_block_io_on_trim=" << stats.trimmer.should_block_io_on_trim() + << ", should_(trim_dirty=" << stats.trimmer.should_trim_dirty() + << ", trim_alloc=" << stats.trimmer.should_trim_alloc() << ")"; + } else { + os << "not-ready"; + } + if (stats.detailed) { + os << ", journal_head=" << stats.trimmer.get_journal_head() + << ", alloc_tail=" << stats.trimmer.get_alloc_tail() + << ", dirty_tail=" << stats.trimmer.get_dirty_tail(); + if (stats.trimmer.background_callback->is_ready()) { + os << ", alloc_tail_target=" << stats.trimmer.get_alloc_tail_target() + << ", dirty_tail_target=" << stats.trimmer.get_dirty_tail_target() + << ", tail_limit=" << stats.trimmer.get_tail_limit(); + } + } + os << ")"; + return os; +} + +bool SpaceTrackerSimple::equals(const SpaceTrackerI &_other) const +{ + LOG_PREFIX(SpaceTrackerSimple::equals); + const auto &other = static_cast<const SpaceTrackerSimple&>(_other); + + if (other.live_bytes_by_segment.size() != live_bytes_by_segment.size()) { + ERROR("different segment counts, bug in test"); + assert(0 == "segment counts should match"); + return false; + } + + bool all_match = true; + for (auto i = live_bytes_by_segment.begin(), j = other.live_bytes_by_segment.begin(); + i != live_bytes_by_segment.end(); ++i, ++j) { + if (i->second.live_bytes != j->second.live_bytes) { + all_match = false; + DEBUG("segment_id {} live bytes mismatch *this: {}, other: {}", + i->first, i->second.live_bytes, j->second.live_bytes); + } + } + return all_match; +} + +int64_t SpaceTrackerDetailed::SegmentMap::allocate( + device_segment_id_t segment, + segment_off_t offset, + extent_len_t len, + const extent_len_t block_size) +{ + LOG_PREFIX(SegmentMap::allocate); + assert(offset % block_size == 0); + assert(len % block_size == 0); + + const auto b = (offset / block_size); + const auto e = (offset + len) / block_size; + + bool error = false; + for (auto i = b; i < e; ++i) { + if (bitmap[i]) { + if (!error) { + ERROR("found allocated in {}, {} ~ {}", segment, offset, len); + error = true; + } + DEBUG("block {} allocated", i * block_size); + } + bitmap[i] = true; + } + return update_usage(len); +} + +int64_t SpaceTrackerDetailed::SegmentMap::release( + device_segment_id_t segment, + segment_off_t offset, + extent_len_t len, + const extent_len_t block_size) +{ + LOG_PREFIX(SegmentMap::release); + assert(offset % block_size == 0); + assert(len % block_size == 0); + + const auto b = (offset / block_size); + const auto e = (offset + len) / block_size; + + bool error = false; + for (auto i = b; i < e; ++i) { + if (!bitmap[i]) { + if (!error) { + ERROR("found unallocated in {}, {} ~ {}", segment, offset, len); + error = true; + } + DEBUG("block {} unallocated", i * block_size); + } + bitmap[i] = false; + } + return update_usage(-(int64_t)len); +} + +bool SpaceTrackerDetailed::equals(const SpaceTrackerI &_other) const +{ + LOG_PREFIX(SpaceTrackerDetailed::equals); + const auto &other = static_cast<const SpaceTrackerDetailed&>(_other); + + if (other.segment_usage.size() != segment_usage.size()) { + ERROR("different segment counts, bug in test"); + assert(0 == "segment counts should match"); + return false; + } + + bool all_match = true; + for (auto i = segment_usage.begin(), j = other.segment_usage.begin(); + i != segment_usage.end(); ++i, ++j) { + if (i->second.get_usage() != j->second.get_usage()) { + all_match = false; + ERROR("segment_id {} live bytes mismatch *this: {}, other: {}", + i->first, i->second.get_usage(), j->second.get_usage()); + } + } + return all_match; +} + +void SpaceTrackerDetailed::SegmentMap::dump_usage(extent_len_t block_size) const +{ + LOG_PREFIX(SegmentMap::dump_usage); + INFO("dump start"); + for (unsigned i = 0; i < bitmap.size(); ++i) { + if (bitmap[i]) { + LOCAL_LOGGER.info(" {} still live", i * block_size); + } + } +} + +void SpaceTrackerDetailed::dump_usage(segment_id_t id) const +{ + LOG_PREFIX(SpaceTrackerDetailed::dump_usage); + INFO("{}", id); + segment_usage[id].dump_usage( + block_size_by_segment_manager[id.device_id()]); +} + +void SpaceTrackerSimple::dump_usage(segment_id_t id) const +{ + LOG_PREFIX(SpaceTrackerSimple::dump_usage); + INFO("id: {}, live_bytes: {}", + id, live_bytes_by_segment[id].live_bytes); +} + +std::ostream &operator<<( + std::ostream &os, const AsyncCleaner::stat_printer_t &stats) +{ + stats.cleaner.print(os, stats.detailed); + return os; +} + +SegmentCleaner::SegmentCleaner( + config_t config, + SegmentManagerGroupRef&& sm_group, + BackrefManager &backref_manager, + SegmentSeqAllocator &segment_seq_allocator, + bool detailed, + bool is_cold) + : detailed(detailed), + is_cold(is_cold), + config(config), + sm_group(std::move(sm_group)), + backref_manager(backref_manager), + ool_segment_seq_allocator(segment_seq_allocator) +{ + config.validate(); +} + +void SegmentCleaner::register_metrics() +{ + namespace sm = seastar::metrics; + stats.segment_util.buckets.resize(UTIL_BUCKETS); + std::size_t i; + for (i = 0; i < UTIL_BUCKETS; ++i) { + stats.segment_util.buckets[i].upper_bound = ((double)(i + 1)) / 10; + stats.segment_util.buckets[i].count = 0; + } + // NOTE: by default the segments are empty + i = get_bucket_index(UTIL_STATE_EMPTY); + stats.segment_util.buckets[i].count = segments.get_num_segments(); + + std::string prefix; + if (is_cold) { + prefix.append("cold_"); + } + prefix.append("segment_cleaner"); + + metrics.add_group(prefix, { + sm::make_counter("segments_number", + [this] { return segments.get_num_segments(); }, + sm::description("the number of segments")), + sm::make_counter("segment_size", + [this] { return segments.get_segment_size(); }, + sm::description("the bytes of a segment")), + sm::make_counter("segments_in_journal", + [this] { return get_segments_in_journal(); }, + sm::description("the number of segments in journal")), + sm::make_counter("segments_type_journal", + [this] { return segments.get_num_type_journal(); }, + sm::description("the number of segments typed journal")), + sm::make_counter("segments_type_ool", + [this] { return segments.get_num_type_ool(); }, + sm::description("the number of segments typed out-of-line")), + sm::make_counter("segments_open", + [this] { return segments.get_num_open(); }, + sm::description("the number of open segments")), + sm::make_counter("segments_empty", + [this] { return segments.get_num_empty(); }, + sm::description("the number of empty segments")), + sm::make_counter("segments_closed", + [this] { return segments.get_num_closed(); }, + sm::description("the number of closed segments")), + + sm::make_counter("segments_count_open_journal", + [this] { return segments.get_count_open_journal(); }, + sm::description("the count of open journal segment operations")), + sm::make_counter("segments_count_open_ool", + [this] { return segments.get_count_open_ool(); }, + sm::description("the count of open ool segment operations")), + sm::make_counter("segments_count_release_journal", + [this] { return segments.get_count_release_journal(); }, + sm::description("the count of release journal segment operations")), + sm::make_counter("segments_count_release_ool", + [this] { return segments.get_count_release_ool(); }, + sm::description("the count of release ool segment operations")), + sm::make_counter("segments_count_close_journal", + [this] { return segments.get_count_close_journal(); }, + sm::description("the count of close journal segment operations")), + sm::make_counter("segments_count_close_ool", + [this] { return segments.get_count_close_ool(); }, + sm::description("the count of close ool segment operations")), + + sm::make_counter("total_bytes", + [this] { return segments.get_total_bytes(); }, + sm::description("the size of the space")), + sm::make_counter("available_bytes", + [this] { return segments.get_available_bytes(); }, + sm::description("the size of the space is available")), + sm::make_counter("unavailable_unreclaimable_bytes", + [this] { return get_unavailable_unreclaimable_bytes(); }, + sm::description("the size of the space is unavailable and unreclaimable")), + sm::make_counter("unavailable_reclaimable_bytes", + [this] { return get_unavailable_reclaimable_bytes(); }, + sm::description("the size of the space is unavailable and reclaimable")), + sm::make_counter("used_bytes", stats.used_bytes, + sm::description("the size of the space occupied by live extents")), + sm::make_counter("unavailable_unused_bytes", + [this] { return get_unavailable_unused_bytes(); }, + sm::description("the size of the space is unavailable and not alive")), + + sm::make_counter("projected_count", stats.projected_count, + sm::description("the number of projected usage reservations")), + sm::make_counter("projected_used_bytes_sum", stats.projected_used_bytes_sum, + sm::description("the sum of the projected usage in bytes")), + + sm::make_counter("reclaimed_bytes", stats.reclaimed_bytes, + sm::description("rewritten bytes due to reclaim")), + sm::make_counter("reclaimed_segment_bytes", stats.reclaimed_segment_bytes, + sm::description("rewritten bytes due to reclaim")), + sm::make_counter("closed_journal_used_bytes", stats.closed_journal_used_bytes, + sm::description("used bytes when close a journal segment")), + sm::make_counter("closed_journal_total_bytes", stats.closed_journal_total_bytes, + sm::description("total bytes of closed journal segments")), + sm::make_counter("closed_ool_used_bytes", stats.closed_ool_used_bytes, + sm::description("used bytes when close a ool segment")), + sm::make_counter("closed_ool_total_bytes", stats.closed_ool_total_bytes, + sm::description("total bytes of closed ool segments")), + + sm::make_gauge("available_ratio", + [this] { return segments.get_available_ratio(); }, + sm::description("ratio of available space to total space")), + sm::make_gauge("reclaim_ratio", + [this] { return get_reclaim_ratio(); }, + sm::description("ratio of reclaimable space to unavailable space")), + + sm::make_histogram("segment_utilization_distribution", + [this]() -> seastar::metrics::histogram& { + return stats.segment_util; + }, + sm::description("utilization distribution of all segments")) + }); +} + +segment_id_t SegmentCleaner::allocate_segment( + segment_seq_t seq, + segment_type_t type, + data_category_t category, + rewrite_gen_t generation) +{ + LOG_PREFIX(SegmentCleaner::allocate_segment); + assert(seq != NULL_SEG_SEQ); + ceph_assert(type == segment_type_t::OOL || + trimmer != nullptr); // segment_type_t::JOURNAL + for (auto it = segments.begin(); + it != segments.end(); + ++it) { + auto seg_id = it->first; + auto& segment_info = it->second; + if (segment_info.is_empty()) { + auto old_usage = calc_utilization(seg_id); + segments.mark_open(seg_id, seq, type, category, generation); + background_callback->maybe_wake_background(); + auto new_usage = calc_utilization(seg_id); + adjust_segment_util(old_usage, new_usage); + INFO("opened, {}", stat_printer_t{*this, false}); + return seg_id; + } + } + ERROR("out of space with {} {} {} {}", + type, segment_seq_printer_t{seq}, category, + rewrite_gen_printer_t{generation}); + ceph_abort("seastore device size setting is too small"); + return NULL_SEG_ID; +} + +void SegmentCleaner::close_segment(segment_id_t segment) +{ + LOG_PREFIX(SegmentCleaner::close_segment); + auto old_usage = calc_utilization(segment); + segments.mark_closed(segment); + auto &seg_info = segments[segment]; + if (seg_info.type == segment_type_t::JOURNAL) { + stats.closed_journal_used_bytes += space_tracker->get_usage(segment); + stats.closed_journal_total_bytes += segments.get_segment_size(); + } else { + stats.closed_ool_used_bytes += space_tracker->get_usage(segment); + stats.closed_ool_total_bytes += segments.get_segment_size(); + } + auto new_usage = calc_utilization(segment); + adjust_segment_util(old_usage, new_usage); + INFO("closed, {} -- {}", stat_printer_t{*this, false}, seg_info); +} + +double SegmentCleaner::calc_gc_benefit_cost( + segment_id_t id, + const sea_time_point &now_time, + const sea_time_point &bound_time) const +{ + double util = calc_utilization(id); + ceph_assert(util >= 0 && util < 1); + if constexpr (gc_formula == gc_formula_t::GREEDY) { + return 1 - util; + } + + if constexpr (gc_formula == gc_formula_t::COST_BENEFIT) { + if (util == 0) { + return std::numeric_limits<double>::max(); + } + auto modify_time = segments[id].modify_time; + double age_segment = modify_time.time_since_epoch().count(); + double age_now = now_time.time_since_epoch().count(); + if (likely(age_now > age_segment)) { + return (1 - util) * (age_now - age_segment) / (2 * util); + } else { + // time is wrong + return (1 - util) / (2 * util); + } + } + + assert(gc_formula == gc_formula_t::BENEFIT); + auto modify_time = segments[id].modify_time; + double age_factor = 0.5; // middle value if age is invalid + if (likely(bound_time != NULL_TIME && + modify_time != NULL_TIME && + now_time > modify_time)) { + assert(modify_time >= bound_time); + double age_bound = bound_time.time_since_epoch().count(); + double age_now = now_time.time_since_epoch().count(); + double age_segment = modify_time.time_since_epoch().count(); + age_factor = (age_now - age_segment) / (age_now - age_bound); + } + return ((1 - 2 * age_factor) * util * util + + (2 * age_factor - 2) * util + 1); +} + +SegmentCleaner::do_reclaim_space_ret +SegmentCleaner::do_reclaim_space( + const std::vector<CachedExtentRef> &backref_extents, + const backref_pin_list_t &pin_list, + std::size_t &reclaimed, + std::size_t &runs) +{ + return repeat_eagain([this, &backref_extents, + &pin_list, &reclaimed, &runs] { + reclaimed = 0; + runs++; + auto src = Transaction::src_t::CLEANER_MAIN; + if (is_cold) { + src = Transaction::src_t::CLEANER_COLD; + } + return extent_callback->with_transaction_intr( + src, + "clean_reclaim_space", + [this, &backref_extents, &pin_list, &reclaimed](auto &t) + { + return seastar::do_with( + std::vector<CachedExtentRef>(backref_extents), + [this, &t, &reclaimed, &pin_list](auto &extents) + { + LOG_PREFIX(SegmentCleaner::do_reclaim_space); + // calculate live extents + auto cached_backref_entries = + backref_manager.get_cached_backref_entries_in_range( + reclaim_state->start_pos, reclaim_state->end_pos); + backref_entry_query_set_t backref_entries; + for (auto &pin : pin_list) { + backref_entries.emplace( + pin->get_key(), + pin->get_val(), + pin->get_length(), + pin->get_type(), + JOURNAL_SEQ_NULL); + } + for (auto &cached_backref : cached_backref_entries) { + if (cached_backref.laddr == L_ADDR_NULL) { + auto it = backref_entries.find(cached_backref.paddr); + assert(it->len == cached_backref.len); + backref_entries.erase(it); + } else { + backref_entries.emplace(cached_backref); + } + } + // retrieve live extents + DEBUGT("start, backref_entries={}, backref_extents={}", + t, backref_entries.size(), extents.size()); + return seastar::do_with( + std::move(backref_entries), + [this, &extents, &t](auto &backref_entries) { + return trans_intr::parallel_for_each( + backref_entries, + [this, &extents, &t](auto &ent) + { + LOG_PREFIX(SegmentCleaner::do_reclaim_space); + TRACET("getting extent of type {} at {}~{}", + t, + ent.type, + ent.paddr, + ent.len); + return extent_callback->get_extents_if_live( + t, ent.type, ent.paddr, ent.laddr, ent.len + ).si_then([FNAME, &extents, &ent, &t](auto list) { + if (list.empty()) { + TRACET("addr {} dead, skipping", t, ent.paddr); + } else { + for (auto &e : list) { + extents.emplace_back(std::move(e)); + } + } + }); + }); + }).si_then([FNAME, &extents, this, &reclaimed, &t] { + DEBUGT("reclaim {} extents", t, extents.size()); + // rewrite live extents + auto modify_time = segments[reclaim_state->get_segment_id()].modify_time; + return trans_intr::do_for_each( + extents, + [this, modify_time, &t, &reclaimed](auto ext) + { + reclaimed += ext->get_length(); + return extent_callback->rewrite_extent( + t, ext, reclaim_state->target_generation, modify_time); + }); + }); + }).si_then([this, &t] { + return extent_callback->submit_transaction_direct(t); + }); + }); + }); +} + +SegmentCleaner::clean_space_ret SegmentCleaner::clean_space() +{ + LOG_PREFIX(SegmentCleaner::clean_space); + assert(background_callback->is_ready()); + ceph_assert(can_clean_space()); + if (!reclaim_state) { + segment_id_t seg_id = get_next_reclaim_segment(); + auto &segment_info = segments[seg_id]; + INFO("reclaim {} {} start, usage={}, time_bound={}", + seg_id, segment_info, + space_tracker->calc_utilization(seg_id), + sea_time_point_printer_t{segments.get_time_bound()}); + ceph_assert(segment_info.is_closed()); + reclaim_state = reclaim_state_t::create( + seg_id, segment_info.generation, segments.get_segment_size()); + } + reclaim_state->advance(config.reclaim_bytes_per_cycle); + + DEBUG("reclaiming {} {}~{}", + rewrite_gen_printer_t{reclaim_state->generation}, + reclaim_state->start_pos, + reclaim_state->end_pos); + double pavail_ratio = get_projected_available_ratio(); + sea_time_point start = seastar::lowres_system_clock::now(); + + // Backref-tree doesn't support tree-read during tree-updates with parallel + // transactions. So, concurrent transactions between trim and reclaim are + // not allowed right now. + return seastar::do_with( + std::pair<std::vector<CachedExtentRef>, backref_pin_list_t>(), + [this](auto &weak_read_ret) { + return repeat_eagain([this, &weak_read_ret] { + return extent_callback->with_transaction_intr( + Transaction::src_t::READ, + "retrieve_from_backref_tree", + [this, &weak_read_ret](auto &t) { + return backref_manager.get_mappings( + t, + reclaim_state->start_pos, + reclaim_state->end_pos + ).si_then([this, &t, &weak_read_ret](auto pin_list) { + if (!pin_list.empty()) { + auto it = pin_list.begin(); + auto &first_pin = *it; + if (first_pin->get_key() < reclaim_state->start_pos) { + // BackrefManager::get_mappings may include a entry before + // reclaim_state->start_pos, which is semantically inconsistent + // with the requirements of the cleaner + pin_list.erase(it); + } + } + return backref_manager.retrieve_backref_extents_in_range( + t, + reclaim_state->start_pos, + reclaim_state->end_pos + ).si_then([pin_list=std::move(pin_list), + &weak_read_ret](auto extents) mutable { + weak_read_ret = std::make_pair(std::move(extents), std::move(pin_list)); + }); + }); + }); + }).safe_then([&weak_read_ret] { + return std::move(weak_read_ret); + }); + }).safe_then([this, FNAME, pavail_ratio, start](auto weak_read_ret) { + return seastar::do_with( + std::move(weak_read_ret.first), + std::move(weak_read_ret.second), + (size_t)0, + (size_t)0, + [this, FNAME, pavail_ratio, start]( + auto &backref_extents, auto &pin_list, auto &reclaimed, auto &runs) + { + return do_reclaim_space( + backref_extents, + pin_list, + reclaimed, + runs + ).safe_then([this, FNAME, pavail_ratio, start, &reclaimed, &runs] { + stats.reclaiming_bytes += reclaimed; + auto d = seastar::lowres_system_clock::now() - start; + DEBUG("duration: {}, pavail_ratio before: {}, repeats: {}", + d, pavail_ratio, runs); + if (reclaim_state->is_complete()) { + auto segment_to_release = reclaim_state->get_segment_id(); + INFO("reclaim {} finish, reclaimed alive/total={}", + segment_to_release, + stats.reclaiming_bytes/(double)segments.get_segment_size()); + stats.reclaimed_bytes += stats.reclaiming_bytes; + stats.reclaimed_segment_bytes += segments.get_segment_size(); + stats.reclaiming_bytes = 0; + reclaim_state.reset(); + return sm_group->release_segment(segment_to_release + ).handle_error( + clean_space_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "SegmentCleaner::clean_space encountered invalid error in release_segment" + } + ).safe_then([this, FNAME, segment_to_release] { + auto old_usage = calc_utilization(segment_to_release); + if(unlikely(old_usage != 0)) { + space_tracker->dump_usage(segment_to_release); + ERROR("segment {} old_usage {} != 0", + segment_to_release, old_usage); + ceph_abort(); + } + segments.mark_empty(segment_to_release); + auto new_usage = calc_utilization(segment_to_release); + adjust_segment_util(old_usage, new_usage); + INFO("released {}, {}", + segment_to_release, stat_printer_t{*this, false}); + background_callback->maybe_wake_blocked_io(); + }); + } else { + return clean_space_ertr::now(); + } + }); + }); + }); +} + +SegmentCleaner::mount_ret SegmentCleaner::mount() +{ + LOG_PREFIX(SegmentCleaner::mount); + const auto& sms = sm_group->get_segment_managers(); + INFO("{} segment managers", sms.size()); + + assert(background_callback->get_state() == state_t::MOUNT); + + space_tracker.reset( + detailed ? + (SpaceTrackerI*)new SpaceTrackerDetailed( + sms) : + (SpaceTrackerI*)new SpaceTrackerSimple( + sms)); + + segments.reset(); + for (auto sm : sms) { + segments.add_segment_manager(*sm); + } + segments.assign_ids(); + + stats = {}; + metrics.clear(); + register_metrics(); + + INFO("{} segments", segments.get_num_segments()); + return crimson::do_for_each( + segments.begin(), + segments.end(), + [this, FNAME](auto& it) + { + auto segment_id = it.first; + return sm_group->read_segment_header( + segment_id + ).safe_then([segment_id, this, FNAME](auto header) { + DEBUG("segment_id={} -- {}", segment_id, header); + auto s_type = header.get_type(); + if (s_type == segment_type_t::NULL_SEG) { + ERROR("got null segment, segment_id={} -- {}", segment_id, header); + ceph_abort(); + } + return sm_group->read_segment_tail( + segment_id + ).safe_then([this, FNAME, segment_id, header](auto tail) + -> scan_extents_ertr::future<> { + if (tail.segment_nonce != header.segment_nonce) { + return scan_no_tail_segment(header, segment_id); + } + ceph_assert(header.get_type() == tail.get_type()); + + sea_time_point modify_time = mod_to_timepoint(tail.modify_time); + std::size_t num_extents = tail.num_extents; + if ((modify_time == NULL_TIME && num_extents == 0) || + (modify_time != NULL_TIME && num_extents != 0)) { + segments.update_modify_time(segment_id, modify_time, num_extents); + } else { + ERROR("illegal modify time {}", tail); + return crimson::ct_error::input_output_error::make(); + } + + init_mark_segment_closed( + segment_id, + header.segment_seq, + header.type, + header.category, + header.generation); + return seastar::now(); + }).handle_error( + crimson::ct_error::enodata::handle( + [this, header, segment_id](auto) { + return scan_no_tail_segment(header, segment_id); + }), + crimson::ct_error::pass_further_all{} + ); + }).handle_error( + crimson::ct_error::enoent::handle([](auto) { + return mount_ertr::now(); + }), + crimson::ct_error::enodata::handle([](auto) { + return mount_ertr::now(); + }), + crimson::ct_error::input_output_error::pass_further{}, + crimson::ct_error::assert_all{"unexpected error"} + ); + }).safe_then([this, FNAME] { + INFO("done, {}", segments); + }); +} + +SegmentCleaner::scan_extents_ret SegmentCleaner::scan_no_tail_segment( + const segment_header_t &segment_header, + segment_id_t segment_id) +{ + LOG_PREFIX(SegmentCleaner::scan_no_tail_segment); + INFO("scan {} {}", segment_id, segment_header); + return seastar::do_with( + scan_valid_records_cursor({ + segments[segment_id].seq, + paddr_t::make_seg_paddr(segment_id, 0) + }), + SegmentManagerGroup::found_record_handler_t( + [this, segment_id, segment_header, FNAME]( + record_locator_t locator, + const record_group_header_t &record_group_header, + const bufferlist& mdbuf + ) mutable -> SegmentManagerGroup::scan_valid_records_ertr::future<> + { + DEBUG("{} {}, decoding {} records", + segment_id, segment_header.get_type(), record_group_header.records); + + auto maybe_headers = try_decode_record_headers( + record_group_header, mdbuf); + if (!maybe_headers) { + // This should be impossible, we did check the crc on the mdbuf + ERROR("unable to decode record headers for record group {}", + locator.record_block_base); + return crimson::ct_error::input_output_error::make(); + } + + for (auto &record_header : *maybe_headers) { + auto modify_time = mod_to_timepoint(record_header.modify_time); + if (record_header.extents == 0 || modify_time != NULL_TIME) { + segments.update_modify_time( + segment_id, modify_time, record_header.extents); + } else { + ERROR("illegal modify time {}", record_header); + return crimson::ct_error::input_output_error::make(); + } + } + return seastar::now(); + }), + [this, segment_header](auto &cursor, auto &handler) + { + return sm_group->scan_valid_records( + cursor, + segment_header.segment_nonce, + segments.get_segment_size(), + handler).discard_result(); + }).safe_then([this, segment_id, segment_header] { + init_mark_segment_closed( + segment_id, + segment_header.segment_seq, + segment_header.type, + segment_header.category, + segment_header.generation); + }); +} + +bool SegmentCleaner::check_usage() +{ + SpaceTrackerIRef tracker(space_tracker->make_empty()); + extent_callback->with_transaction_weak( + "check_usage", + [this, &tracker](auto &t) { + return backref_manager.scan_mapped_space( + t, + [&tracker]( + paddr_t paddr, + paddr_t backref_key, + extent_len_t len, + extent_types_t type, + laddr_t laddr) + { + if (paddr.get_addr_type() == paddr_types_t::SEGMENT) { + if (is_backref_node(type)) { + assert(laddr == L_ADDR_NULL); + assert(backref_key != P_ADDR_NULL); + tracker->allocate( + paddr.as_seg_paddr().get_segment_id(), + paddr.as_seg_paddr().get_segment_off(), + len); + } else if (laddr == L_ADDR_NULL) { + assert(backref_key == P_ADDR_NULL); + tracker->release( + paddr.as_seg_paddr().get_segment_id(), + paddr.as_seg_paddr().get_segment_off(), + len); + } else { + assert(backref_key == P_ADDR_NULL); + tracker->allocate( + paddr.as_seg_paddr().get_segment_id(), + paddr.as_seg_paddr().get_segment_off(), + len); + } + } + }); + }).unsafe_get0(); + return space_tracker->equals(*tracker); +} + +void SegmentCleaner::mark_space_used( + paddr_t addr, + extent_len_t len) +{ + LOG_PREFIX(SegmentCleaner::mark_space_used); + assert(background_callback->get_state() >= state_t::SCAN_SPACE); + assert(len); + // TODO: drop + if (addr.get_addr_type() != paddr_types_t::SEGMENT) { + return; + } + + auto& seg_addr = addr.as_seg_paddr(); + stats.used_bytes += len; + auto old_usage = calc_utilization(seg_addr.get_segment_id()); + [[maybe_unused]] auto ret = space_tracker->allocate( + seg_addr.get_segment_id(), + seg_addr.get_segment_off(), + len); + auto new_usage = calc_utilization(seg_addr.get_segment_id()); + adjust_segment_util(old_usage, new_usage); + + background_callback->maybe_wake_background(); + assert(ret > 0); + DEBUG("segment {} new len: {}~{}, live_bytes: {}", + seg_addr.get_segment_id(), + addr, + len, + space_tracker->get_usage(seg_addr.get_segment_id())); +} + +void SegmentCleaner::mark_space_free( + paddr_t addr, + extent_len_t len) +{ + LOG_PREFIX(SegmentCleaner::mark_space_free); + assert(background_callback->get_state() >= state_t::SCAN_SPACE); + assert(len); + // TODO: drop + if (addr.get_addr_type() != paddr_types_t::SEGMENT) { + return; + } + + ceph_assert(stats.used_bytes >= len); + stats.used_bytes -= len; + auto& seg_addr = addr.as_seg_paddr(); + + DEBUG("segment {} free len: {}~{}", + seg_addr.get_segment_id(), addr, len); + auto old_usage = calc_utilization(seg_addr.get_segment_id()); + [[maybe_unused]] auto ret = space_tracker->release( + seg_addr.get_segment_id(), + seg_addr.get_segment_off(), + len); + auto new_usage = calc_utilization(seg_addr.get_segment_id()); + adjust_segment_util(old_usage, new_usage); + background_callback->maybe_wake_blocked_io(); + assert(ret >= 0); + DEBUG("segment {} free len: {}~{}, live_bytes: {}", + seg_addr.get_segment_id(), + addr, + len, + space_tracker->get_usage(seg_addr.get_segment_id())); +} + +segment_id_t SegmentCleaner::get_next_reclaim_segment() const +{ + LOG_PREFIX(SegmentCleaner::get_next_reclaim_segment); + segment_id_t id = NULL_SEG_ID; + double max_benefit_cost = 0; + sea_time_point now_time; + if constexpr (gc_formula != gc_formula_t::GREEDY) { + now_time = seastar::lowres_system_clock::now(); + } else { + now_time = NULL_TIME; + } + sea_time_point bound_time; + if constexpr (gc_formula == gc_formula_t::BENEFIT) { + bound_time = segments.get_time_bound(); + if (bound_time == NULL_TIME) { + WARN("BENEFIT -- bound_time is NULL_TIME"); + } + } else { + bound_time = NULL_TIME; + } + for (auto& [_id, segment_info] : segments) { + if (segment_info.is_closed() && + (trimmer == nullptr || + !segment_info.is_in_journal(trimmer->get_journal_tail()))) { + double benefit_cost = calc_gc_benefit_cost(_id, now_time, bound_time); + if (benefit_cost > max_benefit_cost) { + id = _id; + max_benefit_cost = benefit_cost; + } + } + } + if (id != NULL_SEG_ID) { + DEBUG("segment {}, benefit_cost {}", + id, max_benefit_cost); + return id; + } else { + ceph_assert(get_segments_reclaimable() == 0); + // see should_clean_space() + ceph_abort("impossible!"); + return NULL_SEG_ID; + } +} + +bool SegmentCleaner::try_reserve_projected_usage(std::size_t projected_usage) +{ + assert(background_callback->is_ready()); + stats.projected_used_bytes += projected_usage; + if (should_block_io_on_clean()) { + stats.projected_used_bytes -= projected_usage; + return false; + } else { + ++stats.projected_count; + stats.projected_used_bytes_sum += stats.projected_used_bytes; + return true; + } +} + +void SegmentCleaner::release_projected_usage(std::size_t projected_usage) +{ + assert(background_callback->is_ready()); + ceph_assert(stats.projected_used_bytes >= projected_usage); + stats.projected_used_bytes -= projected_usage; + background_callback->maybe_wake_blocked_io(); +} + +void SegmentCleaner::print(std::ostream &os, bool is_detailed) const +{ + os << "SegmentCleaner("; + if (background_callback->is_ready()) { + os << "should_block_io_on_clean=" << should_block_io_on_clean() + << ", should_clean=" << should_clean_space(); + } else { + os << "not-ready"; + } + os << ", projected_avail_ratio=" << get_projected_available_ratio() + << ", reclaim_ratio=" << get_reclaim_ratio() + << ", alive_ratio=" << get_alive_ratio(); + if (is_detailed) { + os << ", unavailable_unreclaimable=" + << get_unavailable_unreclaimable_bytes() << "B" + << ", unavailable_reclaimble=" + << get_unavailable_reclaimable_bytes() << "B" + << ", alive=" << stats.used_bytes << "B" + << ", " << segments; + } + os << ")"; +} + +RBMCleaner::RBMCleaner( + RBMDeviceGroupRef&& rb_group, + BackrefManager &backref_manager, + bool detailed) + : detailed(detailed), + rb_group(std::move(rb_group)), + backref_manager(backref_manager) +{} + +void RBMCleaner::print(std::ostream &os, bool is_detailed) const +{ + // TODO + return; +} + +void RBMCleaner::mark_space_used( + paddr_t addr, + extent_len_t len) +{ + LOG_PREFIX(RBMCleaner::mark_space_used); + assert(addr.get_addr_type() == paddr_types_t::RANDOM_BLOCK); + auto rbms = rb_group->get_rb_managers(); + for (auto rbm : rbms) { + if (addr.get_device_id() == rbm->get_device_id()) { + if (rbm->get_start() <= addr) { + INFO("allocate addr: {} len: {}", addr, len); + stats.used_bytes += len; + rbm->mark_space_used(addr, len); + } + return; + } + } +} + +void RBMCleaner::mark_space_free( + paddr_t addr, + extent_len_t len) +{ + LOG_PREFIX(RBMCleaner::mark_space_free); + assert(addr.get_addr_type() == paddr_types_t::RANDOM_BLOCK); + auto rbms = rb_group->get_rb_managers(); + for (auto rbm : rbms) { + if (addr.get_device_id() == rbm->get_device_id()) { + if (rbm->get_start() <= addr) { + INFO("free addr: {} len: {}", addr, len); + ceph_assert(stats.used_bytes >= len); + stats.used_bytes -= len; + rbm->mark_space_free(addr, len); + } + return; + } + } +} + +void RBMCleaner::commit_space_used(paddr_t addr, extent_len_t len) +{ + auto rbms = rb_group->get_rb_managers(); + for (auto rbm : rbms) { + if (addr.get_device_id() == rbm->get_device_id()) { + if (rbm->get_start() <= addr) { + rbm->complete_allocation(addr, len); + } + return; + } + } +} + +bool RBMCleaner::try_reserve_projected_usage(std::size_t projected_usage) +{ + assert(background_callback->is_ready()); + stats.projected_used_bytes += projected_usage; + return true; +} + +void RBMCleaner::release_projected_usage(std::size_t projected_usage) +{ + assert(background_callback->is_ready()); + ceph_assert(stats.projected_used_bytes >= projected_usage); + stats.projected_used_bytes -= projected_usage; + background_callback->maybe_wake_blocked_io(); +} + +RBMCleaner::clean_space_ret RBMCleaner::clean_space() +{ + // TODO + return clean_space_ertr::now(); +} + +RBMCleaner::mount_ret RBMCleaner::mount() +{ + stats = {}; + register_metrics(); + return seastar::do_with( + rb_group->get_rb_managers(), + [](auto &rbs) { + return crimson::do_for_each( + rbs.begin(), + rbs.end(), + [](auto& it) { + return it->open( + ).handle_error( + crimson::ct_error::input_output_error::pass_further(), + crimson::ct_error::assert_all{ + "Invalid error when opening RBM"} + ); + }); + }); +} + +bool RBMCleaner::check_usage() +{ + assert(detailed); + const auto& rbms = rb_group->get_rb_managers(); + RBMSpaceTracker tracker(rbms); + extent_callback->with_transaction_weak( + "check_usage", + [this, &tracker, &rbms](auto &t) { + return backref_manager.scan_mapped_space( + t, + [&tracker, &rbms]( + paddr_t paddr, + paddr_t backref_key, + extent_len_t len, + extent_types_t type, + laddr_t laddr) + { + for (auto rbm : rbms) { + if (rbm->get_device_id() == paddr.get_device_id()) { + if (is_backref_node(type)) { + assert(laddr == L_ADDR_NULL); + assert(backref_key != P_ADDR_NULL); + tracker.allocate( + paddr, + len); + } else if (laddr == L_ADDR_NULL) { + assert(backref_key == P_ADDR_NULL); + tracker.release( + paddr, + len); + } else { + assert(backref_key == P_ADDR_NULL); + tracker.allocate( + paddr, + len); + } + } + } + }); + }).unsafe_get0(); + return equals(tracker); +} + +bool RBMCleaner::equals(const RBMSpaceTracker &_other) const +{ + LOG_PREFIX(RBMSpaceTracker::equals); + const auto &other = static_cast<const RBMSpaceTracker&>(_other); + auto rbs = rb_group->get_rb_managers(); + //TODO: multiple rbm allocator + auto rbm = rbs[0]; + assert(rbm); + + if (rbm->get_device()->get_available_size() / rbm->get_block_size() + != other.block_usage.size()) { + assert(0 == "block counts should match"); + return false; + } + bool all_match = true; + for (auto i = other.block_usage.begin(); + i != other.block_usage.end(); ++i) { + if (i->first < rbm->get_start().as_blk_paddr().get_device_off()) { + continue; + } + auto addr = i->first; + auto state = rbm->get_extent_state( + convert_abs_addr_to_paddr(addr, rbm->get_device_id()), + rbm->get_block_size()); + if ((i->second.used && state == rbm_extent_state_t::ALLOCATED) || + (!i->second.used && (state == rbm_extent_state_t::FREE || + state == rbm_extent_state_t::RESERVED))) { + // pass + } else { + all_match = false; + ERROR("block addr {} mismatch other used: {}", + addr, i->second.used); + } + } + return all_match; +} + +void RBMCleaner::register_metrics() +{ + namespace sm = seastar::metrics; + + metrics.add_group("rbm_cleaner", { + sm::make_counter("total_bytes", + [this] { return get_total_bytes(); }, + sm::description("the size of the space")), + sm::make_counter("available_bytes", + [this] { return get_total_bytes() - get_journal_bytes() - stats.used_bytes; }, + sm::description("the size of the space is available")), + sm::make_counter("used_bytes", stats.used_bytes, + sm::description("the size of the space occupied by live extents")), + }); +} + +} diff --git a/src/crimson/os/seastore/async_cleaner.h b/src/crimson/os/seastore/async_cleaner.h new file mode 100644 index 000000000..fb8e03bb4 --- /dev/null +++ b/src/crimson/os/seastore/async_cleaner.h @@ -0,0 +1,1761 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <boost/intrusive/set.hpp> +#include <seastar/core/metrics_types.hh> + +#include "common/ceph_time.h" + +#include "osd/osd_types.h" + +#include "crimson/os/seastore/cached_extent.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/segment_manager.h" +#include "crimson/os/seastore/segment_manager_group.h" +#include "crimson/os/seastore/randomblock_manager_group.h" +#include "crimson/os/seastore/transaction.h" +#include "crimson/os/seastore/segment_seq_allocator.h" + +namespace crimson::os::seastore { + +/* + * segment_info_t + * + * Maintains the tracked information for a segment. + * It is read-only outside segments_info_t. + */ +struct segment_info_t { + segment_id_t id = NULL_SEG_ID; + + // segment_info_t is initiated as set_empty() + Segment::segment_state_t state = Segment::segment_state_t::EMPTY; + + // Will be non-null for any segments in the current journal + segment_seq_t seq = NULL_SEG_SEQ; + + segment_type_t type = segment_type_t::NULL_SEG; + + data_category_t category = data_category_t::NUM; + + rewrite_gen_t generation = NULL_GENERATION; + + sea_time_point modify_time = NULL_TIME; + + std::size_t num_extents = 0; + + segment_off_t written_to = 0; + + bool is_in_journal(journal_seq_t tail_committed) const { + return type == segment_type_t::JOURNAL && + tail_committed.segment_seq <= seq; + } + + bool is_empty() const { + return state == Segment::segment_state_t::EMPTY; + } + + bool is_closed() const { + return state == Segment::segment_state_t::CLOSED; + } + + bool is_open() const { + return state == Segment::segment_state_t::OPEN; + } + + void init_closed(segment_seq_t, segment_type_t, + data_category_t, rewrite_gen_t, + segment_off_t); + + void set_open(segment_seq_t, segment_type_t, + data_category_t, rewrite_gen_t); + + void set_empty(); + + void set_closed(); + + void update_modify_time(sea_time_point _modify_time, std::size_t _num_extents) { + ceph_assert(!is_closed()); + assert(_modify_time != NULL_TIME); + assert(_num_extents != 0); + if (modify_time == NULL_TIME) { + modify_time = _modify_time; + num_extents = _num_extents; + } else { + modify_time = get_average_time( + modify_time, num_extents, _modify_time, _num_extents); + num_extents += _num_extents; + } + } +}; + +std::ostream& operator<<(std::ostream&, const segment_info_t&); + +/* + * segments_info_t + * + * Keep track of all segments and related information. + */ +class segments_info_t { +public: + segments_info_t() { + reset(); + } + + const segment_info_t& operator[](segment_id_t id) const { + return segments[id]; + } + + auto begin() const { + return segments.begin(); + } + + auto end() const { + return segments.end(); + } + + std::size_t get_num_segments() const { + assert(segments.size() > 0); + return segments.size(); + } + segment_off_t get_segment_size() const { + assert(segment_size > 0); + return segment_size; + } + std::size_t get_num_in_journal_open() const { + return num_in_journal_open; + } + std::size_t get_num_type_journal() const { + return num_type_journal; + } + std::size_t get_num_type_ool() const { + return num_type_ool; + } + std::size_t get_num_open() const { + return num_open; + } + std::size_t get_num_empty() const { + return num_empty; + } + std::size_t get_num_closed() const { + return num_closed; + } + std::size_t get_count_open_journal() const { + return count_open_journal; + } + std::size_t get_count_open_ool() const { + return count_open_ool; + } + std::size_t get_count_release_journal() const { + return count_release_journal; + } + std::size_t get_count_release_ool() const { + return count_release_ool; + } + std::size_t get_count_close_journal() const { + return count_close_journal; + } + std::size_t get_count_close_ool() const { + return count_close_ool; + } + + std::size_t get_total_bytes() const { + return total_bytes; + } + /// the available space that is writable, including in open segments + std::size_t get_available_bytes() const { + return num_empty * get_segment_size() + avail_bytes_in_open; + } + /// the unavailable space that is not writable + std::size_t get_unavailable_bytes() const { + assert(total_bytes >= get_available_bytes()); + return total_bytes - get_available_bytes(); + } + std::size_t get_available_bytes_in_open() const { + return avail_bytes_in_open; + } + double get_available_ratio() const { + return (double)get_available_bytes() / (double)total_bytes; + } + + journal_seq_t get_submitted_journal_head() const { + if (unlikely(journal_segment_id == NULL_SEG_ID)) { + return JOURNAL_SEQ_NULL; + } + auto &segment_info = segments[journal_segment_id]; + assert(!segment_info.is_empty()); + assert(segment_info.type == segment_type_t::JOURNAL); + assert(segment_info.seq != NULL_SEG_SEQ); + return journal_seq_t{ + segment_info.seq, + paddr_t::make_seg_paddr( + journal_segment_id, + segment_info.written_to) + }; + } + + sea_time_point get_time_bound() const { + if (!modify_times.empty()) { + return *modify_times.begin(); + } else { + return NULL_TIME; + } + } + + void reset(); + + void add_segment_manager(SegmentManager &segment_manager); + + void assign_ids() { + for (auto &item : segments) { + item.second.id = item.first; + } + } + + // initiate non-empty segments, the others are by default empty + void init_closed(segment_id_t, segment_seq_t, segment_type_t, + data_category_t, rewrite_gen_t); + + void mark_open(segment_id_t, segment_seq_t, segment_type_t, + data_category_t, rewrite_gen_t); + + void mark_empty(segment_id_t); + + void mark_closed(segment_id_t); + + void update_written_to(segment_type_t, paddr_t); + + void update_modify_time( + segment_id_t id, sea_time_point tp, std::size_t num) { + if (num == 0) { + return; + } + + assert(tp != NULL_TIME); + segments[id].update_modify_time(tp, num); + } + +private: + // See reset() for member initialization + segment_map_t<segment_info_t> segments; + + segment_off_t segment_size; + + segment_id_t journal_segment_id; + std::size_t num_in_journal_open; + std::size_t num_type_journal; + std::size_t num_type_ool; + + std::size_t num_open; + std::size_t num_empty; + std::size_t num_closed; + + std::size_t count_open_journal; + std::size_t count_open_ool; + std::size_t count_release_journal; + std::size_t count_release_ool; + std::size_t count_close_journal; + std::size_t count_close_ool; + + std::size_t total_bytes; + std::size_t avail_bytes_in_open; + + std::multiset<sea_time_point> modify_times; +}; + +std::ostream &operator<<(std::ostream &, const segments_info_t &); + +/** + * Callback interface for querying extents and operating on transactions. + */ +class ExtentCallbackInterface { +public: + using base_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + using base_iertr = trans_iertr<base_ertr>; + + virtual ~ExtentCallbackInterface() = default; + + /// Creates empty transaction + /// weak transaction should be type READ + virtual TransactionRef create_transaction( + Transaction::src_t, const char *name, bool is_weak=false) = 0; + + /// Creates empty transaction with interruptible context + template <typename Func> + auto with_transaction_intr( + Transaction::src_t src, + const char* name, + Func &&f) { + return do_with_transaction_intr<Func, false>( + src, name, std::forward<Func>(f)); + } + + template <typename Func> + auto with_transaction_weak( + const char* name, + Func &&f) { + return do_with_transaction_intr<Func, true>( + Transaction::src_t::READ, name, std::forward<Func>(f) + ).handle_error( + crimson::ct_error::eagain::handle([] { + ceph_assert(0 == "eagain impossible"); + }), + crimson::ct_error::pass_further_all{} + ); + } + + /// See Cache::get_next_dirty_extents + using get_next_dirty_extents_iertr = base_iertr; + using get_next_dirty_extents_ret = get_next_dirty_extents_iertr::future< + std::vector<CachedExtentRef>>; + virtual get_next_dirty_extents_ret get_next_dirty_extents( + Transaction &t, ///< [in] current transaction + journal_seq_t bound,///< [in] return extents with dirty_from < bound + size_t max_bytes ///< [in] return up to max_bytes of extents + ) = 0; + + /** + * rewrite_extent + * + * Updates t with operations moving the passed extents to a new + * segment. extent may be invalid, implementation must correctly + * handle finding the current instance if it is still alive and + * otherwise ignore it. + */ + using rewrite_extent_iertr = base_iertr; + using rewrite_extent_ret = rewrite_extent_iertr::future<>; + virtual rewrite_extent_ret rewrite_extent( + Transaction &t, + CachedExtentRef extent, + rewrite_gen_t target_generation, + sea_time_point modify_time) = 0; + + /** + * get_extent_if_live + * + * Returns extent at specified location if still referenced by + * lba_manager and not removed by t. + * + * See TransactionManager::get_extent_if_live and + * LBAManager::get_physical_extent_if_live. + */ + using get_extents_if_live_iertr = base_iertr; + using get_extents_if_live_ret = get_extents_if_live_iertr::future< + std::list<CachedExtentRef>>; + virtual get_extents_if_live_ret get_extents_if_live( + Transaction &t, + extent_types_t type, + paddr_t addr, + laddr_t laddr, + extent_len_t len) = 0; + + /** + * submit_transaction_direct + * + * Submits transaction without any space throttling. + */ + using submit_transaction_direct_iertr = base_iertr; + using submit_transaction_direct_ret = + submit_transaction_direct_iertr::future<>; + virtual submit_transaction_direct_ret submit_transaction_direct( + Transaction &t, + std::optional<journal_seq_t> seq_to_trim = std::nullopt) = 0; + +private: + template <typename Func, bool IsWeak> + auto do_with_transaction_intr( + Transaction::src_t src, + const char* name, + Func &&f) { + return seastar::do_with( + create_transaction(src, name, IsWeak), + [f=std::forward<Func>(f)](auto &ref_t) mutable { + return with_trans_intr( + *ref_t, + [f=std::forward<Func>(f)](auto& t) mutable { + return f(t); + } + ); + } + ); + } +}; + +/** + * Callback interface to wake up background works + */ +struct BackgroundListener { + enum class state_t { + STOP, + MOUNT, + SCAN_SPACE, + RUNNING, + HALT, + }; + + virtual ~BackgroundListener() = default; + virtual void maybe_wake_background() = 0; + virtual void maybe_wake_blocked_io() = 0; + virtual state_t get_state() const = 0; + + bool is_ready() const { + return get_state() >= state_t::RUNNING; + } +}; + +/** + * Callback interface for Journal + */ +class JournalTrimmer { +public: + // get the committed journal head + virtual journal_seq_t get_journal_head() const = 0; + + // set the committed journal head + virtual void set_journal_head(journal_seq_t) = 0; + + // get the committed journal dirty tail + virtual journal_seq_t get_dirty_tail() const = 0; + + // get the committed journal alloc tail + virtual journal_seq_t get_alloc_tail() const = 0; + + // set the committed journal tails + virtual void update_journal_tails( + journal_seq_t dirty_tail, journal_seq_t alloc_tail) = 0; + + // try reserve the projected usage in journal + // returns if the reservation is successful + // if the reservation is successful, user should call + // release_inline_usage to restore. + virtual bool try_reserve_inline_usage(std::size_t usage) = 0; + + // release the projected usage in journal + virtual void release_inline_usage(std::size_t usage) = 0; + + virtual ~JournalTrimmer() {} + + journal_seq_t get_journal_tail() const { + return std::min(get_alloc_tail(), get_dirty_tail()); + } + + virtual std::size_t get_trim_size_per_cycle() const = 0; + + bool check_is_ready() const { + return (get_journal_head() != JOURNAL_SEQ_NULL && + get_dirty_tail() != JOURNAL_SEQ_NULL && + get_alloc_tail() != JOURNAL_SEQ_NULL); + } + + std::size_t get_num_rolls() const { + if (!check_is_ready()) { + return 0; + } + assert(get_journal_head().segment_seq >= + get_journal_tail().segment_seq); + return get_journal_head().segment_seq + 1 - + get_journal_tail().segment_seq; + } +}; + +class BackrefManager; +class JournalTrimmerImpl; +using JournalTrimmerImplRef = std::unique_ptr<JournalTrimmerImpl>; + +/** + * Journal trimming implementation + */ +class JournalTrimmerImpl : public JournalTrimmer { +public: + struct config_t { + /// Number of minimum bytes to stop trimming dirty. + std::size_t target_journal_dirty_bytes = 0; + /// Number of minimum bytes to stop trimming allocation + /// (having the corresponding backrefs unmerged) + std::size_t target_journal_alloc_bytes = 0; + /// Number of maximum bytes to block user transactions. + std::size_t max_journal_bytes = 0; + /// Number of bytes to rewrite dirty per cycle + std::size_t rewrite_dirty_bytes_per_cycle = 0; + /// Number of bytes to rewrite backref per cycle + std::size_t rewrite_backref_bytes_per_cycle = 0; + + void validate() const; + + static config_t get_default( + std::size_t roll_size, journal_type_t type); + + static config_t get_test( + std::size_t roll_size, journal_type_t type); + }; + + JournalTrimmerImpl( + BackrefManager &backref_manager, + config_t config, + journal_type_t type, + device_off_t roll_start, + device_off_t roll_size); + + ~JournalTrimmerImpl() = default; + + /* + * JournalTrimmer interfaces + */ + + journal_seq_t get_journal_head() const final { + return journal_head; + } + + void set_journal_head(journal_seq_t) final; + + journal_seq_t get_dirty_tail() const final { + return journal_dirty_tail; + } + + journal_seq_t get_alloc_tail() const final { + return journal_alloc_tail; + } + + void update_journal_tails( + journal_seq_t dirty_tail, journal_seq_t alloc_tail) final; + + std::size_t get_trim_size_per_cycle() const final { + return config.rewrite_backref_bytes_per_cycle + + config.rewrite_dirty_bytes_per_cycle; + } + + journal_type_t get_journal_type() const { + return journal_type; + } + + void set_extent_callback(ExtentCallbackInterface *cb) { + extent_callback = cb; + } + + void set_background_callback(BackgroundListener *cb) { + background_callback = cb; + } + + void reset() { + journal_head = JOURNAL_SEQ_NULL; + journal_dirty_tail = JOURNAL_SEQ_NULL; + journal_alloc_tail = JOURNAL_SEQ_NULL; + } + + bool should_trim() const { + return should_trim_alloc() || should_trim_dirty(); + } + + bool should_block_io_on_trim() const { + return get_tail_limit() > + get_journal_tail().add_offset( + journal_type, reserved_usage, roll_start, roll_size); + } + + bool try_reserve_inline_usage(std::size_t usage) final { + reserved_usage += usage; + if (should_block_io_on_trim()) { + reserved_usage -= usage; + return false; + } else { + return true; + } + } + + void release_inline_usage(std::size_t usage) final { + ceph_assert(reserved_usage >= usage); + reserved_usage -= usage; + } + + seastar::future<> trim(); + + static JournalTrimmerImplRef create( + BackrefManager &backref_manager, + config_t config, + journal_type_t type, + device_off_t roll_start, + device_off_t roll_size) { + return std::make_unique<JournalTrimmerImpl>( + backref_manager, config, type, roll_start, roll_size); + } + + struct stat_printer_t { + const JournalTrimmerImpl &trimmer; + bool detailed = false; + }; + friend std::ostream &operator<<(std::ostream &, const stat_printer_t &); + +private: + bool should_trim_dirty() const { + return get_dirty_tail_target() > journal_dirty_tail; + } + + bool should_trim_alloc() const { + return get_alloc_tail_target() > journal_alloc_tail; + } + + using trim_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + trim_ertr::future<> trim_dirty(); + + trim_ertr::future<> trim_alloc(); + + journal_seq_t get_tail_limit() const; + journal_seq_t get_dirty_tail_target() const; + journal_seq_t get_alloc_tail_target() const; + std::size_t get_dirty_journal_size() const; + std::size_t get_alloc_journal_size() const; + void register_metrics(); + + ExtentCallbackInterface *extent_callback = nullptr; + BackgroundListener *background_callback = nullptr; + BackrefManager &backref_manager; + + config_t config; + journal_type_t journal_type; + device_off_t roll_start; + device_off_t roll_size; + + journal_seq_t journal_head; + journal_seq_t journal_dirty_tail; + journal_seq_t journal_alloc_tail; + + std::size_t reserved_usage; + + seastar::metrics::metric_group metrics; +}; + +std::ostream &operator<<( + std::ostream &, const JournalTrimmerImpl::stat_printer_t &); + +/** + * Callback interface for managing available segments + */ +class SegmentProvider { +public: + virtual const segment_info_t& get_seg_info(segment_id_t id) const = 0; + + virtual segment_id_t allocate_segment( + segment_seq_t, segment_type_t, data_category_t, rewrite_gen_t) = 0; + + virtual void close_segment(segment_id_t) = 0; + + // set the submitted segment writes in order + virtual void update_segment_avail_bytes(segment_type_t, paddr_t) = 0; + + virtual void update_modify_time( + segment_id_t, sea_time_point, std::size_t) = 0; + + virtual SegmentManagerGroup* get_segment_manager_group() = 0; + + virtual ~SegmentProvider() {} +}; + +class SpaceTrackerI { +public: + virtual int64_t allocate( + segment_id_t segment, + segment_off_t offset, + extent_len_t len) = 0; + + virtual int64_t release( + segment_id_t segment, + segment_off_t offset, + extent_len_t len) = 0; + + virtual int64_t get_usage( + segment_id_t segment) const = 0; + + virtual bool equals(const SpaceTrackerI &other) const = 0; + + virtual std::unique_ptr<SpaceTrackerI> make_empty() const = 0; + + virtual void dump_usage(segment_id_t) const = 0; + + virtual double calc_utilization(segment_id_t segment) const = 0; + + virtual void reset() = 0; + + virtual ~SpaceTrackerI() = default; +}; +using SpaceTrackerIRef = std::unique_ptr<SpaceTrackerI>; + +class SpaceTrackerSimple : public SpaceTrackerI { + struct segment_bytes_t { + int64_t live_bytes = 0; + segment_off_t total_bytes = 0; + }; + // Tracks live space for each segment + segment_map_t<segment_bytes_t> live_bytes_by_segment; + + int64_t update_usage(segment_id_t segment, int64_t delta) { + live_bytes_by_segment[segment].live_bytes += delta; + assert(live_bytes_by_segment[segment].live_bytes >= 0); + return live_bytes_by_segment[segment].live_bytes; + } +public: + SpaceTrackerSimple(const SpaceTrackerSimple &) = default; + SpaceTrackerSimple(const std::vector<SegmentManager*> &sms) { + for (auto sm : sms) { + live_bytes_by_segment.add_device( + sm->get_device_id(), + sm->get_num_segments(), + {0, sm->get_segment_size()}); + } + } + + int64_t allocate( + segment_id_t segment, + segment_off_t offset, + extent_len_t len) final { + return update_usage(segment, len); + } + + int64_t release( + segment_id_t segment, + segment_off_t offset, + extent_len_t len) final { + return update_usage(segment, -(int64_t)len); + } + + int64_t get_usage(segment_id_t segment) const final { + return live_bytes_by_segment[segment].live_bytes; + } + + double calc_utilization(segment_id_t segment) const final { + auto& seg_bytes = live_bytes_by_segment[segment]; + return (double)seg_bytes.live_bytes / (double)seg_bytes.total_bytes; + } + + void dump_usage(segment_id_t) const final; + + void reset() final { + for (auto &i : live_bytes_by_segment) { + i.second = {0, 0}; + } + } + + SpaceTrackerIRef make_empty() const final { + auto ret = SpaceTrackerIRef(new SpaceTrackerSimple(*this)); + ret->reset(); + return ret; + } + + bool equals(const SpaceTrackerI &other) const; +}; + +class SpaceTrackerDetailed : public SpaceTrackerI { + class SegmentMap { + int64_t used = 0; + segment_off_t total_bytes = 0; + std::vector<bool> bitmap; + + public: + SegmentMap( + size_t blocks, + segment_off_t total_bytes) + : total_bytes(total_bytes), + bitmap(blocks, false) {} + + int64_t update_usage(int64_t delta) { + used += delta; + return used; + } + + int64_t allocate( + device_segment_id_t segment, + segment_off_t offset, + extent_len_t len, + const extent_len_t block_size); + + int64_t release( + device_segment_id_t segment, + segment_off_t offset, + extent_len_t len, + const extent_len_t block_size); + + int64_t get_usage() const { + return used; + } + + void dump_usage(extent_len_t block_size) const; + + double calc_utilization() const { + return (double)used / (double)total_bytes; + } + + void reset() { + used = 0; + for (auto &&i: bitmap) { + i = false; + } + } + }; + + // Tracks live space for each segment + segment_map_t<SegmentMap> segment_usage; + std::vector<size_t> block_size_by_segment_manager; + +public: + SpaceTrackerDetailed(const SpaceTrackerDetailed &) = default; + SpaceTrackerDetailed(const std::vector<SegmentManager*> &sms) + { + block_size_by_segment_manager.resize(DEVICE_ID_MAX, 0); + for (auto sm : sms) { + segment_usage.add_device( + sm->get_device_id(), + sm->get_num_segments(), + SegmentMap( + sm->get_segment_size() / sm->get_block_size(), + sm->get_segment_size())); + block_size_by_segment_manager[sm->get_device_id()] = sm->get_block_size(); + } + } + + int64_t allocate( + segment_id_t segment, + segment_off_t offset, + extent_len_t len) final { + return segment_usage[segment].allocate( + segment.device_segment_id(), + offset, + len, + block_size_by_segment_manager[segment.device_id()]); + } + + int64_t release( + segment_id_t segment, + segment_off_t offset, + extent_len_t len) final { + return segment_usage[segment].release( + segment.device_segment_id(), + offset, + len, + block_size_by_segment_manager[segment.device_id()]); + } + + int64_t get_usage(segment_id_t segment) const final { + return segment_usage[segment].get_usage(); + } + + double calc_utilization(segment_id_t segment) const final { + return segment_usage[segment].calc_utilization(); + } + + void dump_usage(segment_id_t seg) const final; + + void reset() final { + for (auto &i: segment_usage) { + i.second.reset(); + } + } + + SpaceTrackerIRef make_empty() const final { + auto ret = SpaceTrackerIRef(new SpaceTrackerDetailed(*this)); + ret->reset(); + return ret; + } + + bool equals(const SpaceTrackerI &other) const; +}; + +template <typename T> +class block_map_t { +public: + block_map_t() { + device_to_blocks.resize(DEVICE_ID_MAX_VALID); + device_block_size.resize(DEVICE_ID_MAX_VALID); + } + void add_device(device_id_t device, std::size_t blocks, const T& init, + size_t block_size) { + ceph_assert(device <= DEVICE_ID_MAX_VALID); + ceph_assert(device_to_blocks[device].size() == 0); + ceph_assert(blocks > 0); + device_to_blocks[device].resize(blocks, init); + total_blocks += blocks; + device_block_size[device] = block_size; + } + void clear() { + device_to_blocks.clear(); + device_to_blocks.resize(DEVICE_ID_MAX_VALID); + total_blocks = 0; + } + + T& operator[](paddr_t block) { + ceph_assert(device_to_blocks[block.get_device_id()].size() != 0); + auto &blk = block.as_blk_paddr(); + auto block_id = get_block_id(block.get_device_id(), blk.get_device_off()); + return device_to_blocks[block.get_device_id()][block_id]; + } + const T& operator[](paddr_t block) const { + ceph_assert(device_to_blocks[block.get_device_id()].size() != 0); + auto &blk = block.as_blk_paddr(); + auto block_id = get_block_id(block.get_device_id(), blk.get_device_off()); + return device_to_blocks[block.get_device_id()][block_id]; + } + + auto begin() { + return iterator<false>::lower_bound(*this, 0, 0); + } + auto begin() const { + return iterator<true>::lower_bound(*this, 0, 0); + } + + auto end() { + return iterator<false>::end_iterator(*this); + } + auto end() const { + return iterator<true>::end_iterator(*this); + } + + size_t size() const { + return total_blocks; + } + + uint64_t get_block_size(device_id_t device_id) { + return device_block_size[device_id]; + } + + uint32_t get_block_id(device_id_t device_id, device_off_t blk_off) const { + auto block_size = device_block_size[device_id]; + return blk_off == 0 ? 0 : blk_off/block_size; + } + + template <bool is_const = false> + class iterator { + /// points at set being iterated over + std::conditional_t< + is_const, + const block_map_t &, + block_map_t &> parent; + + /// points at current device, or DEVICE_ID_MAX_VALID if is_end() + device_id_t device_id; + + /// segment at which we are pointing, 0 if is_end() + device_off_t blk_off; + + /// holds referent for operator* and operator-> when !is_end() + std::optional< + std::pair< + const device_off_t, + std::conditional_t<is_const, const T&, T&> + >> current; + + bool is_end() const { + return device_id == DEVICE_ID_MAX_VALID; + } + + uint32_t get_block_id() { + return parent.get_block_id(device_id, blk_off); + } + + void find_valid() { + assert(!is_end()); + auto &device_vec = parent.device_to_blocks[device_id]; + if (device_vec.size() == 0 || + get_block_id() == device_vec.size()) { + while (++device_id < DEVICE_ID_MAX_VALID&& + parent.device_to_blocks[device_id].size() == 0); + blk_off = 0; + } + if (is_end()) { + current = std::nullopt; + } else { + current.emplace( + blk_off, + parent.device_to_blocks[device_id][get_block_id()] + ); + } + } + + iterator( + decltype(parent) &parent, + device_id_t device_id, + device_off_t device_block_off) + : parent(parent), device_id(device_id), + blk_off(device_block_off) {} + + public: + static iterator lower_bound( + decltype(parent) &parent, + device_id_t device_id, + device_off_t block_off) { + if (device_id == DEVICE_ID_MAX_VALID) { + return end_iterator(parent); + } else { + auto ret = iterator{parent, device_id, block_off}; + ret.find_valid(); + return ret; + } + } + + static iterator end_iterator( + decltype(parent) &parent) { + return iterator{parent, DEVICE_ID_MAX_VALID, 0}; + } + + iterator<is_const>& operator++() { + assert(!is_end()); + auto block_size = parent.device_block_size[device_id]; + blk_off += block_size; + find_valid(); + return *this; + } + + bool operator==(iterator<is_const> rit) { + return (device_id == rit.device_id && + blk_off == rit.blk_off); + } + + bool operator!=(iterator<is_const> rit) { + return !(*this == rit); + } + template <bool c = is_const, std::enable_if_t<c, int> = 0> + const std::pair<const device_off_t, const T&> *operator->() { + assert(!is_end()); + return &*current; + } + template <bool c = is_const, std::enable_if_t<!c, int> = 0> + std::pair<const device_off_t, T&> *operator->() { + assert(!is_end()); + return &*current; + } + template <bool c = is_const, std::enable_if_t<c, int> = 0> + const std::pair<const device_off_t, const T&> &operator*() { + assert(!is_end()); + return *current; + } + template <bool c = is_const, std::enable_if_t<!c, int> = 0> + std::pair<const device_off_t, T&> &operator*() { + assert(!is_end()); + return *current; + } + }; + std::vector<std::vector<T>> device_to_blocks; + std::vector<size_t> device_block_size; + size_t total_blocks = 0; +}; + +class RBMSpaceTracker { + struct random_block_t { + bool used = false; + void allocate() { + used = true; + } + void release() { + used = false; + } + }; + block_map_t<random_block_t> block_usage; + +public: + RBMSpaceTracker(const RBMSpaceTracker &) = default; + RBMSpaceTracker(const std::vector<RandomBlockManager*> &rbms) { + for (auto rbm : rbms) { + block_usage.add_device( + rbm->get_device_id(), + rbm->get_device()->get_available_size() / rbm->get_block_size(), + {false}, + rbm->get_block_size()); + } + } + + void allocate( + paddr_t addr, + extent_len_t len) { + paddr_t cursor = addr; + paddr_t end = addr.add_offset(len); + do { + block_usage[cursor].allocate(); + cursor = cursor.add_offset( + block_usage.get_block_size(addr.get_device_id())); + } while (cursor < end); + } + + void release( + paddr_t addr, + extent_len_t len) { + paddr_t cursor = addr; + paddr_t end = addr.add_offset(len); + do { + block_usage[cursor].release(); + cursor = cursor.add_offset( + block_usage.get_block_size(addr.get_device_id())); + } while (cursor < end); + } + + void reset() { + for (auto &i : block_usage) { + i.second = {false}; + } + } + + std::unique_ptr<RBMSpaceTracker> make_empty() const { + auto ret = std::make_unique<RBMSpaceTracker>(*this); + ret->reset(); + return ret; + } + friend class RBMCleaner; +}; +using RBMSpaceTrackerRef = std::unique_ptr<RBMSpaceTracker>; + +/* + * AsyncCleaner + * + * Interface for ExtentPlacementManager::BackgroundProcess + * to do background cleaning. + */ +class AsyncCleaner { +public: + using state_t = BackgroundListener::state_t; + using base_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + + virtual void set_background_callback(BackgroundListener *) = 0; + + virtual void set_extent_callback(ExtentCallbackInterface *) = 0; + + virtual store_statfs_t get_stat() const = 0; + + virtual void print(std::ostream &, bool is_detailed) const = 0; + + virtual bool check_usage_is_empty() const = 0; + + using mount_ertr = base_ertr; + using mount_ret = mount_ertr::future<>; + virtual mount_ret mount() = 0; + + virtual void mark_space_used(paddr_t, extent_len_t) = 0; + + virtual void mark_space_free(paddr_t, extent_len_t) = 0; + + virtual void commit_space_used(paddr_t, extent_len_t) = 0; + + // try reserve the projected usage in cleaner + // returns if the reservation is successful + // if the reservation is successful, user should call + // release_projected_usage to restore. + virtual bool try_reserve_projected_usage(std::size_t) = 0; + + virtual void release_projected_usage(std::size_t) = 0; + + virtual bool should_block_io_on_clean() const = 0; + + virtual bool can_clean_space() const = 0; + + virtual bool should_clean_space() const = 0; + + using clean_space_ertr = base_ertr; + using clean_space_ret = clean_space_ertr::future<>; + virtual clean_space_ret clean_space() = 0; + + virtual const std::set<device_id_t>& get_device_ids() const = 0; + + virtual std::size_t get_reclaim_size_per_cycle() const = 0; + + // test only + virtual bool check_usage() = 0; + + struct stat_printer_t { + const AsyncCleaner &cleaner; + bool detailed = false; + }; + + virtual ~AsyncCleaner() {} +}; + +using AsyncCleanerRef = std::unique_ptr<AsyncCleaner>; + +std::ostream &operator<<( + std::ostream &, const AsyncCleaner::stat_printer_t &); + +class SegmentCleaner; +using SegmentCleanerRef = std::unique_ptr<SegmentCleaner>; + +class SegmentCleaner : public SegmentProvider, public AsyncCleaner { +public: + /// Config + struct config_t { + /// Ratio of maximum available space to disable reclaiming. + double available_ratio_gc_max = 0; + /// Ratio of minimum available space to force reclaiming. + double available_ratio_hard_limit = 0; + /// Ratio of minimum reclaimable space to stop reclaiming. + double reclaim_ratio_gc_threshold = 0; + /// Number of bytes to reclaim per cycle + std::size_t reclaim_bytes_per_cycle = 0; + + void validate() const { + ceph_assert(available_ratio_gc_max > available_ratio_hard_limit); + ceph_assert(reclaim_bytes_per_cycle > 0); + } + + static config_t get_default() { + return config_t{ + .15, // available_ratio_gc_max + .1, // available_ratio_hard_limit + .1, // reclaim_ratio_gc_threshold + 1<<20 // reclaim_bytes_per_cycle + }; + } + + static config_t get_test() { + return config_t{ + .99, // available_ratio_gc_max + .2, // available_ratio_hard_limit + .6, // reclaim_ratio_gc_threshold + 1<<20 // reclaim_bytes_per_cycle + }; + } + }; + + SegmentCleaner( + config_t config, + SegmentManagerGroupRef&& sm_group, + BackrefManager &backref_manager, + SegmentSeqAllocator &segment_seq_allocator, + bool detailed, + bool is_cold); + + void set_journal_trimmer(JournalTrimmer &_trimmer) { + trimmer = &_trimmer; + } + + static SegmentCleanerRef create( + config_t config, + SegmentManagerGroupRef&& sm_group, + BackrefManager &backref_manager, + SegmentSeqAllocator &ool_seq_allocator, + bool detailed, + bool is_cold = false) { + return std::make_unique<SegmentCleaner>( + config, std::move(sm_group), backref_manager, + ool_seq_allocator, detailed, is_cold); + } + + /* + * SegmentProvider interfaces + */ + + const segment_info_t& get_seg_info(segment_id_t id) const final { + return segments[id]; + } + + segment_id_t allocate_segment( + segment_seq_t, segment_type_t, data_category_t, rewrite_gen_t) final; + + void close_segment(segment_id_t segment) final; + + void update_segment_avail_bytes(segment_type_t type, paddr_t offset) final { + assert(type == segment_type_t::OOL || + trimmer != nullptr); // segment_type_t::JOURNAL + segments.update_written_to(type, offset); + background_callback->maybe_wake_background(); + } + + void update_modify_time( + segment_id_t id, sea_time_point tp, std::size_t num_extents) final { + ceph_assert(num_extents == 0 || tp != NULL_TIME); + segments.update_modify_time(id, tp, num_extents); + } + + SegmentManagerGroup* get_segment_manager_group() final { + return sm_group.get(); + } + + /* + * AsyncCleaner interfaces + */ + + void set_background_callback(BackgroundListener *cb) final { + background_callback = cb; + } + + void set_extent_callback(ExtentCallbackInterface *cb) final { + extent_callback = cb; + } + + store_statfs_t get_stat() const final { + store_statfs_t st; + st.total = segments.get_total_bytes(); + st.available = segments.get_total_bytes() - stats.used_bytes; + st.allocated = stats.used_bytes; + st.data_stored = stats.used_bytes; + + // TODO add per extent type counters for omap_allocated and + // internal metadata + return st; + } + + void print(std::ostream &, bool is_detailed) const final; + + bool check_usage_is_empty() const final { + return space_tracker->equals(*space_tracker->make_empty()); + } + + mount_ret mount() final; + + void mark_space_used(paddr_t, extent_len_t) final; + + void mark_space_free(paddr_t, extent_len_t) final; + + void commit_space_used(paddr_t addr, extent_len_t len) final { + mark_space_used(addr, len); + } + + bool try_reserve_projected_usage(std::size_t) final; + + void release_projected_usage(size_t) final; + + bool should_block_io_on_clean() const final { + assert(background_callback->is_ready()); + if (get_segments_reclaimable() == 0) { + return false; + } + auto aratio = get_projected_available_ratio(); + return aratio < config.available_ratio_hard_limit; + } + + bool can_clean_space() const final { + assert(background_callback->is_ready()); + return get_segments_reclaimable() > 0; + } + + bool should_clean_space() const final { + assert(background_callback->is_ready()); + if (get_segments_reclaimable() == 0) { + return false; + } + auto aratio = segments.get_available_ratio(); + auto rratio = get_reclaim_ratio(); + return ( + (aratio < config.available_ratio_hard_limit) || + ((aratio < config.available_ratio_gc_max) && + (rratio > config.reclaim_ratio_gc_threshold)) + ); + } + + clean_space_ret clean_space() final; + + const std::set<device_id_t>& get_device_ids() const final { + return sm_group->get_device_ids(); + } + + std::size_t get_reclaim_size_per_cycle() const final { + return config.reclaim_bytes_per_cycle; + } + + // Testing interfaces + + bool check_usage() final; + +private: + /* + * 10 buckets for the number of closed segments by usage + * 2 extra buckets for the number of open and empty segments + */ + static constexpr double UTIL_STATE_OPEN = 1.05; + static constexpr double UTIL_STATE_EMPTY = 1.15; + static constexpr std::size_t UTIL_BUCKETS = 12; + static std::size_t get_bucket_index(double util) { + auto index = std::floor(util * 10); + assert(index < UTIL_BUCKETS); + return index; + } + double calc_utilization(segment_id_t id) const { + auto& info = segments[id]; + if (info.is_open()) { + return UTIL_STATE_OPEN; + } else if (info.is_empty()) { + return UTIL_STATE_EMPTY; + } else { + auto ret = space_tracker->calc_utilization(id); + assert(ret >= 0 && ret < 1); + return ret; + } + } + + // journal status helpers + + double calc_gc_benefit_cost( + segment_id_t id, + const sea_time_point &now_time, + const sea_time_point &bound_time) const; + + segment_id_t get_next_reclaim_segment() const; + + struct reclaim_state_t { + rewrite_gen_t generation; + rewrite_gen_t target_generation; + segment_off_t segment_size; + paddr_t start_pos; + paddr_t end_pos; + + static reclaim_state_t create( + segment_id_t segment_id, + rewrite_gen_t generation, + segment_off_t segment_size) { + ceph_assert(is_rewrite_generation(generation)); + + rewrite_gen_t target_gen; + if (generation < MIN_REWRITE_GENERATION) { + target_gen = MIN_REWRITE_GENERATION; + } else { + // tolerate the target_gen to exceed MAX_REWRETE_GENERATION to make EPM + // aware of its original generation for the decisions. + target_gen = generation + 1; + } + + assert(is_target_rewrite_generation(target_gen)); + return {generation, + target_gen, + segment_size, + P_ADDR_NULL, + paddr_t::make_seg_paddr(segment_id, 0)}; + } + + segment_id_t get_segment_id() const { + return end_pos.as_seg_paddr().get_segment_id(); + } + + bool is_complete() const { + return end_pos.as_seg_paddr().get_segment_off() >= segment_size; + } + + void advance(std::size_t bytes) { + assert(!is_complete()); + start_pos = end_pos; + auto &end_seg_paddr = end_pos.as_seg_paddr(); + auto next_off = end_seg_paddr.get_segment_off() + bytes; + if (next_off > (std::size_t)segment_size) { + end_seg_paddr.set_segment_off(segment_size); + } else { + end_seg_paddr.set_segment_off(next_off); + } + } + }; + std::optional<reclaim_state_t> reclaim_state; + + using do_reclaim_space_ertr = base_ertr; + using do_reclaim_space_ret = do_reclaim_space_ertr::future<>; + do_reclaim_space_ret do_reclaim_space( + const std::vector<CachedExtentRef> &backref_extents, + const backref_pin_list_t &pin_list, + std::size_t &reclaimed, + std::size_t &runs); + + /* + * Segments calculations + */ + std::size_t get_segments_in_journal() const { + if (trimmer != nullptr) { + return trimmer->get_num_rolls(); + } else { + return 0; + } + } + std::size_t get_segments_in_journal_closed() const { + auto in_journal = get_segments_in_journal(); + auto in_journal_open = segments.get_num_in_journal_open(); + if (in_journal >= in_journal_open) { + return in_journal - in_journal_open; + } else { + return 0; + } + } + std::size_t get_segments_reclaimable() const { + assert(segments.get_num_closed() >= get_segments_in_journal_closed()); + return segments.get_num_closed() - get_segments_in_journal_closed(); + } + + /* + * Space calculations + */ + /// the unavailable space that is not reclaimable yet + std::size_t get_unavailable_unreclaimable_bytes() const { + auto ret = (segments.get_num_open() + get_segments_in_journal_closed()) * + segments.get_segment_size(); + assert(ret >= segments.get_available_bytes_in_open()); + return ret - segments.get_available_bytes_in_open(); + } + /// the unavailable space that can be reclaimed + std::size_t get_unavailable_reclaimable_bytes() const { + auto ret = get_segments_reclaimable() * segments.get_segment_size(); + ceph_assert(ret + get_unavailable_unreclaimable_bytes() == segments.get_unavailable_bytes()); + return ret; + } + /// the unavailable space that is not alive + std::size_t get_unavailable_unused_bytes() const { + assert(segments.get_unavailable_bytes() > stats.used_bytes); + return segments.get_unavailable_bytes() - stats.used_bytes; + } + double get_reclaim_ratio() const { + if (segments.get_unavailable_bytes() == 0) return 0; + return (double)get_unavailable_unused_bytes() / (double)segments.get_unavailable_bytes(); + } + double get_alive_ratio() const { + return stats.used_bytes / (double)segments.get_total_bytes(); + } + + /* + * Space calculations (projected) + */ + std::size_t get_projected_available_bytes() const { + return (segments.get_available_bytes() > stats.projected_used_bytes) ? + segments.get_available_bytes() - stats.projected_used_bytes: + 0; + } + double get_projected_available_ratio() const { + return (double)get_projected_available_bytes() / + (double)segments.get_total_bytes(); + } + + using scan_extents_ertr = SegmentManagerGroup::scan_valid_records_ertr; + using scan_extents_ret = scan_extents_ertr::future<>; + scan_extents_ret scan_no_tail_segment( + const segment_header_t& header, + segment_id_t segment_id); + + void adjust_segment_util(double old_usage, double new_usage) { + auto old_index = get_bucket_index(old_usage); + auto new_index = get_bucket_index(new_usage); + assert(stats.segment_util.buckets[old_index].count > 0); + stats.segment_util.buckets[old_index].count--; + stats.segment_util.buckets[new_index].count++; + } + + void init_mark_segment_closed( + segment_id_t segment, + segment_seq_t seq, + segment_type_t s_type, + data_category_t category, + rewrite_gen_t generation) { + assert(background_callback->get_state() == state_t::MOUNT); + ceph_assert(s_type == segment_type_t::OOL || + trimmer != nullptr); // segment_type_t::JOURNAL + auto old_usage = calc_utilization(segment); + segments.init_closed(segment, seq, s_type, category, generation); + auto new_usage = calc_utilization(segment); + adjust_segment_util(old_usage, new_usage); + if (s_type == segment_type_t::OOL) { + ool_segment_seq_allocator.set_next_segment_seq(seq); + } + } + + const bool detailed; + const bool is_cold; + const config_t config; + + SegmentManagerGroupRef sm_group; + BackrefManager &backref_manager; + + SpaceTrackerIRef space_tracker; + segments_info_t segments; + + struct { + /** + * used_bytes + * + * Bytes occupied by live extents + */ + uint64_t used_bytes = 0; + + /** + * projected_used_bytes + * + * Sum of projected bytes used by each transaction between throttle + * acquisition and commit completion. See try_reserve_projected_usage() + */ + uint64_t projected_used_bytes = 0; + uint64_t projected_count = 0; + uint64_t projected_used_bytes_sum = 0; + + uint64_t closed_journal_used_bytes = 0; + uint64_t closed_journal_total_bytes = 0; + uint64_t closed_ool_used_bytes = 0; + uint64_t closed_ool_total_bytes = 0; + + uint64_t reclaiming_bytes = 0; + uint64_t reclaimed_bytes = 0; + uint64_t reclaimed_segment_bytes = 0; + + seastar::metrics::histogram segment_util; + } stats; + seastar::metrics::metric_group metrics; + void register_metrics(); + + // optional, set if this cleaner is assigned to SegmentedJournal + JournalTrimmer *trimmer = nullptr; + + ExtentCallbackInterface *extent_callback = nullptr; + + BackgroundListener *background_callback = nullptr; + + // TODO: drop once paddr->journal_seq_t is introduced + SegmentSeqAllocator &ool_segment_seq_allocator; +}; + +class RBMCleaner; +using RBMCleanerRef = std::unique_ptr<RBMCleaner>; + +class RBMCleaner : public AsyncCleaner { +public: + RBMCleaner( + RBMDeviceGroupRef&& rb_group, + BackrefManager &backref_manager, + bool detailed); + + static RBMCleanerRef create( + RBMDeviceGroupRef&& rb_group, + BackrefManager &backref_manager, + bool detailed) { + return std::make_unique<RBMCleaner>( + std::move(rb_group), backref_manager, detailed); + } + + RBMDeviceGroup* get_rb_group() { + return rb_group.get(); + } + + /* + * AsyncCleaner interfaces + */ + + void set_background_callback(BackgroundListener *cb) final { + background_callback = cb; + } + + void set_extent_callback(ExtentCallbackInterface *cb) final { + extent_callback = cb; + } + + store_statfs_t get_stat() const final { + store_statfs_t st; + st.total = get_total_bytes(); + st.available = get_total_bytes() - get_journal_bytes() - stats.used_bytes; + st.allocated = get_journal_bytes() + stats.used_bytes; + st.data_stored = get_journal_bytes() + stats.used_bytes; + return st; + } + + void print(std::ostream &, bool is_detailed) const final; + + mount_ret mount() final; + + void mark_space_used(paddr_t, extent_len_t) final; + + void mark_space_free(paddr_t, extent_len_t) final; + + void commit_space_used(paddr_t, extent_len_t) final; + + bool try_reserve_projected_usage(std::size_t) final; + + void release_projected_usage(size_t) final; + + bool should_block_io_on_clean() const final { + return false; + } + + bool can_clean_space() const final { + return false; + } + + bool should_clean_space() const final { + return false; + } + + clean_space_ret clean_space() final; + + const std::set<device_id_t>& get_device_ids() const final { + return rb_group->get_device_ids(); + } + + std::size_t get_reclaim_size_per_cycle() const final { + return 0; + } + + RandomBlockManager* get_rbm(paddr_t paddr) { + auto rbs = rb_group->get_rb_managers(); + for (auto p : rbs) { + if (p->get_device_id() == paddr.get_device_id()) { + return p; + } + } + return nullptr; + } + + paddr_t alloc_paddr(extent_len_t length) { + // TODO: implement allocation strategy (dirty metadata and multiple devices) + auto rbs = rb_group->get_rb_managers(); + auto paddr = rbs[0]->alloc_extent(length); + stats.used_bytes += length; + return paddr; + } + + size_t get_total_bytes() const { + auto rbs = rb_group->get_rb_managers(); + size_t total = 0; + for (auto p : rbs) { + total += p->get_device()->get_available_size(); + } + return total; + } + + size_t get_journal_bytes() const { + auto rbs = rb_group->get_rb_managers(); + size_t total = 0; + for (auto p : rbs) { + total += p->get_journal_size(); + } + return total; + } + + // Testing interfaces + + bool check_usage() final; + + bool check_usage_is_empty() const final { + // TODO + return true; + } + +private: + bool equals(const RBMSpaceTracker &other) const; + + const bool detailed; + RBMDeviceGroupRef rb_group; + BackrefManager &backref_manager; + + struct { + /** + * used_bytes + * + * Bytes occupied by live extents + */ + uint64_t used_bytes = 0; + + /** + * projected_used_bytes + * + * Sum of projected bytes used by each transaction between throttle + * acquisition and commit completion. See reserve_projected_usage() + */ + uint64_t projected_used_bytes = 0; + } stats; + seastar::metrics::metric_group metrics; + void register_metrics(); + + ExtentCallbackInterface *extent_callback = nullptr; + BackgroundListener *background_callback = nullptr; +}; +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::segment_info_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::segments_info_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::AsyncCleaner::stat_printer_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::JournalTrimmerImpl::stat_printer_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/backref/backref_tree_node.cc b/src/crimson/os/seastore/backref/backref_tree_node.cc new file mode 100644 index 000000000..513c29994 --- /dev/null +++ b/src/crimson/os/seastore/backref/backref_tree_node.cc @@ -0,0 +1,14 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/backref/backref_tree_node.h" + +namespace crimson::os::seastore::backref { + +std::ostream& operator<<(std::ostream &out, const backref_map_val_t& val) { + return out << "backref_map_val_t(" + << val.laddr + << "~" << val.len << ")"; +} + +} // namespace crimson::os::seastore::backref diff --git a/src/crimson/os/seastore/backref/backref_tree_node.h b/src/crimson/os/seastore/backref/backref_tree_node.h new file mode 100644 index 000000000..c3ff52520 --- /dev/null +++ b/src/crimson/os/seastore/backref/backref_tree_node.h @@ -0,0 +1,137 @@ +// -*- 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/btree/fixed_kv_node.h" + +namespace crimson::os::seastore::backref { + +using backref_node_meta_t = fixed_kv_node_meta_t<paddr_t>; +using backref_node_meta_le_t = fixed_kv_node_meta_le_t<paddr_t>; + +constexpr size_t INTERNAL_NODE_CAPACITY = 254; +constexpr size_t LEAF_NODE_CAPACITY = 169; + +using BackrefNode = FixedKVNode<paddr_t>; + +struct backref_map_val_t { + extent_len_t len = 0; ///< length of extents + laddr_t laddr = 0; ///< logical address of extents + extent_types_t type = extent_types_t::ROOT; + + backref_map_val_t() = default; + backref_map_val_t( + extent_len_t len, + laddr_t laddr, + extent_types_t type) + : len(len), laddr(laddr), type(type) {} + + bool operator==(const backref_map_val_t& rhs) const noexcept { + return len == rhs.len && laddr == rhs.laddr; + } +}; + +std::ostream& operator<<(std::ostream &out, const backref_map_val_t& val); + +struct backref_map_val_le_t { + extent_len_le_t len = init_extent_len_le(0); + laddr_le_t laddr = laddr_le_t(0); + extent_types_le_t type = 0; + + backref_map_val_le_t() = default; + backref_map_val_le_t(const backref_map_val_le_t &) = default; + explicit backref_map_val_le_t(const backref_map_val_t &val) + : len(init_extent_len_le(val.len)), + laddr(val.laddr), + type(extent_types_le_t(val.type)) {} + + operator backref_map_val_t() const { + return backref_map_val_t{len, laddr, (extent_types_t)type}; + } +}; + +class BackrefInternalNode + : public FixedKVInternalNode< + INTERNAL_NODE_CAPACITY, + paddr_t, paddr_le_t, + BACKREF_NODE_SIZE, + BackrefInternalNode> { +public: + template <typename... T> + BackrefInternalNode(T&&... t) : + FixedKVInternalNode(std::forward<T>(t)...) {} + + static constexpr extent_types_t TYPE = extent_types_t::BACKREF_INTERNAL; + + extent_types_t get_type() const final { + return TYPE; + } +}; +using BackrefInternalNodeRef = BackrefInternalNode::Ref; + +class BackrefLeafNode + : public FixedKVLeafNode< + LEAF_NODE_CAPACITY, + paddr_t, paddr_le_t, + backref_map_val_t, backref_map_val_le_t, + BACKREF_NODE_SIZE, + BackrefLeafNode, + false> { +public: + template <typename... T> + BackrefLeafNode(T&&... t) : + FixedKVLeafNode(std::forward<T>(t)...) {} + + static constexpr extent_types_t TYPE = extent_types_t::BACKREF_LEAF; + + extent_types_t get_type() const final { + return TYPE; + } + + const_iterator insert( + const_iterator iter, + paddr_t key, + backref_map_val_t val, + LogicalCachedExtent*) final { + journal_insert( + iter, + key, + val, + maybe_get_delta_buffer()); + return iter; + } + + void update( + const_iterator iter, + backref_map_val_t val, + LogicalCachedExtent*) final { + return journal_update( + iter, + val, + maybe_get_delta_buffer()); + } + + void remove(const_iterator iter) final { + return journal_remove( + iter, + maybe_get_delta_buffer()); + } + + // backref leaf nodes don't have to resolve relative addresses + void resolve_relative_addrs(paddr_t base) final {} + + void node_resolve_vals(iterator from, iterator to) const final {} + + void node_unresolve_vals(iterator from, iterator to) const final {} +}; +using BackrefLeafNodeRef = BackrefLeafNode::Ref; + +} // namespace crimson::os::seastore::backref + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::backref::backref_map_val_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::backref::BackrefInternalNode> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::backref::BackrefLeafNode> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::backref::backref_node_meta_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/backref/btree_backref_manager.cc b/src/crimson/os/seastore/backref/btree_backref_manager.cc new file mode 100644 index 000000000..30ff45540 --- /dev/null +++ b/src/crimson/os/seastore/backref/btree_backref_manager.cc @@ -0,0 +1,609 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/backref/btree_backref_manager.h" + +SET_SUBSYS(seastore_backref); + +namespace crimson::os::seastore { + +template<> +Transaction::tree_stats_t& get_tree_stats< + crimson::os::seastore::backref::BackrefBtree>(Transaction &t) { + return t.get_backref_tree_stats(); +} + +template<> +phy_tree_root_t& get_phy_tree_root< + crimson::os::seastore::backref::BackrefBtree>(root_t &r) { + return r.backref_root; +} + +template<> +const get_phy_tree_root_node_ret get_phy_tree_root_node< + crimson::os::seastore::backref::BackrefBtree>( + const RootBlockRef &root_block, op_context_t<paddr_t> c) { + auto backref_root = root_block->backref_root_node; + if (backref_root) { + ceph_assert(backref_root->is_initial_pending() + == root_block->is_pending()); + return {true, + trans_intr::make_interruptible( + c.cache.get_extent_viewable_by_trans(c.trans, backref_root))}; + } else if (root_block->is_pending()) { + auto &prior = static_cast<RootBlock&>(*root_block->get_prior_instance()); + backref_root = prior.backref_root_node; + if (backref_root) { + return {true, + trans_intr::make_interruptible( + c.cache.get_extent_viewable_by_trans(c.trans, backref_root))}; + } else { + return {false, + trans_intr::make_interruptible( + Cache::get_extent_ertr::make_ready_future< + CachedExtentRef>())}; + } + } else { + return {false, + trans_intr::make_interruptible( + Cache::get_extent_ertr::make_ready_future< + CachedExtentRef>())}; + } +} + +template <typename ROOT> +void link_phy_tree_root_node(RootBlockRef &root_block, ROOT* backref_root) { + root_block->backref_root_node = backref_root; + ceph_assert(backref_root != nullptr); + backref_root->root_block = root_block; +} + +template void link_phy_tree_root_node( + RootBlockRef &root_block, backref::BackrefInternalNode* backref_root); +template void link_phy_tree_root_node( + RootBlockRef &root_block, backref::BackrefLeafNode* backref_root); +template void link_phy_tree_root_node( + RootBlockRef &root_block, backref::BackrefNode* backref_root); + +template <> +void unlink_phy_tree_root_node<paddr_t>(RootBlockRef &root_block) { + root_block->backref_root_node = nullptr; +} + +} + +namespace crimson::os::seastore::backref { + +BtreeBackrefManager::mkfs_ret +BtreeBackrefManager::mkfs( + Transaction &t) +{ + LOG_PREFIX(BtreeBackrefManager::mkfs); + INFOT("start", t); + return cache.get_root(t).si_then([this, &t](auto croot) { + assert(croot->is_mutation_pending()); + croot->get_root().backref_root = BackrefBtree::mkfs(croot, get_context(t)); + return mkfs_iertr::now(); + }).handle_error_interruptible( + mkfs_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in BtreeBackrefManager::mkfs" + } + ); +} + +BtreeBackrefManager::get_mapping_ret +BtreeBackrefManager::get_mapping( + Transaction &t, + paddr_t offset) +{ + LOG_PREFIX(BtreeBackrefManager::get_mapping); + TRACET("{}", t, offset); + auto c = get_context(t); + return with_btree_ret<BackrefBtree, BackrefMappingRef>( + cache, + c, + [c, offset](auto &btree) { + return btree.lower_bound( + c, offset + ).si_then([offset, c](auto iter) -> get_mapping_ret { + LOG_PREFIX(BtreeBackrefManager::get_mapping); + if (iter.is_end() || iter.get_key() != offset) { + ERRORT("{} doesn't exist", c.trans, offset); + return crimson::ct_error::enoent::make(); + } else { + TRACET("{} got {}, {}", + c.trans, offset, iter.get_key(), iter.get_val()); + return get_mapping_ret( + interruptible::ready_future_marker{}, + iter.get_pin(c)); + } + }); + }); +} + +BtreeBackrefManager::get_mappings_ret +BtreeBackrefManager::get_mappings( + Transaction &t, + paddr_t offset, + paddr_t end) +{ + LOG_PREFIX(BtreeBackrefManager::get_mappings); + TRACET("{}~{}", t, offset, end); + auto c = get_context(t); + return with_btree_state<BackrefBtree, backref_pin_list_t>( + cache, + c, + [c, offset, end](auto &btree, auto &ret) { + return BackrefBtree::iterate_repeat( + c, + btree.upper_bound_right(c, offset), + [&ret, offset, end, c](auto &pos) { + LOG_PREFIX(BtreeBackrefManager::get_mappings); + if (pos.is_end() || pos.get_key() >= end) { + TRACET("{}~{} done with {} results", + c.trans, offset, end, ret.size()); + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } + TRACET("{}~{} got {}, {}, repeat ...", + c.trans, offset, end, pos.get_key(), pos.get_val()); + ceph_assert((pos.get_key().add_offset(pos.get_val().len)) > offset); + ret.emplace_back(pos.get_pin(c)); + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::no); + }); + }); +} + +BtreeBackrefManager::new_mapping_ret +BtreeBackrefManager::new_mapping( + Transaction &t, + paddr_t key, + extent_len_t len, + laddr_t addr, + extent_types_t type) +{ + ceph_assert( + is_aligned( + key.get_addr_type() == paddr_types_t::SEGMENT ? + key.as_seg_paddr().get_segment_off() : + key.as_blk_paddr().get_device_off(), + cache.get_block_size())); + struct state_t { + paddr_t last_end; + + std::optional<BackrefBtree::iterator> insert_iter; + std::optional<BackrefBtree::iterator> ret; + + state_t(paddr_t hint) : last_end(hint) {} + }; + + LOG_PREFIX(BtreeBackrefManager::new_mapping); + DEBUGT("{}~{}, paddr={}", t, addr, len, key); + backref_map_val_t val{len, addr, type}; + auto c = get_context(t); + //++stats.num_alloc_extents; + //auto lookup_attempts = stats.num_alloc_extents_iter_nexts; + return crimson::os::seastore::with_btree_state<BackrefBtree, state_t>( + cache, + c, + key, + [val, c, key, len, addr, /*lookup_attempts,*/ &t] + (auto &btree, auto &state) { + return BackrefBtree::iterate_repeat( + c, + btree.upper_bound_right(c, key), + [&state, len, addr, &t, key/*, lookup_attempts*/](auto &pos) { + LOG_PREFIX(BtreeBackrefManager::new_mapping); + //++stats.num_alloc_extents_iter_nexts; + if (pos.is_end()) { + DEBUGT("{}~{}, paddr={}, state: end, insert at {}", + t, addr, len, key, + //stats.num_alloc_extents_iter_nexts - lookup_attempts, + state.last_end); + state.insert_iter = pos; + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } else if (pos.get_key() >= (state.last_end.add_offset(len))) { + DEBUGT("{}~{}, paddr={}, state: {}~{}, " + "insert at {} -- {}", + t, addr, len, key, + pos.get_key(), pos.get_val().len, + //stats.num_alloc_extents_iter_nexts - lookup_attempts, + state.last_end, + pos.get_val()); + state.insert_iter = pos; + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } else { + ERRORT("{}~{}, paddr={}, state: {}~{}, repeat ... -- {}", + t, addr, len, key, + pos.get_key(), pos.get_val().len, + pos.get_val()); + ceph_abort("not possible for the backref tree"); + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::no); + } + }).si_then([c, addr, len, key, &btree, &state, val] { + return btree.insert( + c, + *state.insert_iter, + state.last_end, + val, + nullptr + ).si_then([&state, c, addr, len, key](auto &&p) { + LOG_PREFIX(BtreeBackrefManager::new_mapping); + auto [iter, inserted] = std::move(p); + TRACET("{}~{}, paddr={}, inserted at {}, leaf {}", + c.trans, addr, len, key, state.last_end, *iter.get_leaf_node()); + ceph_assert(inserted); + state.ret = iter; + }); + }); + }).si_then([c](auto &&state) { + return new_mapping_iertr::make_ready_future<BackrefMappingRef>( + state.ret->get_pin(c)); + }); +} + +BtreeBackrefManager::merge_cached_backrefs_ret +BtreeBackrefManager::merge_cached_backrefs( + Transaction &t, + const journal_seq_t &limit, + const uint64_t max) +{ + LOG_PREFIX(BtreeBackrefManager::merge_cached_backrefs); + DEBUGT("insert up to {}", t, limit); + return seastar::do_with( + limit, + JOURNAL_SEQ_NULL, + [this, &t, max](auto &limit, auto &inserted_to) { + auto &backref_entryrefs_by_seq = cache.get_backref_entryrefs_by_seq(); + return seastar::do_with( + backref_entryrefs_by_seq.begin(), + JOURNAL_SEQ_NULL, + [this, &t, &limit, &backref_entryrefs_by_seq, max](auto &iter, auto &inserted_to) { + return trans_intr::repeat( + [&iter, this, &t, &limit, &backref_entryrefs_by_seq, max, &inserted_to]() + -> merge_cached_backrefs_iertr::future<seastar::stop_iteration> { + if (iter == backref_entryrefs_by_seq.end()) { + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + auto &seq = iter->first; + auto &backref_entry_refs = iter->second; + LOG_PREFIX(BtreeBackrefManager::merge_cached_backrefs); + DEBUGT("seq {}, limit {}, num_fresh_backref {}" + , t, seq, limit, t.get_num_fresh_backref()); + if (seq <= limit && t.get_num_fresh_backref() * BACKREF_NODE_SIZE < max) { + inserted_to = seq; + return trans_intr::do_for_each( + backref_entry_refs, + [this, &t](auto &backref_entry_ref) { + LOG_PREFIX(BtreeBackrefManager::merge_cached_backrefs); + auto &backref_entry = *backref_entry_ref; + if (backref_entry.laddr != L_ADDR_NULL) { + DEBUGT("new mapping: {}~{} -> {}", + t, + backref_entry.paddr, + backref_entry.len, + backref_entry.laddr); + return new_mapping( + t, + backref_entry.paddr, + backref_entry.len, + backref_entry.laddr, + backref_entry.type).si_then([](auto &&pin) { + return seastar::now(); + }); + } else { + DEBUGT("remove mapping: {}", t, backref_entry.paddr); + return remove_mapping( + t, + backref_entry.paddr + ).si_then([](auto&&) { + return seastar::now(); + }).handle_error_interruptible( + crimson::ct_error::input_output_error::pass_further(), + crimson::ct_error::assert_all("no enoent possible") + ); + } + }).si_then([&iter] { + iter++; + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::no); + }); + } + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + }).si_then([&inserted_to] { + return seastar::make_ready_future<journal_seq_t>( + std::move(inserted_to)); + }); + }); + return merge_cached_backrefs_iertr::make_ready_future<journal_seq_t>( + std::move(inserted_to)); + }); +} + +BtreeBackrefManager::scan_mapped_space_ret +BtreeBackrefManager::scan_mapped_space( + Transaction &t, + BtreeBackrefManager::scan_mapped_space_func_t &&f) +{ + LOG_PREFIX(BtreeBackrefManager::scan_mapped_space); + DEBUGT("scan backref tree", t); + auto c = get_context(t); + return seastar::do_with( + std::move(f), + [this, c, FNAME](auto &scan_visitor) + { + auto block_size = cache.get_block_size(); + // traverse leaf-node entries + return with_btree<BackrefBtree>( + cache, c, + [c, &scan_visitor, block_size, FNAME](auto &btree) + { + return BackrefBtree::iterate_repeat( + c, + btree.lower_bound( + c, + P_ADDR_MIN), + [c, &scan_visitor, block_size, FNAME](auto &pos) { + if (pos.is_end()) { + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } + TRACET("tree value {}~{} {}~{} {} used", + c.trans, + pos.get_key(), + pos.get_val().len, + pos.get_val().laddr, + pos.get_val().len, + pos.get_val().type); + ceph_assert(pos.get_key().is_absolute()); + ceph_assert(pos.get_val().len > 0 && + pos.get_val().len % block_size == 0); + ceph_assert(!is_backref_node(pos.get_val().type)); + ceph_assert(pos.get_val().laddr != L_ADDR_NULL); + scan_visitor( + pos.get_key(), + P_ADDR_NULL, + pos.get_val().len, + pos.get_val().type, + pos.get_val().laddr); + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::no); + } + ); + }).si_then([this, &scan_visitor, c, FNAME, block_size] { + // traverse alloc-deltas in order + auto &backref_entryrefs = cache.get_backref_entryrefs_by_seq(); + for (auto &[seq, refs] : backref_entryrefs) { + boost::ignore_unused(seq); + DEBUGT("scan {} backref entries", c.trans, refs.size()); + for (auto &backref_entry : refs) { + if (backref_entry->laddr == L_ADDR_NULL) { + TRACET("backref entry {}~{} {} free", + c.trans, + backref_entry->paddr, + backref_entry->len, + backref_entry->type); + } else { + TRACET("backref entry {}~{} {}~{} {} used", + c.trans, + backref_entry->paddr, + backref_entry->len, + backref_entry->laddr, + backref_entry->len, + backref_entry->type); + } + ceph_assert(backref_entry->paddr.is_absolute()); + ceph_assert(backref_entry->len > 0 && + backref_entry->len % block_size == 0); + ceph_assert(!is_backref_node(backref_entry->type)); + scan_visitor( + backref_entry->paddr, + P_ADDR_NULL, + backref_entry->len, + backref_entry->type, + backref_entry->laddr); + } + } + }).si_then([this, &scan_visitor, block_size, c, FNAME] { + BackrefBtree::mapped_space_visitor_t f = + [&scan_visitor, block_size, FNAME, c]( + paddr_t paddr, paddr_t key, extent_len_t len, + depth_t depth, extent_types_t type, BackrefBtree::iterator&) { + TRACET("tree node {}~{} {}, depth={} used", + c.trans, paddr, len, type, depth); + ceph_assert(paddr.is_absolute()); + ceph_assert(len > 0 && len % block_size == 0); + ceph_assert(depth >= 1); + ceph_assert(is_backref_node(type)); + return scan_visitor(paddr, key, len, type, L_ADDR_NULL); + }; + return seastar::do_with( + std::move(f), + [this, c](auto &tree_visitor) + { + // traverse internal-node entries + return with_btree<BackrefBtree>( + cache, c, + [c, &tree_visitor](auto &btree) + { + return BackrefBtree::iterate_repeat( + c, + btree.lower_bound( + c, + P_ADDR_MIN, + &tree_visitor), + [](auto &pos) { + if (pos.is_end()) { + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } + return BackrefBtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::no); + }, + &tree_visitor + ); + }); + }); + }); + }); +} + +BtreeBackrefManager::base_iertr::future<> _init_cached_extent( + op_context_t<paddr_t> c, + const CachedExtentRef &e, + BackrefBtree &btree, + bool &ret) +{ + return btree.init_cached_extent(c, e + ).si_then([&ret](bool is_alive) { + ret = is_alive; + }); +} + +BtreeBackrefManager::init_cached_extent_ret BtreeBackrefManager::init_cached_extent( + Transaction &t, + CachedExtentRef e) +{ + LOG_PREFIX(BtreeBackrefManager::init_cached_extent); + TRACET("{}", t, *e); + return seastar::do_with(bool(), [this, e, &t](bool &ret) { + auto c = get_context(t); + return with_btree<BackrefBtree>(cache, c, [c, e, &ret](auto &btree) + -> base_iertr::future<> { + LOG_PREFIX(BtreeBackrefManager::init_cached_extent); + DEBUGT("extent {}", c.trans, *e); + return _init_cached_extent(c, e, btree, ret); + }).si_then([&ret] { return ret; }); + }); +} + +BtreeBackrefManager::rewrite_extent_ret +BtreeBackrefManager::rewrite_extent( + Transaction &t, + CachedExtentRef extent) +{ + auto c = get_context(t); + return with_btree<BackrefBtree>( + cache, + c, + [c, extent](auto &btree) mutable { + return btree.rewrite_extent(c, extent); + }); +} + +BtreeBackrefManager::remove_mapping_ret +BtreeBackrefManager::remove_mapping( + Transaction &t, + paddr_t addr) +{ + auto c = get_context(t); + return with_btree_ret<BackrefBtree, remove_mapping_result_t>( + cache, + c, + [c, addr](auto &btree) mutable { + return btree.lower_bound( + c, addr + ).si_then([&btree, c, addr](auto iter) + -> remove_mapping_ret { + if (iter.is_end() || iter.get_key() != addr) { + LOG_PREFIX(BtreeBackrefManager::remove_mapping); + WARNT("paddr={} doesn't exist, state: {}, leaf {}", + c.trans, addr, iter.get_key(), *iter.get_leaf_node()); + return remove_mapping_iertr::make_ready_future< + remove_mapping_result_t>(remove_mapping_result_t()); + } + + auto ret = remove_mapping_result_t{ + iter.get_key(), + iter.get_val().len, + iter.get_val().laddr}; + return btree.remove( + c, + iter + ).si_then([ret] { + return ret; + }); + }); + }); +} + +Cache::backref_entry_query_mset_t +BtreeBackrefManager::get_cached_backref_entries_in_range( + paddr_t start, + paddr_t end) +{ + return cache.get_backref_entries_in_range(start, end); +} + +void BtreeBackrefManager::cache_new_backref_extent( + paddr_t paddr, + paddr_t key, + extent_types_t type) +{ + return cache.add_backref_extent(paddr, key, type); +} + +BtreeBackrefManager::retrieve_backref_extents_in_range_ret +BtreeBackrefManager::retrieve_backref_extents_in_range( + Transaction &t, + paddr_t start, + paddr_t end) +{ + auto backref_extents = cache.get_backref_extents_in_range(start, end); + return seastar::do_with( + std::vector<CachedExtentRef>(), + std::move(backref_extents), + [this, &t](auto &extents, auto &backref_extents) { + return trans_intr::parallel_for_each( + backref_extents, + [this, &extents, &t](auto &ent) { + // only the gc fiber which is single can rewrite backref extents, + // so it must be alive + assert(is_backref_node(ent.type)); + LOG_PREFIX(BtreeBackrefManager::retrieve_backref_extents_in_range); + DEBUGT("getting backref extent of type {} at {}, key {}", + t, + ent.type, + ent.paddr, + ent.key); + + auto c = get_context(t); + return with_btree_ret<BackrefBtree, CachedExtentRef>( + cache, + c, + [c, &ent](auto &btree) { + if (ent.type == extent_types_t::BACKREF_INTERNAL) { + return btree.get_internal_if_live( + c, ent.paddr, ent.key, BACKREF_NODE_SIZE); + } else { + assert(ent.type == extent_types_t::BACKREF_LEAF); + return btree.get_leaf_if_live( + c, ent.paddr, ent.key, BACKREF_NODE_SIZE); + } + }).si_then([&extents](auto ext) { + ceph_assert(ext); + extents.emplace_back(std::move(ext)); + }); + }).si_then([&extents] { + return std::move(extents); + }); + }); +} + +} // namespace crimson::os::seastore::backref diff --git a/src/crimson/os/seastore/backref/btree_backref_manager.h b/src/crimson/os/seastore/backref/btree_backref_manager.h new file mode 100644 index 000000000..952e78b65 --- /dev/null +++ b/src/crimson/os/seastore/backref/btree_backref_manager.h @@ -0,0 +1,121 @@ +// -*- 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/backref_manager.h" +#include "crimson/os/seastore/backref/backref_tree_node.h" +#include "crimson/os/seastore/btree/fixed_kv_btree.h" + +namespace crimson::os::seastore::backref { + +constexpr size_t BACKREF_BLOCK_SIZE = 4096; + +class BtreeBackrefMapping : public BtreeNodeMapping<paddr_t, laddr_t> { + extent_types_t type; +public: + BtreeBackrefMapping(op_context_t<paddr_t> ctx) + : BtreeNodeMapping(ctx) {} + BtreeBackrefMapping( + op_context_t<paddr_t> ctx, + CachedExtentRef parent, + uint16_t pos, + backref_map_val_t &val, + backref_node_meta_t &&meta) + : BtreeNodeMapping( + ctx, + parent, + pos, + val.laddr, + val.len, + std::forward<backref_node_meta_t>(meta)), + type(val.type) + {} + extent_types_t get_type() const final { + return type; + } + +protected: + std::unique_ptr<BtreeNodeMapping<paddr_t, laddr_t>> _duplicate( + op_context_t<paddr_t> ctx) const final { + return std::unique_ptr<BtreeNodeMapping<paddr_t, laddr_t>>( + new BtreeBackrefMapping(ctx)); + } +}; + +using BackrefBtree = FixedKVBtree< + paddr_t, backref_map_val_t, BackrefInternalNode, + BackrefLeafNode, BtreeBackrefMapping, BACKREF_BLOCK_SIZE, false>; + +class BtreeBackrefManager : public BackrefManager { +public: + + BtreeBackrefManager(Cache &cache) + : cache(cache) + {} + + mkfs_ret mkfs( + Transaction &t) final; + + get_mapping_ret get_mapping( + Transaction &t, + paddr_t offset) final; + + get_mappings_ret get_mappings( + Transaction &t, + paddr_t offset, + paddr_t end) final; + + new_mapping_ret new_mapping( + Transaction &t, + paddr_t key, + extent_len_t len, + laddr_t val, + extent_types_t type) final; + + merge_cached_backrefs_ret merge_cached_backrefs( + Transaction &t, + const journal_seq_t &limit, + const uint64_t max) final; + + remove_mapping_ret remove_mapping( + Transaction &t, + paddr_t offset) final; + + scan_mapped_space_ret scan_mapped_space( + Transaction &t, + scan_mapped_space_func_t &&f) final; + + init_cached_extent_ret init_cached_extent( + Transaction &t, + CachedExtentRef e) final; + + rewrite_extent_ret rewrite_extent( + Transaction &t, + CachedExtentRef extent) final; + + Cache::backref_entry_query_mset_t + get_cached_backref_entries_in_range( + paddr_t start, + paddr_t end) final; + + retrieve_backref_extents_in_range_ret + retrieve_backref_extents_in_range( + Transaction &t, + paddr_t start, + paddr_t end) final; + + void cache_new_backref_extent( + paddr_t paddr, + paddr_t key, + extent_types_t type) final; + +private: + Cache &cache; + + op_context_t<paddr_t> get_context(Transaction &t) { + return op_context_t<paddr_t>{cache, t}; + } +}; + +} // namespace crimson::os::seastore::backref diff --git a/src/crimson/os/seastore/backref_manager.cc b/src/crimson/os/seastore/backref_manager.cc new file mode 100644 index 000000000..c596ee41d --- /dev/null +++ b/src/crimson/os/seastore/backref_manager.cc @@ -0,0 +1,18 @@ +// -*- 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/os/seastore/backref_manager.h" +#include "crimson/os/seastore/backref/btree_backref_manager.h" + +namespace crimson::os::seastore { + +BackrefManagerRef create_backref_manager( + Cache &cache) +{ + return BackrefManagerRef( + new backref::BtreeBackrefManager(cache)); +} + +} // namespace crimson::os::seastore::backref + diff --git a/src/crimson/os/seastore/backref_manager.h b/src/crimson/os/seastore/backref_manager.h new file mode 100644 index 000000000..3feedb997 --- /dev/null +++ b/src/crimson/os/seastore/backref_manager.h @@ -0,0 +1,152 @@ +// -*- 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/cache.h" +#include "crimson/os/seastore/cached_extent.h" +#include "crimson/os/seastore/transaction.h" + +namespace crimson::os::seastore { + +/** + * Abstract interface for managing back references that map paddr_t to laddr_t + */ +class BackrefManager { +public: + using base_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + using base_iertr = trans_iertr<base_ertr>; + + using mkfs_iertr = base_iertr; + using mkfs_ret = mkfs_iertr::future<>; + virtual mkfs_ret mkfs( + Transaction &t) = 0; + + /** + * Fetches mappings for paddr_t in range [offset, offset + len) + * + * Future will not resolve until all pins have resolved + */ + using get_mappings_iertr = base_iertr; + using get_mappings_ret = get_mappings_iertr::future<backref_pin_list_t>; + virtual get_mappings_ret get_mappings( + Transaction &t, + paddr_t offset, + paddr_t end) = 0; + + /** + * Fetches the mapping for paddr_t + * + * Future will not resolve until the pin has resolved + */ + using get_mapping_iertr = base_iertr::extend< + crimson::ct_error::enoent>; + using get_mapping_ret = get_mapping_iertr::future<BackrefMappingRef>; + virtual get_mapping_ret get_mapping( + Transaction &t, + paddr_t offset) = 0; + + /** + * rewrite_extent + * + * rewrite extent into passed transaction + */ + using rewrite_extent_iertr = base_iertr; + using rewrite_extent_ret = rewrite_extent_iertr::future<>; + virtual rewrite_extent_ret rewrite_extent( + Transaction &t, + CachedExtentRef extent) = 0; + + /** + * Insert new paddr_t -> laddr_t mapping + */ + using new_mapping_iertr = base_iertr; + using new_mapping_ret = new_mapping_iertr::future<BackrefMappingRef>; + virtual new_mapping_ret new_mapping( + Transaction &t, + paddr_t key, + extent_len_t len, + laddr_t val, + extent_types_t type) = 0; + + /** + * Check if a CachedExtent is alive, should be called + * after replay on each cached extent. + * + * @return returns whether the extent is alive + */ + using init_cached_extent_iertr = base_iertr; + using init_cached_extent_ret = init_cached_extent_iertr::future<bool>; + virtual init_cached_extent_ret init_cached_extent( + Transaction &t, + CachedExtentRef e) = 0; + + virtual Cache::backref_entry_query_mset_t + get_cached_backref_entries_in_range( + paddr_t start, + paddr_t end) = 0; + + using retrieve_backref_extents_in_range_iertr = base_iertr; + using retrieve_backref_extents_in_range_ret = + retrieve_backref_extents_in_range_iertr::future<std::vector<CachedExtentRef>>; + virtual retrieve_backref_extents_in_range_ret + retrieve_backref_extents_in_range( + Transaction &t, + paddr_t start, + paddr_t end) = 0; + + virtual void cache_new_backref_extent( + paddr_t paddr, + paddr_t key, + extent_types_t type) = 0; + + /** + * merge in-cache paddr_t -> laddr_t mappings to the on-disk backref tree + */ + using merge_cached_backrefs_iertr = base_iertr; + using merge_cached_backrefs_ret = merge_cached_backrefs_iertr::future<journal_seq_t>; + virtual merge_cached_backrefs_ret merge_cached_backrefs( + Transaction &t, + const journal_seq_t &limit, + const uint64_t max) = 0; + + struct remove_mapping_result_t { + paddr_t offset = P_ADDR_NULL; + extent_len_t len = 0; + laddr_t laddr = L_ADDR_NULL; + }; + + /** + * delete the mapping for paddr_t offset + */ + using remove_mapping_iertr = base_iertr::extend< + crimson::ct_error::enoent>; + using remove_mapping_ret = remove_mapping_iertr::future<remove_mapping_result_t>; + virtual remove_mapping_ret remove_mapping( + Transaction &t, + paddr_t offset) = 0; + + /** + * scan all extents in both tree and cache, + * including backref extents, logical extents and lba extents, + * visit them with scan_mapped_space_func_t + */ + using scan_mapped_space_iertr = base_iertr; + using scan_mapped_space_ret = scan_mapped_space_iertr::future<>; + using scan_mapped_space_func_t = std::function< + void(paddr_t, paddr_t, extent_len_t, extent_types_t, laddr_t)>; + virtual scan_mapped_space_ret scan_mapped_space( + Transaction &t, + scan_mapped_space_func_t &&f) = 0; + + virtual ~BackrefManager() {} +}; + +using BackrefManagerRef = + std::unique_ptr<BackrefManager>; + +BackrefManagerRef create_backref_manager( + Cache &cache); + +} // namespace crimson::os::seastore::backref diff --git a/src/crimson/os/seastore/btree/btree_range_pin.cc b/src/crimson/os/seastore/btree/btree_range_pin.cc new file mode 100644 index 000000000..2f801dcf1 --- /dev/null +++ b/src/crimson/os/seastore/btree/btree_range_pin.cc @@ -0,0 +1,27 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/btree/btree_range_pin.h" +#include "crimson/os/seastore/btree/fixed_kv_node.h" + +namespace crimson::os::seastore { + +template <typename key_t, typename val_t> +get_child_ret_t<LogicalCachedExtent> +BtreeNodeMapping<key_t, val_t>::get_logical_extent( + Transaction &t) +{ + assert(parent); + assert(parent->is_valid()); + assert(pos != std::numeric_limits<uint16_t>::max()); + auto &p = (FixedKVNode<key_t>&)*parent; + auto v = p.get_logical_child(ctx, pos); + if (!v.has_child()) { + this->child_pos = v.get_child_pos(); + } + return v; +} + +template class BtreeNodeMapping<laddr_t, paddr_t>; +template class BtreeNodeMapping<paddr_t, laddr_t>; +} // namespace crimson::os::seastore diff --git a/src/crimson/os/seastore/btree/btree_range_pin.h b/src/crimson/os/seastore/btree/btree_range_pin.h new file mode 100644 index 000000000..68188e9ff --- /dev/null +++ b/src/crimson/os/seastore/btree/btree_range_pin.h @@ -0,0 +1,204 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <boost/intrusive/set.hpp> + +#include "crimson/common/log.h" + +#include "crimson/os/seastore/cache.h" +#include "crimson/os/seastore/cached_extent.h" +#include "crimson/os/seastore/seastore_types.h" + +namespace crimson::os::seastore { + +template <typename node_key_t> +struct op_context_t { + Cache &cache; + Transaction &trans; +}; + +constexpr uint16_t MAX_FIXEDKVBTREE_DEPTH = 8; + +template <typename bound_t> +struct fixed_kv_node_meta_t { + bound_t begin = min_max_t<bound_t>::min; + bound_t end = min_max_t<bound_t>::min; + depth_t depth = 0; + + bool is_parent_of(const fixed_kv_node_meta_t &other) const { + return (depth == other.depth + 1) && + (begin <= other.begin) && + (end > other.begin); + } + + bool is_in_range(const bound_t key) const { + return begin <= key && end > key; + } + + std::pair<fixed_kv_node_meta_t, fixed_kv_node_meta_t> split_into(bound_t pivot) const { + return std::make_pair( + fixed_kv_node_meta_t{begin, pivot, depth}, + fixed_kv_node_meta_t{pivot, end, depth}); + } + + static fixed_kv_node_meta_t merge_from( + const fixed_kv_node_meta_t &lhs, const fixed_kv_node_meta_t &rhs) { + ceph_assert(lhs.depth == rhs.depth); + return fixed_kv_node_meta_t{lhs.begin, rhs.end, lhs.depth}; + } + + static std::pair<fixed_kv_node_meta_t, fixed_kv_node_meta_t> + rebalance(const fixed_kv_node_meta_t &lhs, const fixed_kv_node_meta_t &rhs, bound_t pivot) { + ceph_assert(lhs.depth == rhs.depth); + return std::make_pair( + fixed_kv_node_meta_t{lhs.begin, pivot, lhs.depth}, + fixed_kv_node_meta_t{pivot, rhs.end, lhs.depth}); + } + + bool is_root() const { + return begin == min_max_t<bound_t>::min && end == min_max_t<bound_t>::max; + } +}; + +template <typename bound_t> +inline std::ostream &operator<<( + std::ostream &lhs, + const fixed_kv_node_meta_t<bound_t> &rhs) +{ + return lhs << "btree_node_meta_t(" + << "begin=" << rhs.begin + << ", end=" << rhs.end + << ", depth=" << rhs.depth + << ")"; +} + +/** + * fixed_kv_node_meta_le_t + * + * On disk layout for fixed_kv_node_meta_t + */ +template <typename bound_le_t> +struct fixed_kv_node_meta_le_t { + bound_le_t begin = bound_le_t(0); + bound_le_t end = bound_le_t(0); + depth_le_t depth = init_depth_le(0); + + fixed_kv_node_meta_le_t() = default; + fixed_kv_node_meta_le_t( + const fixed_kv_node_meta_le_t<bound_le_t> &) = default; + explicit fixed_kv_node_meta_le_t( + const fixed_kv_node_meta_t<typename bound_le_t::orig_type> &val) + : begin(val.begin), + end(val.end), + depth(init_depth_le(val.depth)) {} + + operator fixed_kv_node_meta_t<typename bound_le_t::orig_type>() const { + return fixed_kv_node_meta_t<typename bound_le_t::orig_type>{ + begin, end, depth }; + } +}; + +template <typename key_t, typename val_t> +class BtreeNodeMapping : public PhysicalNodeMapping<key_t, val_t> { +protected: + op_context_t<key_t> ctx; + /** + * parent + * + * populated until link_extent is called to ensure cache residence + * until add_pin is called. + */ + CachedExtentRef parent; + + pladdr_t value; + extent_len_t len = 0; + fixed_kv_node_meta_t<key_t> range; + uint16_t pos = std::numeric_limits<uint16_t>::max(); + + virtual std::unique_ptr<BtreeNodeMapping> _duplicate(op_context_t<key_t>) const = 0; + fixed_kv_node_meta_t<key_t> _get_pin_range() const { + return range; + } + +public: + using val_type = val_t; + BtreeNodeMapping(op_context_t<key_t> ctx) : ctx(ctx) {} + + BtreeNodeMapping( + op_context_t<key_t> ctx, + CachedExtentRef parent, + uint16_t pos, + pladdr_t value, + extent_len_t len, + fixed_kv_node_meta_t<key_t> meta) + : ctx(ctx), + parent(parent), + value(value), + len(len), + range(meta), + pos(pos) + { + if (!parent->is_pending()) { + this->child_pos = {parent, pos}; + } + } + + CachedExtentRef get_parent() const final { + return parent; + } + + CachedExtentRef get_parent() { + return parent; + } + + void set_parent(CachedExtentRef ext) { + parent = ext; + } + + uint16_t get_pos() const final { + return pos; + } + + extent_len_t get_length() const final { + ceph_assert(range.end > range.begin); + return len; + } + + extent_types_t get_type() const override { + ceph_abort("should never happen"); + return extent_types_t::ROOT; + } + + val_t get_val() const final { + if constexpr (std::is_same_v<val_t, paddr_t>) { + return value.get_paddr(); + } else { + static_assert(std::is_same_v<val_t, laddr_t>); + return value.get_laddr(); + } + } + + key_t get_key() const override { + return range.begin; + } + + PhysicalNodeMappingRef<key_t, val_t> duplicate() const final { + auto ret = _duplicate(ctx); + ret->range = range; + ret->value = value; + ret->parent = parent; + ret->len = len; + ret->pos = pos; + return ret; + } + + bool has_been_invalidated() const final { + return parent->has_been_invalidated(); + } + + get_child_ret_t<LogicalCachedExtent> get_logical_extent(Transaction&) final; +}; + +} diff --git a/src/crimson/os/seastore/btree/fixed_kv_btree.h b/src/crimson/os/seastore/btree/fixed_kv_btree.h new file mode 100644 index 000000000..2970d0440 --- /dev/null +++ b/src/crimson/os/seastore/btree/fixed_kv_btree.h @@ -0,0 +1,2251 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include <boost/container/static_vector.hpp> +#include <sys/mman.h> +#include <memory> +#include <string.h> + +#include "crimson/os/seastore/logging.h" + +#include "crimson/os/seastore/cache.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/btree/btree_range_pin.h" +#include "crimson/os/seastore/root_block.h" + +#define RESERVATION_PTR reinterpret_cast<ChildableCachedExtent*>(0x1) + +namespace crimson::os::seastore::lba_manager::btree { +struct lba_map_val_t; +} + +namespace crimson::os::seastore { + +bool is_valid_child_ptr(ChildableCachedExtent* child); + +template <typename T> +phy_tree_root_t& get_phy_tree_root(root_t& r); + +using get_child_iertr = + ::crimson::interruptible::interruptible_errorator< + typename trans_intr::condition, + get_child_ertr>; +using get_phy_tree_root_node_ret = + std::pair<bool, get_child_iertr::future<CachedExtentRef>>; + +template <typename T, typename key_t> +const get_phy_tree_root_node_ret get_phy_tree_root_node( + const RootBlockRef &root_block, + op_context_t<key_t> c); + +template <typename ROOT_T> +void link_phy_tree_root_node(RootBlockRef &root_block, ROOT_T* root_node); + +template <typename T> +void unlink_phy_tree_root_node(RootBlockRef &root_block); + +template <typename T> +Transaction::tree_stats_t& get_tree_stats(Transaction &t); + +template < + typename node_key_t, + typename node_val_t, + typename internal_node_t, + typename leaf_node_t, + typename pin_t, + size_t node_size, + bool leaf_has_children> +class FixedKVBtree { + static constexpr size_t MAX_DEPTH = 16; + using self_type = FixedKVBtree< + node_key_t, + node_val_t, + internal_node_t, + leaf_node_t, + pin_t, + node_size, + leaf_has_children>; +public: + using InternalNodeRef = TCachedExtentRef<internal_node_t>; + using LeafNodeRef = TCachedExtentRef<leaf_node_t>; + + using base_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + using base_iertr = trans_iertr<base_ertr>; + + class iterator; + using iterator_fut = base_iertr::future<iterator>; + + using mapped_space_visitor_t = std::function< + void(paddr_t, node_key_t, extent_len_t, depth_t, extent_types_t, iterator&)>; + + class iterator { + public: + iterator(const iterator &rhs) noexcept : + internal(rhs.internal), leaf(rhs.leaf) {} + iterator(iterator &&rhs) noexcept : + internal(std::move(rhs.internal)), leaf(std::move(rhs.leaf)) {} + + iterator &operator=(const iterator &) = default; + iterator &operator=(iterator &&) = default; + + iterator_fut next( + op_context_t<node_key_t> c, + mapped_space_visitor_t *visitor=nullptr) const + { + assert_valid(); + assert(!is_end()); + + auto ret = *this; + ret.leaf.pos++; + if (ret.at_boundary()) { + return seastar::do_with( + ret, + [c, visitor](auto &ret) mutable { + return ret.handle_boundary( + c, visitor + ).si_then([&ret] { + return std::move(ret); + }); + }); + } else { + return iterator_fut( + interruptible::ready_future_marker{}, + ret); + } + + } + + iterator_fut prev(op_context_t<node_key_t> c) const + { + assert_valid(); + assert(!is_begin()); + + auto ret = *this; + + if (ret.leaf.pos > 0) { + ret.leaf.pos--; + return iterator_fut( + interruptible::ready_future_marker{}, + ret); + } + + depth_t depth_with_space = 2; + for (; depth_with_space <= get_depth(); ++depth_with_space) { + if (ret.get_internal(depth_with_space).pos > 0) { + break; + } + } + + assert(depth_with_space <= ret.get_depth()); // must not be begin() + return seastar::do_with( + std::move(ret), + [](const internal_node_t &internal) { return --internal.end(); }, + [](const leaf_node_t &leaf) { return --leaf.end(); }, + [c, depth_with_space](auto &ret, auto &li, auto &ll) { + for (depth_t depth = 2; depth < depth_with_space; ++depth) { + ret.get_internal(depth).reset(); + } + ret.leaf.reset(); + ret.get_internal(depth_with_space).pos--; + // note, cannot result in at_boundary() by construction + return lookup_depth_range( + c, ret, depth_with_space - 1, 0, li, ll, nullptr + ).si_then([&ret] { + assert(!ret.at_boundary()); + return std::move(ret); + }); + }); + } + + void assert_valid() const { + assert(leaf.node); + assert(leaf.pos <= leaf.node->get_size()); + + for (auto &i: internal) { + (void)i; + assert(i.node); + assert(i.pos < i.node->get_size()); + } + } + + depth_t get_depth() const { + return internal.size() + 1; + } + + auto &get_internal(depth_t depth) { + assert(depth > 1); + assert((depth - 2) < internal.size()); + return internal[depth - 2]; + } + + const auto &get_internal(depth_t depth) const { + assert(depth > 1); + assert((depth - 2) < internal.size()); + return internal[depth - 2]; + } + + node_key_t get_key() const { + assert(!is_end()); + return leaf.node->iter_idx(leaf.pos).get_key(); + } + node_val_t get_val() const { + assert(!is_end()); + auto ret = leaf.node->iter_idx(leaf.pos).get_val(); + if constexpr ( + std::is_same_v<crimson::os::seastore::lba_manager::btree::lba_map_val_t, + node_val_t>) { + if (ret.pladdr.is_paddr()) { + ret.pladdr = ret.pladdr.get_paddr().maybe_relative_to( + leaf.node->get_paddr()); + } + } + return ret; + } + + bool is_end() const { + // external methods may only resolve at a boundary if at end + return at_boundary(); + } + + bool is_begin() const { + for (auto &i: internal) { + if (i.pos != 0) + return false; + } + return leaf.pos == 0; + } + + std::unique_ptr<pin_t> get_pin(op_context_t<node_key_t> ctx) const { + assert(!is_end()); + auto val = get_val(); + auto key = get_key(); + return std::make_unique<pin_t>( + ctx, + leaf.node, + leaf.pos, + val, + fixed_kv_node_meta_t<node_key_t>{ key, key + val.len, 0 }); + } + + typename leaf_node_t::Ref get_leaf_node() { + return leaf.node; + } + + uint16_t get_leaf_pos() { + return leaf.pos; + } + private: + iterator() noexcept {} + iterator(depth_t depth) noexcept : internal(depth - 1) {} + + friend class FixedKVBtree; + static constexpr uint16_t INVALID = std::numeric_limits<uint16_t>::max(); + template <typename NodeType> + struct node_position_t { + typename NodeType::Ref node; + uint16_t pos = INVALID; + + node_position_t() = default; + node_position_t( + typename NodeType::Ref node, + uint16_t pos) + : node(node), pos(pos) {} + + void reset() { + *this = node_position_t{}; + } + + auto get_iter() { + assert(pos != INVALID); + assert(pos < node->get_size()); + return node->iter_idx(pos); + } + }; + boost::container::static_vector< + node_position_t<internal_node_t>, MAX_DEPTH> internal; + node_position_t<leaf_node_t> leaf; + + bool at_boundary() const { + assert(leaf.pos <= leaf.node->get_size()); + return leaf.pos == leaf.node->get_size(); + } + + using handle_boundary_ertr = base_iertr; + using handle_boundary_ret = handle_boundary_ertr::future<>; + handle_boundary_ret handle_boundary( + op_context_t<node_key_t> c, + mapped_space_visitor_t *visitor) + { + assert(at_boundary()); + depth_t depth_with_space = 2; + for (; depth_with_space <= get_depth(); ++depth_with_space) { + if ((get_internal(depth_with_space).pos + 1) < + get_internal(depth_with_space).node->get_size()) { + break; + } + } + + if (depth_with_space <= get_depth()) { + return seastar::do_with( + [](const internal_node_t &internal) { return internal.begin(); }, + [](const leaf_node_t &leaf) { return leaf.begin(); }, + [this, c, depth_with_space, visitor](auto &li, auto &ll) { + for (depth_t depth = 2; depth < depth_with_space; ++depth) { + get_internal(depth).reset(); + } + leaf.reset(); + get_internal(depth_with_space).pos++; + // note, cannot result in at_boundary() by construction + return lookup_depth_range( + c, *this, depth_with_space - 1, 0, li, ll, visitor + ); + }); + } else { + // end + return seastar::now(); + } + } + + depth_t check_split() const { + if (!leaf.node->at_max_capacity()) { + return 0; + } + for (depth_t split_from = 1; split_from < get_depth(); ++split_from) { + if (!get_internal(split_from + 1).node->at_max_capacity()) + return split_from; + } + return get_depth(); + } + + depth_t check_merge() const { + if (!leaf.node->below_min_capacity()) { + return 0; + } + for (depth_t merge_from = 1; merge_from < get_depth(); ++merge_from) { + if (!get_internal(merge_from + 1).node->below_min_capacity()) + return merge_from; + } + return get_depth(); + } + }; + + FixedKVBtree(RootBlockRef &root_block) : root_block(root_block) {} + + auto& get_root() { + return get_phy_tree_root<self_type>(root_block->get_root()); + } + + auto& get_root() const { + return get_phy_tree_root<self_type>(root_block->get_root()); + } + + template <typename T> + void set_root_node(const TCachedExtentRef<T> &root_node) { + static_assert(std::is_base_of_v<typename internal_node_t::base_t, T>); + link_phy_tree_root_node(root_block, root_node.get()); + } + + auto get_root_node(op_context_t<node_key_t> c) const { + return get_phy_tree_root_node<self_type>(root_block, c); + } + + /// mkfs + using mkfs_ret = phy_tree_root_t; + static mkfs_ret mkfs(RootBlockRef &root_block, op_context_t<node_key_t> c) { + assert(root_block->is_mutation_pending()); + auto root_leaf = c.cache.template alloc_new_extent<leaf_node_t>( + c.trans, + node_size, + placement_hint_t::HOT, + INIT_GENERATION); + root_leaf->set_size(0); + fixed_kv_node_meta_t<node_key_t> meta{min_max_t<node_key_t>::min, min_max_t<node_key_t>::max, 1}; + root_leaf->set_meta(meta); + root_leaf->range = meta; + get_tree_stats<self_type>(c.trans).depth = 1u; + get_tree_stats<self_type>(c.trans).extents_num_delta++; + link_phy_tree_root_node(root_block, root_leaf.get()); + return phy_tree_root_t{root_leaf->get_paddr(), 1u}; + } + + /** + * lower_bound + * + * @param c [in] context + * @param addr [in] ddr + * @return least iterator >= key + */ + iterator_fut lower_bound( + op_context_t<node_key_t> c, + node_key_t addr, + mapped_space_visitor_t *visitor=nullptr, + depth_t min_depth = 1) const + { + LOG_PREFIX(FixedKVBtree::lower_bound); + return lookup( + c, + [addr](const internal_node_t &internal) { + assert(internal.get_size() > 0); + auto iter = internal.upper_bound(addr); + assert(iter != internal.begin()); + --iter; + return iter; + }, + [FNAME, c, addr](const leaf_node_t &leaf) { + auto ret = leaf.lower_bound(addr); + SUBTRACET( + seastore_fixedkv_tree, + "leaf addr {}, got ret offset {}, size {}, end {}", + c.trans, + addr, + ret.get_offset(), + leaf.get_size(), + ret == leaf.end()); + return ret; + }, + min_depth, + visitor + ).si_then([FNAME, c, min_depth](auto &&ret) { + SUBTRACET( + seastore_fixedkv_tree, + "ret.leaf.pos {}", + c.trans, + ret.leaf.pos); + if (min_depth == 1) { + ret.assert_valid(); + } + return std::move(ret); + }); + } + + + /** + * upper_bound + * + * @param c [in] context + * @param addr [in] ddr + * @return least iterator > key + */ + iterator_fut upper_bound( + op_context_t<node_key_t> c, + node_key_t addr + ) const { + return lower_bound( + c, addr + ).si_then([c, addr](auto iter) { + if (!iter.is_end() && iter.get_key() == addr) { + return iter.next(c); + } else { + return iterator_fut( + interruptible::ready_future_marker{}, + iter); + } + }); + } + + /** + * upper_bound_right + * + * @param c [in] context + * @param addr [in] addr + * @return least iterator i s.t. i.get_key() + i.get_val().len > key + */ + iterator_fut upper_bound_right( + op_context_t<node_key_t> c, + node_key_t addr) const + { + return lower_bound( + c, addr + ).si_then([c, addr](auto iter) { + if (iter.is_begin()) { + return iterator_fut( + interruptible::ready_future_marker{}, + iter); + } else { + return iter.prev( + c + ).si_then([iter, addr](auto prev) { + if ((prev.get_key() + prev.get_val().len) > addr) { + return iterator_fut( + interruptible::ready_future_marker{}, + prev); + } else { + return iterator_fut( + interruptible::ready_future_marker{}, + iter); + } + }); + } + }); + } + + iterator_fut begin(op_context_t<node_key_t> c) const { + return lower_bound(c, 0); + } + iterator_fut end(op_context_t<node_key_t> c) const { + return upper_bound(c, min_max_t<node_key_t>::max); + } + + template <typename child_node_t, typename node_t, bool lhc = leaf_has_children, + typename std::enable_if<lhc, int>::type = 0> + void check_node( + op_context_t<node_key_t> c, + TCachedExtentRef<node_t> node) + { + assert(leaf_has_children); + for (auto i : *node) { + CachedExtentRef child_node; + Transaction::get_extent_ret ret; + + if constexpr (std::is_base_of_v<typename internal_node_t::base_t, child_node_t>) { + ret = c.trans.get_extent( + i->get_val().maybe_relative_to(node->get_paddr()), + &child_node); + } else { + assert(i->get_val().pladdr.is_paddr()); + ret = c.trans.get_extent( + i->get_val().pladdr.get_paddr().maybe_relative_to(node->get_paddr()), + &child_node); + } + if (ret == Transaction::get_extent_ret::PRESENT) { + if (child_node->is_stable()) { + assert(child_node->is_valid()); + auto cnode = child_node->template cast<child_node_t>(); + assert(cnode->has_parent_tracker()); + if (node->is_pending()) { + auto &n = node->get_stable_for_key(i->get_key()); + assert(cnode->get_parent_node().get() == &n); + auto pos = n.lower_bound_offset(i->get_key()); + assert(pos < n.get_node_size()); + assert(n.children[pos] == cnode.get()); + } else { + assert(cnode->get_parent_node().get() == node.get()); + assert(node->children[i->get_offset()] == cnode.get()); + } + } else if (child_node->is_pending()) { + if (child_node->is_mutation_pending()) { + auto &prior = (child_node_t &)*child_node->prior_instance; + assert(prior.is_valid()); + assert(prior.is_parent_valid()); + if (node->is_mutation_pending()) { + auto &n = node->get_stable_for_key(i->get_key()); + assert(prior.get_parent_node().get() == &n); + auto pos = n.lower_bound_offset(i->get_key()); + assert(pos < n.get_node_size()); + assert(n.children[pos] == &prior); + } else { + assert(prior.get_parent_node().get() == node.get()); + assert(node->children[i->get_offset()] == &prior); + } + } else { + auto cnode = child_node->template cast<child_node_t>(); + auto pos = node->find(i->get_key()).get_offset(); + auto child = node->children[pos]; + assert(child); + assert(child == cnode.get()); + assert(cnode->is_parent_valid()); + } + } else { + ceph_assert(!child_node->is_valid()); + ceph_abort("impossible"); + } + } else if (ret == Transaction::get_extent_ret::ABSENT) { + ChildableCachedExtent* child = nullptr; + if (node->is_pending()) { + auto &n = node->get_stable_for_key(i->get_key()); + auto pos = n.lower_bound_offset(i->get_key()); + assert(pos < n.get_node_size()); + child = n.children[pos]; + if (is_valid_child_ptr(child)) { + auto c = (child_node_t*)child; + assert(c->has_parent_tracker()); + assert(c->get_parent_node().get() == &n); + } + } else { + child = node->children[i->get_offset()]; + if (is_valid_child_ptr(child)) { + auto c = (child_node_t*)child; + assert(c->has_parent_tracker()); + assert(c->get_parent_node().get() == node.get()); + } + } + + if (!is_valid_child_ptr(child)) { + if constexpr ( + std::is_base_of_v<typename internal_node_t::base_t, child_node_t>) + { + assert(!c.cache.query_cache(i->get_val(), nullptr)); + } else { + if constexpr (leaf_has_children) { + assert(i->get_val().pladdr.is_paddr() + ? (bool)!c.cache.query_cache( + i->get_val().pladdr.get_paddr(), nullptr) + : true); + } + } + } + } else { + ceph_abort("impossible"); + } + } + } + + using check_child_trackers_ret = base_iertr::future<>; + template <bool lhc = leaf_has_children, + typename std::enable_if<lhc, int>::type = 0> + check_child_trackers_ret check_child_trackers( + op_context_t<node_key_t> c) { + mapped_space_visitor_t checker = [c, this]( + paddr_t, + node_key_t, + extent_len_t, + depth_t depth, + extent_types_t, + iterator& iter) { + if constexpr (!leaf_has_children) { + if (depth == 1) { + return seastar::now(); + } + } + if (depth > 1) { + auto &node = iter.get_internal(depth).node; + assert(node->is_valid()); + check_node<typename internal_node_t::base_t>(c, node); + } else { + assert(depth == 1); + auto &node = iter.leaf.node; + assert(node->is_valid()); + check_node<LogicalCachedExtent>(c, node); + } + return seastar::now(); + }; + + return seastar::do_with( + std::move(checker), + [this, c](auto &checker) { + return iterate_repeat( + c, + lower_bound( + c, + min_max_t<node_key_t>::min, + &checker), + [](auto &pos) { + if (pos.is_end()) { + return base_iertr::make_ready_future< + seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + return base_iertr::make_ready_future< + seastar::stop_iteration>( + seastar::stop_iteration::no); + }, + &checker); + }); + } + + using iterate_repeat_ret_inner = base_iertr::future< + seastar::stop_iteration>; + template <typename F> + static base_iertr::future<> iterate_repeat( + op_context_t<node_key_t> c, + iterator_fut &&iter_fut, + F &&f, + mapped_space_visitor_t *visitor=nullptr) { + return std::move( + iter_fut + ).si_then([c, visitor, f=std::forward<F>(f)](auto iter) { + return seastar::do_with( + iter, + std::move(f), + [c, visitor](auto &pos, auto &f) { + return trans_intr::repeat( + [c, visitor, &f, &pos] { + return f( + pos + ).si_then([c, visitor, &pos](auto done) { + if (done == seastar::stop_iteration::yes) { + return iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } else { + ceph_assert(!pos.is_end()); + return pos.next( + c, visitor + ).si_then([&pos](auto next) { + pos = next; + return iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::no); + }); + } + }); + }); + }); + }); + } + + /** + * insert + * + * Inserts val at laddr with iter as a hint. If element at laddr already + * exists returns iterator to that element unchanged and returns false. + * + * Invalidates all outstanding iterators for this tree on this transaction. + * + * @param c [in] op context + * @param iter [in] hint, insertion constant if immediately prior to iter + * @param laddr [in] addr at which to insert + * @param val [in] val to insert + * @return pair<iter, bool> where iter points to element at addr, bool true + * iff element at laddr did not exist. + */ + using insert_iertr = base_iertr; + using insert_ret = insert_iertr::future<std::pair<iterator, bool>>; + insert_ret insert( + op_context_t<node_key_t> c, + iterator iter, + node_key_t laddr, + node_val_t val, + LogicalCachedExtent* nextent + ) { + LOG_PREFIX(FixedKVBtree::insert); + SUBTRACET( + seastore_fixedkv_tree, + "inserting laddr {} at iter {}", + c.trans, + laddr, + iter.is_end() ? min_max_t<node_key_t>::max : iter.get_key()); + return seastar::do_with( + iter, + [this, c, laddr, val, nextent](auto &ret) { + return find_insertion( + c, laddr, ret + ).si_then([this, c, laddr, val, &ret, nextent] { + if (!ret.at_boundary() && ret.get_key() == laddr) { + return insert_ret( + interruptible::ready_future_marker{}, + std::make_pair(ret, false)); + } else { + ++(get_tree_stats<self_type>(c.trans).num_inserts); + return handle_split( + c, ret + ).si_then([c, laddr, val, &ret, nextent] { + if (!ret.leaf.node->is_mutable()) { + CachedExtentRef mut = c.cache.duplicate_for_write( + c.trans, ret.leaf.node + ); + ret.leaf.node = mut->cast<leaf_node_t>(); + } + auto iter = typename leaf_node_t::const_iterator( + ret.leaf.node.get(), ret.leaf.pos); + assert(iter == ret.leaf.node->lower_bound(laddr)); + assert(iter == ret.leaf.node->end() || iter->get_key() > laddr); + assert(laddr >= ret.leaf.node->get_meta().begin && + laddr < ret.leaf.node->get_meta().end); + ret.leaf.node->insert(iter, laddr, val, nextent); + return insert_ret( + interruptible::ready_future_marker{}, + std::make_pair(ret, true)); + }); + } + }); + }); + } + + insert_ret insert( + op_context_t<node_key_t> c, + node_key_t laddr, + node_val_t val, + LogicalCachedExtent* nextent) { + return lower_bound( + c, laddr + ).si_then([this, c, laddr, val, nextent](auto iter) { + return this->insert(c, iter, laddr, val, nextent); + }); + } + + /** + * update + * + * Invalidates all outstanding iterators for this tree on this transaction. + * + * @param c [in] op context + * @param iter [in] iterator to element to update, must not be end + * @param val [in] val with which to update + * @return iterator to newly updated element + */ + using update_iertr = base_iertr; + using update_ret = update_iertr::future<iterator>; + update_ret update( + op_context_t<node_key_t> c, + iterator iter, + node_val_t val, + LogicalCachedExtent* nextent) + { + LOG_PREFIX(FixedKVBtree::update); + SUBTRACET( + seastore_fixedkv_tree, + "update element at {}", + c.trans, + iter.is_end() ? min_max_t<node_key_t>::max : iter.get_key()); + if (!iter.leaf.node->is_mutable()) { + CachedExtentRef mut = c.cache.duplicate_for_write( + c.trans, iter.leaf.node + ); + iter.leaf.node = mut->cast<leaf_node_t>(); + } + ++(get_tree_stats<self_type>(c.trans).num_updates); + iter.leaf.node->update( + iter.leaf.node->iter_idx(iter.leaf.pos), + val, + nextent); + return update_ret( + interruptible::ready_future_marker{}, + iter); + } + + + /** + * remove + * + * Invalidates all outstanding iterators for this tree on this transaction. + * + * @param c [in] op context + * @param iter [in] iterator to element to remove, must not be end + */ + using remove_iertr = base_iertr; + using remove_ret = remove_iertr::future<>; + remove_ret remove( + op_context_t<node_key_t> c, + iterator iter) + { + LOG_PREFIX(FixedKVBtree::remove); + SUBTRACET( + seastore_fixedkv_tree, + "remove element at {}", + c.trans, + iter.is_end() ? min_max_t<node_key_t>::max : iter.get_key()); + assert(!iter.is_end()); + ++(get_tree_stats<self_type>(c.trans).num_erases); + return seastar::do_with( + iter, + [this, c](auto &ret) { + if (!ret.leaf.node->is_mutable()) { + CachedExtentRef mut = c.cache.duplicate_for_write( + c.trans, ret.leaf.node + ); + ret.leaf.node = mut->cast<leaf_node_t>(); + } + ret.leaf.node->remove( + ret.leaf.node->iter_idx(ret.leaf.pos)); + + return handle_merge( + c, ret + ); + }); + } + + /** + * init_cached_extent + * + * Checks whether e is live (reachable from fixed kv tree) and drops or initializes + * accordingly. + * + * Returns if e is live. + */ + using init_cached_extent_iertr = base_iertr; + using init_cached_extent_ret = init_cached_extent_iertr::future<bool>; + init_cached_extent_ret init_cached_extent( + op_context_t<node_key_t> c, + CachedExtentRef e) + { + assert(!e->is_logical()); + LOG_PREFIX(FixedKVTree::init_cached_extent); + SUBTRACET(seastore_fixedkv_tree, "extent {}", c.trans, *e); + if (e->get_type() == internal_node_t::TYPE) { + auto eint = e->cast<internal_node_t>(); + return lower_bound( + c, eint->get_node_meta().begin + ).si_then([e, c, eint](auto iter) { + // Note, this check is valid even if iter.is_end() + LOG_PREFIX(FixedKVTree::init_cached_extent); + depth_t cand_depth = eint->get_node_meta().depth; + if (cand_depth <= iter.get_depth() && + &*iter.get_internal(cand_depth).node == &*eint) { + SUBTRACET( + seastore_fixedkv_tree, + "extent {} is live", + c.trans, + *eint); + return true; + } else { + SUBTRACET( + seastore_fixedkv_tree, + "extent {} is not live", + c.trans, + *eint); + return false; + } + }); + } else if (e->get_type() == leaf_node_t::TYPE) { + auto eleaf = e->cast<leaf_node_t>(); + return lower_bound( + c, eleaf->get_node_meta().begin + ).si_then([c, e, eleaf](auto iter) { + // Note, this check is valid even if iter.is_end() + LOG_PREFIX(FixedKVTree::init_cached_extent); + if (iter.leaf.node == &*eleaf) { + SUBTRACET( + seastore_fixedkv_tree, + "extent {} is live", + c.trans, + *eleaf); + return true; + } else { + SUBTRACET( + seastore_fixedkv_tree, + "extent {} is not live", + c.trans, + *eleaf); + return false; + } + }); + } else { + SUBTRACET( + seastore_fixedkv_tree, + "found other extent {} type {}", + c.trans, + *e, + e->get_type()); + return init_cached_extent_ret( + interruptible::ready_future_marker{}, + true); + } + } + + /// get_leaf_if_live: get leaf node at laddr/addr if still live + using get_leaf_if_live_iertr = base_iertr; + using get_leaf_if_live_ret = get_leaf_if_live_iertr::future<CachedExtentRef>; + get_leaf_if_live_ret get_leaf_if_live( + op_context_t<node_key_t> c, + paddr_t addr, + node_key_t laddr, + extent_len_t len) + { + LOG_PREFIX(FixedKVBtree::get_leaf_if_live); + return lower_bound( + c, laddr + ).si_then([FNAME, c, addr, laddr, len](auto iter) { + if (iter.leaf.node->get_paddr() == addr) { + SUBTRACET( + seastore_fixedkv_tree, + "extent laddr {} addr {}~{} found: {}", + c.trans, + laddr, + addr, + len, + *iter.leaf.node); + return CachedExtentRef(iter.leaf.node); + } else { + SUBTRACET( + seastore_fixedkv_tree, + "extent laddr {} addr {}~{} is not live, does not match node {}", + c.trans, + laddr, + addr, + len, + *iter.leaf.node); + return CachedExtentRef(); + } + }); + } + + + /// get_internal_if_live: get internal node at laddr/addr if still live + using get_internal_if_live_iertr = base_iertr; + using get_internal_if_live_ret = get_internal_if_live_iertr::future<CachedExtentRef>; + get_internal_if_live_ret get_internal_if_live( + op_context_t<node_key_t> c, + paddr_t addr, + node_key_t laddr, + extent_len_t len) + { + LOG_PREFIX(FixedKVBtree::get_internal_if_live); + return lower_bound( + c, laddr + ).si_then([FNAME, c, addr, laddr, len](auto iter) { + for (depth_t d = 2; d <= iter.get_depth(); ++d) { + CachedExtent &node = *iter.get_internal(d).node; + auto internal_node = node.cast<internal_node_t>(); + if (internal_node->get_paddr() == addr) { + SUBTRACET( + seastore_fixedkv_tree, + "extent laddr {} addr {}~{} found: {}", + c.trans, + laddr, + addr, + len, + *internal_node); + assert(internal_node->get_node_meta().begin == laddr); + return CachedExtentRef(internal_node); + } + } + SUBTRACET( + seastore_fixedkv_tree, + "extent laddr {} addr {}~{} is not live, no matching internal node", + c.trans, + laddr, + addr, + len); + return CachedExtentRef(); + }); + } + + + /** + * rewrite_extent + * + * Rewrites a fresh copy of extent into transaction and updates internal + * references. + */ + using rewrite_extent_iertr = base_iertr; + using rewrite_extent_ret = rewrite_extent_iertr::future<>; + rewrite_extent_ret rewrite_extent( + op_context_t<node_key_t> c, + CachedExtentRef e) { + LOG_PREFIX(FixedKVBtree::rewrite_extent); + assert(is_lba_backref_node(e->get_type())); + + auto do_rewrite = [&](auto &fixed_kv_extent) { + auto n_fixed_kv_extent = c.cache.template alloc_new_extent< + std::remove_reference_t<decltype(fixed_kv_extent)> + >( + c.trans, + fixed_kv_extent.get_length(), + fixed_kv_extent.get_user_hint(), + // get target rewrite generation + fixed_kv_extent.get_rewrite_generation()); + fixed_kv_extent.get_bptr().copy_out( + 0, + fixed_kv_extent.get_length(), + n_fixed_kv_extent->get_bptr().c_str()); + n_fixed_kv_extent->set_modify_time(fixed_kv_extent.get_modify_time()); + n_fixed_kv_extent->range = n_fixed_kv_extent->get_node_meta(); + + if (fixed_kv_extent.get_type() == internal_node_t::TYPE || + leaf_node_t::do_has_children) { + if (!fixed_kv_extent.is_pending()) { + n_fixed_kv_extent->copy_sources.emplace(&fixed_kv_extent); + n_fixed_kv_extent->prior_instance = &fixed_kv_extent; + } else { + ceph_assert(fixed_kv_extent.is_mutation_pending()); + n_fixed_kv_extent->copy_sources.emplace( + (typename internal_node_t::base_t* + )fixed_kv_extent.get_prior_instance().get()); + n_fixed_kv_extent->children = std::move(fixed_kv_extent.children); + n_fixed_kv_extent->prior_instance = fixed_kv_extent.get_prior_instance(); + n_fixed_kv_extent->adjust_ptracker_for_children(); + } + } + + /* This is a bit underhanded. Any relative addrs here must necessarily + * be record relative as we are rewriting a dirty extent. Thus, we + * are using resolve_relative_addrs with a (likely negative) block + * relative offset to correct them to block-relative offsets adjusted + * for our new transaction location. + * + * Upon commit, these now block relative addresses will be interpretted + * against the real final address. + */ + if (!n_fixed_kv_extent->get_paddr().is_absolute()) { + // backend_type_t::SEGMENTED + assert(n_fixed_kv_extent->get_paddr().is_record_relative()); + n_fixed_kv_extent->resolve_relative_addrs( + make_record_relative_paddr(0).block_relative_to( + n_fixed_kv_extent->get_paddr())); + } // else: backend_type_t::RANDOM_BLOCK + + SUBTRACET( + seastore_fixedkv_tree, + "rewriting {} into {}", + c.trans, + fixed_kv_extent, + *n_fixed_kv_extent); + + return update_internal_mapping( + c, + n_fixed_kv_extent->get_node_meta().depth, + n_fixed_kv_extent->get_node_meta().begin, + e->get_paddr(), + n_fixed_kv_extent->get_paddr(), + n_fixed_kv_extent + ).si_then([c, e] { + c.cache.retire_extent(c.trans, e); + }); + }; + + CachedExtentRef n_fixed_kv_extent; + if (e->get_type() == internal_node_t::TYPE) { + auto lint = e->cast<internal_node_t>(); + return do_rewrite(*lint); + } else { + assert(e->get_type() == leaf_node_t::TYPE); + auto lleaf = e->cast<leaf_node_t>(); + return do_rewrite(*lleaf); + } + } + + using update_internal_mapping_iertr = base_iertr; + using update_internal_mapping_ret = update_internal_mapping_iertr::future<>; + update_internal_mapping_ret update_internal_mapping( + op_context_t<node_key_t> c, + depth_t depth, + node_key_t laddr, + paddr_t old_addr, + paddr_t new_addr, + typename internal_node_t::base_ref nextent) + { + LOG_PREFIX(FixedKVBtree::update_internal_mapping); + SUBTRACET( + seastore_fixedkv_tree, + "updating laddr {} at depth {} from {} to {}, nextent {}", + c.trans, + laddr, + depth, + old_addr, + new_addr, + *nextent); + + return lower_bound( + c, laddr, nullptr, depth + 1 + ).si_then([=, this](auto iter) { + assert(iter.get_depth() >= depth); + if (depth == iter.get_depth()) { + SUBTRACET(seastore_fixedkv_tree, "update at root", c.trans); + + if (laddr != min_max_t<node_key_t>::min) { + SUBERRORT( + seastore_fixedkv_tree, + "updating root laddr {} at depth {} from {} to {}," + "laddr is not 0", + c.trans, + laddr, + depth, + old_addr, + new_addr, + get_root().get_location()); + ceph_assert(0 == "impossible"); + } + + if (get_root().get_location() != old_addr) { + SUBERRORT( + seastore_fixedkv_tree, + "updating root laddr {} at depth {} from {} to {}," + "root addr {} does not match", + c.trans, + laddr, + depth, + old_addr, + new_addr, + get_root().get_location()); + ceph_assert(0 == "impossible"); + } + + root_block = c.cache.duplicate_for_write( + c.trans, root_block)->template cast<RootBlock>(); + get_root().set_location(new_addr); + set_root_node(nextent); + } else { + auto &parent = iter.get_internal(depth + 1); + assert(parent.node); + assert(parent.pos < parent.node->get_size()); + auto piter = parent.node->iter_idx(parent.pos); + + if (piter->get_key() != laddr) { + SUBERRORT( + seastore_fixedkv_tree, + "updating laddr {} at depth {} from {} to {}," + "node {} pos {} val pivot addr {} does not match", + c.trans, + laddr, + depth, + old_addr, + new_addr, + *(parent.node), + parent.pos, + piter->get_key()); + ceph_assert(0 == "impossible"); + } + + + if (piter->get_val() != old_addr) { + SUBERRORT( + seastore_fixedkv_tree, + "updating laddr {} at depth {} from {} to {}," + "node {} pos {} val addr {} does not match", + c.trans, + laddr, + depth, + old_addr, + new_addr, + *(parent.node), + parent.pos, + piter->get_val()); + ceph_assert(0 == "impossible"); + } + + CachedExtentRef mut = c.cache.duplicate_for_write( + c.trans, + parent.node + ); + typename internal_node_t::Ref mparent = mut->cast<internal_node_t>(); + mparent->update(piter, new_addr, nextent.get()); + + /* Note, iter is now invalid as we didn't udpate either the parent + * node reference to the new mutable instance nor did we update the + * child pointer to the new node. Not a problem as we'll now just + * destruct it. + */ + } + return seastar::now(); + }); + } + + +private: + RootBlockRef root_block; + + template <typename T> + using node_position_t = typename iterator::template node_position_t<T>; + + using get_internal_node_iertr = base_iertr; + using get_internal_node_ret = get_internal_node_iertr::future<InternalNodeRef>; + static get_internal_node_ret get_internal_node( + op_context_t<node_key_t> c, + depth_t depth, + paddr_t offset, + node_key_t begin, + node_key_t end, + typename std::optional<node_position_t<internal_node_t>> parent_pos) + { + LOG_PREFIX(FixedKVBtree::get_internal_node); + SUBTRACET( + seastore_fixedkv_tree, + "reading internal at offset {}, depth {}, begin {}, end {}", + c.trans, + offset, + depth, + begin, + end); + assert(depth > 1); + auto init_internal = [c, depth, begin, end, + parent_pos=std::move(parent_pos)] + (internal_node_t &node) { + assert(!node.is_pending()); + assert(!node.is_linked()); + node.range = fixed_kv_node_meta_t<node_key_t>{begin, end, depth}; + if (parent_pos) { + auto &parent = parent_pos->node; + parent->link_child(&node, parent_pos->pos); + } else { + assert(node.range.is_root()); + auto root_block = c.cache.get_root_fast(c.trans); + if (root_block->is_mutation_pending()) { + auto &stable_root = (RootBlockRef&)*root_block->get_prior_instance(); + link_phy_tree_root_node(stable_root, &node); + } else { + assert(!root_block->is_pending()); + link_phy_tree_root_node(root_block, &node); + } + } + }; + return c.cache.template get_absent_extent<internal_node_t>( + c.trans, + offset, + node_size, + init_internal + ).si_then([FNAME, c, offset, init_internal, depth, begin, end]( + typename internal_node_t::Ref ret) { + SUBTRACET( + seastore_fixedkv_tree, + "read internal at offset {} {}", + c.trans, + offset, + *ret); + // This can only happen during init_cached_extent + // or when backref extent being rewritten by gc space reclaiming + if (!ret->is_pending() && !ret->is_linked()) { + assert(ret->is_dirty() + || (is_backref_node(ret->get_type()) + && ret->is_clean())); + init_internal(*ret); + } + auto meta = ret->get_meta(); + if (ret->get_size()) { + ceph_assert(meta.begin <= ret->begin()->get_key()); + ceph_assert(meta.end > (ret->end() - 1)->get_key()); + } + ceph_assert(depth == meta.depth); + ceph_assert(begin == meta.begin); + ceph_assert(end == meta.end); + return get_internal_node_ret( + interruptible::ready_future_marker{}, + ret); + }); + } + + + using get_leaf_node_iertr = base_iertr; + using get_leaf_node_ret = get_leaf_node_iertr::future<LeafNodeRef>; + static get_leaf_node_ret get_leaf_node( + op_context_t<node_key_t> c, + paddr_t offset, + node_key_t begin, + node_key_t end, + typename std::optional<node_position_t<leaf_node_t>> parent_pos) + { + LOG_PREFIX(FixedKVBtree::get_leaf_node); + SUBTRACET( + seastore_fixedkv_tree, + "reading leaf at offset {}, begin {}, end {}", + c.trans, + offset, + begin, + end); + auto init_leaf = [c, begin, end, + parent_pos=std::move(parent_pos)] + (leaf_node_t &node) { + assert(!node.is_pending()); + assert(!node.is_linked()); + node.range = fixed_kv_node_meta_t<node_key_t>{begin, end, 1}; + if (parent_pos) { + auto &parent = parent_pos->node; + parent->link_child(&node, parent_pos->pos); + } else { + assert(node.range.is_root()); + auto root_block = c.cache.get_root_fast(c.trans); + if (root_block->is_mutation_pending()) { + auto &stable_root = (RootBlockRef&)*root_block->get_prior_instance(); + link_phy_tree_root_node(stable_root, &node); + } else { + assert(!root_block->is_pending()); + link_phy_tree_root_node(root_block, &node); + } + } + }; + return c.cache.template get_absent_extent<leaf_node_t>( + c.trans, + offset, + node_size, + init_leaf + ).si_then([FNAME, c, offset, init_leaf, begin, end] + (typename leaf_node_t::Ref ret) { + SUBTRACET( + seastore_fixedkv_tree, + "read leaf at offset {} {}", + c.trans, + offset, + *ret); + // This can only happen during init_cached_extent + // or when backref extent being rewritten by gc space reclaiming + if (!ret->is_pending() && !ret->is_linked()) { + assert(ret->is_dirty() + || (is_backref_node(ret->get_type()) + && ret->is_clean())); + init_leaf(*ret); + } + auto meta = ret->get_meta(); + if (ret->get_size()) { + ceph_assert(meta.begin <= ret->begin()->get_key()); + ceph_assert(meta.end > (ret->end() - 1)->get_key()); + } + ceph_assert(1 == meta.depth); + ceph_assert(begin == meta.begin); + ceph_assert(end == meta.end); + return get_leaf_node_ret( + interruptible::ready_future_marker{}, + ret); + }); + } + + using lookup_root_iertr = base_iertr; + using lookup_root_ret = lookup_root_iertr::future<>; + lookup_root_ret lookup_root( + op_context_t<node_key_t> c, + iterator &iter, + mapped_space_visitor_t *visitor) const { + LOG_PREFIX(FixedKVBtree::lookup_root); + SUBTRACET(seastore_fixedkv_tree, + "looking up root on {}", + c.trans, + *root_block); + auto [found, fut] = get_root_node(c); + + auto on_found_internal = + [this, visitor, &iter](InternalNodeRef &root_node) { + iter.get_internal(get_root().get_depth()).node = root_node; + if (visitor) (*visitor)( + root_node->get_paddr(), + root_node->get_node_meta().begin, + root_node->get_length(), + get_root().get_depth(), + internal_node_t::TYPE, + iter); + return lookup_root_iertr::now(); + }; + auto on_found_leaf = + [visitor, &iter, this](LeafNodeRef root_node) { + iter.leaf.node = root_node; + if (visitor) (*visitor)( + root_node->get_paddr(), + root_node->get_node_meta().begin, + root_node->get_length(), + get_root().get_depth(), + leaf_node_t::TYPE, + iter); + return lookup_root_iertr::now(); + }; + + if (found) { + return fut.si_then( + [this, c, on_found_internal=std::move(on_found_internal), + on_found_leaf=std::move(on_found_leaf)](auto root) { + LOG_PREFIX(FixedKVBtree::lookup_root); + ceph_assert(root); + SUBTRACET(seastore_fixedkv_tree, + "got root node on {}, res: {}", + c.trans, + *root_block, + *root); + + if (get_root().get_depth() > 1) { + auto root_node = root->template cast<internal_node_t>(); + return on_found_internal(root_node); + } else { + auto root_node = root->template cast<leaf_node_t>(); + return on_found_leaf(root_node); + } + }); + } else { + if (get_root().get_depth() > 1) { + return get_internal_node( + c, + get_root().get_depth(), + get_root().get_location(), + min_max_t<node_key_t>::min, + min_max_t<node_key_t>::max, + std::nullopt + ).si_then([on_found=std::move(on_found_internal)](InternalNodeRef root_node) { + return on_found(root_node); + }); + } else { + return get_leaf_node( + c, + get_root().get_location(), + min_max_t<node_key_t>::min, + min_max_t<node_key_t>::max, + std::nullopt + ).si_then([on_found=std::move(on_found_leaf)](LeafNodeRef root_node) { + return on_found(root_node); + }); + } + } + } + + using lookup_internal_level_iertr = base_iertr; + using lookup_internal_level_ret = lookup_internal_level_iertr::future<>; + template <typename F> + static lookup_internal_level_ret lookup_internal_level( + op_context_t<node_key_t> c, + depth_t depth, + iterator &iter, + F &f, + mapped_space_visitor_t *visitor + ) { + assert(depth > 1); + auto &parent_entry = iter.get_internal(depth + 1); + auto parent = parent_entry.node; + auto node_iter = parent->iter_idx(parent_entry.pos); + + auto on_found = [depth, visitor, &iter, &f](InternalNodeRef node) { + auto &entry = iter.get_internal(depth); + entry.node = node; + auto node_iter = f(*node); + assert(node_iter != node->end()); + entry.pos = node_iter->get_offset(); + if (visitor) + (*visitor)( + node->get_paddr(), + node->get_node_meta().begin, + node->get_length(), + depth, + node->get_type(), + iter); + return seastar::now(); + }; + + auto v = parent->template get_child<internal_node_t>(c, node_iter); + if (v.has_child()) { + return v.get_child_fut().safe_then( + [on_found=std::move(on_found), node_iter, c, + parent_entry](auto child) mutable { + LOG_PREFIX(FixedKVBtree::lookup_internal_level); + SUBTRACET(seastore_fixedkv_tree, + "got child on {}, pos: {}, res: {}", + c.trans, + *parent_entry.node, + parent_entry.pos, + *child); + auto &cnode = (typename internal_node_t::base_t &)*child; + assert(cnode.get_node_meta().begin == node_iter.get_key()); + assert(cnode.get_node_meta().end > node_iter.get_key()); + return on_found(child->template cast<internal_node_t>()); + }); + } + + auto child_pos = v.get_child_pos(); + auto next_iter = node_iter + 1; + auto begin = node_iter->get_key(); + auto end = next_iter == parent->end() + ? parent->get_node_meta().end + : next_iter->get_key(); + return get_internal_node( + c, + depth, + node_iter->get_val().maybe_relative_to(parent->get_paddr()), + begin, + end, + std::make_optional<node_position_t<internal_node_t>>( + child_pos.template get_parent<internal_node_t>(), + child_pos.get_pos()) + ).si_then([on_found=std::move(on_found)](InternalNodeRef node) { + return on_found(node); + }); + } + + using lookup_leaf_iertr = base_iertr; + using lookup_leaf_ret = lookup_leaf_iertr::future<>; + template <typename F> + static lookup_internal_level_ret lookup_leaf( + op_context_t<node_key_t> c, + iterator &iter, + F &f, + mapped_space_visitor_t *visitor + ) { + auto &parent_entry = iter.get_internal(2); + auto parent = parent_entry.node; + assert(parent); + auto node_iter = parent->iter_idx(parent_entry.pos); + + auto on_found = [visitor, &iter, &f](LeafNodeRef node) { + iter.leaf.node = node; + auto node_iter = f(*node); + iter.leaf.pos = node_iter->get_offset(); + if (visitor) + (*visitor)( + node->get_paddr(), + node->get_node_meta().begin, + node->get_length(), + 1, + node->get_type(), + iter); + return seastar::now(); + }; + + auto v = parent->template get_child<leaf_node_t>(c, node_iter); + if (v.has_child()) { + return v.get_child_fut().safe_then( + [on_found=std::move(on_found), node_iter, c, + parent_entry](auto child) mutable { + LOG_PREFIX(FixedKVBtree::lookup_leaf); + SUBTRACET(seastore_fixedkv_tree, + "got child on {}, pos: {}, res: {}", + c.trans, + *parent_entry.node, + parent_entry.pos, + *child); + auto &cnode = (typename internal_node_t::base_t &)*child; + assert(cnode.get_node_meta().begin == node_iter.get_key()); + assert(cnode.get_node_meta().end > node_iter.get_key()); + return on_found(child->template cast<leaf_node_t>()); + }); + } + + auto child_pos = v.get_child_pos(); + auto next_iter = node_iter + 1; + auto begin = node_iter->get_key(); + auto end = next_iter == parent->end() + ? parent->get_node_meta().end + : next_iter->get_key(); + + return get_leaf_node( + c, + node_iter->get_val().maybe_relative_to(parent->get_paddr()), + begin, + end, + std::make_optional<node_position_t<leaf_node_t>>( + child_pos.template get_parent<leaf_node_t>(), + child_pos.get_pos()) + ).si_then([on_found=std::move(on_found)](LeafNodeRef node) { + return on_found(node); + }); + } + + /** + * lookup_depth_range + * + * Performs node lookups on depths [from, to) using li and ll to + * specific target at each level. Note, may leave the iterator + * at_boundary(), call handle_boundary() prior to returning out + * lf FixedKVBtree. + */ + using lookup_depth_range_iertr = base_iertr; + using lookup_depth_range_ret = lookup_depth_range_iertr::future<>; + template <typename LI, typename LL> + static lookup_depth_range_ret lookup_depth_range( + op_context_t<node_key_t> c, ///< [in] context + iterator &iter, ///< [in,out] iterator to populate + depth_t from, ///< [in] from inclusive + depth_t to, ///< [in] to exclusive, (to <= from, to == from is a noop) + LI &li, ///< [in] internal->iterator + LL &ll, ///< [in] leaf->iterator + mapped_space_visitor_t *visitor ///< [in] mapped space visitor + ) { + LOG_PREFIX(FixedKVBtree::lookup_depth_range); + SUBTRACET(seastore_fixedkv_tree, "{} -> {}", c.trans, from, to); + return seastar::do_with( + from, + [c, to, visitor, &iter, &li, &ll](auto &d) { + return trans_intr::repeat( + [c, to, visitor, &iter, &li, &ll, &d] { + if (d > to) { + return [&] { + if (d > 1) { + return lookup_internal_level( + c, + d, + iter, + li, + visitor); + } else { + assert(d == 1); + return lookup_leaf( + c, + iter, + ll, + visitor); + } + }().si_then([&d] { + --d; + return lookup_depth_range_iertr::make_ready_future< + seastar::stop_iteration + >(seastar::stop_iteration::no); + }); + } else { + return lookup_depth_range_iertr::make_ready_future< + seastar::stop_iteration + >(seastar::stop_iteration::yes); + } + }); + }); + } + + using lookup_iertr = base_iertr; + using lookup_ret = lookup_iertr::future<iterator>; + template <typename LI, typename LL> + lookup_ret lookup( + op_context_t<node_key_t> c, + LI &&lookup_internal, + LL &&lookup_leaf, + depth_t min_depth, + mapped_space_visitor_t *visitor + ) const { + LOG_PREFIX(FixedKVBtree::lookup); + assert(min_depth > 0); + return seastar::do_with( + iterator{get_root().get_depth()}, + std::forward<LI>(lookup_internal), + std::forward<LL>(lookup_leaf), + [FNAME, this, visitor, c, min_depth](auto &iter, auto &li, auto &ll) { + return lookup_root( + c, iter, visitor + ).si_then([FNAME, this, visitor, c, &iter, &li, &ll, min_depth] { + if (iter.get_depth() > 1) { + auto &root_entry = *(iter.internal.rbegin()); + root_entry.pos = li(*(root_entry.node)).get_offset(); + } else { + auto &root_entry = iter.leaf; + auto riter = ll(*(root_entry.node)); + root_entry.pos = riter->get_offset(); + } + SUBTRACET(seastore_fixedkv_tree, "got root, depth {}", + c.trans, get_root().get_depth()); + return lookup_depth_range( + c, + iter, + get_root().get_depth() - 1, + min_depth - 1, + li, + ll, + visitor + ).si_then([c, visitor, &iter, min_depth] { + // It's only when the lookup is triggered by + // update_internal_mapping() that min_depth is + // NOT 1 + if (min_depth == 1 && iter.at_boundary()) { + return iter.handle_boundary(c, visitor); + } else { + return lookup_iertr::now(); + } + }); + }).si_then([&iter] { + return std::move(iter); + }); + }); + } + + /** + * find_insertion + * + * Prepare iter for insertion. iter should begin pointing at + * the valid insertion point (lower_bound(laddr)). + * + * Upon completion, iter will point at the + * position at which laddr should be inserted. iter may, upon completion, + * point at the end of a leaf other than the end leaf if that's the correct + * insertion point. + */ + using find_insertion_iertr = base_iertr; + using find_insertion_ret = find_insertion_iertr::future<>; + static find_insertion_ret find_insertion( + op_context_t<node_key_t> c, + node_key_t laddr, + iterator &iter) + { + assert(iter.is_end() || iter.get_key() >= laddr); + if (!iter.is_end() && iter.get_key() == laddr) { + return seastar::now(); + } else if (iter.leaf.node->get_node_meta().begin <= laddr) { +#ifndef NDEBUG + auto p = iter; + if (p.leaf.pos > 0) { + --p.leaf.pos; + assert(p.get_key() < laddr); + } +#endif + return seastar::now(); + } else { + assert(iter.leaf.pos == 0); + return iter.prev( + c + ).si_then([laddr, &iter](auto p) { + boost::ignore_unused(laddr); // avoid clang warning; + assert(p.leaf.node->get_node_meta().begin <= laddr); + assert(p.get_key() < laddr); + // Note, this is specifically allowed to violate the iterator + // invariant that pos is a valid index for the node in the event + // that the insertion point is at the end of a node. + p.leaf.pos++; + assert(p.at_boundary()); + iter = p; + return seastar::now(); + }); + } + } + + /** + * handle_split + * + * Split nodes in iter as needed for insertion. First, scan iter from leaf + * to find first non-full level. Then, split from there towards leaf. + * + * Upon completion, iter will point at the newly split insertion point. As + * with find_insertion, iter's leaf pointer may be end without iter being + * end. + */ + using handle_split_iertr = base_iertr; + using handle_split_ret = handle_split_iertr::future<>; + handle_split_ret handle_split( + op_context_t<node_key_t> c, + iterator &iter) + { + LOG_PREFIX(FixedKVBtree::handle_split); + + depth_t split_from = iter.check_split(); + + SUBTRACET(seastore_fixedkv_tree, "split_from {}, depth {}", c.trans, split_from, iter.get_depth()); + + if (split_from == iter.get_depth()) { + auto nroot = c.cache.template alloc_new_extent<internal_node_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + fixed_kv_node_meta_t<node_key_t> meta{ + min_max_t<node_key_t>::min, min_max_t<node_key_t>::max, iter.get_depth() + 1}; + nroot->set_meta(meta); + nroot->range = meta; + nroot->journal_insert( + nroot->begin(), + min_max_t<node_key_t>::min, + get_root().get_location(), + nullptr); + iter.internal.push_back({nroot, 0}); + + get_tree_stats<self_type>(c.trans).depth = iter.get_depth(); + get_tree_stats<self_type>(c.trans).extents_num_delta++; + + root_block = c.cache.duplicate_for_write( + c.trans, root_block)->template cast<RootBlock>(); + get_root().set_location(nroot->get_paddr()); + get_root().set_depth(iter.get_depth()); + ceph_assert(get_root().get_depth() <= MAX_FIXEDKVBTREE_DEPTH); + set_root_node(nroot); + } + + /* pos may be either node_position_t<leaf_node_t> or + * node_position_t<internal_node_t> */ + auto split_level = [&](auto &parent_pos, auto &pos) { + LOG_PREFIX(FixedKVBtree::handle_split); + auto [left, right, pivot] = pos.node->make_split_children(c); + + auto parent_node = parent_pos.node; + auto parent_iter = parent_pos.get_iter(); + + parent_node->update( + parent_iter, + left->get_paddr(), + left.get()); + parent_node->insert( + parent_iter + 1, + pivot, + right->get_paddr(), + right.get()); + + SUBTRACET( + seastore_fixedkv_tree, + "splitted {} into left: {}, right: {}", + c.trans, + *pos.node, + *left, + *right); + c.cache.retire_extent(c.trans, pos.node); + + get_tree_stats<self_type>(c.trans).extents_num_delta++; + return std::make_pair(left, right); + }; + + for (; split_from > 0; --split_from) { + auto &parent_pos = iter.get_internal(split_from + 1); + if (!parent_pos.node->is_mutable()) { + parent_pos.node = c.cache.duplicate_for_write( + c.trans, parent_pos.node + )->template cast<internal_node_t>(); + } + + if (split_from > 1) { + auto &pos = iter.get_internal(split_from); + SUBTRACET( + seastore_fixedkv_tree, + "splitting internal {} at depth {}, parent: {} at pos: {}", + c.trans, + *pos.node, + split_from, + *parent_pos.node, + parent_pos.pos); + auto [left, right] = split_level(parent_pos, pos); + + if (pos.pos < left->get_size()) { + pos.node = left; + } else { + pos.node = right; + pos.pos -= left->get_size(); + + parent_pos.pos += 1; + } + } else { + auto &pos = iter.leaf; + SUBTRACET( + seastore_fixedkv_tree, + "splitting leaf {}, parent: {} at pos: {}", + c.trans, + *pos.node, + *parent_pos.node, + parent_pos.pos); + auto [left, right] = split_level(parent_pos, pos); + + /* right->get_node_meta().begin == pivot == right->begin()->get_key() + * Thus, if pos.pos == left->get_size(), we want iter to point to + * left with pos.pos at the end rather than right with pos.pos = 0 + * since the insertion would be to the left of the first element + * of right and thus necessarily less than right->get_node_meta().begin. + */ + if (pos.pos <= left->get_size()) { + pos.node = left; + } else { + pos.node = right; + pos.pos -= left->get_size(); + + parent_pos.pos += 1; + } + } + } + + return seastar::now(); + } + + + using handle_merge_iertr = base_iertr; + using handle_merge_ret = handle_merge_iertr::future<>; + handle_merge_ret handle_merge( + op_context_t<node_key_t> c, + iterator &iter) + { + LOG_PREFIX(FixedKVBtree::handle_merge); + if (iter.get_depth() == 1 || + !iter.leaf.node->below_min_capacity()) { + SUBTRACET( + seastore_fixedkv_tree, + "no need to merge leaf, leaf size {}, depth {}", + c.trans, + iter.leaf.node->get_size(), + iter.get_depth()); + return seastar::now(); + } + + return seastar::do_with( + depth_t{1}, + [FNAME, this, c, &iter](auto &to_merge) { + return trans_intr::repeat( + [FNAME, this, c, &iter, &to_merge] { + SUBTRACET( + seastore_fixedkv_tree, + "merging depth {}", + c.trans, + to_merge); + auto &parent_pos = iter.get_internal(to_merge + 1); + auto merge_fut = handle_merge_iertr::now(); + if (to_merge > 1) { + auto &pos = iter.get_internal(to_merge); + merge_fut = merge_level(c, to_merge, parent_pos, pos); + } else { + auto &pos = iter.leaf; + merge_fut = merge_level(c, to_merge, parent_pos, pos); + } + + return merge_fut.si_then([FNAME, this, c, &iter, &to_merge] { + ++to_merge; + auto &pos = iter.get_internal(to_merge); + if (to_merge == iter.get_depth()) { + if (pos.node->get_size() == 1) { + SUBTRACET(seastore_fixedkv_tree, "collapsing root", c.trans); + c.cache.retire_extent(c.trans, pos.node); + assert(pos.pos == 0); + auto node_iter = pos.get_iter(); + iter.internal.pop_back(); + get_tree_stats<self_type>(c.trans).depth = iter.get_depth(); + get_tree_stats<self_type>(c.trans).extents_num_delta--; + + root_block = c.cache.duplicate_for_write( + c.trans, root_block + )->template cast<RootBlock>(); + get_root().set_location( + node_iter->get_val().maybe_relative_to(pos.node->get_paddr())); + get_root().set_depth(iter.get_depth()); + if (iter.get_depth() > 1) { + auto root_node = iter.get_internal(iter.get_depth()).node; + set_root_node(root_node); + } else { + set_root_node(iter.leaf.node); + } + } else { + SUBTRACET(seastore_fixedkv_tree, "no need to collapse root", c.trans); + } + return seastar::stop_iteration::yes; + } else if (pos.node->below_min_capacity()) { + SUBTRACET( + seastore_fixedkv_tree, + "continuing, next node {} depth {} at min", + c.trans, + *pos.node, + to_merge); + return seastar::stop_iteration::no; + } else { + SUBTRACET( + seastore_fixedkv_tree, + "complete, next node {} depth {} not min", + c.trans, + *pos.node, + to_merge); + return seastar::stop_iteration::yes; + } + }); + }); + }); + } + + template <typename NodeType, + std::enable_if_t<std::is_same_v<NodeType, leaf_node_t>, int> = 0> + base_iertr::future<typename NodeType::Ref> get_node( + op_context_t<node_key_t> c, + depth_t depth, + paddr_t addr, + node_key_t begin, + node_key_t end, + typename std::optional<node_position_t<leaf_node_t>> parent_pos) { + assert(depth == 1); + return get_leaf_node(c, addr, begin, end, std::move(parent_pos)); + } + + template <typename NodeType, + std::enable_if_t<std::is_same_v<NodeType, internal_node_t>, int> = 0> + base_iertr::future<typename NodeType::Ref> get_node( + op_context_t<node_key_t> c, + depth_t depth, + paddr_t addr, + node_key_t begin, + node_key_t end, + typename std::optional<node_position_t<internal_node_t>> parent_pos) { + return get_internal_node(c, depth, addr, begin, end, std::move(parent_pos)); + } + + template <typename NodeType> + handle_merge_ret merge_level( + op_context_t<node_key_t> c, + depth_t depth, + node_position_t<internal_node_t> &parent_pos, + node_position_t<NodeType> &pos) + { + LOG_PREFIX(FixedKVBtree::merge_level); + if (!parent_pos.node->is_mutable()) { + parent_pos.node = c.cache.duplicate_for_write( + c.trans, parent_pos.node + )->template cast<internal_node_t>(); + } + + auto iter = parent_pos.get_iter(); + assert(iter.get_offset() < parent_pos.node->get_size()); + bool donor_is_left = ((iter.get_offset() + 1) == parent_pos.node->get_size()); + auto donor_iter = donor_is_left ? (iter - 1) : (iter + 1); + auto next_iter = donor_iter + 1; + auto begin = donor_iter->get_key(); + auto end = next_iter == parent_pos.node->end() + ? parent_pos.node->get_node_meta().end + : next_iter->get_key(); + + SUBTRACET(seastore_fixedkv_tree, "parent: {}, node: {}", c.trans, *parent_pos.node, *pos.node); + auto do_merge = [c, iter, donor_iter, donor_is_left, &parent_pos, &pos]( + typename NodeType::Ref donor) { + LOG_PREFIX(FixedKVBtree::merge_level); + auto [l, r] = donor_is_left ? + std::make_pair(donor, pos.node) : std::make_pair(pos.node, donor); + + auto [liter, riter] = donor_is_left ? + std::make_pair(donor_iter, iter) : std::make_pair(iter, donor_iter); + + if (donor->at_min_capacity()) { + auto replacement = l->make_full_merge(c, r); + + parent_pos.node->update( + liter, + replacement->get_paddr(), + replacement.get()); + parent_pos.node->remove(riter); + + pos.node = replacement; + if (donor_is_left) { + pos.pos += l->get_size(); + parent_pos.pos--; + } + + SUBTRACET(seastore_fixedkv_tree, "l: {}, r: {}, replacement: {}", c.trans, *l, *r, *replacement); + c.cache.retire_extent(c.trans, l); + c.cache.retire_extent(c.trans, r); + get_tree_stats<self_type>(c.trans).extents_num_delta--; + } else { + LOG_PREFIX(FixedKVBtree::merge_level); + auto [replacement_l, replacement_r, pivot] = + l->make_balanced( + c, + r, + !donor_is_left); + + parent_pos.node->update( + liter, + replacement_l->get_paddr(), + replacement_l.get()); + parent_pos.node->replace( + riter, + pivot, + replacement_r->get_paddr(), + replacement_r.get()); + + if (donor_is_left) { + assert(parent_pos.pos > 0); + parent_pos.pos--; + } + + auto orig_position = donor_is_left ? + l->get_size() + pos.pos : + pos.pos; + if (orig_position < replacement_l->get_size()) { + pos.node = replacement_l; + pos.pos = orig_position; + } else { + parent_pos.pos++; + pos.node = replacement_r; + pos.pos = orig_position - replacement_l->get_size(); + } + + SUBTRACET( + seastore_fixedkv_tree, + "l: {}, r: {}, replacement_l: {}, replacement_r: {}", + c.trans, *l, *r, *replacement_l, *replacement_r); + c.cache.retire_extent(c.trans, l); + c.cache.retire_extent(c.trans, r); + } + + return seastar::now(); + }; + + auto v = parent_pos.node->template get_child<NodeType>(c, donor_iter); + if (v.has_child()) { + return v.get_child_fut().safe_then( + [do_merge=std::move(do_merge), &pos, + donor_iter, donor_is_left, c, parent_pos](auto child) mutable { + LOG_PREFIX(FixedKVBtree::merge_level); + SUBTRACET(seastore_fixedkv_tree, + "got child on {}, pos: {}, res: {}", + c.trans, + *parent_pos.node, + donor_iter.get_offset(), + *child); + auto &node = (typename internal_node_t::base_t&)*child; + assert(donor_is_left ? + node.get_node_meta().end == pos.node->get_node_meta().begin : + node.get_node_meta().begin == pos.node->get_node_meta().end); + assert(node.get_node_meta().begin == donor_iter.get_key()); + assert(node.get_node_meta().end > donor_iter.get_key()); + return do_merge(child->template cast<NodeType>()); + }); + } + + auto child_pos = v.get_child_pos(); + return get_node<NodeType>( + c, + depth, + donor_iter.get_val().maybe_relative_to(parent_pos.node->get_paddr()), + begin, + end, + std::make_optional<node_position_t<NodeType>>( + child_pos.template get_parent<NodeType>(), + child_pos.get_pos()) + ).si_then([do_merge=std::move(do_merge)](typename NodeType::Ref donor) { + return do_merge(donor); + }); + } +}; + +template <typename T> +struct is_fixed_kv_tree : std::false_type {}; + +template < + typename node_key_t, + typename node_val_t, + typename internal_node_t, + typename leaf_node_t, + typename pin_t, + size_t node_size, + bool leaf_has_children> +struct is_fixed_kv_tree< + FixedKVBtree< + node_key_t, + node_val_t, + internal_node_t, + leaf_node_t, + pin_t, + node_size, + leaf_has_children>> : std::true_type {}; + +template < + typename tree_type_t, + typename node_key_t, + typename F, + std::enable_if_t<is_fixed_kv_tree<tree_type_t>::value, int> = 0> +auto with_btree( + Cache &cache, + op_context_t<node_key_t> c, + F &&f) { + return cache.get_root( + c.trans + ).si_then([f=std::forward<F>(f)](RootBlockRef croot) mutable { + return seastar::do_with( + tree_type_t(croot), + [f=std::move(f)](auto &btree) mutable { + return f(btree); + }); + }); +} + +template < + typename tree_type_t, + typename State, + typename node_key_t, + typename F, + std::enable_if_t<is_fixed_kv_tree<tree_type_t>::value, int> = 0> +auto with_btree_state( + Cache &cache, + op_context_t<node_key_t> c, + State &&init, + F &&f) { + return seastar::do_with( + std::forward<State>(init), + [&cache, c, f=std::forward<F>(f)](auto &state) mutable { + return with_btree<tree_type_t>( + cache, + c, + [&state, f=std::move(f)](auto &btree) mutable { + return f(btree, state); + }).si_then([&state] { + return seastar::make_ready_future<State>(std::move(state)); + }); + }); +} + +template < + typename tree_type_t, + typename State, + typename node_key_t, + typename F, + std::enable_if_t<is_fixed_kv_tree<tree_type_t>::value, int> = 0> +auto with_btree_state( + Cache &cache, + op_context_t<node_key_t> c, + F &&f) { + return crimson::os::seastore::with_btree_state<tree_type_t, State>( + cache, c, State{}, std::forward<F>(f)); +} + +template < + typename tree_type_t, + typename Ret, + typename node_key_t, + typename F> +auto with_btree_ret( + Cache &cache, + op_context_t<node_key_t> c, + F &&f) { + return with_btree_state<tree_type_t, Ret>( + cache, + c, + [f=std::forward<F>(f)](auto &btree, auto &ret) mutable { + return f( + btree + ).si_then([&ret](auto &&_ret) { + ret = std::move(_ret); + }); + }); +} + +} + diff --git a/src/crimson/os/seastore/btree/fixed_kv_node.cc b/src/crimson/os/seastore/btree/fixed_kv_node.cc new file mode 100644 index 000000000..00aceab92 --- /dev/null +++ b/src/crimson/os/seastore/btree/fixed_kv_node.cc @@ -0,0 +1,12 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/btree/fixed_kv_node.h" + +namespace crimson::os::seastore { + +bool is_valid_child_ptr(ChildableCachedExtent* child) { + return child != nullptr && child != RESERVATION_PTR; +} + +} // namespace crimson::os::seastore diff --git a/src/crimson/os/seastore/btree/fixed_kv_node.h b/src/crimson/os/seastore/btree/fixed_kv_node.h new file mode 100644 index 000000000..956a1824e --- /dev/null +++ b/src/crimson/os/seastore/btree/fixed_kv_node.h @@ -0,0 +1,1220 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <sys/mman.h> +#include <memory> +#include <string.h> + + +#include "include/buffer.h" + +#include "crimson/common/fixed_kv_node_layout.h" +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/cache.h" +#include "crimson/os/seastore/cached_extent.h" + +#include "crimson/os/seastore/btree/btree_range_pin.h" +#include "crimson/os/seastore/btree/fixed_kv_btree.h" +#include "crimson/os/seastore/root_block.h" + +namespace crimson::os::seastore { + +/** + * FixedKVNode + * + * Base class enabling recursive lookup between internal and leaf nodes. + */ +template <typename node_key_t> +struct FixedKVNode : ChildableCachedExtent { + using FixedKVNodeRef = TCachedExtentRef<FixedKVNode>; + fixed_kv_node_meta_t<node_key_t> range; + + struct copy_source_cmp_t { + using is_transparent = node_key_t; + bool operator()(const FixedKVNodeRef &l, const FixedKVNodeRef &r) const { + assert(l->range.end <= r->range.begin + || r->range.end <= l->range.begin + || (l->range.begin == r->range.begin + && l->range.end == r->range.end)); + return l->range.begin < r->range.begin; + } + bool operator()(const node_key_t &l, const FixedKVNodeRef &r) const { + return l < r->range.begin; + } + bool operator()(const FixedKVNodeRef &l, const node_key_t &r) const { + return l->range.begin < r; + } + }; + + /* + * + * Nodes of fixed-kv-btree connect to their child nodes by pointers following + * invariants below: + * + * 1. if nodes are stable: + * a. parent points at the node's stable parent + * b. prior_instance is empty + * c. child pointers point at stable children. Child resolution is done + * directly via this array. + * c. copy_sources is empty + * 2. if nodes are mutation_pending: + * a. parent is empty and needs to be fixed upon commit + * b. prior_instance points to its stable version + * c. child pointers are null except for initial_pending() children of + * this transaction. Child resolution is done by first checking this + * array, and then recursively resolving via the parent. We copy child + * pointers from parent on commit. + * c. copy_sources is empty + * 3. if nodes are initial_pending + * a. parent points at its pending parent on this transaction (must exist) + * b. prior_instance is empty or, if it's the result of rewrite, points to + * its stable predecessor + * c. child pointers are null except for initial_pending() children of + * this transaction (live due to 3a below). Child resolution is done + * by first checking this array, and then recursively resolving via + * the correct copy_sources entry. We copy child pointers from copy_sources + * on commit. + * d. copy_sources contains the set of stable nodes at the same tree-level(only + * its "prior_instance" if the node is the result of a rewrite), with which + * the lba range of this node overlaps. + */ + std::vector<ChildableCachedExtent*> children; + std::set<FixedKVNodeRef, copy_source_cmp_t> copy_sources; + uint16_t capacity = 0; + parent_tracker_t* my_tracker = nullptr; + RootBlockRef root_block; + + bool is_linked() { + assert(!has_parent_tracker() || !(bool)root_block); + return (bool)has_parent_tracker() || (bool)root_block; + } + + FixedKVNode(uint16_t capacity, ceph::bufferptr &&ptr) + : ChildableCachedExtent(std::move(ptr)), + children(capacity, nullptr), + capacity(capacity) {} + FixedKVNode(const FixedKVNode &rhs) + : ChildableCachedExtent(rhs), + range(rhs.range), + children(rhs.capacity, nullptr), + capacity(rhs.capacity) {} + + virtual fixed_kv_node_meta_t<node_key_t> get_node_meta() const = 0; + virtual uint16_t get_node_size() const = 0; + + virtual ~FixedKVNode() = default; + virtual node_key_t get_key_from_idx(uint16_t idx) const = 0; + + template<typename iter_t> + void update_child_ptr(iter_t iter, ChildableCachedExtent* child) { + children[iter.get_offset()] = child; + set_child_ptracker(child); + } + + virtual bool is_leaf_and_has_children() const = 0; + + template<typename iter_t> + void insert_child_ptr(iter_t iter, ChildableCachedExtent* child) { + auto raw_children = children.data(); + auto offset = iter.get_offset(); + std::memmove( + &raw_children[offset + 1], + &raw_children[offset], + (get_node_size() - offset) * sizeof(ChildableCachedExtent*)); + if (child) { + children[offset] = child; + set_child_ptracker(child); + } else { + // this can only happen when reserving lba spaces + ceph_assert(is_leaf_and_has_children()); + // this is to avoid mistakenly copying pointers from + // copy sources when committing this lba node, because + // we rely on pointers' "nullness" to avoid copying + // pointers for updated values + children[offset] = RESERVATION_PTR; + } + } + + template<typename iter_t> + void remove_child_ptr(iter_t iter) { + LOG_PREFIX(FixedKVNode::remove_child_ptr); + auto raw_children = children.data(); + auto offset = iter.get_offset(); + SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, total size {}, extent {}", + this->pending_for_transaction, + offset, + get_node_size(), + (void*)raw_children[offset]); + // parent tracker of the child being removed will be + // reset when the child is invalidated, so no need to + // reset it here + std::memmove( + &raw_children[offset], + &raw_children[offset + 1], + (get_node_size() - offset - 1) * sizeof(ChildableCachedExtent*)); + } + + FixedKVNode& get_stable_for_key(node_key_t key) { + ceph_assert(is_pending()); + if (is_mutation_pending()) { + return (FixedKVNode&)*get_prior_instance(); + } else { + ceph_assert(!copy_sources.empty()); + auto it = copy_sources.upper_bound(key); + it--; + auto ©_source = *it; + ceph_assert(copy_source->get_node_meta().is_in_range(key)); + return *copy_source; + } + } + + static void push_copy_sources( + FixedKVNode &dest, + FixedKVNode &src) + { + ceph_assert(dest.is_initial_pending()); + if (!src.is_pending()) { + dest.copy_sources.emplace(&src); + } else if (src.is_mutation_pending()) { + dest.copy_sources.emplace( + src.get_prior_instance()->template cast<FixedKVNode>()); + } else { + ceph_assert(src.is_initial_pending()); + dest.copy_sources.insert( + src.copy_sources.begin(), + src.copy_sources.end()); + } + } + + virtual uint16_t get_node_split_pivot() = 0; + + static void move_child_ptrs( + FixedKVNode &dest, + FixedKVNode &src, + size_t dest_start, + size_t src_start, + size_t src_end) + { + std::memmove( + dest.children.data() + dest_start, + src.children.data() + src_start, + (src_end - src_start) * sizeof(ChildableCachedExtent*)); + + ceph_assert(src_start < src_end); + ceph_assert(src.children.size() >= src_end); + for (auto it = src.children.begin() + src_start; + it != src.children.begin() + src_end; + it++) + { + auto child = *it; + if (is_valid_child_ptr(child)) { + dest.set_child_ptracker(child); + } + } + } + + void link_child(ChildableCachedExtent* child, uint16_t pos) { + assert(pos < get_node_size()); + assert(child); + ceph_assert(!is_pending()); + ceph_assert(child->is_valid() && !child->is_pending()); + assert(!children[pos]); + children[pos] = child; + set_child_ptracker(child); + } + + virtual get_child_ret_t<LogicalCachedExtent> + get_logical_child(op_context_t<node_key_t> c, uint16_t pos) = 0; + + template <typename T, typename iter_t> + get_child_ret_t<T> get_child(op_context_t<node_key_t> c, iter_t iter) { + auto pos = iter.get_offset(); + assert(children.capacity()); + auto child = children[pos]; + if (is_valid_child_ptr(child)) { + ceph_assert(child->get_type() == T::TYPE); + return c.cache.template get_extent_viewable_by_trans<T>(c.trans, (T*)child); + } else if (is_pending()) { + auto key = iter.get_key(); + auto &sparent = get_stable_for_key(key); + auto spos = sparent.child_pos_for_key(key); + auto child = sparent.children[spos]; + if (is_valid_child_ptr(child)) { + ceph_assert(child->get_type() == T::TYPE); + return c.cache.template get_extent_viewable_by_trans<T>(c.trans, (T*)child); + } else { + return child_pos_t(&sparent, spos); + } + } else { + return child_pos_t(this, pos); + } + } + + void split_child_ptrs( + FixedKVNode &left, + FixedKVNode &right) + { + assert(!left.my_tracker); + assert(!right.my_tracker); + push_copy_sources(left, *this); + push_copy_sources(right, *this); + if (is_pending()) { + uint16_t pivot = get_node_split_pivot(); + move_child_ptrs(left, *this, 0, 0, pivot); + move_child_ptrs(right, *this, 0, pivot, get_node_size()); + my_tracker = nullptr; + } + } + + void merge_child_ptrs( + FixedKVNode &left, + FixedKVNode &right) + { + ceph_assert(!my_tracker); + push_copy_sources(*this, left); + push_copy_sources(*this, right); + + if (left.is_pending()) { + move_child_ptrs(*this, left, 0, 0, left.get_node_size()); + left.my_tracker = nullptr; + } + + if (right.is_pending()) { + move_child_ptrs(*this, right, left.get_node_size(), 0, right.get_node_size()); + right.my_tracker = nullptr; + } + } + + static void balance_child_ptrs( + FixedKVNode &left, + FixedKVNode &right, + bool prefer_left, + FixedKVNode &replacement_left, + FixedKVNode &replacement_right) + { + size_t l_size = left.get_node_size(); + size_t r_size = right.get_node_size(); + size_t total = l_size + r_size; + size_t pivot_idx = (l_size + r_size) / 2; + if (total % 2 && prefer_left) { + pivot_idx++; + } + + assert(!replacement_left.my_tracker); + assert(!replacement_right.my_tracker); + if (pivot_idx < l_size) { + // deal with left + push_copy_sources(replacement_left, left); + push_copy_sources(replacement_right, left); + if (left.is_pending()) { + move_child_ptrs(replacement_left, left, 0, 0, pivot_idx); + move_child_ptrs(replacement_right, left, 0, pivot_idx, l_size); + left.my_tracker = nullptr; + } + + // deal with right + push_copy_sources(replacement_right, right); + if (right.is_pending()) { + move_child_ptrs(replacement_right, right, l_size - pivot_idx, 0, r_size); + right.my_tracker= nullptr; + } + } else { + // deal with left + push_copy_sources(replacement_left, left); + if (left.is_pending()) { + move_child_ptrs(replacement_left, left, 0, 0, l_size); + left.my_tracker = nullptr; + } + + // deal with right + push_copy_sources(replacement_left, right); + push_copy_sources(replacement_right, right); + if (right.is_pending()) { + move_child_ptrs(replacement_left, right, l_size, 0, pivot_idx - l_size); + move_child_ptrs(replacement_right, right, 0, pivot_idx - l_size, r_size); + right.my_tracker= nullptr; + } + } + } + + void set_parent_tracker_from_prior_instance() { + assert(is_mutation_pending()); + auto &prior = (FixedKVNode&)(*get_prior_instance()); + if (range.is_root()) { + ceph_assert(prior.root_block); + ceph_assert(pending_for_transaction); + root_block = prior.root_block; + link_phy_tree_root_node(root_block, this); + return; + } + ceph_assert(!root_block); + take_prior_parent_tracker(); + assert(is_parent_valid()); + auto parent = get_parent_node<FixedKVNode>(); + //TODO: can this search be avoided? + auto off = parent->lower_bound_offset(get_node_meta().begin); + assert(parent->get_key_from_idx(off) == get_node_meta().begin); + parent->children[off] = this; + } + + bool is_children_empty() const { + for (auto it = children.begin(); + it != children.begin() + get_node_size(); + it++) { + if (is_valid_child_ptr(*it) + && (*it)->is_valid()) { + return false; + } + } + return true; + } + + void set_children_from_prior_instance() { + assert(get_prior_instance()); + auto &prior = (FixedKVNode&)(*get_prior_instance()); + assert(prior.my_tracker || prior.is_children_empty()); + + if (prior.my_tracker) { + prior.my_tracker->reset_parent(this); + my_tracker = prior.my_tracker; + // All my initial pending children is pointing to the original + // tracker which has been dropped by the above line, so need + // to adjust them to point to the new tracker + adjust_ptracker_for_children(); + } + assert(my_tracker || is_children_empty()); + } + + void adjust_ptracker_for_children() { + auto begin = children.begin(); + auto end = begin + get_node_size(); + ceph_assert(end <= children.end()); + for (auto it = begin; it != end; it++) { + auto child = *it; + if (is_valid_child_ptr(child)) { + set_child_ptracker(child); + } + } + } + + void on_delta_write(paddr_t record_block_offset) final { + // All in-memory relative addrs are necessarily record-relative + assert(get_prior_instance()); + assert(pending_for_transaction); + resolve_relative_addrs(record_block_offset); + } + + virtual uint16_t lower_bound_offset(node_key_t) const = 0; + virtual uint16_t upper_bound_offset(node_key_t) const = 0; + virtual uint16_t child_pos_for_key(node_key_t) const = 0; + + virtual bool validate_stable_children() = 0; + + template<typename iter_t> + uint16_t copy_children_from_stable_source( + FixedKVNode &source, + iter_t foreign_start_it, + iter_t foreign_end_it, + iter_t local_start_it) { + auto foreign_it = foreign_start_it, local_it = local_start_it; + while (foreign_it != foreign_end_it + && local_it.get_offset() < get_node_size()) + { + auto &child = children[local_it.get_offset()]; + if (foreign_it.get_key() == local_it.get_key()) { + // the foreign key is preserved + if (!child) { + child = source.children[foreign_it.get_offset()]; + } + foreign_it++; + local_it++; + } else if (foreign_it.get_key() < local_it.get_key()) { + // the foreign key has been removed, because, if it hasn't, + // there must have been a local key before the one pointed + // by the current "local_it" that's equal to this foreign key + // and has pushed the foreign_it forward. + foreign_it++; + } else { + // the local key must be a newly inserted one. + local_it++; + } + } + return local_it.get_offset(); + } + + template<typename Func> + void copy_children_from_stable_sources(Func &&get_iter) { + if (!copy_sources.empty()) { + auto it = --copy_sources.upper_bound(get_node_meta().begin); + auto &cs = *it; + uint16_t start_pos = cs->lower_bound_offset( + get_node_meta().begin); + if (start_pos == cs->get_node_size()) { + it++; + start_pos = 0; + } + uint16_t local_next_pos = 0; + for (; it != copy_sources.end(); it++) { + auto& copy_source = *it; + auto end_pos = copy_source->get_node_size(); + if (copy_source->get_node_meta().is_in_range(get_node_meta().end)) { + end_pos = copy_source->upper_bound_offset(get_node_meta().end); + } + auto local_start_iter = get_iter(*this, local_next_pos); + auto foreign_start_iter = get_iter(*copy_source, start_pos); + auto foreign_end_iter = get_iter(*copy_source, end_pos); + local_next_pos = copy_children_from_stable_source( + *copy_source, foreign_start_iter, foreign_end_iter, local_start_iter); + if (end_pos != copy_source->get_node_size()) { + break; + } + start_pos = 0; + } + } + } + + void on_invalidated(Transaction &t) final { + reset_parent_tracker(); + } + + bool is_rewrite() { + return is_initial_pending() && get_prior_instance(); + } + + void on_initial_write() final { + // All in-memory relative addrs are necessarily block-relative + resolve_relative_addrs(get_paddr()); + if (range.is_root()) { + reset_parent_tracker(); + } + assert(has_parent_tracker() ? (is_parent_valid()) : true); + } + + void set_child_ptracker(ChildableCachedExtent *child) { + if (!this->my_tracker) { + this->my_tracker = new parent_tracker_t(this); + } + child->reset_parent_tracker(this->my_tracker); + } + + void on_clean_read() final { + // From initial write of block, relative addrs are necessarily block-relative + resolve_relative_addrs(get_paddr()); + } + + virtual void resolve_relative_addrs(paddr_t base) = 0; +}; + +/** + * FixedKVInternalNode + * + * Abstracts operations on and layout of internal nodes for the + * FixedKVBTree. + */ +template < + size_t CAPACITY, + typename NODE_KEY, + typename NODE_KEY_LE, + size_t node_size, + typename node_type_t> +struct FixedKVInternalNode + : FixedKVNode<NODE_KEY>, + common::FixedKVNodeLayout< + CAPACITY, + fixed_kv_node_meta_t<NODE_KEY>, + fixed_kv_node_meta_le_t<NODE_KEY_LE>, + NODE_KEY, NODE_KEY_LE, + paddr_t, paddr_le_t> { + using Ref = TCachedExtentRef<node_type_t>; + using base_t = FixedKVNode<NODE_KEY>; + using base_ref = typename FixedKVNode<NODE_KEY>::FixedKVNodeRef; + using node_layout_t = + common::FixedKVNodeLayout< + CAPACITY, + fixed_kv_node_meta_t<NODE_KEY>, + fixed_kv_node_meta_le_t<NODE_KEY_LE>, + NODE_KEY, + NODE_KEY_LE, + paddr_t, + paddr_le_t>; + using internal_const_iterator_t = typename node_layout_t::const_iterator; + using internal_iterator_t = typename node_layout_t::iterator; + using this_type_t = FixedKVInternalNode< + CAPACITY, + NODE_KEY, + NODE_KEY_LE, + node_size, + node_type_t>; + + FixedKVInternalNode(ceph::bufferptr &&ptr) + : FixedKVNode<NODE_KEY>(CAPACITY, std::move(ptr)), + node_layout_t(this->get_bptr().c_str()) {} + FixedKVInternalNode(const FixedKVInternalNode &rhs) + : FixedKVNode<NODE_KEY>(rhs), + node_layout_t(this->get_bptr().c_str()) {} + + bool is_leaf_and_has_children() const final { + return false; + } + + uint16_t get_node_split_pivot() final { + return this->get_split_pivot().get_offset(); + } + + void prepare_commit() final { + if (this->is_initial_pending()) { + if (this->is_rewrite()) { + this->set_children_from_prior_instance(); + } + this->copy_children_from_stable_sources( + [this](base_t &node, uint16_t pos) { + ceph_assert(node.get_type() == this->get_type()); + auto &n = static_cast<this_type_t&>(node); + return n.iter_idx(pos); + } + ); + if (this->is_rewrite()) { + this->reset_prior_instance(); + } else { + this->adjust_ptracker_for_children(); + } + assert(this->validate_stable_children()); + this->copy_sources.clear(); + } + } + + get_child_ret_t<LogicalCachedExtent> + get_logical_child(op_context_t<NODE_KEY>, uint16_t pos) final { + ceph_abort("impossible"); + return get_child_ret_t<LogicalCachedExtent>(child_pos_t(nullptr, 0)); + } + + bool validate_stable_children() final { + LOG_PREFIX(FixedKVInternalNode::validate_stable_children); + if (this->children.empty()) { + return false; + } + + for (auto i : *this) { + auto child = (FixedKVNode<NODE_KEY>*)this->children[i.get_offset()]; + if (child && child->range.begin != i.get_key()) { + SUBERROR(seastore_fixedkv_tree, + "stable child not valid: child {}, child meta{}, key {}", + *child, + child->get_node_meta(), + i.get_key()); + ceph_abort(); + return false; + } + } + return true; + } + + virtual ~FixedKVInternalNode() { + if (this->is_valid() && !this->is_pending()) { + if (this->range.is_root()) { + ceph_assert(this->root_block); + unlink_phy_tree_root_node<NODE_KEY>(this->root_block); + } else { + ceph_assert(this->is_parent_valid()); + auto parent = this->template get_parent_node<FixedKVNode<NODE_KEY>>(); + auto off = parent->lower_bound_offset(this->get_meta().begin); + assert(parent->get_key_from_idx(off) == this->get_meta().begin); + assert(parent->children[off] == this); + parent->children[off] = nullptr; + } + } + } + + uint16_t lower_bound_offset(NODE_KEY key) const final { + return this->lower_bound(key).get_offset(); + } + + uint16_t upper_bound_offset(NODE_KEY key) const final { + return this->upper_bound(key).get_offset(); + } + + uint16_t child_pos_for_key(NODE_KEY key) const final { + auto it = this->upper_bound(key); + assert(it != this->begin()); + --it; + return it.get_offset(); + } + + NODE_KEY get_key_from_idx(uint16_t idx) const final { + return this->iter_idx(idx).get_key(); + } + + fixed_kv_node_meta_t<NODE_KEY> get_node_meta() const { + return this->get_meta(); + } + + uint16_t get_node_size() const final { + return this->get_size(); + } + + typename node_layout_t::delta_buffer_t delta_buffer; + typename node_layout_t::delta_buffer_t *maybe_get_delta_buffer() { + return this->is_mutation_pending() + ? &delta_buffer : nullptr; + } + + CachedExtentRef duplicate_for_write(Transaction&) override { + assert(delta_buffer.empty()); + return CachedExtentRef(new node_type_t(*this)); + }; + + void on_replace_prior(Transaction&) final { + ceph_assert(!this->is_rewrite()); + this->set_children_from_prior_instance(); + auto &prior = (this_type_t&)(*this->get_prior_instance()); + auto copied = this->copy_children_from_stable_source( + prior, + prior.begin(), + prior.end(), + this->begin()); + ceph_assert(copied <= get_node_size()); + assert(this->validate_stable_children()); + this->set_parent_tracker_from_prior_instance(); + } + + void update( + internal_const_iterator_t iter, + paddr_t addr, + FixedKVNode<NODE_KEY>* nextent) { + LOG_PREFIX(FixedKVInternalNode::update); + SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, {}", + this->pending_for_transaction, + iter.get_offset(), + *nextent); + this->update_child_ptr(iter, nextent); + return this->journal_update( + iter, + this->maybe_generate_relative(addr), + maybe_get_delta_buffer()); + } + + void insert( + internal_const_iterator_t iter, + NODE_KEY pivot, + paddr_t addr, + FixedKVNode<NODE_KEY>* nextent) { + LOG_PREFIX(FixedKVInternalNode::insert); + SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, key {}, {}", + this->pending_for_transaction, + iter.get_offset(), + pivot, + *nextent); + this->insert_child_ptr(iter, nextent); + return this->journal_insert( + iter, + pivot, + this->maybe_generate_relative(addr), + maybe_get_delta_buffer()); + } + + void remove(internal_const_iterator_t iter) { + LOG_PREFIX(FixedKVInternalNode::remove); + SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, key {}", + this->pending_for_transaction, + iter.get_offset(), + iter.get_key()); + this->remove_child_ptr(iter); + return this->journal_remove( + iter, + maybe_get_delta_buffer()); + } + + void replace( + internal_const_iterator_t iter, + NODE_KEY pivot, + paddr_t addr, + FixedKVNode<NODE_KEY>* nextent) { + LOG_PREFIX(FixedKVInternalNode::replace); + SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, old key {}, key {}, {}", + this->pending_for_transaction, + iter.get_offset(), + iter.get_key(), + pivot, + *nextent); + this->update_child_ptr(iter, nextent); + return this->journal_replace( + iter, + pivot, + this->maybe_generate_relative(addr), + maybe_get_delta_buffer()); + } + + std::tuple<Ref, Ref, NODE_KEY> + make_split_children(op_context_t<NODE_KEY> c) { + auto left = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + auto right = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + this->split_child_ptrs(*left, *right); + auto pivot = this->split_into(*left, *right); + left->range = left->get_meta(); + right->range = right->get_meta(); + return std::make_tuple( + left, + right, + pivot); + } + + Ref make_full_merge( + op_context_t<NODE_KEY> c, + Ref &right) { + auto replacement = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + replacement->merge_child_ptrs(*this, *right); + replacement->merge_from(*this, *right->template cast<node_type_t>()); + replacement->range = replacement->get_meta(); + return replacement; + } + + std::tuple<Ref, Ref, NODE_KEY> + make_balanced( + op_context_t<NODE_KEY> c, + Ref &_right, + bool prefer_left) { + ceph_assert(_right->get_type() == this->get_type()); + auto &right = *_right->template cast<node_type_t>(); + auto replacement_left = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + auto replacement_right = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + + auto pivot = this->balance_into_new_nodes( + *this, + right, + prefer_left, + *replacement_left, + *replacement_right); + this->balance_child_ptrs( + *this, + right, + prefer_left, + *replacement_left, + *replacement_right); + + replacement_left->range = replacement_left->get_meta(); + replacement_right->range = replacement_right->get_meta(); + return std::make_tuple( + replacement_left, + replacement_right, + pivot); + } + + /** + * Internal relative addresses on read or in memory prior to commit + * are either record or block relative depending on whether this + * physical node is is_initial_pending() or just is_mutable(). + * + * User passes appropriate base depending on lifecycle and + * resolve_relative_addrs fixes up relative internal references + * based on base. + */ + void resolve_relative_addrs(paddr_t base) + { + LOG_PREFIX(FixedKVInternalNode::resolve_relative_addrs); + for (auto i: *this) { + if (i->get_val().is_relative()) { + auto updated = base.add_relative(i->get_val()); + SUBTRACE(seastore_fixedkv_tree, "{} -> {}", i->get_val(), updated); + i->set_val(updated); + } + } + } + + void node_resolve_vals( + internal_iterator_t from, + internal_iterator_t to) const { + if (this->is_initial_pending()) { + for (auto i = from; i != to; ++i) { + if (i->get_val().is_relative()) { + assert(i->get_val().is_block_relative()); + i->set_val(this->get_paddr().add_relative(i->get_val())); + } + } + } + } + void node_unresolve_vals( + internal_iterator_t from, + internal_iterator_t to) const { + if (this->is_initial_pending()) { + for (auto i = from; i != to; ++i) { + if (i->get_val().is_relative()) { + assert(i->get_val().is_record_relative()); + i->set_val(i->get_val().block_relative_to(this->get_paddr())); + } + } + } + } + + std::ostream &_print_detail(std::ostream &out) const + { + out << ", size=" << this->get_size() + << ", meta=" << this->get_meta() + << ", my_tracker=" << (void*)this->my_tracker; + if (this->my_tracker) { + out << ", my_tracker->parent=" << (void*)this->my_tracker->get_parent().get(); + } + return out << ", root_block=" << (void*)this->root_block.get(); + } + + ceph::bufferlist get_delta() { + ceph::buffer::ptr bptr(delta_buffer.get_bytes()); + delta_buffer.copy_out(bptr.c_str(), bptr.length()); + ceph::bufferlist bl; + bl.push_back(bptr); + return bl; + } + + void apply_delta_and_adjust_crc( + paddr_t base, const ceph::bufferlist &_bl) { + assert(_bl.length()); + ceph::bufferlist bl = _bl; + bl.rebuild(); + typename node_layout_t::delta_buffer_t buffer; + buffer.copy_in(bl.front().c_str(), bl.front().length()); + buffer.replay(*this); + this->set_last_committed_crc(this->get_crc32c()); + resolve_relative_addrs(base); + } + + constexpr static size_t get_min_capacity() { + return (node_layout_t::get_capacity() - 1) / 2; + } + + bool at_max_capacity() const { + assert(this->get_size() <= node_layout_t::get_capacity()); + return this->get_size() == node_layout_t::get_capacity(); + } + + bool at_min_capacity() const { + assert(this->get_size() >= (get_min_capacity() - 1)); + return this->get_size() <= get_min_capacity(); + } + + bool below_min_capacity() const { + assert(this->get_size() >= (get_min_capacity() - 1)); + return this->get_size() < get_min_capacity(); + } +}; + +template < + size_t CAPACITY, + typename NODE_KEY, + typename NODE_KEY_LE, + typename VAL, + typename VAL_LE, + size_t node_size, + typename node_type_t, + bool has_children> +struct FixedKVLeafNode + : FixedKVNode<NODE_KEY>, + common::FixedKVNodeLayout< + CAPACITY, + fixed_kv_node_meta_t<NODE_KEY>, + fixed_kv_node_meta_le_t<NODE_KEY_LE>, + NODE_KEY, NODE_KEY_LE, + VAL, VAL_LE> { + using Ref = TCachedExtentRef<node_type_t>; + using node_layout_t = + common::FixedKVNodeLayout< + CAPACITY, + fixed_kv_node_meta_t<NODE_KEY>, + fixed_kv_node_meta_le_t<NODE_KEY_LE>, + NODE_KEY, + NODE_KEY_LE, + VAL, + VAL_LE>; + using internal_const_iterator_t = typename node_layout_t::const_iterator; + using this_type_t = FixedKVLeafNode< + CAPACITY, + NODE_KEY, + NODE_KEY_LE, + VAL, + VAL_LE, + node_size, + node_type_t, + has_children>; + using base_t = FixedKVNode<NODE_KEY>; + FixedKVLeafNode(ceph::bufferptr &&ptr) + : FixedKVNode<NODE_KEY>(has_children ? CAPACITY : 0, std::move(ptr)), + node_layout_t(this->get_bptr().c_str()) {} + FixedKVLeafNode(const FixedKVLeafNode &rhs) + : FixedKVNode<NODE_KEY>(rhs), + node_layout_t(this->get_bptr().c_str()) {} + + static constexpr bool do_has_children = has_children; + + bool is_leaf_and_has_children() const final { + return has_children; + } + + uint16_t get_node_split_pivot() final { + return this->get_split_pivot().get_offset(); + } + + get_child_ret_t<LogicalCachedExtent> + get_logical_child(op_context_t<NODE_KEY> c, uint16_t pos) final { + auto child = this->children[pos]; + if (is_valid_child_ptr(child)) { + ceph_assert(child->is_logical()); + return c.cache.template get_extent_viewable_by_trans< + LogicalCachedExtent>(c.trans, (LogicalCachedExtent*)child); + } else if (this->is_pending()) { + auto key = this->iter_idx(pos).get_key(); + auto &sparent = this->get_stable_for_key(key); + auto spos = sparent.child_pos_for_key(key); + auto child = sparent.children[spos]; + if (is_valid_child_ptr(child)) { + ceph_assert(child->is_logical()); + return c.cache.template get_extent_viewable_by_trans< + LogicalCachedExtent>(c.trans, (LogicalCachedExtent*)child); + } else { + return child_pos_t(&sparent, spos); + } + } else { + return child_pos_t(this, pos); + } + } + + bool validate_stable_children() override { + return true; + } + + virtual ~FixedKVLeafNode() { + if (this->is_valid() && !this->is_pending()) { + if (this->range.is_root()) { + ceph_assert(this->root_block); + unlink_phy_tree_root_node<NODE_KEY>(this->root_block); + } else { + ceph_assert(this->is_parent_valid()); + auto parent = this->template get_parent_node<FixedKVNode<NODE_KEY>>(); + auto off = parent->lower_bound_offset(this->get_meta().begin); + assert(parent->get_key_from_idx(off) == this->get_meta().begin); + assert(parent->children[off] == this); + parent->children[off] = nullptr; + } + } + } + + void prepare_commit() final { + if constexpr (has_children) { + if (this->is_initial_pending()) { + if (this->is_rewrite()) { + this->set_children_from_prior_instance(); + } + this->copy_children_from_stable_sources( + [this](base_t &node, uint16_t pos) { + ceph_assert(node.get_type() == this->get_type()); + auto &n = static_cast<this_type_t&>(node); + return n.iter_idx(pos); + } + ); + if (this->is_rewrite()) { + this->reset_prior_instance(); + } else { + this->adjust_ptracker_for_children(); + } + assert(this->validate_stable_children()); + this->copy_sources.clear(); + } + } + assert(this->is_initial_pending() + ? this->copy_sources.empty(): + true); + } + + void on_replace_prior(Transaction&) final { + ceph_assert(!this->is_rewrite()); + if constexpr (has_children) { + this->set_children_from_prior_instance(); + auto &prior = (this_type_t&)(*this->get_prior_instance()); + auto copied = this->copy_children_from_stable_source( + prior, + prior.begin(), + prior.end(), + this->begin()); + ceph_assert(copied <= get_node_size()); + assert(this->validate_stable_children()); + this->set_parent_tracker_from_prior_instance(); + } else { + this->set_parent_tracker_from_prior_instance(); + } + } + + uint16_t lower_bound_offset(NODE_KEY key) const final { + return this->lower_bound(key).get_offset(); + } + + uint16_t upper_bound_offset(NODE_KEY key) const final { + return this->upper_bound(key).get_offset(); + } + + uint16_t child_pos_for_key(NODE_KEY key) const final { + return lower_bound_offset(key); + } + + NODE_KEY get_key_from_idx(uint16_t idx) const final { + return this->iter_idx(idx).get_key(); + } + + fixed_kv_node_meta_t<NODE_KEY> get_node_meta() const { + return this->get_meta(); + } + + uint16_t get_node_size() const final { + return this->get_size(); + } + + typename node_layout_t::delta_buffer_t delta_buffer; + virtual typename node_layout_t::delta_buffer_t *maybe_get_delta_buffer() { + return this->is_mutation_pending() ? &delta_buffer : nullptr; + } + + CachedExtentRef duplicate_for_write(Transaction&) override { + assert(delta_buffer.empty()); + return CachedExtentRef(new node_type_t(*this)); + }; + + virtual void update( + internal_const_iterator_t iter, + VAL val, + LogicalCachedExtent* nextent) = 0; + virtual internal_const_iterator_t insert( + internal_const_iterator_t iter, + NODE_KEY addr, + VAL val, + LogicalCachedExtent* nextent) = 0; + virtual void remove(internal_const_iterator_t iter) = 0; + + std::tuple<Ref, Ref, NODE_KEY> + make_split_children(op_context_t<NODE_KEY> c) { + auto left = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + auto right = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + if constexpr (has_children) { + this->split_child_ptrs(*left, *right); + } + auto pivot = this->split_into(*left, *right); + left->range = left->get_meta(); + right->range = right->get_meta(); + return std::make_tuple( + left, + right, + pivot); + } + + Ref make_full_merge( + op_context_t<NODE_KEY> c, + Ref &right) { + auto replacement = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + if constexpr (has_children) { + replacement->merge_child_ptrs(*this, *right); + } + replacement->merge_from(*this, *right->template cast<node_type_t>()); + replacement->range = replacement->get_meta(); + return replacement; + } + + std::tuple<Ref, Ref, NODE_KEY> + make_balanced( + op_context_t<NODE_KEY> c, + Ref &_right, + bool prefer_left) { + ceph_assert(_right->get_type() == this->get_type()); + auto &right = *_right->template cast<node_type_t>(); + auto replacement_left = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + auto replacement_right = c.cache.template alloc_new_extent<node_type_t>( + c.trans, node_size, placement_hint_t::HOT, INIT_GENERATION); + + auto pivot = this->balance_into_new_nodes( + *this, + right, + prefer_left, + *replacement_left, + *replacement_right); + if constexpr (has_children) { + this->balance_child_ptrs( + *this, + right, + prefer_left, + *replacement_left, + *replacement_right); + } + + replacement_left->range = replacement_left->get_meta(); + replacement_right->range = replacement_right->get_meta(); + return std::make_tuple( + replacement_left, + replacement_right, + pivot); + } + + ceph::bufferlist get_delta() { + ceph::buffer::ptr bptr(delta_buffer.get_bytes()); + delta_buffer.copy_out(bptr.c_str(), bptr.length()); + ceph::bufferlist bl; + bl.push_back(bptr); + return bl; + } + + void apply_delta_and_adjust_crc( + paddr_t base, const ceph::bufferlist &_bl) { + assert(_bl.length()); + ceph::bufferlist bl = _bl; + bl.rebuild(); + typename node_layout_t::delta_buffer_t buffer; + buffer.copy_in(bl.front().c_str(), bl.front().length()); + buffer.replay(*this); + this->set_last_committed_crc(this->get_crc32c()); + this->resolve_relative_addrs(base); + } + + std::ostream &_print_detail(std::ostream &out) const + { + return out << ", size=" << this->get_size() + << ", meta=" << this->get_meta(); + } + + constexpr static size_t get_min_capacity() { + return (node_layout_t::get_capacity() - 1) / 2; + } + + bool at_max_capacity() const { + assert(this->get_size() <= node_layout_t::get_capacity()); + return this->get_size() == node_layout_t::get_capacity(); + } + + bool at_min_capacity() const { + assert(this->get_size() >= (get_min_capacity() - 1)); + return this->get_size() <= get_min_capacity(); + } + + bool below_min_capacity() const { + assert(this->get_size() >= (get_min_capacity() - 1)); + return this->get_size() < get_min_capacity(); + } +}; + +} // namespace crimson::os::seastore + +#if FMT_VERSION >= 90000 +template <> +struct fmt::formatter< + crimson::os::seastore::FixedKVNode< + crimson::os::seastore::laddr_t>> : fmt::ostream_formatter {}; +template <> +struct fmt::formatter< + crimson::os::seastore::FixedKVNode< + crimson::os::seastore::paddr_t>> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/cache.cc b/src/crimson/os/seastore/cache.cc new file mode 100644 index 000000000..4d1dc9296 --- /dev/null +++ b/src/crimson/os/seastore/cache.cc @@ -0,0 +1,2040 @@ +// -*- 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 <sstream> +#include <string_view> + +#include <seastar/core/metrics.hh> + +#include "crimson/os/seastore/logging.h" +#include "crimson/common/config_proxy.h" +#include "crimson/os/seastore/async_cleaner.h" + +// included for get_extent_by_type +#include "crimson/os/seastore/collection_manager/collection_flat_node.h" +#include "crimson/os/seastore/lba_manager/btree/lba_btree_node.h" +#include "crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.h" +#include "crimson/os/seastore/object_data_handler.h" +#include "crimson/os/seastore/collection_manager/collection_flat_node.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/seastore.h" +#include "crimson/os/seastore/backref/backref_tree_node.h" +#include "test/crimson/seastore/test_block.h" + +using std::string_view; + +SET_SUBSYS(seastore_cache); + +namespace crimson::os::seastore { + +std::ostream &operator<<(std::ostream &out, const backref_entry_t &ent) { + return out << "backref_entry_t{" + << ent.paddr << "~" << ent.len << ", " + << "laddr: " << ent.laddr << ", " + << "type: " << ent.type << ", " + << "seq: " << ent.seq << ", " + << "}"; +} + +Cache::Cache( + ExtentPlacementManager &epm) + : epm(epm), + lru(crimson::common::get_conf<Option::size_t>( + "seastore_cache_lru_size")) +{ + LOG_PREFIX(Cache::Cache); + INFO("created, lru_size={}", lru.get_capacity()); + register_metrics(); + segment_providers_by_device_id.resize(DEVICE_ID_MAX, nullptr); +} + +Cache::~Cache() +{ + LOG_PREFIX(Cache::~Cache); + for (auto &i: extents) { + ERROR("extent is still alive -- {}", i); + } + ceph_assert(extents.empty()); +} + +Cache::retire_extent_ret Cache::retire_extent_addr( + Transaction &t, paddr_t addr, extent_len_t length) +{ + LOG_PREFIX(Cache::retire_extent_addr); + TRACET("retire {}~{}", t, addr, length); + + assert(addr.is_real() && !addr.is_block_relative()); + + CachedExtentRef ext; + auto result = t.get_extent(addr, &ext); + if (result == Transaction::get_extent_ret::PRESENT) { + DEBUGT("retire {}~{} on t -- {}", t, addr, length, *ext); + t.add_to_retired_set(CachedExtentRef(&*ext)); + return retire_extent_iertr::now(); + } else if (result == Transaction::get_extent_ret::RETIRED) { + ERRORT("retire {}~{} failed, already retired -- {}", t, addr, length, *ext); + ceph_abort(); + } + + // any relative addr must have been on the transaction + assert(!addr.is_relative()); + + // absent from transaction + // retiring is not included by the cache hit metrics + ext = query_cache(addr, nullptr); + if (ext) { + DEBUGT("retire {}~{} in cache -- {}", t, addr, length, *ext); + } else { + // add a new placeholder to Cache + ext = CachedExtent::make_cached_extent_ref< + RetiredExtentPlaceholder>(length); + ext->init(CachedExtent::extent_state_t::CLEAN, + addr, + PLACEMENT_HINT_NULL, + NULL_GENERATION, + TRANS_ID_NULL); + DEBUGT("retire {}~{} as placeholder, add extent -- {}", + t, addr, length, *ext); + const auto t_src = t.get_src(); + add_extent(ext, &t_src); + } + t.add_to_read_set(ext); + t.add_to_retired_set(ext); + return retire_extent_iertr::now(); +} + +void Cache::dump_contents() +{ + LOG_PREFIX(Cache::dump_contents); + DEBUG("enter"); + for (auto &&i: extents) { + DEBUG("live {}", i); + } + DEBUG("exit"); +} + +void Cache::register_metrics() +{ + LOG_PREFIX(Cache::register_metrics); + DEBUG(""); + + stats = {}; + + namespace sm = seastar::metrics; + using src_t = Transaction::src_t; + + std::map<src_t, sm::label_instance> labels_by_src { + {src_t::MUTATE, sm::label_instance("src", "MUTATE")}, + {src_t::READ, sm::label_instance("src", "READ")}, + {src_t::TRIM_DIRTY, sm::label_instance("src", "TRIM_DIRTY")}, + {src_t::TRIM_ALLOC, sm::label_instance("src", "TRIM_ALLOC")}, + {src_t::CLEANER_MAIN, sm::label_instance("src", "CLEANER_MAIN")}, + {src_t::CLEANER_COLD, sm::label_instance("src", "CLEANER_COLD")}, + }; + assert(labels_by_src.size() == (std::size_t)src_t::MAX); + + std::map<extent_types_t, sm::label_instance> labels_by_ext { + {extent_types_t::ROOT, sm::label_instance("ext", "ROOT")}, + {extent_types_t::LADDR_INTERNAL, sm::label_instance("ext", "LADDR_INTERNAL")}, + {extent_types_t::LADDR_LEAF, sm::label_instance("ext", "LADDR_LEAF")}, + {extent_types_t::DINK_LADDR_LEAF, sm::label_instance("ext", "DINK_LADDR_LEAF")}, + {extent_types_t::OMAP_INNER, sm::label_instance("ext", "OMAP_INNER")}, + {extent_types_t::OMAP_LEAF, sm::label_instance("ext", "OMAP_LEAF")}, + {extent_types_t::ONODE_BLOCK_STAGED, sm::label_instance("ext", "ONODE_BLOCK_STAGED")}, + {extent_types_t::COLL_BLOCK, sm::label_instance("ext", "COLL_BLOCK")}, + {extent_types_t::OBJECT_DATA_BLOCK, sm::label_instance("ext", "OBJECT_DATA_BLOCK")}, + {extent_types_t::RETIRED_PLACEHOLDER, sm::label_instance("ext", "RETIRED_PLACEHOLDER")}, + {extent_types_t::ALLOC_INFO, sm::label_instance("ext", "ALLOC_INFO")}, + {extent_types_t::JOURNAL_TAIL, sm::label_instance("ext", "JOURNAL_TAIL")}, + {extent_types_t::TEST_BLOCK, sm::label_instance("ext", "TEST_BLOCK")}, + {extent_types_t::TEST_BLOCK_PHYSICAL, sm::label_instance("ext", "TEST_BLOCK_PHYSICAL")}, + {extent_types_t::BACKREF_INTERNAL, sm::label_instance("ext", "BACKREF_INTERNAL")}, + {extent_types_t::BACKREF_LEAF, sm::label_instance("ext", "BACKREF_LEAF")} + }; + assert(labels_by_ext.size() == (std::size_t)extent_types_t::NONE); + + /* + * trans_created + */ + for (auto& [src, src_label] : labels_by_src) { + metrics.add_group( + "cache", + { + sm::make_counter( + "trans_created", + get_by_src(stats.trans_created_by_src, src), + sm::description("total number of transaction created"), + {src_label} + ), + } + ); + } + + /* + * cache_query: cache_access and cache_hit + */ + for (auto& [src, src_label] : labels_by_src) { + metrics.add_group( + "cache", + { + sm::make_counter( + "cache_access", + get_by_src(stats.cache_query_by_src, src).access, + sm::description("total number of cache accesses"), + {src_label} + ), + sm::make_counter( + "cache_hit", + get_by_src(stats.cache_query_by_src, src).hit, + sm::description("total number of cache hits"), + {src_label} + ), + } + ); + } + + { + /* + * efforts discarded/committed + */ + auto effort_label = sm::label("effort"); + + // invalidated efforts + using namespace std::literals::string_view_literals; + const string_view invalidated_effort_names[] = { + "READ"sv, + "MUTATE"sv, + "RETIRE"sv, + "FRESH"sv, + "FRESH_OOL_WRITTEN"sv, + }; + for (auto& [src, src_label] : labels_by_src) { + auto& efforts = get_by_src(stats.invalidated_efforts_by_src, src); + for (auto& [ext, ext_label] : labels_by_ext) { + auto& counter = get_by_ext(efforts.num_trans_invalidated, ext); + metrics.add_group( + "cache", + { + sm::make_counter( + "trans_invalidated_by_extent", + counter, + sm::description("total number of transactions invalidated by extents"), + {src_label, ext_label} + ), + } + ); + } + + if (src == src_t::READ) { + // read transaction won't have non-read efforts + auto read_effort_label = effort_label("READ"); + metrics.add_group( + "cache", + { + sm::make_counter( + "invalidated_extents", + efforts.read.num, + sm::description("extents of invalidated transactions"), + {src_label, read_effort_label} + ), + sm::make_counter( + "invalidated_extent_bytes", + efforts.read.bytes, + sm::description("extent bytes of invalidated transactions"), + {src_label, read_effort_label} + ), + } + ); + continue; + } + + // non READ invalidated efforts + for (auto& effort_name : invalidated_effort_names) { + auto& effort = [&effort_name, &efforts]() -> io_stat_t& { + if (effort_name == "READ") { + return efforts.read; + } else if (effort_name == "MUTATE") { + return efforts.mutate; + } else if (effort_name == "RETIRE") { + return efforts.retire; + } else if (effort_name == "FRESH") { + return efforts.fresh; + } else { + assert(effort_name == "FRESH_OOL_WRITTEN"); + return efforts.fresh_ool_written; + } + }(); + metrics.add_group( + "cache", + { + sm::make_counter( + "invalidated_extents", + effort.num, + sm::description("extents of invalidated transactions"), + {src_label, effort_label(effort_name)} + ), + sm::make_counter( + "invalidated_extent_bytes", + effort.bytes, + sm::description("extent bytes of invalidated transactions"), + {src_label, effort_label(effort_name)} + ), + } + ); + } // effort_name + + metrics.add_group( + "cache", + { + sm::make_counter( + "trans_invalidated", + efforts.total_trans_invalidated, + sm::description("total number of transactions invalidated"), + {src_label} + ), + sm::make_counter( + "invalidated_delta_bytes", + efforts.mutate_delta_bytes, + sm::description("delta bytes of invalidated transactions"), + {src_label} + ), + sm::make_counter( + "invalidated_ool_records", + efforts.num_ool_records, + sm::description("number of ool-records from invalidated transactions"), + {src_label} + ), + sm::make_counter( + "invalidated_ool_record_bytes", + efforts.ool_record_bytes, + sm::description("bytes of ool-record from invalidated transactions"), + {src_label} + ), + } + ); + } // src + + // committed efforts + const string_view committed_effort_names[] = { + "READ"sv, + "MUTATE"sv, + "RETIRE"sv, + "FRESH_INVALID"sv, + "FRESH_INLINE"sv, + "FRESH_OOL"sv, + }; + for (auto& [src, src_label] : labels_by_src) { + if (src == src_t::READ) { + // READ transaction won't commit + continue; + } + auto& efforts = get_by_src(stats.committed_efforts_by_src, src); + metrics.add_group( + "cache", + { + sm::make_counter( + "trans_committed", + efforts.num_trans, + sm::description("total number of transaction committed"), + {src_label} + ), + sm::make_counter( + "committed_ool_records", + efforts.num_ool_records, + sm::description("number of ool-records from committed transactions"), + {src_label} + ), + sm::make_counter( + "committed_ool_record_metadata_bytes", + efforts.ool_record_metadata_bytes, + sm::description("bytes of ool-record metadata from committed transactions"), + {src_label} + ), + sm::make_counter( + "committed_ool_record_data_bytes", + efforts.ool_record_data_bytes, + sm::description("bytes of ool-record data from committed transactions"), + {src_label} + ), + sm::make_counter( + "committed_inline_record_metadata_bytes", + efforts.inline_record_metadata_bytes, + sm::description("bytes of inline-record metadata from committed transactions" + "(excludes delta buffer)"), + {src_label} + ), + } + ); + for (auto& effort_name : committed_effort_names) { + auto& effort_by_ext = [&efforts, &effort_name]() + -> counter_by_extent_t<io_stat_t>& { + if (effort_name == "READ") { + return efforts.read_by_ext; + } else if (effort_name == "MUTATE") { + return efforts.mutate_by_ext; + } else if (effort_name == "RETIRE") { + return efforts.retire_by_ext; + } else if (effort_name == "FRESH_INVALID") { + return efforts.fresh_invalid_by_ext; + } else if (effort_name == "FRESH_INLINE") { + return efforts.fresh_inline_by_ext; + } else { + assert(effort_name == "FRESH_OOL"); + return efforts.fresh_ool_by_ext; + } + }(); + for (auto& [ext, ext_label] : labels_by_ext) { + auto& effort = get_by_ext(effort_by_ext, ext); + metrics.add_group( + "cache", + { + sm::make_counter( + "committed_extents", + effort.num, + sm::description("extents of committed transactions"), + {src_label, effort_label(effort_name), ext_label} + ), + sm::make_counter( + "committed_extent_bytes", + effort.bytes, + sm::description("extent bytes of committed transactions"), + {src_label, effort_label(effort_name), ext_label} + ), + } + ); + } // ext + } // effort_name + + auto& delta_by_ext = efforts.delta_bytes_by_ext; + for (auto& [ext, ext_label] : labels_by_ext) { + auto& value = get_by_ext(delta_by_ext, ext); + metrics.add_group( + "cache", + { + sm::make_counter( + "committed_delta_bytes", + value, + sm::description("delta bytes of committed transactions"), + {src_label, ext_label} + ), + } + ); + } // ext + } // src + + // successful read efforts + metrics.add_group( + "cache", + { + sm::make_counter( + "trans_read_successful", + stats.success_read_efforts.num_trans, + sm::description("total number of successful read transactions") + ), + sm::make_counter( + "successful_read_extents", + stats.success_read_efforts.read.num, + sm::description("extents of successful read transactions") + ), + sm::make_counter( + "successful_read_extent_bytes", + stats.success_read_efforts.read.bytes, + sm::description("extent bytes of successful read transactions") + ), + } + ); + } + + /** + * Cached extents (including placeholders) + * + * Dirty extents + */ + metrics.add_group( + "cache", + { + sm::make_counter( + "cached_extents", + [this] { + return extents.size(); + }, + sm::description("total number of cached extents") + ), + sm::make_counter( + "cached_extent_bytes", + [this] { + return extents.get_bytes(); + }, + sm::description("total bytes of cached extents") + ), + sm::make_counter( + "dirty_extents", + [this] { + return dirty.size(); + }, + sm::description("total number of dirty extents") + ), + sm::make_counter( + "dirty_extent_bytes", + stats.dirty_bytes, + sm::description("total bytes of dirty extents") + ), + sm::make_counter( + "cache_lru_size_bytes", + [this] { + return lru.get_current_contents_bytes(); + }, + sm::description("total bytes pinned by the lru") + ), + sm::make_counter( + "cache_lru_size_extents", + [this] { + return lru.get_current_contents_extents(); + }, + sm::description("total extents pinned by the lru") + ), + } + ); + + /** + * tree stats + */ + auto tree_label = sm::label("tree"); + auto onode_label = tree_label("ONODE"); + auto omap_label = tree_label("OMAP"); + auto lba_label = tree_label("LBA"); + auto backref_label = tree_label("BACKREF"); + auto register_tree_metrics = [&labels_by_src, &onode_label, &omap_label, this]( + const sm::label_instance& tree_label, + uint64_t& tree_depth, + int64_t& tree_extents_num, + counter_by_src_t<tree_efforts_t>& committed_tree_efforts, + counter_by_src_t<tree_efforts_t>& invalidated_tree_efforts) { + metrics.add_group( + "cache", + { + sm::make_counter( + "tree_depth", + tree_depth, + sm::description("the depth of tree"), + {tree_label} + ), + sm::make_counter( + "tree_extents_num", + tree_extents_num, + sm::description("num of extents of the tree"), + {tree_label} + ) + } + ); + for (auto& [src, src_label] : labels_by_src) { + if (src == src_t::READ) { + // READ transaction won't contain any tree inserts and erases + continue; + } + if (is_background_transaction(src) && + (tree_label == onode_label || + tree_label == omap_label)) { + // CLEANER transaction won't contain any onode/omap tree operations + continue; + } + auto& committed_efforts = get_by_src(committed_tree_efforts, src); + auto& invalidated_efforts = get_by_src(invalidated_tree_efforts, src); + metrics.add_group( + "cache", + { + sm::make_counter( + "tree_inserts_committed", + committed_efforts.num_inserts, + sm::description("total number of committed insert operations"), + {tree_label, src_label} + ), + sm::make_counter( + "tree_erases_committed", + committed_efforts.num_erases, + sm::description("total number of committed erase operations"), + {tree_label, src_label} + ), + sm::make_counter( + "tree_updates_committed", + committed_efforts.num_updates, + sm::description("total number of committed update operations"), + {tree_label, src_label} + ), + sm::make_counter( + "tree_inserts_invalidated", + invalidated_efforts.num_inserts, + sm::description("total number of invalidated insert operations"), + {tree_label, src_label} + ), + sm::make_counter( + "tree_erases_invalidated", + invalidated_efforts.num_erases, + sm::description("total number of invalidated erase operations"), + {tree_label, src_label} + ), + sm::make_counter( + "tree_updates_invalidated", + invalidated_efforts.num_updates, + sm::description("total number of invalidated update operations"), + {tree_label, src_label} + ), + } + ); + } + }; + register_tree_metrics( + onode_label, + stats.onode_tree_depth, + stats.onode_tree_extents_num, + stats.committed_onode_tree_efforts, + stats.invalidated_onode_tree_efforts); + register_tree_metrics( + omap_label, + stats.omap_tree_depth, + stats.omap_tree_extents_num, + stats.committed_omap_tree_efforts, + stats.invalidated_omap_tree_efforts); + register_tree_metrics( + lba_label, + stats.lba_tree_depth, + stats.lba_tree_extents_num, + stats.committed_lba_tree_efforts, + stats.invalidated_lba_tree_efforts); + register_tree_metrics( + backref_label, + stats.backref_tree_depth, + stats.backref_tree_extents_num, + stats.committed_backref_tree_efforts, + stats.invalidated_backref_tree_efforts); + + /** + * conflict combinations + */ + auto srcs_label = sm::label("srcs"); + auto num_srcs = static_cast<std::size_t>(Transaction::src_t::MAX); + std::size_t srcs_index = 0; + for (uint8_t src2_int = 0; src2_int < num_srcs; ++src2_int) { + auto src2 = static_cast<Transaction::src_t>(src2_int); + for (uint8_t src1_int = src2_int; src1_int < num_srcs; ++src1_int) { + ++srcs_index; + auto src1 = static_cast<Transaction::src_t>(src1_int); + // impossible combinations + // should be consistent with checks in account_conflict() + if ((src1 == Transaction::src_t::READ && + src2 == Transaction::src_t::READ) || + (src1 == Transaction::src_t::TRIM_DIRTY && + src2 == Transaction::src_t::TRIM_DIRTY) || + (src1 == Transaction::src_t::CLEANER_MAIN && + src2 == Transaction::src_t::CLEANER_MAIN) || + (src1 == Transaction::src_t::CLEANER_COLD && + src2 == Transaction::src_t::CLEANER_COLD) || + (src1 == Transaction::src_t::TRIM_ALLOC && + src2 == Transaction::src_t::TRIM_ALLOC)) { + continue; + } + std::ostringstream oss; + oss << src1 << "," << src2; + metrics.add_group( + "cache", + { + sm::make_counter( + "trans_srcs_invalidated", + stats.trans_conflicts_by_srcs[srcs_index - 1], + sm::description("total number conflicted transactions by src pair"), + {srcs_label(oss.str())} + ), + } + ); + } + } + assert(srcs_index == NUM_SRC_COMB); + srcs_index = 0; + for (uint8_t src_int = 0; src_int < num_srcs; ++src_int) { + ++srcs_index; + auto src = static_cast<Transaction::src_t>(src_int); + std::ostringstream oss; + oss << "UNKNOWN," << src; + metrics.add_group( + "cache", + { + sm::make_counter( + "trans_srcs_invalidated", + stats.trans_conflicts_by_unknown[srcs_index - 1], + sm::description("total number conflicted transactions by src pair"), + {srcs_label(oss.str())} + ), + } + ); + } + + /** + * rewrite version + */ + metrics.add_group( + "cache", + { + sm::make_counter( + "version_count_dirty", + stats.committed_dirty_version.num, + sm::description("total number of rewrite-dirty extents") + ), + sm::make_counter( + "version_sum_dirty", + stats.committed_dirty_version.version, + sm::description("sum of the version from rewrite-dirty extents") + ), + sm::make_counter( + "version_count_reclaim", + stats.committed_reclaim_version.num, + sm::description("total number of rewrite-reclaim extents") + ), + sm::make_counter( + "version_sum_reclaim", + stats.committed_reclaim_version.version, + sm::description("sum of the version from rewrite-reclaim extents") + ), + } + ); +} + +void Cache::add_extent( + CachedExtentRef ref, + const Transaction::src_t* p_src=nullptr) +{ + assert(ref->is_valid()); + assert(ref->user_hint == PLACEMENT_HINT_NULL); + assert(ref->rewrite_generation == NULL_GENERATION); + extents.insert(*ref); + if (ref->is_dirty()) { + add_to_dirty(ref); + } else { + touch_extent(*ref, p_src); + } +} + +void Cache::mark_dirty(CachedExtentRef ref) +{ + if (ref->is_dirty()) { + assert(ref->primary_ref_list_hook.is_linked()); + return; + } + + lru.remove_from_lru(*ref); + ref->state = CachedExtent::extent_state_t::DIRTY; + add_to_dirty(ref); +} + +void Cache::add_to_dirty(CachedExtentRef ref) +{ + assert(ref->is_dirty()); + assert(!ref->primary_ref_list_hook.is_linked()); + ceph_assert(ref->get_modify_time() != NULL_TIME); + intrusive_ptr_add_ref(&*ref); + dirty.push_back(*ref); + stats.dirty_bytes += ref->get_length(); +} + +void Cache::remove_from_dirty(CachedExtentRef ref) +{ + if (ref->is_dirty()) { + ceph_assert(ref->primary_ref_list_hook.is_linked()); + stats.dirty_bytes -= ref->get_length(); + dirty.erase(dirty.s_iterator_to(*ref)); + intrusive_ptr_release(&*ref); + } else { + ceph_assert(!ref->primary_ref_list_hook.is_linked()); + } +} + +void Cache::remove_extent(CachedExtentRef ref) +{ + assert(ref->is_valid()); + if (ref->is_dirty()) { + remove_from_dirty(ref); + } else if (!ref->is_placeholder()) { + lru.remove_from_lru(*ref); + } + extents.erase(*ref); +} + +void Cache::commit_retire_extent( + Transaction& t, + CachedExtentRef ref) +{ + remove_extent(ref); + + ref->dirty_from_or_retired_at = JOURNAL_SEQ_NULL; + invalidate_extent(t, *ref); +} + +void Cache::commit_replace_extent( + Transaction& t, + CachedExtentRef next, + CachedExtentRef prev) +{ + assert(next->is_dirty()); + assert(next->get_paddr() == prev->get_paddr()); + assert(next->version == prev->version + 1); + extents.replace(*next, *prev); + + if (prev->get_type() == extent_types_t::ROOT) { + assert(prev->is_stable_clean() + || prev->primary_ref_list_hook.is_linked()); + if (prev->is_dirty()) { + stats.dirty_bytes -= prev->get_length(); + dirty.erase(dirty.s_iterator_to(*prev)); + intrusive_ptr_release(&*prev); + } + add_to_dirty(next); + } else if (prev->is_dirty()) { + assert(prev->get_dirty_from() == next->get_dirty_from()); + assert(prev->primary_ref_list_hook.is_linked()); + auto prev_it = dirty.iterator_to(*prev); + dirty.insert(prev_it, *next); + dirty.erase(prev_it); + intrusive_ptr_release(&*prev); + intrusive_ptr_add_ref(&*next); + } else { + lru.remove_from_lru(*prev); + add_to_dirty(next); + } + + next->on_replace_prior(t); + invalidate_extent(t, *prev); +} + +void Cache::invalidate_extent( + Transaction& t, + CachedExtent& extent) +{ + if (!extent.may_conflict()) { + assert(extent.transactions.empty()); + extent.set_invalid(t); + return; + } + + LOG_PREFIX(Cache::invalidate_extent); + bool do_conflict_log = true; + for (auto &&i: extent.transactions) { + if (!i.t->conflicted) { + if (do_conflict_log) { + SUBDEBUGT(seastore_t, "conflict begin -- {}", t, extent); + do_conflict_log = false; + } + assert(!i.t->is_weak()); + account_conflict(t.get_src(), i.t->get_src()); + mark_transaction_conflicted(*i.t, extent); + } + } + extent.set_invalid(t); +} + +void Cache::mark_transaction_conflicted( + Transaction& t, CachedExtent& conflicting_extent) +{ + LOG_PREFIX(Cache::mark_transaction_conflicted); + SUBTRACET(seastore_t, "", t); + assert(!t.conflicted); + t.conflicted = true; + + auto& efforts = get_by_src(stats.invalidated_efforts_by_src, + t.get_src()); + ++efforts.total_trans_invalidated; + + auto& counter = get_by_ext(efforts.num_trans_invalidated, + conflicting_extent.get_type()); + ++counter; + + io_stat_t read_stat; + for (auto &i: t.read_set) { + read_stat.increment(i.ref->get_length()); + } + efforts.read.increment_stat(read_stat); + + if (t.get_src() != Transaction::src_t::READ) { + io_stat_t retire_stat; + for (auto &i: t.retired_set) { + retire_stat.increment(i->get_length()); + } + efforts.retire.increment_stat(retire_stat); + + auto& fresh_stat = t.get_fresh_block_stats(); + efforts.fresh.increment_stat(fresh_stat); + + io_stat_t delta_stat; + for (auto &i: t.mutated_block_list) { + if (!i->is_valid()) { + continue; + } + efforts.mutate.increment(i->get_length()); + delta_stat.increment(i->get_delta().length()); + } + efforts.mutate_delta_bytes += delta_stat.bytes; + + for (auto &i: t.pre_alloc_list) { + epm.mark_space_free(i->get_paddr(), i->get_length()); + } + + auto& ool_stats = t.get_ool_write_stats(); + efforts.fresh_ool_written.increment_stat(ool_stats.extents); + efforts.num_ool_records += ool_stats.num_records; + auto ool_record_bytes = (ool_stats.md_bytes + ool_stats.get_data_bytes()); + efforts.ool_record_bytes += ool_record_bytes; + + if (is_background_transaction(t.get_src())) { + // CLEANER transaction won't contain any onode/omap tree operations + assert(t.onode_tree_stats.is_clear()); + assert(t.omap_tree_stats.is_clear()); + } else { + get_by_src(stats.invalidated_onode_tree_efforts, t.get_src() + ).increment(t.onode_tree_stats); + get_by_src(stats.invalidated_omap_tree_efforts, t.get_src() + ).increment(t.omap_tree_stats); + } + + get_by_src(stats.invalidated_lba_tree_efforts, t.get_src() + ).increment(t.lba_tree_stats); + get_by_src(stats.invalidated_backref_tree_efforts, t.get_src() + ).increment(t.backref_tree_stats); + + SUBDEBUGT(seastore_t, + "discard {} read, {} fresh, {} delta, {} retire, {}({}B) ool-records", + t, + read_stat, + fresh_stat, + delta_stat, + retire_stat, + ool_stats.num_records, + ool_record_bytes); + } else { + // read transaction won't have non-read efforts + assert(t.retired_set.empty()); + assert(t.get_fresh_block_stats().is_clear()); + assert(t.mutated_block_list.empty()); + assert(t.get_ool_write_stats().is_clear()); + assert(t.onode_tree_stats.is_clear()); + assert(t.omap_tree_stats.is_clear()); + assert(t.lba_tree_stats.is_clear()); + assert(t.backref_tree_stats.is_clear()); + SUBDEBUGT(seastore_t, "discard {} read", t, read_stat); + } +} + +void Cache::on_transaction_destruct(Transaction& t) +{ + LOG_PREFIX(Cache::on_transaction_destruct); + SUBTRACET(seastore_t, "", t); + if (t.get_src() == Transaction::src_t::READ && + t.conflicted == false) { + io_stat_t read_stat; + for (auto &i: t.read_set) { + read_stat.increment(i.ref->get_length()); + } + SUBDEBUGT(seastore_t, "done {} read", t, read_stat); + + if (!t.is_weak()) { + // exclude weak transaction as it is impossible to conflict + ++stats.success_read_efforts.num_trans; + stats.success_read_efforts.read.increment_stat(read_stat); + } + + // read transaction won't have non-read efforts + assert(t.retired_set.empty()); + assert(t.get_fresh_block_stats().is_clear()); + assert(t.mutated_block_list.empty()); + assert(t.onode_tree_stats.is_clear()); + assert(t.omap_tree_stats.is_clear()); + assert(t.lba_tree_stats.is_clear()); + assert(t.backref_tree_stats.is_clear()); + } +} + +CachedExtentRef Cache::alloc_new_extent_by_type( + Transaction &t, ///< [in, out] current transaction + extent_types_t type, ///< [in] type tag + extent_len_t length, ///< [in] length + placement_hint_t hint, ///< [in] user hint + rewrite_gen_t gen ///< [in] rewrite generation +) +{ + LOG_PREFIX(Cache::alloc_new_extent_by_type); + SUBDEBUGT(seastore_cache, "allocate {} {}B, hint={}, gen={}", + t, type, length, hint, rewrite_gen_printer_t{gen}); + switch (type) { + case extent_types_t::ROOT: + ceph_assert(0 == "ROOT is never directly alloc'd"); + return CachedExtentRef(); + case extent_types_t::LADDR_INTERNAL: + return alloc_new_extent<lba_manager::btree::LBAInternalNode>(t, length, hint, gen); + case extent_types_t::LADDR_LEAF: + return alloc_new_extent<lba_manager::btree::LBALeafNode>( + t, length, hint, gen); + case extent_types_t::ONODE_BLOCK_STAGED: + return alloc_new_extent<onode::SeastoreNodeExtent>(t, length, hint, gen); + case extent_types_t::OMAP_INNER: + return alloc_new_extent<omap_manager::OMapInnerNode>(t, length, hint, gen); + case extent_types_t::OMAP_LEAF: + return alloc_new_extent<omap_manager::OMapLeafNode>(t, length, hint, gen); + case extent_types_t::COLL_BLOCK: + return alloc_new_extent<collection_manager::CollectionNode>(t, length, hint, gen); + case extent_types_t::OBJECT_DATA_BLOCK: + return alloc_new_extent<ObjectDataBlock>(t, length, hint, gen); + case extent_types_t::RETIRED_PLACEHOLDER: + ceph_assert(0 == "impossible"); + return CachedExtentRef(); + case extent_types_t::TEST_BLOCK: + return alloc_new_extent<TestBlock>(t, length, hint, gen); + case extent_types_t::TEST_BLOCK_PHYSICAL: + return alloc_new_extent<TestBlockPhysical>(t, length, hint, gen); + case extent_types_t::NONE: { + ceph_assert(0 == "NONE is an invalid extent type"); + return CachedExtentRef(); + } + default: + ceph_assert(0 == "impossible"); + return CachedExtentRef(); + } +} + +CachedExtentRef Cache::duplicate_for_write( + Transaction &t, + CachedExtentRef i) { + LOG_PREFIX(Cache::duplicate_for_write); + assert(i->is_fully_loaded()); + + if (i->is_mutable()) + return i; + + if (i->is_exist_clean()) { + i->version++; + i->state = CachedExtent::extent_state_t::EXIST_MUTATION_PENDING; + i->last_committed_crc = i->get_crc32c(); + // deepcopy the buffer of exist clean extent beacuse it shares + // buffer with original clean extent. + auto bp = i->get_bptr(); + auto nbp = ceph::bufferptr(bp.c_str(), bp.length()); + i->set_bptr(std::move(nbp)); + + t.add_mutated_extent(i); + DEBUGT("duplicate existing extent {}", t, *i); + return i; + } + + auto ret = i->duplicate_for_write(t); + ret->pending_for_transaction = t.get_trans_id(); + ret->prior_instance = i; + // duplicate_for_write won't occur after ool write finished + assert(!i->prior_poffset); + auto [iter, inserted] = i->mutation_pendings.insert(*ret); + ceph_assert(inserted); + t.add_mutated_extent(ret); + if (ret->get_type() == extent_types_t::ROOT) { + t.root = ret->cast<RootBlock>(); + } else { + ret->last_committed_crc = i->last_committed_crc; + } + + ret->version++; + ret->state = CachedExtent::extent_state_t::MUTATION_PENDING; + DEBUGT("{} -> {}", t, *i, *ret); + return ret; +} + +record_t Cache::prepare_record( + Transaction &t, + const journal_seq_t &journal_head, + const journal_seq_t &journal_dirty_tail) +{ + LOG_PREFIX(Cache::prepare_record); + SUBTRACET(seastore_t, "enter", t); + + auto trans_src = t.get_src(); + assert(!t.is_weak()); + assert(trans_src != Transaction::src_t::READ); + + auto& efforts = get_by_src(stats.committed_efforts_by_src, + trans_src); + + // Should be valid due to interruptible future + io_stat_t read_stat; + for (auto &i: t.read_set) { + if (!i.ref->is_valid()) { + SUBERRORT(seastore_t, + "read_set got invalid extent, aborting -- {}", t, *i.ref); + ceph_abort("no invalid extent allowed in transactions' read_set"); + } + get_by_ext(efforts.read_by_ext, + i.ref->get_type()).increment(i.ref->get_length()); + read_stat.increment(i.ref->get_length()); + } + t.read_set.clear(); + t.write_set.clear(); + + record_t record(trans_src); + auto commit_time = seastar::lowres_system_clock::now(); + + // Add new copy of mutated blocks, set_io_wait to block until written + record.deltas.reserve(t.mutated_block_list.size()); + io_stat_t delta_stat; + for (auto &i: t.mutated_block_list) { + if (!i->is_valid()) { + DEBUGT("invalid mutated extent -- {}", t, *i); + continue; + } + assert(i->is_exist_mutation_pending() || + i->prior_instance); + get_by_ext(efforts.mutate_by_ext, + i->get_type()).increment(i->get_length()); + + auto delta_bl = i->get_delta(); + auto delta_length = delta_bl.length(); + i->set_modify_time(commit_time); + DEBUGT("mutated extent with {}B delta -- {}", + t, delta_length, *i); + if (!i->is_exist_mutation_pending()) { + DEBUGT("commit replace extent ... -- {}, prior={}", + t, *i, *i->prior_instance); + // extent with EXIST_MUTATION_PENDING doesn't have + // prior_instance field so skip these extents. + // the existing extents should be added into Cache + // during complete_commit to sync with gc transaction. + commit_replace_extent(t, i, i->prior_instance); + } + + i->prepare_write(); + i->set_io_wait(); + i->prepare_commit(); + + assert(i->get_version() > 0); + auto final_crc = i->get_crc32c(); + if (i->get_type() == extent_types_t::ROOT) { + SUBTRACET(seastore_t, "writing out root delta {}B -- {}", + t, delta_length, *i); + assert(t.root == i); + root = t.root; + record.push_back( + delta_info_t{ + extent_types_t::ROOT, + P_ADDR_NULL, + L_ADDR_NULL, + 0, + 0, + 0, + t.root->get_version() - 1, + MAX_SEG_SEQ, + segment_type_t::NULL_SEG, + std::move(delta_bl) + }); + } else { + auto sseq = NULL_SEG_SEQ; + auto stype = segment_type_t::NULL_SEG; + + // FIXME: This is specific to the segmented implementation + if (i->get_paddr().get_addr_type() == paddr_types_t::SEGMENT) { + auto sid = i->get_paddr().as_seg_paddr().get_segment_id(); + auto sinfo = get_segment_info(sid); + if (sinfo) { + sseq = sinfo->seq; + stype = sinfo->type; + } + } + + record.push_back( + delta_info_t{ + i->get_type(), + i->get_paddr(), + (i->is_logical() + ? i->cast<LogicalCachedExtent>()->get_laddr() + : L_ADDR_NULL), + i->last_committed_crc, + final_crc, + i->get_length(), + i->get_version() - 1, + sseq, + stype, + std::move(delta_bl) + }); + i->last_committed_crc = final_crc; + } + assert(delta_length); + get_by_ext(efforts.delta_bytes_by_ext, + i->get_type()) += delta_length; + delta_stat.increment(delta_length); + } + + // Transaction is now a go, set up in-memory cache state + // invalidate now invalid blocks + io_stat_t retire_stat; + std::vector<alloc_delta_t> alloc_deltas; + alloc_delta_t rel_delta; + rel_delta.op = alloc_delta_t::op_types_t::CLEAR; + for (auto &i: t.retired_set) { + get_by_ext(efforts.retire_by_ext, + i->get_type()).increment(i->get_length()); + retire_stat.increment(i->get_length()); + DEBUGT("retired and remove extent -- {}", t, *i); + commit_retire_extent(t, i); + if (is_backref_mapped_extent_node(i) + || is_retired_placeholder(i->get_type())) { + rel_delta.alloc_blk_ranges.emplace_back( + i->get_paddr(), + L_ADDR_NULL, + i->get_length(), + i->get_type()); + } + } + alloc_deltas.emplace_back(std::move(rel_delta)); + + record.extents.reserve(t.inline_block_list.size()); + io_stat_t fresh_stat; + io_stat_t fresh_invalid_stat; + alloc_delta_t alloc_delta; + alloc_delta.op = alloc_delta_t::op_types_t::SET; + for (auto &i: t.inline_block_list) { + if (!i->is_valid()) { + DEBUGT("invalid fresh inline extent -- {}", t, *i); + fresh_invalid_stat.increment(i->get_length()); + get_by_ext(efforts.fresh_invalid_by_ext, + i->get_type()).increment(i->get_length()); + } else { + TRACET("fresh inline extent -- {}", t, *i); + } + fresh_stat.increment(i->get_length()); + get_by_ext(efforts.fresh_inline_by_ext, + i->get_type()).increment(i->get_length()); + assert(i->is_inline() || i->get_paddr().is_fake()); + + bufferlist bl; + i->prepare_write(); + i->prepare_commit(); + bl.append(i->get_bptr()); + if (i->get_type() == extent_types_t::ROOT) { + ceph_assert(0 == "ROOT never gets written as a fresh block"); + } + + assert(bl.length() == i->get_length()); + auto modify_time = i->get_modify_time(); + if (modify_time == NULL_TIME) { + modify_time = commit_time; + } + record.push_back(extent_t{ + i->get_type(), + i->is_logical() + ? i->cast<LogicalCachedExtent>()->get_laddr() + : (is_lba_node(i->get_type()) + ? i->cast<lba_manager::btree::LBANode>()->get_node_meta().begin + : L_ADDR_NULL), + std::move(bl) + }, + modify_time); + if (i->is_valid() + && is_backref_mapped_extent_node(i)) { + alloc_delta.alloc_blk_ranges.emplace_back( + i->get_paddr(), + i->is_logical() + ? i->cast<LogicalCachedExtent>()->get_laddr() + : (is_lba_node(i->get_type()) + ? i->cast<lba_manager::btree::LBANode>()->get_node_meta().begin + : L_ADDR_NULL), + i->get_length(), + i->get_type()); + } + } + + for (auto &i: t.written_ool_block_list) { + TRACET("fresh ool extent -- {}", t, *i); + ceph_assert(i->is_valid()); + assert(!i->is_inline()); + get_by_ext(efforts.fresh_ool_by_ext, + i->get_type()).increment(i->get_length()); + i->prepare_commit(); + if (is_backref_mapped_extent_node(i)) { + alloc_delta.alloc_blk_ranges.emplace_back( + i->get_paddr(), + i->is_logical() + ? i->cast<LogicalCachedExtent>()->get_laddr() + : i->cast<lba_manager::btree::LBANode>()->get_node_meta().begin, + i->get_length(), + i->get_type()); + } + } + + for (auto &i: t.existing_block_list) { + if (i->is_valid()) { + alloc_delta.alloc_blk_ranges.emplace_back( + i->get_paddr(), + i->cast<LogicalCachedExtent>()->get_laddr(), + i->get_length(), + i->get_type()); + } + } + alloc_deltas.emplace_back(std::move(alloc_delta)); + + for (auto b : alloc_deltas) { + bufferlist bl; + encode(b, bl); + delta_info_t delta; + delta.type = extent_types_t::ALLOC_INFO; + delta.bl = bl; + record.push_back(std::move(delta)); + } + + if (is_background_transaction(trans_src)) { + assert(journal_head != JOURNAL_SEQ_NULL); + assert(journal_dirty_tail != JOURNAL_SEQ_NULL); + journal_seq_t dirty_tail; + auto maybe_dirty_tail = get_oldest_dirty_from(); + if (!maybe_dirty_tail.has_value()) { + dirty_tail = journal_head; + SUBINFOT(seastore_t, "dirty_tail all trimmed, set to head {}, src={}", + t, dirty_tail, trans_src); + } else if (*maybe_dirty_tail == JOURNAL_SEQ_NULL) { + dirty_tail = journal_dirty_tail; + SUBINFOT(seastore_t, "dirty_tail is pending, set to {}, src={}", + t, dirty_tail, trans_src); + } else { + dirty_tail = *maybe_dirty_tail; + } + ceph_assert(dirty_tail != JOURNAL_SEQ_NULL); + journal_seq_t alloc_tail; + auto maybe_alloc_tail = get_oldest_backref_dirty_from(); + if (!maybe_alloc_tail.has_value()) { + // FIXME: the replay point of the allocations requires to be accurate. + // Setting the alloc_tail to get_journal_head() cannot skip replaying the + // last unnecessary record. + alloc_tail = journal_head; + SUBINFOT(seastore_t, "alloc_tail all trimmed, set to head {}, src={}", + t, alloc_tail, trans_src); + } else if (*maybe_alloc_tail == JOURNAL_SEQ_NULL) { + ceph_abort("impossible"); + } else { + alloc_tail = *maybe_alloc_tail; + } + ceph_assert(alloc_tail != JOURNAL_SEQ_NULL); + auto tails = journal_tail_delta_t{alloc_tail, dirty_tail}; + SUBDEBUGT(seastore_t, "update tails as delta {}", t, tails); + bufferlist bl; + encode(tails, bl); + delta_info_t delta; + delta.type = extent_types_t::JOURNAL_TAIL; + delta.bl = bl; + record.push_back(std::move(delta)); + } + + ceph_assert(t.get_fresh_block_stats().num == + t.inline_block_list.size() + + t.written_ool_block_list.size() + + t.num_delayed_invalid_extents + + t.num_allocated_invalid_extents); + + auto& ool_stats = t.get_ool_write_stats(); + ceph_assert(ool_stats.extents.num == t.written_ool_block_list.size()); + + if (record.is_empty()) { + SUBINFOT(seastore_t, + "record to submit is empty, src={}", t, trans_src); + assert(t.onode_tree_stats.is_clear()); + assert(t.omap_tree_stats.is_clear()); + assert(t.lba_tree_stats.is_clear()); + assert(t.backref_tree_stats.is_clear()); + assert(ool_stats.is_clear()); + } + + if (record.modify_time == NULL_TIME) { + record.modify_time = commit_time; + } + + SUBDEBUGT(seastore_t, + "commit H{} dirty_from={}, alloc_from={}, " + "{} read, {} fresh with {} invalid, " + "{} delta, {} retire, {}(md={}B, data={}B) ool-records, " + "{}B md, {}B data, modify_time={}", + t, (void*)&t.get_handle(), + get_oldest_dirty_from().value_or(JOURNAL_SEQ_NULL), + get_oldest_backref_dirty_from().value_or(JOURNAL_SEQ_NULL), + read_stat, + fresh_stat, + fresh_invalid_stat, + delta_stat, + retire_stat, + ool_stats.num_records, + ool_stats.md_bytes, + ool_stats.get_data_bytes(), + record.size.get_raw_mdlength(), + record.size.dlength, + sea_time_point_printer_t{record.modify_time}); + if (is_background_transaction(trans_src)) { + // background transaction won't contain any onode tree operations + assert(t.onode_tree_stats.is_clear()); + assert(t.omap_tree_stats.is_clear()); + } else { + if (t.onode_tree_stats.depth) { + stats.onode_tree_depth = t.onode_tree_stats.depth; + } + if (t.omap_tree_stats.depth) { + stats.omap_tree_depth = t.omap_tree_stats.depth; + } + stats.onode_tree_extents_num += t.onode_tree_stats.extents_num_delta; + ceph_assert(stats.onode_tree_extents_num >= 0); + get_by_src(stats.committed_onode_tree_efforts, trans_src + ).increment(t.onode_tree_stats); + stats.omap_tree_extents_num += t.omap_tree_stats.extents_num_delta; + ceph_assert(stats.omap_tree_extents_num >= 0); + get_by_src(stats.committed_omap_tree_efforts, trans_src + ).increment(t.omap_tree_stats); + } + + if (t.lba_tree_stats.depth) { + stats.lba_tree_depth = t.lba_tree_stats.depth; + } + stats.lba_tree_extents_num += t.lba_tree_stats.extents_num_delta; + ceph_assert(stats.lba_tree_extents_num >= 0); + get_by_src(stats.committed_lba_tree_efforts, trans_src + ).increment(t.lba_tree_stats); + if (t.backref_tree_stats.depth) { + stats.backref_tree_depth = t.backref_tree_stats.depth; + } + stats.backref_tree_extents_num += t.backref_tree_stats.extents_num_delta; + ceph_assert(stats.backref_tree_extents_num >= 0); + get_by_src(stats.committed_backref_tree_efforts, trans_src + ).increment(t.backref_tree_stats); + + ++(efforts.num_trans); + efforts.num_ool_records += ool_stats.num_records; + efforts.ool_record_metadata_bytes += ool_stats.md_bytes; + efforts.ool_record_data_bytes += ool_stats.get_data_bytes(); + efforts.inline_record_metadata_bytes += + (record.size.get_raw_mdlength() - record.get_delta_size()); + + auto &rewrite_version_stats = t.get_rewrite_version_stats(); + if (trans_src == Transaction::src_t::TRIM_DIRTY) { + stats.committed_dirty_version.increment_stat(rewrite_version_stats); + } else if (trans_src == Transaction::src_t::CLEANER_MAIN || + trans_src == Transaction::src_t::CLEANER_COLD) { + stats.committed_reclaim_version.increment_stat(rewrite_version_stats); + } else { + assert(rewrite_version_stats.is_clear()); + } + + return record; +} + +void Cache::backref_batch_update( + std::vector<backref_entry_ref> &&list, + const journal_seq_t &seq) +{ + LOG_PREFIX(Cache::backref_batch_update); + DEBUG("inserting {} entries at {}", list.size(), seq); + ceph_assert(seq != JOURNAL_SEQ_NULL); + + for (auto &ent : list) { + backref_entry_mset.insert(*ent); + } + + auto iter = backref_entryrefs_by_seq.find(seq); + if (iter == backref_entryrefs_by_seq.end()) { + backref_entryrefs_by_seq.emplace(seq, std::move(list)); + } else { + iter->second.insert( + iter->second.end(), + std::make_move_iterator(list.begin()), + std::make_move_iterator(list.end())); + } +} + +void Cache::complete_commit( + Transaction &t, + paddr_t final_block_start, + journal_seq_t start_seq) +{ + LOG_PREFIX(Cache::complete_commit); + SUBTRACET(seastore_t, "final_block_start={}, start_seq={}", + t, final_block_start, start_seq); + + std::vector<backref_entry_ref> backref_list; + t.for_each_fresh_block([&](const CachedExtentRef &i) { + if (!i->is_valid()) { + return; + } + + bool is_inline = false; + if (i->is_inline()) { + is_inline = true; + i->set_paddr(final_block_start.add_relative(i->get_paddr())); + } + i->last_committed_crc = i->get_crc32c(); + i->pending_for_transaction = TRANS_ID_NULL; + i->on_initial_write(); + + i->state = CachedExtent::extent_state_t::CLEAN; + DEBUGT("add extent as fresh, inline={} -- {}", + t, is_inline, *i); + const auto t_src = t.get_src(); + i->invalidate_hints(); + add_extent(i, &t_src); + epm.commit_space_used(i->get_paddr(), i->get_length()); + if (is_backref_mapped_extent_node(i)) { + DEBUGT("backref_list new {} len {}", + t, + i->get_paddr(), + i->get_length()); + backref_list.emplace_back( + std::make_unique<backref_entry_t>( + i->get_paddr(), + i->is_logical() + ? i->cast<LogicalCachedExtent>()->get_laddr() + : (is_lba_node(i->get_type()) + ? i->cast<lba_manager::btree::LBANode>()->get_node_meta().begin + : L_ADDR_NULL), + i->get_length(), + i->get_type(), + start_seq)); + } else if (is_backref_node(i->get_type())) { + add_backref_extent( + i->get_paddr(), + i->cast<backref::BackrefNode>()->get_node_meta().begin, + i->get_type()); + } else { + ERRORT("{}", t, *i); + ceph_abort("not possible"); + } + }); + + // Add new copy of mutated blocks, set_io_wait to block until written + for (auto &i: t.mutated_block_list) { + if (!i->is_valid()) { + continue; + } + assert(i->is_exist_mutation_pending() || + i->prior_instance); + i->on_delta_write(final_block_start); + i->pending_for_transaction = TRANS_ID_NULL; + i->prior_instance = CachedExtentRef(); + i->state = CachedExtent::extent_state_t::DIRTY; + assert(i->version > 0); + if (i->version == 1 || i->get_type() == extent_types_t::ROOT) { + i->dirty_from_or_retired_at = start_seq; + DEBUGT("commit extent done, become dirty -- {}", t, *i); + } else { + DEBUGT("commit extent done -- {}", t, *i); + } + } + + for (auto &i: t.retired_set) { + epm.mark_space_free(i->get_paddr(), i->get_length()); + } + for (auto &i: t.existing_block_list) { + if (i->is_valid()) { + epm.mark_space_used(i->get_paddr(), i->get_length()); + } + } + + for (auto &i: t.mutated_block_list) { + if (!i->is_valid()) { + continue; + } + i->complete_io(); + } + + last_commit = start_seq; + for (auto &i: t.retired_set) { + i->dirty_from_or_retired_at = start_seq; + if (is_backref_mapped_extent_node(i) + || is_retired_placeholder(i->get_type())) { + DEBUGT("backref_list free {} len {}", + t, + i->get_paddr(), + i->get_length()); + backref_list.emplace_back( + std::make_unique<backref_entry_t>( + i->get_paddr(), + L_ADDR_NULL, + i->get_length(), + i->get_type(), + start_seq)); + } else if (is_backref_node(i->get_type())) { + remove_backref_extent(i->get_paddr()); + } else { + ERRORT("{}", t, *i); + ceph_abort("not possible"); + } + } + + auto existing_stats = t.get_existing_block_stats(); + DEBUGT("total existing blocks num: {}, exist clean num: {}, " + "exist mutation pending num: {}", + t, + existing_stats.valid_num, + existing_stats.clean_num, + existing_stats.mutated_num); + for (auto &i: t.existing_block_list) { + if (i->is_valid()) { + if (i->is_exist_clean()) { + i->state = CachedExtent::extent_state_t::CLEAN; + } else { + assert(i->state == CachedExtent::extent_state_t::DIRTY); + } + DEBUGT("backref_list new existing {} len {}", + t, + i->get_paddr(), + i->get_length()); + backref_list.emplace_back( + std::make_unique<backref_entry_t>( + i->get_paddr(), + i->cast<LogicalCachedExtent>()->get_laddr(), + i->get_length(), + i->get_type(), + start_seq)); + const auto t_src = t.get_src(); + add_extent(i, &t_src); + } + } + if (!backref_list.empty()) { + backref_batch_update(std::move(backref_list), start_seq); + } + + for (auto &i: t.pre_alloc_list) { + if (!i->is_valid()) { + epm.mark_space_free(i->get_paddr(), i->get_length()); + } + } +} + +void Cache::init() +{ + LOG_PREFIX(Cache::init); + if (root) { + // initial creation will do mkfs followed by mount each of which calls init + DEBUG("remove extent -- prv_root={}", *root); + remove_extent(root); + root = nullptr; + } + root = new RootBlock(); + root->init(CachedExtent::extent_state_t::CLEAN, + P_ADDR_ROOT, + PLACEMENT_HINT_NULL, + NULL_GENERATION, + TRANS_ID_NULL); + INFO("init root -- {}", *root); + extents.insert(*root); +} + +Cache::mkfs_iertr::future<> Cache::mkfs(Transaction &t) +{ + LOG_PREFIX(Cache::mkfs); + INFOT("create root", t); + return get_root(t).si_then([this, &t](auto croot) { + duplicate_for_write(t, croot); + return mkfs_iertr::now(); + }).handle_error_interruptible( + mkfs_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in Cache::mkfs" + } + ); +} + +Cache::close_ertr::future<> Cache::close() +{ + LOG_PREFIX(Cache::close); + INFO("close with {}({}B) dirty, dirty_from={}, alloc_from={}, " + "{}({}B) lru, totally {}({}B) indexed extents", + dirty.size(), + stats.dirty_bytes, + get_oldest_dirty_from().value_or(JOURNAL_SEQ_NULL), + get_oldest_backref_dirty_from().value_or(JOURNAL_SEQ_NULL), + lru.get_current_contents_extents(), + lru.get_current_contents_bytes(), + extents.size(), + extents.get_bytes()); + root.reset(); + for (auto i = dirty.begin(); i != dirty.end(); ) { + auto ptr = &*i; + stats.dirty_bytes -= ptr->get_length(); + dirty.erase(i++); + intrusive_ptr_release(ptr); + } + backref_extents.clear(); + backref_entryrefs_by_seq.clear(); + assert(stats.dirty_bytes == 0); + lru.clear(); + return close_ertr::now(); +} + +Cache::replay_delta_ret +Cache::replay_delta( + journal_seq_t journal_seq, + paddr_t record_base, + const delta_info_t &delta, + const journal_seq_t &dirty_tail, + const journal_seq_t &alloc_tail, + sea_time_point modify_time) +{ + LOG_PREFIX(Cache::replay_delta); + assert(dirty_tail != JOURNAL_SEQ_NULL); + assert(alloc_tail != JOURNAL_SEQ_NULL); + ceph_assert(modify_time != NULL_TIME); + + // FIXME: This is specific to the segmented implementation + /* The journal may validly contain deltas for extents in + * since released segments. We can detect those cases by + * checking whether the segment in question currently has a + * sequence number > the current journal segment seq. We can + * safetly skip these deltas because the extent must already + * have been rewritten. + */ + if (delta.paddr != P_ADDR_NULL && + delta.paddr.get_addr_type() == paddr_types_t::SEGMENT) { + auto& seg_addr = delta.paddr.as_seg_paddr(); + auto seg_info = get_segment_info(seg_addr.get_segment_id()); + if (seg_info) { + auto delta_paddr_segment_seq = seg_info->seq; + auto delta_paddr_segment_type = seg_info->type; + if (delta_paddr_segment_seq != delta.ext_seq || + delta_paddr_segment_type != delta.seg_type) { + DEBUG("delta is obsolete, delta_paddr_segment_seq={}," + " delta_paddr_segment_type={} -- {}", + segment_seq_printer_t{delta_paddr_segment_seq}, + delta_paddr_segment_type, + delta); + return replay_delta_ertr::make_ready_future<bool>(false); + } + } + } + + if (delta.type == extent_types_t::JOURNAL_TAIL) { + // this delta should have been dealt with during segment cleaner mounting + return replay_delta_ertr::make_ready_future<bool>(false); + } + + // replay alloc + if (delta.type == extent_types_t::ALLOC_INFO) { + if (journal_seq < alloc_tail) { + DEBUG("journal_seq {} < alloc_tail {}, don't replay {}", + journal_seq, alloc_tail, delta); + return replay_delta_ertr::make_ready_future<bool>(false); + } + + alloc_delta_t alloc_delta; + decode(alloc_delta, delta.bl); + std::vector<backref_entry_ref> backref_list; + for (auto &alloc_blk : alloc_delta.alloc_blk_ranges) { + if (alloc_blk.paddr.is_relative()) { + assert(alloc_blk.paddr.is_record_relative()); + alloc_blk.paddr = record_base.add_relative(alloc_blk.paddr); + } + DEBUG("replay alloc_blk {}~{} {}, journal_seq: {}", + alloc_blk.paddr, alloc_blk.len, alloc_blk.laddr, journal_seq); + backref_list.emplace_back( + std::make_unique<backref_entry_t>( + alloc_blk.paddr, + alloc_blk.laddr, + alloc_blk.len, + alloc_blk.type, + journal_seq)); + } + if (!backref_list.empty()) { + backref_batch_update(std::move(backref_list), journal_seq); + } + return replay_delta_ertr::make_ready_future<bool>(true); + } + + // replay dirty + if (journal_seq < dirty_tail) { + DEBUG("journal_seq {} < dirty_tail {}, don't replay {}", + journal_seq, dirty_tail, delta); + return replay_delta_ertr::make_ready_future<bool>(false); + } + + if (delta.type == extent_types_t::ROOT) { + TRACE("replay root delta at {} {}, remove extent ... -- {}, prv_root={}", + journal_seq, record_base, delta, *root); + remove_extent(root); + root->apply_delta_and_adjust_crc(record_base, delta.bl); + root->dirty_from_or_retired_at = journal_seq; + root->state = CachedExtent::extent_state_t::DIRTY; + DEBUG("replayed root delta at {} {}, add extent -- {}, root={}", + journal_seq, record_base, delta, *root); + root->set_modify_time(modify_time); + add_extent(root); + return replay_delta_ertr::make_ready_future<bool>(true); + } else { + auto _get_extent_if_cached = [this](paddr_t addr) + -> get_extent_ertr::future<CachedExtentRef> { + // replay is not included by the cache hit metrics + auto ret = query_cache(addr, nullptr); + if (ret) { + // no retired-placeholder should be exist yet because no transaction + // has been created. + assert(ret->get_type() != extent_types_t::RETIRED_PLACEHOLDER); + return ret->wait_io().then([ret] { + return ret; + }); + } else { + return seastar::make_ready_future<CachedExtentRef>(); + } + }; + auto extent_fut = (delta.pversion == 0 ? + // replay is not included by the cache hit metrics + _get_extent_by_type( + delta.type, + delta.paddr, + delta.laddr, + delta.length, + nullptr, + [](CachedExtent &) {}, + [](CachedExtent &) {}) : + _get_extent_if_cached( + delta.paddr) + ).handle_error( + replay_delta_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in Cache::replay_delta" + } + ); + return extent_fut.safe_then([=, this, &delta](auto extent) { + if (!extent) { + DEBUG("replay extent is not present, so delta is obsolete at {} {} -- {}", + journal_seq, record_base, delta); + assert(delta.pversion > 0); + return replay_delta_ertr::make_ready_future<bool>(true); + } + + DEBUG("replay extent delta at {} {} ... -- {}, prv_extent={}", + journal_seq, record_base, delta, *extent); + + assert(extent->last_committed_crc == delta.prev_crc); + assert(extent->version == delta.pversion); + extent->apply_delta_and_adjust_crc(record_base, delta.bl); + extent->set_modify_time(modify_time); + assert(extent->last_committed_crc == delta.final_crc); + + extent->version++; + if (extent->version == 1) { + extent->dirty_from_or_retired_at = journal_seq; + DEBUG("replayed extent delta at {} {}, become dirty -- {}, extent={}" , + journal_seq, record_base, delta, *extent); + } else { + DEBUG("replayed extent delta at {} {} -- {}, extent={}" , + journal_seq, record_base, delta, *extent); + } + mark_dirty(extent); + return replay_delta_ertr::make_ready_future<bool>(true); + }); + } +} + +Cache::get_next_dirty_extents_ret Cache::get_next_dirty_extents( + Transaction &t, + journal_seq_t seq, + size_t max_bytes) +{ + LOG_PREFIX(Cache::get_next_dirty_extents); + if (dirty.empty()) { + DEBUGT("max_bytes={}B, seq={}, dirty is empty", + t, max_bytes, seq); + } else { + DEBUGT("max_bytes={}B, seq={}, dirty_from={}", + t, max_bytes, seq, dirty.begin()->get_dirty_from()); + } + std::vector<CachedExtentRef> cand; + size_t bytes_so_far = 0; + for (auto i = dirty.begin(); + i != dirty.end() && bytes_so_far < max_bytes; + ++i) { + auto dirty_from = i->get_dirty_from(); + //dirty extents must be fully loaded + assert(i->is_fully_loaded()); + if (unlikely(dirty_from == JOURNAL_SEQ_NULL)) { + ERRORT("got dirty extent with JOURNAL_SEQ_NULL -- {}", t, *i); + ceph_abort(); + } + if (dirty_from < seq) { + TRACET("next extent -- {}", t, *i); + if (!cand.empty() && cand.back()->get_dirty_from() > dirty_from) { + ERRORT("dirty extents are not ordered by dirty_from -- last={}, next={}", + t, *cand.back(), *i); + ceph_abort(); + } + bytes_so_far += i->get_length(); + cand.push_back(&*i); + } else { + break; + } + } + return seastar::do_with( + std::move(cand), + decltype(cand)(), + [FNAME, this, &t](auto &cand, auto &ret) { + return trans_intr::do_for_each( + cand, + [FNAME, this, &t, &ret](auto &ext) { + TRACET("waiting on extent -- {}", t, *ext); + return trans_intr::make_interruptible( + ext->wait_io() + ).then_interruptible([FNAME, this, ext, &t, &ret] { + if (!ext->is_valid()) { + ++(get_by_src(stats.trans_conflicts_by_unknown, t.get_src())); + mark_transaction_conflicted(t, *ext); + return; + } + + CachedExtentRef on_transaction; + auto result = t.get_extent(ext->get_paddr(), &on_transaction); + if (result == Transaction::get_extent_ret::ABSENT) { + DEBUGT("extent is absent on t -- {}", t, *ext); + t.add_to_read_set(ext); + if (ext->get_type() == extent_types_t::ROOT) { + if (t.root) { + assert(&*t.root == &*ext); + ceph_assert(0 == "t.root would have to already be in the read set"); + } else { + assert(&*ext == &*root); + t.root = root; + } + } + ret.push_back(ext); + } else if (result == Transaction::get_extent_ret::PRESENT) { + DEBUGT("extent is present on t -- {}, on t {}", t, *ext, *on_transaction); + ret.push_back(on_transaction); + } else { + assert(result == Transaction::get_extent_ret::RETIRED); + DEBUGT("extent is retired on t -- {}", t, *ext); + } + }); + }).then_interruptible([&ret] { + return std::move(ret); + }); + }); +} + +Cache::get_root_ret Cache::get_root(Transaction &t) +{ + LOG_PREFIX(Cache::get_root); + if (t.root) { + TRACET("root already on t -- {}", t, *t.root); + return t.root->wait_io().then([&t] { + return get_root_iertr::make_ready_future<RootBlockRef>( + t.root); + }); + } else { + DEBUGT("root not on t -- {}", t, *root); + t.root = root; + t.add_to_read_set(root); + return root->wait_io().then([root=root] { + return get_root_iertr::make_ready_future<RootBlockRef>( + root); + }); + } +} + +Cache::get_extent_ertr::future<CachedExtentRef> Cache::_get_extent_by_type( + extent_types_t type, + paddr_t offset, + laddr_t laddr, + extent_len_t length, + const Transaction::src_t* p_src, + extent_init_func_t &&extent_init_func, + extent_init_func_t &&on_cache) +{ + return [=, this, extent_init_func=std::move(extent_init_func)]() mutable { + src_ext_t* p_metric_key = nullptr; + src_ext_t metric_key; + if (p_src) { + metric_key = std::make_pair(*p_src, type); + p_metric_key = &metric_key; + } + + switch (type) { + case extent_types_t::ROOT: + ceph_assert(0 == "ROOT is never directly read"); + return get_extent_ertr::make_ready_future<CachedExtentRef>(); + case extent_types_t::BACKREF_INTERNAL: + return get_extent<backref::BackrefInternalNode>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::BACKREF_LEAF: + return get_extent<backref::BackrefLeafNode>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::LADDR_INTERNAL: + return get_extent<lba_manager::btree::LBAInternalNode>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::LADDR_LEAF: + return get_extent<lba_manager::btree::LBALeafNode>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::OMAP_INNER: + return get_extent<omap_manager::OMapInnerNode>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::OMAP_LEAF: + return get_extent<omap_manager::OMapLeafNode>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::COLL_BLOCK: + return get_extent<collection_manager::CollectionNode>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::ONODE_BLOCK_STAGED: + return get_extent<onode::SeastoreNodeExtent>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::OBJECT_DATA_BLOCK: + return get_extent<ObjectDataBlock>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::RETIRED_PLACEHOLDER: + ceph_assert(0 == "impossible"); + return get_extent_ertr::make_ready_future<CachedExtentRef>(); + case extent_types_t::TEST_BLOCK: + return get_extent<TestBlock>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::TEST_BLOCK_PHYSICAL: + return get_extent<TestBlockPhysical>( + offset, length, p_metric_key, std::move(extent_init_func), std::move(on_cache) + ).safe_then([](auto extent) { + return CachedExtentRef(extent.detach(), false /* add_ref */); + }); + case extent_types_t::NONE: { + ceph_assert(0 == "NONE is an invalid extent type"); + return get_extent_ertr::make_ready_future<CachedExtentRef>(); + } + default: + ceph_assert(0 == "impossible"); + return get_extent_ertr::make_ready_future<CachedExtentRef>(); + } + }().safe_then([laddr](CachedExtentRef e) { + assert(e->is_logical() == (laddr != L_ADDR_NULL)); + if (e->is_logical()) { + e->cast<LogicalCachedExtent>()->set_laddr(laddr); + } + return get_extent_ertr::make_ready_future<CachedExtentRef>(e); + }); +} + +} diff --git a/src/crimson/os/seastore/cache.h b/src/crimson/os/seastore/cache.h new file mode 100644 index 000000000..c79473f98 --- /dev/null +++ b/src/crimson/os/seastore/cache.h @@ -0,0 +1,1688 @@ +// -*- 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/common/errorator.h" +#include "crimson/os/seastore/cached_extent.h" +#include "crimson/os/seastore/extent_placement_manager.h" +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/random_block_manager.h" +#include "crimson/os/seastore/root_block.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/segment_manager.h" +#include "crimson/os/seastore/transaction.h" + +namespace crimson::os::seastore::backref { +class BtreeBackrefManager; +} + +namespace crimson::os::seastore { + +template < + typename node_key_t, + typename node_val_t, + typename internal_node_t, + typename leaf_node_t, + typename pin_t, + size_t node_size, + bool leaf_has_children> +class FixedKVBtree; +class BackrefManager; +class SegmentProvider; + +struct backref_entry_t { + backref_entry_t( + const paddr_t paddr, + const laddr_t laddr, + const extent_len_t len, + const extent_types_t type, + const journal_seq_t seq) + : paddr(paddr), + laddr(laddr), + len(len), + type(type), + seq(seq) + {} + backref_entry_t(alloc_blk_t alloc_blk) + : paddr(alloc_blk.paddr), + laddr(alloc_blk.laddr), + len(alloc_blk.len), + type(alloc_blk.type) + {} + paddr_t paddr = P_ADDR_NULL; + laddr_t laddr = L_ADDR_NULL; + extent_len_t len = 0; + extent_types_t type = + extent_types_t::ROOT; + journal_seq_t seq; + friend bool operator< ( + const backref_entry_t &l, + const backref_entry_t &r) { + return l.paddr < r.paddr; + } + friend bool operator> ( + const backref_entry_t &l, + const backref_entry_t &r) { + return l.paddr > r.paddr; + } + friend bool operator== ( + const backref_entry_t &l, + const backref_entry_t &r) { + return l.paddr == r.paddr; + } + + using set_hook_t = + boost::intrusive::set_member_hook< + boost::intrusive::link_mode< + boost::intrusive::auto_unlink>>; + set_hook_t backref_set_hook; + using backref_set_member_options = boost::intrusive::member_hook< + backref_entry_t, + set_hook_t, + &backref_entry_t::backref_set_hook>; + using multiset_t = boost::intrusive::multiset< + backref_entry_t, + backref_set_member_options, + boost::intrusive::constant_time_size<false>>; + + struct cmp_t { + using is_transparent = paddr_t; + bool operator()( + const backref_entry_t &l, + const backref_entry_t &r) const { + return l.paddr < r.paddr; + } + bool operator()(const paddr_t l, const backref_entry_t &r) const { + return l < r.paddr; + } + bool operator()(const backref_entry_t &l, const paddr_t r) const { + return l.paddr < r; + } + }; +}; + +std::ostream &operator<<(std::ostream &out, const backref_entry_t &ent); + +using backref_entry_ref = std::unique_ptr<backref_entry_t>; +using backref_entry_mset_t = backref_entry_t::multiset_t; +using backref_entry_refs_t = std::vector<backref_entry_ref>; +using backref_entryrefs_by_seq_t = std::map<journal_seq_t, backref_entry_refs_t>; +using backref_entry_query_set_t = std::set< + backref_entry_t, backref_entry_t::cmp_t>; + +/** + * 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_commit() 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 + * + * + * Cache logs + * + * levels: + * - INFO: major initiation, closing operations + * - DEBUG: major extent related operations, INFO details + * - TRACE: DEBUG details + * - seastore_t logs + */ +class Cache { +public: + using base_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + using base_iertr = trans_iertr<base_ertr>; + + Cache(ExtentPlacementManager &epm); + ~Cache(); + + /// Creates empty transaction by source + TransactionRef create_transaction( + Transaction::src_t src, + const char* name, + bool is_weak) { + LOG_PREFIX(Cache::create_transaction); + + ++(get_by_src(stats.trans_created_by_src, src)); + + auto ret = std::make_unique<Transaction>( + get_dummy_ordering_handle(), + is_weak, + src, + last_commit, + [this](Transaction& t) { + return on_transaction_destruct(t); + }, + ++next_id + ); + SUBDEBUGT(seastore_t, "created name={}, source={}, is_weak={}", + *ret, name, src, is_weak); + assert(!is_weak || src == Transaction::src_t::READ); + return ret; + } + + /// Resets transaction preserving + void reset_transaction_preserve_handle(Transaction &t) { + LOG_PREFIX(Cache::reset_transaction_preserve_handle); + if (t.did_reset()) { + SUBTRACET(seastore_t, "reset", t); + ++(get_by_src(stats.trans_created_by_src, t.get_src())); + } + t.reset_preserve_handle(last_commit); + } + + /// Declare ref retired in t + void retire_extent(Transaction &t, CachedExtentRef ref) { + LOG_PREFIX(Cache::retire_extent); + SUBDEBUGT(seastore_cache, "retire extent -- {}", t, *ref); + t.add_to_retired_set(ref); + } + + /// Declare paddr retired in t + using retire_extent_iertr = base_iertr; + using retire_extent_ret = base_iertr::future<>; + retire_extent_ret retire_extent_addr( + Transaction &t, paddr_t addr, extent_len_t length); + + /** + * get_root + * + * returns ref to current root or t.root if modified in t + */ + using get_root_iertr = base_iertr; + using get_root_ret = get_root_iertr::future<RootBlockRef>; + get_root_ret get_root(Transaction &t); + + /** + * get_root_fast + * + * returns t.root and assume it is already present/read in t + */ + RootBlockRef get_root_fast(Transaction &t) { + LOG_PREFIX(Cache::get_root_fast); + SUBTRACET(seastore_cache, "root already on t -- {}", t, *t.root); + assert(t.root); + return t.root; + } + + /** + * get_extent + * + * returns ref to extent at offset~length of type T either from + * - extent_set if already in cache + * - disk + */ + using src_ext_t = std::pair<Transaction::src_t, extent_types_t>; + using get_extent_ertr = base_ertr; + template <typename T> + using get_extent_ret = get_extent_ertr::future<TCachedExtentRef<T>>; + template <typename T, typename Func, typename OnCache> + get_extent_ret<T> get_extent( + paddr_t offset, ///< [in] starting addr + extent_len_t length, ///< [in] length + const src_ext_t* p_src_ext, ///< [in] cache query metric key + Func &&extent_init_func, ///< [in] init func for extent + OnCache &&on_cache + ) { + LOG_PREFIX(Cache::get_extent); + auto cached = query_cache(offset, p_src_ext); + if (!cached) { + auto ret = CachedExtent::make_cached_extent_ref<T>( + alloc_cache_buf(length)); + ret->init(CachedExtent::extent_state_t::CLEAN_PENDING, + offset, + PLACEMENT_HINT_NULL, + NULL_GENERATION, + TRANS_ID_NULL); + SUBDEBUG(seastore_cache, + "{} {}~{} is absent, add extent and reading ... -- {}", + T::TYPE, offset, length, *ret); + const auto p_src = p_src_ext ? &p_src_ext->first : nullptr; + add_extent(ret, p_src); + on_cache(*ret); + extent_init_func(*ret); + return read_extent<T>( + std::move(ret)); + } + + // extent PRESENT in cache + if (cached->get_type() == extent_types_t::RETIRED_PLACEHOLDER) { + auto ret = CachedExtent::make_cached_extent_ref<T>( + alloc_cache_buf(length)); + ret->init(CachedExtent::extent_state_t::CLEAN_PENDING, + offset, + PLACEMENT_HINT_NULL, + NULL_GENERATION, + TRANS_ID_NULL); + SUBDEBUG(seastore_cache, + "{} {}~{} is absent(placeholder), reading ... -- {}", + T::TYPE, offset, length, *ret); + extents.replace(*ret, *cached); + on_cache(*ret); + + // replace placeholder in transactions + while (!cached->transactions.empty()) { + auto t = cached->transactions.begin()->t; + t->replace_placeholder(*cached, *ret); + } + + cached->state = CachedExtent::extent_state_t::INVALID; + extent_init_func(*ret); + return read_extent<T>( + std::move(ret)); + } else if (!cached->is_fully_loaded()) { + auto ret = TCachedExtentRef<T>(static_cast<T*>(cached.get())); + on_cache(*ret); + SUBDEBUG(seastore_cache, + "{} {}~{} is present without been fully loaded, reading ... -- {}", + T::TYPE, offset, length, *ret); + auto bp = alloc_cache_buf(length); + ret->set_bptr(std::move(bp)); + return read_extent<T>( + std::move(ret)); + } else { + SUBTRACE(seastore_cache, + "{} {}~{} is present in cache -- {}", + T::TYPE, offset, length, *cached); + auto ret = TCachedExtentRef<T>(static_cast<T*>(cached.get())); + on_cache(*ret); + return ret->wait_io( + ).then([ret=std::move(ret)]() mutable + -> get_extent_ret<T> { + // ret may be invalid, caller must check + return get_extent_ret<T>( + get_extent_ertr::ready_future_marker{}, + std::move(ret)); + }); + } + } + template <typename T> + get_extent_ret<T> get_extent( + paddr_t offset, ///< [in] starting addr + extent_len_t length, ///< [in] length + const src_ext_t* p_metric_key ///< [in] cache query metric key + ) { + return get_extent<T>( + offset, length, p_metric_key, + [](T &){}, [](T &) {}); + } + + + /** + * get_extent_if_cached + * + * Returns extent at offset if in cache + */ + using get_extent_if_cached_iertr = base_iertr; + using get_extent_if_cached_ret = + get_extent_if_cached_iertr::future<CachedExtentRef>; + get_extent_if_cached_ret get_extent_if_cached( + Transaction &t, + paddr_t offset, + extent_types_t type) { + CachedExtentRef ret; + LOG_PREFIX(Cache::get_extent_if_cached); + auto result = t.get_extent(offset, &ret); + if (result == Transaction::get_extent_ret::RETIRED) { + SUBDEBUGT(seastore_cache, "{} {} is retired on t -- {}", + t, type, offset, *ret); + return get_extent_if_cached_iertr::make_ready_future< + CachedExtentRef>(ret); + } else if (result == Transaction::get_extent_ret::PRESENT) { + if (ret->is_fully_loaded()) { + SUBTRACET(seastore_cache, "{} {} is present on t -- {}", + t, type, offset, *ret); + return ret->wait_io().then([ret] { + return get_extent_if_cached_iertr::make_ready_future< + CachedExtentRef>(ret); + }); + } else { + SUBDEBUGT(seastore_cache, "{} {} is present on t -- {}" + " without being fully loaded", t, type, offset, *ret); + return get_extent_if_cached_iertr::make_ready_future< + CachedExtentRef>(); + } + } + + // get_extent_ret::ABSENT from transaction + auto metric_key = std::make_pair(t.get_src(), type); + ret = query_cache(offset, &metric_key); + if (!ret) { + SUBDEBUGT(seastore_cache, "{} {} is absent", t, type, offset); + return get_extent_if_cached_iertr::make_ready_future<CachedExtentRef>(); + } else if (ret->get_type() == extent_types_t::RETIRED_PLACEHOLDER) { + // retired_placeholder is not really cached yet + SUBDEBUGT(seastore_cache, "{} {} is absent(placeholder)", + t, type, offset); + return get_extent_if_cached_iertr::make_ready_future<CachedExtentRef>(); + } else if (!ret->is_fully_loaded()) { + SUBDEBUGT(seastore_cache, "{} {} is present without " + "being fully loaded", t, type, offset); + return get_extent_if_cached_iertr::make_ready_future<CachedExtentRef>(); + } + + // present in cache(fully loaded) and is not a retired_placeholder + SUBDEBUGT(seastore_cache, "{} {} is present in cache -- {}", + t, type, offset, *ret); + t.add_to_read_set(ret); + touch_extent(*ret); + return ret->wait_io().then([ret] { + return get_extent_if_cached_iertr::make_ready_future< + CachedExtentRef>(ret); + }); + } + + /** + * 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 + * + * t *must not* have retired offset + */ + using get_extent_iertr = base_iertr; + template <typename T, typename Func> + get_extent_iertr::future<TCachedExtentRef<T>> get_extent( + Transaction &t, + paddr_t offset, + extent_len_t length, + Func &&extent_init_func) { + CachedExtentRef ret; + LOG_PREFIX(Cache::get_extent); + auto result = t.get_extent(offset, &ret); + if (result == Transaction::get_extent_ret::RETIRED) { + SUBERRORT(seastore_cache, "{} {}~{} is retired on t -- {}", + t, T::TYPE, offset, length, *ret); + ceph_abort("impossible"); + } else if (result == Transaction::get_extent_ret::PRESENT) { + if (ret->is_fully_loaded()) { + SUBTRACET(seastore_cache, "{} {}~{} is present on t -- {}", + t, T::TYPE, offset, length, *ret); + return ret->wait_io().then([ret] { + return seastar::make_ready_future<TCachedExtentRef<T>>( + ret->cast<T>()); + }); + } else { + assert(!ret->is_mutable()); + touch_extent(*ret); + SUBDEBUGT(seastore_cache, "{} {}~{} is present on t without been \ + fully loaded, reading ... {}", t, T::TYPE, offset, length, *ret); + auto bp = alloc_cache_buf(ret->get_length()); + ret->set_bptr(std::move(bp)); + return read_extent<T>( + ret->cast<T>()); + } + } else { + SUBTRACET(seastore_cache, "{} {}~{} is absent on t, query cache ...", + t, T::TYPE, offset, length); + auto f = [&t, this](CachedExtent &ext) { + t.add_to_read_set(CachedExtentRef(&ext)); + touch_extent(ext); + }; + auto metric_key = std::make_pair(t.get_src(), T::TYPE); + return trans_intr::make_interruptible( + get_extent<T>( + offset, length, &metric_key, + std::forward<Func>(extent_init_func), std::move(f)) + ); + } + } + + /* + * get_absent_extent + * + * Mostly the same as Cache::get_extent(), with the only difference + * that get_absent_extent won't search the transaction's context for + * the specific CachedExtent + */ + template <typename T, typename Func> + get_extent_iertr::future<TCachedExtentRef<T>> get_absent_extent( + Transaction &t, + paddr_t offset, + extent_len_t length, + Func &&extent_init_func) { + CachedExtentRef ret; + LOG_PREFIX(Cache::get_absent_extent); + +#ifndef NDEBUG + auto r = t.get_extent(offset, &ret); + if (r != Transaction::get_extent_ret::ABSENT) { + SUBERRORT(seastore_cache, "unexpected non-absent extent {}", t, *ret); + ceph_abort(); + } +#endif + + SUBTRACET(seastore_cache, "{} {}~{} is absent on t, query cache ...", + t, T::TYPE, offset, length); + auto f = [&t, this](CachedExtent &ext) { + t.add_to_read_set(CachedExtentRef(&ext)); + touch_extent(ext); + }; + auto metric_key = std::make_pair(t.get_src(), T::TYPE); + return trans_intr::make_interruptible( + get_extent<T>( + offset, length, &metric_key, + std::forward<Func>(extent_init_func), std::move(f)) + ); + } + + template <typename T> + get_extent_iertr::future<TCachedExtentRef<T>> get_extent( + Transaction &t, + paddr_t offset, + extent_len_t length) { + return get_extent<T>(t, offset, length, [](T &){}); + } + + /* + * get_absent_extent + * + * Mostly the same as Cache::get_extent(), with the only difference + * that get_absent_extent won't search the transaction's context for + * the specific CachedExtent + */ + template <typename T> + get_extent_iertr::future<TCachedExtentRef<T>> get_absent_extent( + Transaction &t, + paddr_t offset, + extent_len_t length) { + return get_absent_extent<T>(t, offset, length, [](T &){}); + } + + get_extent_ertr::future<CachedExtentRef> get_extent_viewable_by_trans( + Transaction &t, + CachedExtentRef extent) + { + auto p_extent = extent->get_transactional_view(t); + if (!p_extent->is_pending_in_trans(t.get_trans_id())) { + t.add_to_read_set(p_extent); + if (!p_extent->is_mutation_pending()) { + touch_extent(*p_extent); + } + } + // user should not see RETIRED_PLACEHOLDER extents + ceph_assert(p_extent->get_type() != extent_types_t::RETIRED_PLACEHOLDER); + if (!p_extent->is_fully_loaded()) { + assert(!p_extent->is_mutable()); + touch_extent(*p_extent); + LOG_PREFIX(Cache::get_extent_viewable_by_trans); + SUBDEBUG(seastore_cache, + "{} {}~{} is present without been fully loaded, reading ... -- {}", + p_extent->get_type(), p_extent->get_paddr(),p_extent->get_length(), + *p_extent); + auto bp = alloc_cache_buf(p_extent->get_length()); + p_extent->set_bptr(std::move(bp)); + return read_extent<CachedExtent>(CachedExtentRef(p_extent)); + } + return p_extent->wait_io( + ).then([p_extent] { + return get_extent_ertr::make_ready_future<CachedExtentRef>( + CachedExtentRef(p_extent)); + }); + } + + template <typename T> + get_extent_ertr::future<TCachedExtentRef<T>> get_extent_viewable_by_trans( + Transaction &t, + TCachedExtentRef<T> extent) + { + return get_extent_viewable_by_trans(t, CachedExtentRef(extent.get()) + ).safe_then([](auto p_extent) { + return p_extent->template cast<T>(); + }); + } + + extent_len_t get_block_size() const { + return epm.get_block_size(); + } + +private: + // This is a workaround std::move_only_function not being available, + // not really worth generalizing at this time. + class extent_init_func_t { + struct callable_i { + virtual void operator()(CachedExtent &extent) = 0; + virtual ~callable_i() = default; + }; + template <typename Func> + struct callable_wrapper final : callable_i { + Func func; + callable_wrapper(Func &&func) : func(std::forward<Func>(func)) {} + void operator()(CachedExtent &extent) final { + return func(extent); + } + ~callable_wrapper() final = default; + }; + public: + std::unique_ptr<callable_i> wrapped; + template <typename Func> + extent_init_func_t(Func &&func) : wrapped( + std::make_unique<callable_wrapper<Func>>(std::forward<Func>(func))) + {} + void operator()(CachedExtent &extent) { + return (*wrapped)(extent); + } + }; + get_extent_ertr::future<CachedExtentRef> _get_extent_by_type( + extent_types_t type, + paddr_t offset, + laddr_t laddr, + extent_len_t length, + const Transaction::src_t* p_src, + extent_init_func_t &&extent_init_func, + extent_init_func_t &&on_cache + ); + + using get_extent_by_type_iertr = get_extent_iertr; + using get_extent_by_type_ret = get_extent_by_type_iertr::future< + CachedExtentRef>; + get_extent_by_type_ret _get_extent_by_type( + Transaction &t, + extent_types_t type, + paddr_t offset, + laddr_t laddr, + extent_len_t length, + extent_init_func_t &&extent_init_func + ) { + LOG_PREFIX(Cache::get_extent_by_type); + CachedExtentRef ret; + auto status = t.get_extent(offset, &ret); + if (status == Transaction::get_extent_ret::RETIRED) { + SUBERRORT(seastore_cache, "{} {}~{} {} is retired on t -- {}", + t, type, offset, length, laddr, *ret); + ceph_abort("impossible"); + } else if (status == Transaction::get_extent_ret::PRESENT) { + if (ret->is_fully_loaded()) { + SUBTRACET(seastore_cache, "{} {}~{} {} is present on t -- {}", + t, type, offset, length, laddr, *ret); + return ret->wait_io().then([ret] { + return seastar::make_ready_future<CachedExtentRef>(ret); + }); + } else { + assert(!ret->is_mutable()); + touch_extent(*ret); + SUBDEBUGT(seastore_cache, "{} {}~{} {} is present on t without been \ + fully loaded, reading ...", t, type, offset, length, laddr); + auto bp = alloc_cache_buf(ret->get_length()); + ret->set_bptr(std::move(bp)); + return read_extent<CachedExtent>( + std::move(ret)); + } + } else { + SUBTRACET(seastore_cache, "{} {}~{} {} is absent on t, query cache ...", + t, type, offset, length, laddr); + auto f = [&t, this](CachedExtent &ext) { + t.add_to_read_set(CachedExtentRef(&ext)); + touch_extent(ext); + }; + auto src = t.get_src(); + return trans_intr::make_interruptible( + _get_extent_by_type( + type, offset, laddr, length, &src, + std::move(extent_init_func), std::move(f)) + ); + } + } + + get_extent_by_type_ret _get_absent_extent_by_type( + Transaction &t, + extent_types_t type, + paddr_t offset, + laddr_t laddr, + extent_len_t length, + extent_init_func_t &&extent_init_func + ) { + LOG_PREFIX(Cache::_get_absent_extent_by_type); + +#ifndef NDEBUG + CachedExtentRef ret; + auto r = t.get_extent(offset, &ret); + if (r != Transaction::get_extent_ret::ABSENT) { + SUBERRORT(seastore_cache, "unexpected non-absent extent {}", t, *ret); + ceph_abort(); + } +#endif + + SUBTRACET(seastore_cache, "{} {}~{} {} is absent on t, query cache ...", + t, type, offset, length, laddr); + auto f = [&t, this](CachedExtent &ext) { + t.add_to_read_set(CachedExtentRef(&ext)); + touch_extent(ext); + }; + auto src = t.get_src(); + return trans_intr::make_interruptible( + _get_extent_by_type( + type, offset, laddr, length, &src, + std::move(extent_init_func), std::move(f)) + ); + } + + backref_entryrefs_by_seq_t backref_entryrefs_by_seq; + backref_entry_mset_t backref_entry_mset; + + using backref_entry_query_mset_t = std::multiset< + backref_entry_t, backref_entry_t::cmp_t>; + backref_entry_query_mset_t get_backref_entries_in_range( + paddr_t start, + paddr_t end) { + auto start_iter = backref_entry_mset.lower_bound( + start, + backref_entry_t::cmp_t()); + auto end_iter = backref_entry_mset.lower_bound( + end, + backref_entry_t::cmp_t()); + backref_entry_query_mset_t res; + for (auto it = start_iter; + it != end_iter; + it++) { + res.emplace(it->paddr, it->laddr, it->len, it->type, it->seq); + } + return res; + } + + const backref_entry_mset_t& get_backref_entry_mset() { + return backref_entry_mset; + } + + backref_entryrefs_by_seq_t& get_backref_entryrefs_by_seq() { + return backref_entryrefs_by_seq; + } + + const segment_info_t* get_segment_info(segment_id_t sid) { + auto provider = segment_providers_by_device_id[sid.device_id()]; + if (provider) { + return &provider->get_seg_info(sid); + } else { + return nullptr; + } + } + +public: + /** + * get_extent_by_type + * + * Based on type, instantiate the correct concrete type + * and read in the extent at location offset~length. + */ + template <typename Func> + get_extent_by_type_ret get_extent_by_type( + Transaction &t, ///< [in] transaction + extent_types_t type, ///< [in] type tag + paddr_t offset, ///< [in] starting addr + laddr_t laddr, ///< [in] logical address if logical + extent_len_t length, ///< [in] length + Func &&extent_init_func ///< [in] extent init func + ) { + return _get_extent_by_type( + t, + type, + offset, + laddr, + length, + extent_init_func_t(std::forward<Func>(extent_init_func))); + } + + /* + * get_absent_extent_by_type + * + * Mostly the same as Cache::get_extent_by_type(), with the only difference + * that get_absent_extent_by_type won't search the transaction's context for + * the specific CachedExtent + */ + template <typename Func> + get_extent_by_type_ret get_absent_extent_by_type( + Transaction &t, ///< [in] transaction + extent_types_t type, ///< [in] type tag + paddr_t offset, ///< [in] starting addr + laddr_t laddr, ///< [in] logical address if logical + extent_len_t length, ///< [in] length + Func &&extent_init_func ///< [in] extent init func + ) { + return _get_absent_extent_by_type( + t, + type, + offset, + laddr, + length, + extent_init_func_t(std::forward<Func>(extent_init_func))); + } + + get_extent_by_type_ret get_extent_by_type( + Transaction &t, + extent_types_t type, + paddr_t offset, + laddr_t laddr, + extent_len_t length + ) { + return get_extent_by_type( + t, type, offset, laddr, length, [](CachedExtent &) {}); + } + + + /* + * get_absent_extent_by_type + * + * Mostly the same as Cache::get_extent_by_type(), with the only difference + * that get_absent_extent_by_type won't search the transaction's context for + * the specific CachedExtent + */ + get_extent_by_type_ret get_absent_extent_by_type( + Transaction &t, + extent_types_t type, + paddr_t offset, + laddr_t laddr, + extent_len_t length + ) { + return get_absent_extent_by_type( + t, type, offset, laddr, length, [](CachedExtent &) {}); + } + + void trim_backref_bufs(const journal_seq_t &trim_to) { + LOG_PREFIX(Cache::trim_backref_bufs); + SUBDEBUG(seastore_cache, "trimming to {}", trim_to); + if (!backref_entryrefs_by_seq.empty()) { + SUBDEBUG(seastore_cache, "backref_entryrefs_by_seq {} ~ {}, size={}", + backref_entryrefs_by_seq.rbegin()->first, + backref_entryrefs_by_seq.begin()->first, + backref_entryrefs_by_seq.size()); + assert(backref_entryrefs_by_seq.rbegin()->first >= trim_to); + auto iter = backref_entryrefs_by_seq.upper_bound(trim_to); + backref_entryrefs_by_seq.erase(backref_entryrefs_by_seq.begin(), iter); + } + if (backref_entryrefs_by_seq.empty()) { + SUBDEBUG(seastore_cache, "backref_entryrefs_by_seq all trimmed"); + } + } + + /** + * alloc_new_extent + * + * Allocates a fresh extent. if delayed is true, addr will be alloc'd later. + * Note that epaddr can only be fed by the btree lba unittest for now + */ + template <typename T> + TCachedExtentRef<T> alloc_new_extent( + Transaction &t, ///< [in, out] current transaction + extent_len_t length, ///< [in] length + placement_hint_t hint, ///< [in] user hint +#ifdef UNIT_TESTS_BUILT + rewrite_gen_t gen, ///< [in] rewrite generation + std::optional<paddr_t> epaddr = std::nullopt ///< [in] paddr fed by callers +#else + rewrite_gen_t gen +#endif + ) { + LOG_PREFIX(Cache::alloc_new_extent); + SUBTRACET(seastore_cache, "allocate {} {}B, hint={}, gen={}", + t, T::TYPE, length, hint, rewrite_gen_printer_t{gen}); +#ifdef UNIT_TESTS_BUILT + auto result = epm.alloc_new_extent(t, T::TYPE, length, hint, gen, epaddr); +#else + auto result = epm.alloc_new_extent(t, T::TYPE, length, hint, gen); +#endif + auto ret = CachedExtent::make_cached_extent_ref<T>(std::move(result.bp)); + ret->init(CachedExtent::extent_state_t::INITIAL_WRITE_PENDING, + result.paddr, + hint, + result.gen, + t.get_trans_id()); + t.add_fresh_extent(ret); + SUBDEBUGT(seastore_cache, + "allocated {} {}B extent at {}, hint={}, gen={} -- {}", + t, T::TYPE, length, result.paddr, + hint, rewrite_gen_printer_t{result.gen}, *ret); + return ret; + } + + /** + * alloc_remapped_extent + * + * Allocates an EXIST_CLEAN extent. Use the buffer to fill the new extent + * if buffer exists. + */ + template <typename T> + TCachedExtentRef<T> alloc_remapped_extent( + Transaction &t, + laddr_t remap_laddr, + paddr_t remap_paddr, + extent_len_t remap_length, + laddr_t original_laddr, + std::optional<ceph::bufferptr> &&original_bptr) { + LOG_PREFIX(Cache::alloc_remapped_extent); + assert(remap_laddr >= original_laddr); + TCachedExtentRef<T> ext; + if (original_bptr.has_value()) { + // shallow copy the buffer from original extent + auto nbp = ceph::bufferptr( + *original_bptr, + remap_laddr - original_laddr, + remap_length); + // ExtentPlacementManager::alloc_new_extent will make a new + // (relative/temp) paddr, so make extent directly + ext = CachedExtent::make_cached_extent_ref<T>(std::move(nbp)); + } else { + ext = CachedExtent::make_placeholder_cached_extent_ref<T>(remap_length); + } + + ext->init(CachedExtent::extent_state_t::EXIST_CLEAN, + remap_paddr, + PLACEMENT_HINT_NULL, + NULL_GENERATION, + t.get_trans_id()); + + t.add_fresh_extent(ext); + SUBTRACET(seastore_cache, "allocated {} {}B, hint={}, has ptr? {} -- {}", + t, T::TYPE, remap_length, remap_laddr, original_bptr.has_value(), *ext); + return ext; + } + + /** + * alloc_new_extent + * + * Allocates a fresh extent. addr will be relative until commit. + */ + CachedExtentRef alloc_new_extent_by_type( + Transaction &t, ///< [in, out] current transaction + extent_types_t type, ///< [in] type tag + extent_len_t length, ///< [in] length + placement_hint_t hint, ///< [in] user hint + rewrite_gen_t gen ///< [in] rewrite generation + ); + + /** + * 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 + ); + + /** + * set_segment_provider + * + * Set to provide segment information to help identify out-dated delta. + * + * FIXME: This is specific to the segmented implementation + */ + void set_segment_providers(std::vector<SegmentProvider*> &&providers) { + segment_providers_by_device_id = std::move(providers); + } + + /** + * prepare_record + * + * Construct the record for Journal from transaction. + */ + record_t prepare_record( + Transaction &t, ///< [in, out] current transaction + const journal_seq_t &journal_head, + const journal_seq_t &journal_dirty_tail + ); + + /** + * 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 + journal_seq_t seq ///< [in] journal commit seq + ); + + /** + * init + */ + void init(); + + /** + * 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_iertr = base_iertr; + mkfs_iertr::future<> mkfs(Transaction &t); + + /** + * close + * + * TODO: should 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. + * + * Returns whether the delta is applied. + */ + using replay_delta_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + using replay_delta_ret = replay_delta_ertr::future<bool>; + replay_delta_ret replay_delta( + journal_seq_t seq, + paddr_t record_block_base, + const delta_info_t &delta, + const journal_seq_t &dirty_tail, + const journal_seq_t &alloc_tail, + sea_time_point modify_time); + + /** + * init_cached_extents + * + * Calls passed lambda for each dirty cached block. Intended for use + * after replay to allow lba_manager (or w/e) to read in any ancestor + * blocks. + */ + using init_cached_extents_iertr = base_iertr; + using init_cached_extents_ret = init_cached_extents_iertr::future<>; + template <typename F> + init_cached_extents_ret init_cached_extents( + Transaction &t, + F &&f) + { + LOG_PREFIX(Cache::init_cached_extents); + SUBINFOT(seastore_cache, + "start with {}({}B) extents, {} dirty, dirty_from={}, alloc_from={}", + t, + extents.size(), + extents.get_bytes(), + dirty.size(), + get_oldest_dirty_from().value_or(JOURNAL_SEQ_NULL), + get_oldest_backref_dirty_from().value_or(JOURNAL_SEQ_NULL)); + + // journal replay should has been finished at this point, + // Cache::root should have been inserted to the dirty list + assert(root->is_dirty()); + std::vector<CachedExtentRef> _dirty; + for (auto &e : extents) { + _dirty.push_back(CachedExtentRef(&e)); + } + return seastar::do_with( + std::forward<F>(f), + std::move(_dirty), + [this, FNAME, &t](auto &f, auto &refs) mutable + { + return trans_intr::do_for_each( + refs, + [this, FNAME, &t, &f](auto &e) + { + SUBTRACET(seastore_cache, "inspecting extent ... -- {}", t, *e); + return f(t, e + ).si_then([this, FNAME, &t, e](bool is_alive) { + if (!is_alive) { + SUBDEBUGT(seastore_cache, "extent is not alive, remove extent -- {}", t, *e); + remove_extent(e); + e->set_invalid(t); + } else { + SUBDEBUGT(seastore_cache, "extent is alive -- {}", t, *e); + } + }); + }); + }).handle_error_interruptible( + init_cached_extents_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in Cache::init_cached_extents" + } + ).si_then([this, FNAME, &t] { + SUBINFOT(seastore_cache, + "finish with {}({}B) extents, {} dirty, dirty_from={}, alloc_from={}", + t, + extents.size(), + extents.get_bytes(), + dirty.size(), + get_oldest_dirty_from().value_or(JOURNAL_SEQ_NULL), + get_oldest_backref_dirty_from().value_or(JOURNAL_SEQ_NULL)); + }); + } + + /** + * update_extent_from_transaction + * + * Updates passed extent based on t. If extent has been retired, + * a null result will be returned. + */ + CachedExtentRef update_extent_from_transaction( + Transaction &t, + CachedExtentRef extent) { + if (extent->get_type() == extent_types_t::ROOT) { + if (t.root) { + return t.root; + } else { + t.add_to_read_set(extent); + t.root = extent->cast<RootBlock>(); + return extent; + } + } else { + auto result = t.get_extent(extent->get_paddr(), &extent); + if (result == Transaction::get_extent_ret::RETIRED) { + return CachedExtentRef(); + } else { + if (result == Transaction::get_extent_ret::ABSENT) { + t.add_to_read_set(extent); + } + return extent; + } + } + } + + /** + * print + * + * Dump summary of contents (TODO) + */ + std::ostream &print( + std::ostream &out) const { + return out; + } + + /** + * get_next_dirty_extents + * + * Returns extents with get_dirty_from() < seq and adds to read set of + * t. + */ + using get_next_dirty_extents_iertr = base_iertr; + using get_next_dirty_extents_ret = get_next_dirty_extents_iertr::future< + std::vector<CachedExtentRef>>; + get_next_dirty_extents_ret get_next_dirty_extents( + Transaction &t, + journal_seq_t seq, + size_t max_bytes); + + /// returns std::nullopt if no pending alloc-infos + std::optional<journal_seq_t> get_oldest_backref_dirty_from() const { + LOG_PREFIX(Cache::get_oldest_backref_dirty_from); + if (backref_entryrefs_by_seq.empty()) { + SUBDEBUG(seastore_cache, "backref_oldest: null"); + return std::nullopt; + } + auto oldest = backref_entryrefs_by_seq.begin()->first; + SUBDEBUG(seastore_cache, "backref_oldest: {}", oldest); + ceph_assert(oldest != JOURNAL_SEQ_NULL); + return oldest; + } + + /// returns std::nullopt if no dirty extents + /// returns JOURNAL_SEQ_NULL if the oldest dirty extent is still pending + std::optional<journal_seq_t> get_oldest_dirty_from() const { + LOG_PREFIX(Cache::get_oldest_dirty_from); + if (dirty.empty()) { + SUBDEBUG(seastore_cache, "dirty_oldest: null"); + return std::nullopt; + } else { + auto oldest = dirty.begin()->get_dirty_from(); + if (oldest == JOURNAL_SEQ_NULL) { + SUBDEBUG(seastore_cache, "dirty_oldest: pending"); + } else { + SUBDEBUG(seastore_cache, "dirty_oldest: {}", oldest); + } + return oldest; + } + } + + /// Dump live extents + void dump_contents(); + + /** + * backref_extent_entry_t + * + * All the backref extent entries have to be indexed by paddr in memory, + * so they can be retrived by range during cleaning. + * + * See BtreeBackrefManager::retrieve_backref_extents_in_range() + */ + struct backref_extent_entry_t { + backref_extent_entry_t( + paddr_t paddr, + paddr_t key, + extent_types_t type) + : paddr(paddr), key(key), type(type) {} + paddr_t paddr = P_ADDR_NULL; + paddr_t key = P_ADDR_NULL; + extent_types_t type = extent_types_t::ROOT; + struct cmp_t { + using is_transparent = paddr_t; + bool operator()( + const backref_extent_entry_t &l, + const backref_extent_entry_t &r) const { + return l.paddr < r.paddr; + } + bool operator()( + const paddr_t &l, + const backref_extent_entry_t &r) const { + return l < r.paddr; + } + bool operator()( + const backref_extent_entry_t &l, + const paddr_t &r) const { + return l.paddr < r; + } + }; + }; + + void update_tree_extents_num(extent_types_t type, int64_t delta) { + switch (type) { + case extent_types_t::LADDR_INTERNAL: + [[fallthrough]]; + case extent_types_t::DINK_LADDR_LEAF: + [[fallthrough]]; + case extent_types_t::LADDR_LEAF: + stats.lba_tree_extents_num += delta; + ceph_assert(stats.lba_tree_extents_num >= 0); + return; + case extent_types_t::OMAP_INNER: + [[fallthrough]]; + case extent_types_t::OMAP_LEAF: + stats.omap_tree_extents_num += delta; + ceph_assert(stats.lba_tree_extents_num >= 0); + return; + case extent_types_t::ONODE_BLOCK_STAGED: + stats.onode_tree_extents_num += delta; + ceph_assert(stats.onode_tree_extents_num >= 0); + return; + case extent_types_t::BACKREF_INTERNAL: + [[fallthrough]]; + case extent_types_t::BACKREF_LEAF: + stats.backref_tree_extents_num += delta; + ceph_assert(stats.backref_tree_extents_num >= 0); + return; + default: + return; + } + } + + uint64_t get_omap_tree_depth() { + return stats.omap_tree_depth; + } + + /// Update lru for access to ref + void touch_extent( + CachedExtent &ext, + const Transaction::src_t* p_src=nullptr) + { + if (p_src && is_background_transaction(*p_src)) + return; + if (ext.is_stable_clean() && !ext.is_placeholder()) { + lru.move_to_top(ext); + } + } + +private: + ExtentPlacementManager& epm; + RootBlockRef root; ///< ref to current root + ExtentIndex extents; ///< set of live extents + + journal_seq_t last_commit = JOURNAL_SEQ_MIN; + + // FIXME: This is specific to the segmented implementation + std::vector<SegmentProvider*> segment_providers_by_device_id; + + transaction_id_t next_id = 0; + + /** + * dirty + * + * holds refs to dirty extents. Ordered by CachedExtent::get_dirty_from(). + */ + CachedExtent::list dirty; + + using backref_extent_entry_query_set_t = + std::set< + backref_extent_entry_t, + backref_extent_entry_t::cmp_t>; + backref_extent_entry_query_set_t backref_extents; + + void add_backref_extent( + paddr_t paddr, + paddr_t key, + extent_types_t type) { + assert(!paddr.is_relative()); + auto [iter, inserted] = backref_extents.emplace(paddr, key, type); + boost::ignore_unused(inserted); + assert(inserted); + } + + void remove_backref_extent(paddr_t paddr) { + auto iter = backref_extents.find(paddr); + if (iter != backref_extents.end()) + backref_extents.erase(iter); + } + + backref_extent_entry_query_set_t get_backref_extents_in_range( + paddr_t start, + paddr_t end) { + auto start_iter = backref_extents.lower_bound(start); + auto end_iter = backref_extents.upper_bound(end); + backref_extent_entry_query_set_t res; + res.insert(start_iter, end_iter); + return res; + } + + friend class crimson::os::seastore::backref::BtreeBackrefManager; + friend class crimson::os::seastore::BackrefManager; + /** + * lru + * + * holds references to recently used extents + */ + class LRU { + // max size (bytes) + const size_t capacity = 0; + + // current size (bytes) + size_t contents = 0; + + CachedExtent::list lru; + + void trim_to_capacity() { + while (contents > capacity) { + assert(lru.size() > 0); + remove_from_lru(lru.front()); + } + } + + void add_to_lru(CachedExtent &extent) { + assert(extent.is_stable_clean() && !extent.is_placeholder()); + + if (!extent.primary_ref_list_hook.is_linked()) { + contents += extent.get_length(); + intrusive_ptr_add_ref(&extent); + lru.push_back(extent); + } + trim_to_capacity(); + } + + public: + LRU(size_t capacity) : capacity(capacity) {} + + size_t get_capacity() const { + return capacity; + } + + size_t get_current_contents_bytes() const { + return contents; + } + + size_t get_current_contents_extents() const { + return lru.size(); + } + + void remove_from_lru(CachedExtent &extent) { + assert(extent.is_stable_clean() && !extent.is_placeholder()); + + if (extent.primary_ref_list_hook.is_linked()) { + lru.erase(lru.s_iterator_to(extent)); + assert(contents >= extent.get_length()); + contents -= extent.get_length(); + intrusive_ptr_release(&extent); + } + } + + void move_to_top(CachedExtent &extent) { + assert(extent.is_stable_clean() && !extent.is_placeholder()); + + if (extent.primary_ref_list_hook.is_linked()) { + lru.erase(lru.s_iterator_to(extent)); + intrusive_ptr_release(&extent); + assert(contents >= extent.get_length()); + contents -= extent.get_length(); + } + add_to_lru(extent); + } + + void clear() { + LOG_PREFIX(Cache::LRU::clear); + for (auto iter = lru.begin(); iter != lru.end();) { + SUBDEBUG(seastore_cache, "clearing {}", *iter); + remove_from_lru(*(iter++)); + } + } + + ~LRU() { + clear(); + } + } lru; + + struct query_counters_t { + uint64_t access = 0; + uint64_t hit = 0; + }; + + template <typename CounterT> + using counter_by_extent_t = std::array<CounterT, EXTENT_TYPES_MAX>; + + struct invalid_trans_efforts_t { + io_stat_t read; + io_stat_t mutate; + uint64_t mutate_delta_bytes = 0; + io_stat_t retire; + io_stat_t fresh; + io_stat_t fresh_ool_written; + counter_by_extent_t<uint64_t> num_trans_invalidated; + uint64_t total_trans_invalidated = 0; + uint64_t num_ool_records = 0; + uint64_t ool_record_bytes = 0; + }; + + struct commit_trans_efforts_t { + counter_by_extent_t<io_stat_t> read_by_ext; + counter_by_extent_t<io_stat_t> mutate_by_ext; + counter_by_extent_t<uint64_t> delta_bytes_by_ext; + counter_by_extent_t<io_stat_t> retire_by_ext; + counter_by_extent_t<io_stat_t> fresh_invalid_by_ext; // inline but is already invalid (retired) + counter_by_extent_t<io_stat_t> fresh_inline_by_ext; + counter_by_extent_t<io_stat_t> fresh_ool_by_ext; + uint64_t num_trans = 0; // the number of inline records + uint64_t num_ool_records = 0; + uint64_t ool_record_metadata_bytes = 0; + uint64_t ool_record_data_bytes = 0; + uint64_t inline_record_metadata_bytes = 0; // metadata exclude the delta bytes + }; + + struct success_read_trans_efforts_t { + io_stat_t read; + uint64_t num_trans = 0; + }; + + struct tree_efforts_t { + uint64_t num_inserts = 0; + uint64_t num_erases = 0; + uint64_t num_updates = 0; + + void increment(const Transaction::tree_stats_t& incremental) { + num_inserts += incremental.num_inserts; + num_erases += incremental.num_erases; + num_updates += incremental.num_updates; + } + }; + + template <typename CounterT> + using counter_by_src_t = std::array<CounterT, TRANSACTION_TYPE_MAX>; + + static constexpr std::size_t NUM_SRC_COMB = + TRANSACTION_TYPE_MAX * (TRANSACTION_TYPE_MAX + 1) / 2; + + struct { + counter_by_src_t<uint64_t> trans_created_by_src; + counter_by_src_t<commit_trans_efforts_t> committed_efforts_by_src; + counter_by_src_t<invalid_trans_efforts_t> invalidated_efforts_by_src; + counter_by_src_t<query_counters_t> cache_query_by_src; + success_read_trans_efforts_t success_read_efforts; + uint64_t dirty_bytes = 0; + + uint64_t onode_tree_depth = 0; + int64_t onode_tree_extents_num = 0; + counter_by_src_t<tree_efforts_t> committed_onode_tree_efforts; + counter_by_src_t<tree_efforts_t> invalidated_onode_tree_efforts; + + uint64_t omap_tree_depth = 0; + int64_t omap_tree_extents_num = 0; + counter_by_src_t<tree_efforts_t> committed_omap_tree_efforts; + counter_by_src_t<tree_efforts_t> invalidated_omap_tree_efforts; + + uint64_t lba_tree_depth = 0; + int64_t lba_tree_extents_num = 0; + counter_by_src_t<tree_efforts_t> committed_lba_tree_efforts; + counter_by_src_t<tree_efforts_t> invalidated_lba_tree_efforts; + + uint64_t backref_tree_depth = 0; + int64_t backref_tree_extents_num = 0; + counter_by_src_t<tree_efforts_t> committed_backref_tree_efforts; + counter_by_src_t<tree_efforts_t> invalidated_backref_tree_efforts; + + std::array<uint64_t, NUM_SRC_COMB> trans_conflicts_by_srcs; + counter_by_src_t<uint64_t> trans_conflicts_by_unknown; + + version_stat_t committed_dirty_version; + version_stat_t committed_reclaim_version; + } stats; + + template <typename CounterT> + CounterT& get_by_src( + counter_by_src_t<CounterT>& counters_by_src, + Transaction::src_t src) { + assert(static_cast<std::size_t>(src) < counters_by_src.size()); + return counters_by_src[static_cast<std::size_t>(src)]; + } + + template <typename CounterT> + CounterT& get_by_ext( + counter_by_extent_t<CounterT>& counters_by_ext, + extent_types_t ext) { + auto index = static_cast<uint8_t>(ext); + assert(index < EXTENT_TYPES_MAX); + return counters_by_ext[index]; + } + + void account_conflict(Transaction::src_t src1, Transaction::src_t src2) { + assert(src1 < Transaction::src_t::MAX); + assert(src2 < Transaction::src_t::MAX); + if (src1 > src2) { + std::swap(src1, src2); + } + // impossible combinations + // should be consistent with trans_srcs_invalidated in register_metrics() + assert(!(src1 == Transaction::src_t::READ && + src2 == Transaction::src_t::READ)); + assert(!(src1 == Transaction::src_t::TRIM_DIRTY && + src2 == Transaction::src_t::TRIM_DIRTY)); + assert(!(src1 == Transaction::src_t::CLEANER_MAIN && + src2 == Transaction::src_t::CLEANER_MAIN)); + assert(!(src1 == Transaction::src_t::CLEANER_COLD && + src2 == Transaction::src_t::CLEANER_COLD)); + assert(!(src1 == Transaction::src_t::TRIM_ALLOC && + src2 == Transaction::src_t::TRIM_ALLOC)); + + auto src1_value = static_cast<std::size_t>(src1); + auto src2_value = static_cast<std::size_t>(src2); + auto num_srcs = static_cast<std::size_t>(Transaction::src_t::MAX); + auto conflict_index = num_srcs * src1_value + src2_value - + src1_value * (src1_value + 1) / 2; + assert(conflict_index < NUM_SRC_COMB); + ++stats.trans_conflicts_by_srcs[conflict_index]; + } + + seastar::metrics::metric_group metrics; + void register_metrics(); + + /// alloc buffer for cached extent + bufferptr alloc_cache_buf(size_t size) { + // TODO: memory pooling etc + auto bp = ceph::bufferptr( + buffer::create_page_aligned(size)); + bp.zero(); + return bp; + } + + void backref_batch_update( + std::vector<backref_entry_ref> &&, + const journal_seq_t &); + + /// Add extent to extents handling dirty and refcounting + void add_extent(CachedExtentRef ref, const Transaction::src_t* t_src); + + /// Mark exising extent ref dirty -- mainly for replay + void mark_dirty(CachedExtentRef ref); + + /// Add dirty extent to dirty list + void add_to_dirty(CachedExtentRef ref); + + /// Remove from dirty list + void remove_from_dirty(CachedExtentRef ref); + + /// Remove extent from extents handling dirty and refcounting + void remove_extent(CachedExtentRef ref); + + /// Retire extent + void commit_retire_extent(Transaction& t, CachedExtentRef ref); + + /// Replace prev with next + void commit_replace_extent(Transaction& t, CachedExtentRef next, CachedExtentRef prev); + + /// Invalidate extent and mark affected transactions + void invalidate_extent(Transaction& t, CachedExtent& extent); + + /// Mark a valid transaction as conflicted + void mark_transaction_conflicted( + Transaction& t, CachedExtent& conflicting_extent); + + /// Introspect transaction when it is being destructed + void on_transaction_destruct(Transaction& t); + + template <typename T> + get_extent_ret<T> read_extent( + TCachedExtentRef<T>&& extent + ) { + assert(extent->state == CachedExtent::extent_state_t::CLEAN_PENDING || + extent->state == CachedExtent::extent_state_t::EXIST_CLEAN || + extent->state == CachedExtent::extent_state_t::CLEAN); + extent->set_io_wait(); + return epm.read( + extent->get_paddr(), + extent->get_length(), + extent->get_bptr() + ).safe_then( + [extent=std::move(extent)]() mutable { + LOG_PREFIX(Cache::read_extent); + if (likely(extent->state == CachedExtent::extent_state_t::CLEAN_PENDING)) { + extent->state = CachedExtent::extent_state_t::CLEAN; + /* TODO: crc should be checked against LBA manager */ + extent->last_committed_crc = extent->get_crc32c(); + + extent->on_clean_read(); + } else if (extent->state == CachedExtent::extent_state_t::EXIST_CLEAN || + extent->state == CachedExtent::extent_state_t::CLEAN) { + /* TODO: crc should be checked against LBA manager */ + extent->last_committed_crc = extent->get_crc32c(); + } else { + ceph_assert(!extent->is_valid()); + } + extent->complete_io(); + SUBDEBUG(seastore_cache, "read extent done -- {}", *extent); + return get_extent_ertr::make_ready_future<TCachedExtentRef<T>>( + std::move(extent)); + }, + get_extent_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Cache::get_extent: invalid error" + } + ); + } + + // Extents in cache may contain placeholders + CachedExtentRef query_cache( + paddr_t offset, + const src_ext_t* p_metric_key) { + query_counters_t* p_counters = nullptr; + if (p_metric_key) { + p_counters = &get_by_src(stats.cache_query_by_src, p_metric_key->first); + ++p_counters->access; + } + if (auto iter = extents.find_offset(offset); + iter != extents.end()) { + if (p_metric_key && + // retired_placeholder is not really cached yet + iter->get_type() != extent_types_t::RETIRED_PLACEHOLDER) { + ++p_counters->hit; + } + return CachedExtentRef(&*iter); + } else { + return CachedExtentRef(); + } + } + + template < + typename node_key_t, + typename node_val_t, + typename internal_node_t, + typename leaf_node_t, + typename pin_t, + size_t node_size, + bool leaf_has_children> + friend class FixedKVBtree; +}; +using CacheRef = std::unique_ptr<Cache>; + +} diff --git a/src/crimson/os/seastore/cached_extent.cc b/src/crimson/os/seastore/cached_extent.cc new file mode 100644 index 000000000..769b0446a --- /dev/null +++ b/src/crimson/os/seastore/cached_extent.cc @@ -0,0 +1,176 @@ +// -*- 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/os/seastore/transaction.h" + +#include "crimson/common/log.h" + +#include "crimson/os/seastore/btree/fixed_kv_node.h" + +namespace { + [[maybe_unused]] seastar::logger& logger() { + return crimson::get_logger(ceph_subsys_seastore_tm); + } +} + +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 + +bool is_backref_mapped_extent_node(const CachedExtentRef &extent) { + return extent->is_logical() + || is_lba_node(extent->get_type()) + || extent->get_type() == extent_types_t::TEST_BLOCK_PHYSICAL; +} + +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_PENDING: + return out << "CLEAN_PENDING"; + case CachedExtent::extent_state_t::CLEAN: + return out << "CLEAN"; + case CachedExtent::extent_state_t::DIRTY: + return out << "DIRTY"; + case CachedExtent::extent_state_t::EXIST_CLEAN: + return out << "EXIST_CLEAN"; + case CachedExtent::extent_state_t::EXIST_MUTATION_PENDING: + return out << "EXIST_MUTATION_PENDING"; + 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) { + assert(is_linked()); + parent_index->erase(*this); + } +} +CachedExtent* CachedExtent::get_transactional_view(Transaction &t) { + return get_transactional_view(t.get_trans_id()); +} + +CachedExtent* CachedExtent::get_transactional_view(transaction_id_t tid) { + auto it = mutation_pendings.find(tid, trans_spec_view_t::cmp_t()); + if (it != mutation_pendings.end()) { + return (CachedExtent*)&(*it); + } else { + return this; + } +} + +std::ostream &operator<<(std::ostream &out, const parent_tracker_t &tracker) { + return out << "parent_tracker=" << (void*)&tracker + << ", parent=" << (void*)tracker.get_parent().get(); +} + +std::ostream &ChildableCachedExtent::print_detail(std::ostream &out) const { + if (parent_tracker) { + out << *parent_tracker; + } else { + out << ", parent_tracker=" << (void*)nullptr; + } + _print_detail(out); + return out; +} + +std::ostream &LogicalCachedExtent::_print_detail(std::ostream &out) const +{ + out << ", laddr=" << laddr; + return print_detail_l(out); +} + +void child_pos_t::link_child(ChildableCachedExtent *c) { + get_parent<FixedKVNode<laddr_t>>()->link_child(c, pos); +} + +void CachedExtent::set_invalid(Transaction &t) { + state = extent_state_t::INVALID; + if (trans_view_hook.is_linked()) { + trans_view_hook.unlink(); + } + on_invalidated(t); +} + +LogicalCachedExtent::~LogicalCachedExtent() { + if (has_parent_tracker() && is_valid() && !is_pending()) { + assert(get_parent_node()); + auto parent = get_parent_node<FixedKVNode<laddr_t>>(); + auto off = parent->lower_bound_offset(laddr); + assert(parent->get_key_from_idx(off) == laddr); + assert(parent->children[off] == this); + parent->children[off] = nullptr; + } +} + +void LogicalCachedExtent::on_replace_prior(Transaction &t) { + assert(is_mutation_pending()); + take_prior_parent_tracker(); + assert(get_parent_node()); + auto parent = get_parent_node<FixedKVNode<laddr_t>>(); + //TODO: can this search be avoided? + auto off = parent->lower_bound_offset(laddr); + assert(parent->get_key_from_idx(off) == laddr); + parent->children[off] = this; +} + +parent_tracker_t::~parent_tracker_t() { + // this is parent's tracker, reset it + auto &p = (FixedKVNode<laddr_t>&)*parent; + if (p.my_tracker == this) { + p.my_tracker = nullptr; + } +} + +std::ostream &operator<<(std::ostream &out, const LBAMapping &rhs) +{ + return out << "LBAMapping(" << rhs.get_key() << "~" << rhs.get_length() + << "->" << rhs.get_val(); +} + +std::ostream &operator<<(std::ostream &out, const lba_pin_list_t &rhs) +{ + bool first = true; + out << '['; + for (const auto &i: rhs) { + out << (first ? "" : ",") << *i; + first = false; + } + return out << ']'; +} + +} diff --git a/src/crimson/os/seastore/cached_extent.h b/src/crimson/os/seastore/cached_extent.h new file mode 100644 index 000000000..02f8ae46c --- /dev/null +++ b/src/crimson/os/seastore/cached_extent.h @@ -0,0 +1,1304 @@ +// -*- 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/common/interruptible_future.h" +#include "crimson/os/seastore/seastore_types.h" + +struct btree_lba_manager_test; + +namespace crimson::os::seastore { + +class Transaction; +class CachedExtent; +using CachedExtentRef = boost::intrusive_ptr<CachedExtent>; +class SegmentedAllocator; +class TransactionManager; +class ExtentPlacementManager; + +template < + typename node_key_t, + typename node_val_t, + typename internal_node_t, + typename leaf_node_t, + typename pin_t, + size_t node_size, + bool leaf_has_children> +class FixedKVBtree; +template <typename, typename> +class BtreeNodeMapping; + +// #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 + */ +namespace onode { + class DummyNodeExtent; + class TestReplayExtent; +} + +template <typename T> +class read_set_item_t { + using set_hook_t = boost::intrusive::set_member_hook< + boost::intrusive::link_mode< + boost::intrusive::auto_unlink>>; + set_hook_t trans_hook; + using set_hook_options = boost::intrusive::member_hook< + read_set_item_t, + set_hook_t, + &read_set_item_t::trans_hook>; + +public: + struct cmp_t { + using is_transparent = paddr_t; + bool operator()(const read_set_item_t<T> &lhs, const read_set_item_t &rhs) const; + bool operator()(const paddr_t &lhs, const read_set_item_t<T> &rhs) const; + bool operator()(const read_set_item_t<T> &lhs, const paddr_t &rhs) const; + }; + + struct trans_cmp_t { + bool operator()( + const read_set_item_t<Transaction> &lhs, + const read_set_item_t<Transaction> &rhs) const { + return lhs.t < rhs.t; + } + bool operator()( + const Transaction *lhs, + const read_set_item_t<Transaction> &rhs) const { + return lhs < rhs.t; + } + bool operator()( + const read_set_item_t<Transaction> &lhs, + const Transaction *rhs) const { + return lhs.t < rhs; + } + }; + + using trans_set_t = boost::intrusive::set< + read_set_item_t, + set_hook_options, + boost::intrusive::constant_time_size<false>, + boost::intrusive::compare<trans_cmp_t>>; + + T *t = nullptr; + CachedExtentRef ref; + + read_set_item_t(T *t, CachedExtentRef ref); + read_set_item_t(const read_set_item_t &) = delete; + read_set_item_t(read_set_item_t &&) = default; + ~read_set_item_t() = default; +}; +template <typename T> +using read_set_t = std::set< + read_set_item_t<T>, + typename read_set_item_t<T>::cmp_t>; + +struct trans_spec_view_t { + // if the extent is pending, contains the id of the owning transaction; + // TRANS_ID_NULL otherwise + transaction_id_t pending_for_transaction = TRANS_ID_NULL; + + struct cmp_t { + bool operator()( + const trans_spec_view_t &lhs, + const trans_spec_view_t &rhs) const + { + return lhs.pending_for_transaction < rhs.pending_for_transaction; + } + bool operator()( + const transaction_id_t &lhs, + const trans_spec_view_t &rhs) const + { + return lhs < rhs.pending_for_transaction; + } + bool operator()( + const trans_spec_view_t &lhs, + const transaction_id_t &rhs) const + { + return lhs.pending_for_transaction < rhs; + } + }; + + using trans_view_hook_t = + boost::intrusive::set_member_hook< + boost::intrusive::link_mode< + boost::intrusive::auto_unlink>>; + trans_view_hook_t trans_view_hook; + + using trans_view_member_options = + boost::intrusive::member_hook< + trans_spec_view_t, + trans_view_hook_t, + &trans_spec_view_t::trans_view_hook>; + using trans_view_set_t = boost::intrusive::set< + trans_spec_view_t, + trans_view_member_options, + boost::intrusive::constant_time_size<false>, + boost::intrusive::compare<cmp_t>>; +}; + +class ExtentIndex; +class CachedExtent + : public boost::intrusive_ref_counter< + CachedExtent, boost::thread_unsafe_counter>, + public trans_spec_view_t { + 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_PENDING, // CLEAN, but not yet read out + 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 + EXIST_CLEAN, // Similar to CLEAN, but its metadata not yet + // persisted to disk. + // In Transaction::write_set and existing_block_list. + // After transaction commits, state becomes CLEAN + // and add extent to Cache. Modifing such extents + // will cause state turn to EXIST_MUTATION_PENDING. + EXIST_MUTATION_PENDING,// Similar to MUTATION_PENDING, but its prior_instance + // is empty. + // In Transaction::write_set, existing_block_list and + // mutated_block_list. State becomes DIRTY and it is + // added to Cache after transaction commits. + INVALID // Part of no ExtentIndex set + } state = extent_state_t::INVALID; + friend std::ostream &operator<<(std::ostream &, extent_state_t); + // allow a dummy extent to pretend it is at a specific state + friend class onode::DummyNodeExtent; + friend class onode::TestReplayExtent; + + template < + typename node_key_t, + typename node_val_t, + typename internal_node_t, + typename leaf_node_t, + typename pin_t, + size_t node_size, + bool leaf_has_children> + friend class FixedKVBtree; + uint32_t last_committed_crc = 0; + + // Points at current version while in state MUTATION_PENDING + CachedExtentRef prior_instance; + + // time of the last modification + sea_time_point modify_time = NULL_TIME; + +public: + void init(extent_state_t _state, + paddr_t paddr, + placement_hint_t hint, + rewrite_gen_t gen, + transaction_id_t trans_id) { + assert(gen == NULL_GENERATION || is_rewrite_generation(gen)); + state = _state; + set_paddr(paddr); + user_hint = hint; + rewrite_generation = gen; + pending_for_transaction = trans_id; + } + + void set_modify_time(sea_time_point t) { + modify_time = t; + } + + sea_time_point get_modify_time() const { + return modify_time; + } + + /** + * 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(Transaction &t) = 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() {} + + /** + * prepare_commit + * + * Called prior to committing the transaction in which this extent + * is living. + */ + virtual void prepare_commit() {} + + /** + * 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_clean_read + * + * Called after read of initially written extent. + * State will be CLEAN. Implentation may use this + * call to fixup the buffer with the newly available + * absolute get_paddr(). + */ + virtual void on_clean_read() {} + + /** + * 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) {} + + /** + * on_replace_prior + * + * Called after the extent has replaced a previous one. State + * of the extent must be MUTATION_PENDING. Implementation + * may use this call to synchronize states that must be synchronized + * with the states of Cache and can't wait till transaction + * completes. + */ + virtual void on_replace_prior(Transaction &t) {} + + /** + * on_invalidated + * + * Called after the extent is invalidated, either by Cache::invalidate_extent + * or Transaction::add_to_retired_set. Implementation may use this + * call to adjust states that must be changed immediately once + * invalidated. + */ + virtual void on_invalidated(Transaction &t) {} + /** + * get_type + * + * Returns concrete type. + */ + virtual extent_types_t get_type() const = 0; + + virtual bool is_logical() const { + return false; + } + + virtual bool may_conflict() const { + return true; + } + + friend std::ostream &operator<<(std::ostream &, extent_state_t); + virtual std::ostream &print_detail(std::ostream &out) const { return out; } + std::ostream &print(std::ostream &out) const { + std::string prior_poffset_str = prior_poffset + ? fmt::format("{}", *prior_poffset) + : "nullopt"; + out << "CachedExtent(addr=" << this + << ", type=" << get_type() + << ", version=" << version + << ", dirty_from_or_retired_at=" << dirty_from_or_retired_at + << ", modify_time=" << sea_time_point_printer_t{modify_time} + << ", paddr=" << get_paddr() + << ", prior_paddr=" << prior_poffset_str + << ", length=" << get_length() + << ", state=" << state + << ", last_committed_crc=" << last_committed_crc + << ", refcount=" << use_count() + << ", user_hint=" << user_hint + << ", fully_loaded=" << is_fully_loaded() + << ", rewrite_gen=" << rewrite_gen_printer_t{rewrite_generation}; + if (state != extent_state_t::INVALID && + state != extent_state_t::CLEAN_PENDING) { + print_detail(out); + } + return out << ")"; + } + + /** + * get_delta + * + * Must return a valid delta usable in apply_delta() in submit_transaction + * if state == MUTATION_PENDING. + */ + virtual ceph::bufferlist get_delta() = 0; + + /** + * apply_delta + * + * 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. + * + * Implementation *must* use set_last_committed_crc to update the crc to + * what the crc of the buffer would have been at submission. For physical + * extents that use base to adjust internal record-relative deltas, this + * means that the crc should be of the buffer after applying the delta, + * but before that adjustment. We do it this way because the crc in the + * commit path does not yet know the record base address. + * + * LogicalCachedExtent overrides this method and provides a simpler + * apply_delta override for LogicalCachedExtent implementers. + */ + virtual void apply_delta_and_adjust_crc( + paddr_t base, const 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)); + } + template <typename T> + TCachedExtentRef<const T> cast() const { + return TCachedExtentRef<const T>(static_cast<const T*>(this)); + } + + /// Returns true if extent can be mutated in an open transaction + bool is_mutable() const { + return state == extent_state_t::INITIAL_WRITE_PENDING || + state == extent_state_t::MUTATION_PENDING || + state == extent_state_t::EXIST_MUTATION_PENDING; + } + + /// Returns true if extent is part of an open transaction + bool is_pending() const { + return is_mutable() || state == extent_state_t::EXIST_CLEAN; + } + + /// Returns true if extent is stable and shared among transactions + bool is_stable() const { + return state == extent_state_t::CLEAN_PENDING || + state == extent_state_t::CLEAN || + state == extent_state_t::DIRTY; + } + + /// 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 || + state == extent_state_t::CLEAN_PENDING || + state == extent_state_t::EXIST_CLEAN; + } + + // Returs true if extent is stable and clean + bool is_stable_clean() const { + ceph_assert(is_valid()); + return state == extent_state_t::CLEAN || + state == extent_state_t::CLEAN_PENDING; + } + + /// Ruturns true if data is persisted while metadata isn't + bool is_exist_clean() const { + return state == extent_state_t::EXIST_CLEAN; + } + + /// Returns true if the extent with EXTIST_CLEAN is modified + bool is_exist_mutation_pending() const { + return state == extent_state_t::EXIST_MUTATION_PENDING; + } + + /// 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; + } + + /// Returns true if extent or prior_instance has been invalidated + bool has_been_invalidated() const { + return !is_valid() || (is_mutation_pending() && !prior_instance->is_valid()); + } + + /// Returns true if extent is a plcaeholder + bool is_placeholder() const { + return get_type() == extent_types_t::RETIRED_PLACEHOLDER; + } + + bool is_pending_io() const { + return !!io_wait_promise; + } + + /// Return journal location of oldest relevant delta, only valid while DIRTY + auto get_dirty_from() const { + ceph_assert(is_dirty()); + return dirty_from_or_retired_at; + } + + /// Return journal location of oldest relevant delta, only valid while RETIRED + auto get_retired_at() const { + ceph_assert(!is_valid()); + return dirty_from_or_retired_at; + } + + /// Return true if extent is fully loaded or is about to be fully loaded (call + /// wait_io() in this case) + bool is_fully_loaded() const { + return ptr.has_value(); + } + + /** + * 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 data in disk + extent_len_t get_length() const { + return length; + } + + extent_len_t get_loaded_length() const { + if (ptr.has_value()) { + return ptr->length(); + } else { + return 0; + } + } + + /// Returns version, get_version() == 0 iff is_clean() + extent_version_t get_version() const { + return version; + } + + /// Returns crc32c of buffer + uint32_t get_crc32c() { + return ceph_crc32c( + 1, + reinterpret_cast<const unsigned char *>(get_bptr().c_str()), + get_length()); + } + + /// Get ref to raw buffer + bufferptr &get_bptr() { + assert(ptr.has_value()); + return *ptr; + } + const bufferptr &get_bptr() const { + assert(ptr.has_value()); + 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(); + + placement_hint_t get_user_hint() const { + return user_hint; + } + + rewrite_gen_t get_rewrite_generation() const { + return rewrite_generation; + } + + void invalidate_hints() { + user_hint = PLACEMENT_HINT_NULL; + rewrite_generation = NULL_GENERATION; + } + + /// assign the target rewrite generation for the followup rewrite + void set_target_rewrite_generation(rewrite_gen_t gen) { + assert(is_target_rewrite_generation(gen)); + + user_hint = placement_hint_t::REWRITE; + rewrite_generation = gen; + } + + bool is_inline() const { + return poffset.is_relative(); + } + + paddr_t get_prior_paddr_and_reset() { + assert(prior_poffset); + auto ret = *prior_poffset; + prior_poffset.reset(); + return ret; + } + + void set_invalid(Transaction &t); + + // a rewrite extent has an invalid prior_instance, + // and a mutation_pending extent has a valid prior_instance + CachedExtentRef get_prior_instance() { + return prior_instance; + } + +private: + template <typename T> + friend class read_set_item_t; + + 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; + + bool is_linked() { + return extent_index_hook.is_linked(); + } + + /// set bufferptr + void set_bptr(ceph::bufferptr &&nptr) { + ptr = nptr; + } + + /// Returns true if the extent part of the open transaction + bool is_pending_in_trans(transaction_id_t id) const { + return is_pending() && pending_for_transaction == id; + } + + /// 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>; + + /** + * dirty_from_or_retired_at + * + * Encodes ordering token for primary_ref_list -- dirty_from when + * dirty or retired_at if retired. + */ + journal_seq_t dirty_from_or_retired_at; + + /// cache data contents, std::nullopt if no data in cache + std::optional<ceph::bufferptr> ptr; + + /// disk data length + extent_len_t length; + + /// number of deltas since initial write + extent_version_t version = 0; + + /// address of original block -- record relative iff is_initial_pending() + paddr_t poffset; + + /// relative address before ool write, used to update mapping + std::optional<paddr_t> prior_poffset = std::nullopt; + + /// 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(); + } + } + + CachedExtent* get_transactional_view(Transaction &t); + CachedExtent* get_transactional_view(transaction_id_t tid); + + read_set_item_t<Transaction>::trans_set_t transactions; + + placement_hint_t user_hint = PLACEMENT_HINT_NULL; + + // the target rewrite generation for the followup rewrite + // or the rewrite generation for the fresh write + rewrite_gen_t rewrite_generation = NULL_GENERATION; + +protected: + trans_view_set_t mutation_pendings; + + CachedExtent(CachedExtent &&other) = delete; + CachedExtent(ceph::bufferptr &&_ptr) : ptr(std::move(_ptr)) { + length = ptr->length(); + assert(length > 0); + } + + /// construct new CachedExtent, will deep copy the buffer + CachedExtent(const CachedExtent &other) + : state(other.state), + dirty_from_or_retired_at(other.dirty_from_or_retired_at), + length(other.get_length()), + version(other.version), + poffset(other.poffset) { + assert((length % CEPH_PAGE_SIZE) == 0); + if (other.is_fully_loaded()) { + ptr.emplace(buffer::create_page_aligned(length)); + other.ptr->copy_out(0, length, ptr->c_str()); + } else { + // the extent must be fully loaded before CoW + assert(length == 0); // in case of root + } + } + + struct share_buffer_t {}; + /// construct new CachedExtent, will shallow copy the buffer + CachedExtent(const CachedExtent &other, share_buffer_t) + : state(other.state), + dirty_from_or_retired_at(other.dirty_from_or_retired_at), + ptr(other.ptr), + length(other.get_length()), + version(other.version), + poffset(other.poffset) {} + + // 0 length is only possible for the RootBlock + struct zero_length_t {}; + CachedExtent(zero_length_t) : ptr(ceph::bufferptr(0)), length(0) {}; + + struct retired_placeholder_t{}; + CachedExtent(retired_placeholder_t, extent_len_t _length) + : state(extent_state_t::INVALID), + length(_length) { + assert(length > 0); + } + + /// no buffer extent, for lazy read + CachedExtent(extent_len_t _length) : length(_length) { + assert(length > 0); + } + + friend class Cache; + template <typename T, typename... Args> + static TCachedExtentRef<T> make_cached_extent_ref( + Args&&... args) { + return new T(std::forward<Args>(args)...); + } + + template <typename T> + static TCachedExtentRef<T> make_placeholder_cached_extent_ref( + extent_len_t length) { + return new T(length); + } + + void reset_prior_instance() { + prior_instance.reset(); + } + + /// Sets last_committed_crc + void set_last_committed_crc(uint32_t crc) { + last_committed_crc = crc; + } + + void set_paddr(paddr_t offset, bool need_update_mapping = false) { + if (need_update_mapping) { + assert(!prior_poffset); + prior_poffset = poffset; + } + 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 (is_initial_pending() && addr.is_record_relative()) { + return addr.block_relative_to(get_paddr()); + } else { + ceph_assert(!addr.is_record_relative() || is_mutation_pending()); + return addr; + } + } + + friend class crimson::os::seastore::SegmentedAllocator; + friend class crimson::os::seastore::TransactionManager; + friend class crimson::os::seastore::ExtentPlacementManager; + template <typename, typename> + friend class BtreeNodeMapping; + friend class ::btree_lba_manager_test; +}; + +std::ostream &operator<<(std::ostream &, CachedExtent::extent_state_t); +std::ostream &operator<<(std::ostream &, const CachedExtent&); + +bool is_backref_mapped_extent_node(const CachedExtentRef &extent); + +/// 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, extent_len_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.lower_bound(addr.add_offset(len), paddr_cmp()); + return std::make_pair( + bottom, + top + ); + } + + void clear() { + struct cached_extent_disposer { + void operator() (CachedExtent* extent) { + extent->parent_index = nullptr; + } + }; + extent_index.clear_and_dispose(cached_extent_disposer()); + bytes = 0; + } + + void insert(CachedExtent &extent) { + // sanity check + ceph_assert(!extent.parent_index); + auto [a, b] = get_overlap( + extent.get_paddr(), + extent.get_length()); + ceph_assert(a == b); + + [[maybe_unused]] auto [iter, inserted] = extent_index.insert(extent); + assert(inserted); + extent.parent_index = this; + + bytes += extent.get_length(); + } + + void erase(CachedExtent &extent) { + assert(extent.parent_index); + assert(extent.is_linked()); + [[maybe_unused]] auto erased = extent_index.erase( + extent_index.s_iterator_to(extent)); + extent.parent_index = nullptr; + + assert(erased); + bytes -= extent.get_length(); + } + + void replace(CachedExtent &to, CachedExtent &from) { + assert(to.get_length() == from.get_length()); + extent_index.replace_node(extent_index.s_iterator_to(from), to); + from.parent_index = nullptr; + to.parent_index = this; + } + + 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(); + } + + auto size() const { + return extent_index.size(); + } + + auto get_bytes() const { + return bytes; + } + + ~ExtentIndex() { + assert(extent_index.empty()); + assert(bytes == 0); + } + +private: + uint64_t bytes = 0; +}; + +class ChildableCachedExtent; +class LogicalCachedExtent; + +class child_pos_t { +public: + child_pos_t(CachedExtentRef stable_parent, uint16_t pos) + : stable_parent(stable_parent), pos(pos) {} + + template <typename parent_t> + TCachedExtentRef<parent_t> get_parent() { + ceph_assert(stable_parent); + return stable_parent->template cast<parent_t>(); + } + uint16_t get_pos() { + return pos; + } + void link_child(ChildableCachedExtent *c); +private: + CachedExtentRef stable_parent; + uint16_t pos = std::numeric_limits<uint16_t>::max(); +}; + +using get_child_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; +template <typename T> +struct get_child_ret_t { + std::variant<child_pos_t, get_child_ertr::future<TCachedExtentRef<T>>> ret; + get_child_ret_t(child_pos_t pos) + : ret(std::move(pos)) {} + get_child_ret_t(get_child_ertr::future<TCachedExtentRef<T>> child) + : ret(std::move(child)) {} + + bool has_child() const { + return ret.index() == 1; + } + + child_pos_t &get_child_pos() { + ceph_assert(ret.index() == 0); + return std::get<0>(ret); + } + + get_child_ertr::future<TCachedExtentRef<T>> &get_child_fut() { + ceph_assert(ret.index() == 1); + return std::get<1>(ret); + } +}; + +template <typename key_t, typename> +class PhysicalNodeMapping; + +template <typename key_t, typename val_t> +using PhysicalNodeMappingRef = std::unique_ptr<PhysicalNodeMapping<key_t, val_t>>; + +template <typename key_t, typename val_t> +class PhysicalNodeMapping { +public: + virtual extent_len_t get_length() const = 0; + virtual extent_types_t get_type() const = 0; + virtual val_t get_val() const = 0; + virtual key_t get_key() const = 0; + virtual PhysicalNodeMappingRef<key_t, val_t> duplicate() const = 0; + virtual bool has_been_invalidated() const = 0; + virtual CachedExtentRef get_parent() const = 0; + virtual uint16_t get_pos() const = 0; + // An lba pin may be indirect, see comments in lba_manager/btree/btree_lba_manager.h + virtual bool is_indirect() const { return false; } + virtual key_t get_intermediate_key() const { return min_max_t<key_t>::null; } + virtual key_t get_intermediate_base() const { return min_max_t<key_t>::null; } + virtual extent_len_t get_intermediate_length() const { return 0; } + // The start offset of the pin, must be 0 if the pin is not indirect + virtual extent_len_t get_intermediate_offset() const { + return std::numeric_limits<extent_len_t>::max(); + } + + virtual get_child_ret_t<LogicalCachedExtent> + get_logical_extent(Transaction &t) = 0; + + void link_child(ChildableCachedExtent *c) { + ceph_assert(child_pos); + child_pos->link_child(c); + } + + virtual ~PhysicalNodeMapping() {} +protected: + std::optional<child_pos_t> child_pos = std::nullopt; +}; + +using LBAMapping = PhysicalNodeMapping<laddr_t, paddr_t>; +using LBAMappingRef = PhysicalNodeMappingRef<laddr_t, paddr_t>; + +std::ostream &operator<<(std::ostream &out, const LBAMapping &rhs); + +using lba_pin_list_t = std::list<LBAMappingRef>; + +std::ostream &operator<<(std::ostream &out, const lba_pin_list_t &rhs); + +using BackrefMapping = PhysicalNodeMapping<paddr_t, laddr_t>; +using BackrefMappingRef = PhysicalNodeMappingRef<paddr_t, laddr_t>; + +using backref_pin_list_t = std::list<BackrefMappingRef>; + +/** + * RetiredExtentPlaceholder + * + * Cache::retire_extent_addr(Transaction&, paddr_t, extent_len_t) can retire an + * extent not currently in cache. In that case, in order to detect transaction + * invalidation, we need to add a placeholder to the cache to create the + * mapping back to the transaction. And whenever there is a transaction tries + * to read the placeholder extent out, Cache is responsible to replace the + * placeholder by the real one. Anyway, No placeholder extents should escape + * the Cache interface boundary. + */ +class RetiredExtentPlaceholder : public CachedExtent { + +public: + RetiredExtentPlaceholder(extent_len_t length) + : CachedExtent(CachedExtent::retired_placeholder_t{}, length) {} + + CachedExtentRef duplicate_for_write(Transaction&) final { + ceph_assert(0 == "Should never happen for a placeholder"); + return CachedExtentRef(); + } + + ceph::bufferlist get_delta() final { + ceph_assert(0 == "Should never happen for a placeholder"); + return ceph::bufferlist(); + } + + static constexpr extent_types_t TYPE = extent_types_t::RETIRED_PLACEHOLDER; + extent_types_t get_type() const final { + return TYPE; + } + + void apply_delta_and_adjust_crc( + paddr_t base, const ceph::bufferlist &bl) final { + ceph_assert(0 == "Should never happen for a placeholder"); + } + + bool is_logical() const final { + return false; + } + + std::ostream &print_detail(std::ostream &out) const final { + return out << ", RetiredExtentPlaceholder"; + } + + void on_delta_write(paddr_t record_block_offset) final { + ceph_assert(0 == "Should never happen for a placeholder"); + } +}; + +class parent_tracker_t + : public boost::intrusive_ref_counter< + parent_tracker_t, boost::thread_unsafe_counter> { +public: + parent_tracker_t(CachedExtentRef parent) + : parent(parent) {} + parent_tracker_t(CachedExtent* parent) + : parent(parent) {} + ~parent_tracker_t(); + template <typename T = CachedExtent> + TCachedExtentRef<T> get_parent() const { + ceph_assert(parent); + if constexpr (std::is_same_v<T, CachedExtent>) { + return parent; + } else { + return parent->template cast<T>(); + } + } + void reset_parent(CachedExtentRef p) { + parent = p; + } + bool is_valid() const { + return parent && parent->is_valid(); + } +private: + CachedExtentRef parent; +}; + +std::ostream &operator<<(std::ostream &, const parent_tracker_t &); + +using parent_tracker_ref = boost::intrusive_ptr<parent_tracker_t>; + +class ChildableCachedExtent : public CachedExtent { +public: + template <typename... T> + ChildableCachedExtent(T&&... t) : CachedExtent(std::forward<T>(t)...) {} + bool has_parent_tracker() const { + return (bool)parent_tracker; + } + void reset_parent_tracker(parent_tracker_t *p = nullptr) { + parent_tracker.reset(p); + } + bool is_parent_valid() const { + return parent_tracker && parent_tracker->is_valid(); + } + template <typename T = CachedExtent> + TCachedExtentRef<T> get_parent_node() const { + assert(parent_tracker); + return parent_tracker->template get_parent<T>(); + } + void take_prior_parent_tracker() { + auto &prior = (ChildableCachedExtent&)(*get_prior_instance()); + parent_tracker = prior.parent_tracker; + } + std::ostream &print_detail(std::ostream &out) const final; +private: + parent_tracker_ref parent_tracker; + virtual std::ostream &_print_detail(std::ostream &out) const { + return out; + } +}; +/** + * LogicalCachedExtent + * + * CachedExtent with associated lba mapping. + * + * Users of TransactionManager should be using extents derived from + * LogicalCachedExtent. + */ +class LogicalCachedExtent : public ChildableCachedExtent { +public: + template <typename... T> + LogicalCachedExtent(T&&... t) + : ChildableCachedExtent(std::forward<T>(t)...) + {} + + bool has_laddr() const { + return laddr != L_ADDR_NULL; + } + + laddr_t get_laddr() const { + assert(laddr != L_ADDR_NULL); + return laddr; + } + + void set_laddr(laddr_t nladdr) { + laddr = nladdr; + } + + void maybe_set_intermediate_laddr(LBAMapping &mapping) { + laddr = mapping.is_indirect() + ? mapping.get_intermediate_base() + : mapping.get_key(); + } + + void apply_delta_and_adjust_crc( + paddr_t base, const ceph::bufferlist &bl) final { + apply_delta(bl); + set_last_committed_crc(get_crc32c()); + } + + bool is_logical() const final { + return true; + } + + std::ostream &_print_detail(std::ostream &out) const final; + + void on_replace_prior(Transaction &t) final; + + virtual ~LogicalCachedExtent(); +protected: + + virtual void apply_delta(const ceph::bufferlist &bl) = 0; + virtual std::ostream &print_detail_l(std::ostream &out) const { + return out; + } + + virtual void logical_on_delta_write() {} + + void on_delta_write(paddr_t record_block_offset) final { + assert(is_exist_mutation_pending() || + get_prior_instance()); + logical_on_delta_write(); + } + +private: + // the logical address of the extent, and if shared, + // it is the intermediate_base, see BtreeLBAMapping comments. + laddr_t laddr = L_ADDR_NULL; +}; + +using LogicalCachedExtentRef = TCachedExtentRef<LogicalCachedExtent>; +struct ref_laddr_cmp { + using is_transparent = laddr_t; + bool operator()(const LogicalCachedExtentRef &lhs, + const LogicalCachedExtentRef &rhs) const { + return lhs->get_laddr() < rhs->get_laddr(); + } + bool operator()(const laddr_t &lhs, + const LogicalCachedExtentRef &rhs) const { + return lhs < rhs->get_laddr(); + } + bool operator()(const LogicalCachedExtentRef &lhs, + const laddr_t &rhs) const { + return lhs->get_laddr() < rhs; + } +}; + +template <typename T> +read_set_item_t<T>::read_set_item_t(T *t, CachedExtentRef ref) + : t(t), ref(ref) +{} + +template <typename T> +inline bool read_set_item_t<T>::cmp_t::operator()( + const read_set_item_t<T> &lhs, const read_set_item_t<T> &rhs) const { + return lhs.ref->poffset < rhs.ref->poffset; +} +template <typename T> +inline bool read_set_item_t<T>::cmp_t::operator()( + const paddr_t &lhs, const read_set_item_t<T> &rhs) const { + return lhs < rhs.ref->poffset; +} +template <typename T> +inline bool read_set_item_t<T>::cmp_t::operator()( + const read_set_item_t<T> &lhs, const paddr_t &rhs) const { + return lhs.ref->poffset < rhs; +} + +using lextent_set_t = addr_extent_set_base_t< + laddr_t, + LogicalCachedExtentRef, + ref_laddr_cmp + >; + +template <typename T> +using lextent_list_t = addr_extent_list_base_t< + laddr_t, TCachedExtentRef<T>>; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::lba_pin_list_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::CachedExtent> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::LogicalCachedExtent> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::LBAMapping> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/collection_manager.cc b/src/crimson/os/seastore/collection_manager.cc new file mode 100644 index 000000000..4f5b58d01 --- /dev/null +++ b/src/crimson/os/seastore/collection_manager.cc @@ -0,0 +1,14 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab +// +#include "crimson/os/seastore/collection_manager.h" +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/collection_manager/flat_collection_manager.h" + +namespace crimson::os::seastore::collection_manager { + +CollectionManagerRef create_coll_manager(TransactionManager &trans_manager) { + return CollectionManagerRef(new FlatCollectionManager(trans_manager)); +} + +} diff --git a/src/crimson/os/seastore/collection_manager.h b/src/crimson/os/seastore/collection_manager.h new file mode 100644 index 000000000..37913abb4 --- /dev/null +++ b/src/crimson/os/seastore/collection_manager.h @@ -0,0 +1,84 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <seastar/core/future.hh> + +#include "osd/osd_types.h" + +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/transaction_manager.h" + +namespace crimson::os::seastore { + +struct coll_info_t { + unsigned split_bits; + + coll_info_t(unsigned bits) + : split_bits(bits) {} + + bool operator==(const coll_info_t &rhs) const { + return split_bits == rhs.split_bits; + } +}; + +/// Interface for maintaining set of collections +class CollectionManager { +public: + using base_iertr = TransactionManager::read_extent_iertr; + + /// Initialize collection manager instance for an empty store + using mkfs_iertr = TransactionManager::alloc_extent_iertr; + using mkfs_ret = mkfs_iertr::future<coll_root_t>; + virtual mkfs_ret mkfs( + Transaction &t) = 0; + + /// Create collection + using create_iertr = base_iertr; + using create_ret = create_iertr::future<>; + virtual create_ret create( + coll_root_t &root, + Transaction &t, + coll_t cid, + coll_info_t info + ) = 0; + + /// List collections with info + using list_iertr = base_iertr; + using list_ret_bare = std::vector<std::pair<coll_t, coll_info_t>>; + using list_ret = list_iertr::future<list_ret_bare>; + virtual list_ret list( + const coll_root_t &root, + Transaction &t) = 0; + + /// Remove cid + using remove_iertr = base_iertr; + using remove_ret = remove_iertr::future<>; + virtual remove_ret remove( + const coll_root_t &coll_root, + Transaction &t, + coll_t cid) = 0; + + /// Update info for cid + using update_iertr = base_iertr; + using update_ret = base_iertr::future<>; + virtual update_ret update( + const coll_root_t &coll_root, + Transaction &t, + coll_t cid, + coll_info_t info + ) = 0; + + virtual ~CollectionManager() {} +}; +using CollectionManagerRef = std::unique_ptr<CollectionManager>; + +namespace collection_manager { +/* creat CollectionMapManager for Collection */ +CollectionManagerRef create_coll_manager( + TransactionManager &trans_manager); + +} + +} diff --git a/src/crimson/os/seastore/collection_manager/collection_flat_node.cc b/src/crimson/os/seastore/collection_manager/collection_flat_node.cc new file mode 100644 index 000000000..ed17e2b12 --- /dev/null +++ b/src/crimson/os/seastore/collection_manager/collection_flat_node.cc @@ -0,0 +1,120 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "include/buffer.h" +#include "osd/osd_types.h" +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/collection_manager/collection_flat_node.h" + +namespace { + seastar::logger& logger() { + return crimson::get_logger(ceph_subsys_seastore); + } +} + +namespace crimson::os::seastore::collection_manager { + +void delta_t::replay(coll_map_t &l) const +{ + switch (op) { + case op_t::INSERT: { + l.insert(coll, bits); + break; + } + case op_t::UPDATE: { + l.update(coll, bits); + break; + } + case op_t::REMOVE: { + l.erase(coll); + break; + } + case op_t::INVALID: { + assert(0 == "impossible"); + break; + } + __builtin_unreachable(); + } +} + + +std::ostream &CollectionNode::print_detail_l(std::ostream &out) const +{ + return out; +} + +CollectionNode::list_ret +CollectionNode::list() +{ + logger().debug("CollectionNode:{}, {}", __func__, *this); + CollectionManager::list_ret_bare list_result; + for (auto &[coll, bits] : decoded) { + list_result.emplace_back(coll, bits); + } + return list_ret( + interruptible::ready_future_marker{}, + std::move(list_result)); +} + +CollectionNode::create_ret +CollectionNode::create(coll_context_t cc, coll_t coll, unsigned bits) +{ + logger().debug("CollectionNode:{}", __func__); + if (!is_mutable()) { + auto mut = cc.tm.get_mutable_extent(cc.t, this)->cast<CollectionNode>(); + return mut->create(cc, coll, bits); + } + logger().debug("CollectionNode::create {} {} {}", coll, bits, *this); + auto [iter, inserted] = decoded.insert(coll, bits); + assert(inserted); + if (encoded_sizeof((base_coll_map_t&)decoded) > get_bptr().length()) { + decoded.erase(iter); + return create_ret( + interruptible::ready_future_marker{}, + create_result_t::OVERFLOW); + } else { + if (auto buffer = maybe_get_delta_buffer(); buffer) { + buffer->insert(coll, bits); + } + copy_to_node(); + return create_ret( + interruptible::ready_future_marker{}, + create_result_t::SUCCESS); + } +} + +CollectionNode::update_ret +CollectionNode::update(coll_context_t cc, coll_t coll, unsigned bits) +{ + logger().debug("trans.{} CollectionNode:{} {} {}", + cc.t.get_trans_id(), __func__, coll, bits); + if (!is_mutable()) { + auto mut = cc.tm.get_mutable_extent(cc.t, this)->cast<CollectionNode>(); + return mut->update(cc, coll, bits); + } + if (auto buffer = maybe_get_delta_buffer(); buffer) { + buffer->update(coll, bits); + } + decoded.update(coll, bits); + copy_to_node(); + return seastar::now(); +} + +CollectionNode::remove_ret +CollectionNode::remove(coll_context_t cc, coll_t coll) +{ + logger().debug("trans.{} CollectionNode:{} {}", + cc.t.get_trans_id(),__func__, coll); + if (!is_mutable()) { + auto mut = cc.tm.get_mutable_extent(cc.t, this)->cast<CollectionNode>(); + return mut->remove(cc, coll); + } + if (auto buffer = maybe_get_delta_buffer(); buffer) { + buffer->remove(coll); + } + decoded.remove(coll); + copy_to_node(); + return seastar::now(); +} + +} diff --git a/src/crimson/os/seastore/collection_manager/collection_flat_node.h b/src/crimson/os/seastore/collection_manager/collection_flat_node.h new file mode 100644 index 000000000..2690fb5fd --- /dev/null +++ b/src/crimson/os/seastore/collection_manager/collection_flat_node.h @@ -0,0 +1,184 @@ +// -*- 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/seastore_types.h" +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/collection_manager.h" + +namespace crimson::os::seastore::collection_manager { +struct coll_context_t { + TransactionManager &tm; + Transaction &t; +}; + +using base_coll_map_t = std::map<denc_coll_t, uint32_t>; +struct coll_map_t : base_coll_map_t { + auto insert(coll_t coll, unsigned bits) { + return emplace( + std::make_pair(denc_coll_t{coll}, bits) + ); + } + + void update(coll_t coll, unsigned bits) { + (*this)[denc_coll_t{coll}] = bits; + } + + void remove(coll_t coll) { + erase(denc_coll_t{coll}); + } +}; + +struct delta_t { + enum class op_t : uint_fast8_t { + INSERT, + UPDATE, + REMOVE, + INVALID + } op = op_t::INVALID; + + denc_coll_t coll; + uint32_t bits = 0; + + DENC(delta_t, v, p) { + DENC_START(1, 1, p); + denc(v.op, p); + denc(v.coll, p); + denc(v.bits, p); + DENC_FINISH(p); + } + + void replay(coll_map_t &l) const; +}; +} +WRITE_CLASS_DENC(crimson::os::seastore::collection_manager::delta_t) + +namespace crimson::os::seastore::collection_manager { +class delta_buffer_t { + std::vector<delta_t> buffer; +public: + bool empty() const { + return buffer.empty(); + } + + void insert(coll_t coll, uint32_t bits) { + buffer.push_back(delta_t{delta_t::op_t::INSERT, denc_coll_t(coll), bits}); + } + void update(coll_t coll, uint32_t bits) { + buffer.push_back(delta_t{delta_t::op_t::UPDATE, denc_coll_t(coll), bits}); + } + void remove(coll_t coll) { + buffer.push_back(delta_t{delta_t::op_t::REMOVE, denc_coll_t(coll), 0}); + } + void replay(coll_map_t &l) { + for (auto &i: buffer) { + i.replay(l); + } + } + + void clear() { buffer.clear(); } + + DENC(delta_buffer_t, v, p) { + DENC_START(1, 1, p); + denc(v.buffer, p); + DENC_FINISH(p); + } +}; +} +WRITE_CLASS_DENC(crimson::os::seastore::collection_manager::delta_buffer_t) + +namespace crimson::os::seastore::collection_manager { + +struct CollectionNode + : LogicalCachedExtent { + using CollectionNodeRef = TCachedExtentRef<CollectionNode>; + + explicit CollectionNode(ceph::bufferptr &&ptr) + : LogicalCachedExtent(std::move(ptr)) {} + explicit CollectionNode(const CollectionNode &other) + : LogicalCachedExtent(other), + decoded(other.decoded) {} + + static constexpr extent_types_t type = extent_types_t::COLL_BLOCK; + + coll_map_t decoded; + delta_buffer_t delta_buffer; + + CachedExtentRef duplicate_for_write(Transaction&) final { + assert(delta_buffer.empty()); + return CachedExtentRef(new CollectionNode(*this)); + } + delta_buffer_t *maybe_get_delta_buffer() { + return is_mutation_pending() ? &delta_buffer : nullptr; + } + + using list_iertr = CollectionManager::list_iertr; + using list_ret = CollectionManager::list_ret; + list_ret list(); + + + enum class create_result_t : uint8_t { + SUCCESS, + OVERFLOW + }; + using create_iertr = CollectionManager::create_iertr; + using create_ret = create_iertr::future<create_result_t>; + create_ret create(coll_context_t cc, coll_t coll, unsigned bits); + + using remove_iertr = CollectionManager::remove_iertr; + using remove_ret = CollectionManager::remove_ret; + remove_ret remove(coll_context_t cc, coll_t coll); + + using update_iertr = CollectionManager::update_iertr; + using update_ret = CollectionManager::update_ret; + update_ret update(coll_context_t cc, coll_t coll, unsigned bits); + + void on_clean_read() final { + bufferlist bl; + bl.append(get_bptr()); + auto iter = bl.cbegin(); + decode((base_coll_map_t&)decoded, iter); + } + + void copy_to_node() { + bufferlist bl; + encode((base_coll_map_t&)decoded, bl); + auto iter = bl.begin(); + auto size = encoded_sizeof((base_coll_map_t&)decoded); + assert(size <= get_bptr().length()); + get_bptr().zero(); + iter.copy(size, get_bptr().c_str()); + + } + + ceph::bufferlist get_delta() final { + assert(!delta_buffer.empty()); + ceph::bufferlist bl; + encode(delta_buffer, bl); + delta_buffer.clear(); + return bl; + } + + void apply_delta(const ceph::bufferlist &bl) final { + assert(bl.length()); + delta_buffer_t buffer; + auto bptr = bl.begin(); + decode(buffer, bptr); + buffer.replay(decoded); + copy_to_node(); + } + + static constexpr extent_types_t TYPE = extent_types_t::COLL_BLOCK; + extent_types_t get_type() const final { + return TYPE; + } + + std::ostream &print_detail_l(std::ostream &out) const final; +}; +using CollectionNodeRef = CollectionNode::CollectionNodeRef; +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::collection_manager::CollectionNode> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/collection_manager/flat_collection_manager.cc b/src/crimson/os/seastore/collection_manager/flat_collection_manager.cc new file mode 100644 index 000000000..decb095f6 --- /dev/null +++ b/src/crimson/os/seastore/collection_manager/flat_collection_manager.cc @@ -0,0 +1,133 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <string.h> + +#include "crimson/common/log.h" + +#include "include/buffer.h" +#include "include/stringify.h" +#include "crimson/os/seastore/collection_manager/flat_collection_manager.h" +#include "crimson/os/seastore/collection_manager/collection_flat_node.h" + +namespace { + seastar::logger& logger() { + return crimson::get_logger(ceph_subsys_seastore); + } +} + +namespace crimson::os::seastore::collection_manager { + +constexpr static extent_len_t MIN_FLAT_BLOCK_SIZE = 4<<10; +[[maybe_unused]] constexpr static extent_len_t MAX_FLAT_BLOCK_SIZE = 4<<20; + +FlatCollectionManager::FlatCollectionManager( + TransactionManager &tm) + : tm(tm) {} + +FlatCollectionManager::mkfs_ret +FlatCollectionManager::mkfs(Transaction &t) +{ + + logger().debug("FlatCollectionManager: {}", __func__); + return tm.alloc_extent<CollectionNode>( + t, L_ADDR_MIN, MIN_FLAT_BLOCK_SIZE + ).si_then([](auto&& root_extent) { + coll_root_t coll_root = coll_root_t( + root_extent->get_laddr(), + MIN_FLAT_BLOCK_SIZE + ); + return mkfs_iertr::make_ready_future<coll_root_t>(coll_root); + }); +} + +FlatCollectionManager::get_root_ret +FlatCollectionManager::get_coll_root(const coll_root_t &coll_root, Transaction &t) +{ + logger().debug("FlatCollectionManager: {}", __func__); + assert(coll_root.get_location() != L_ADDR_NULL); + auto cc = get_coll_context(t); + return cc.tm.read_extent<CollectionNode>( + cc.t, + coll_root.get_location(), + coll_root.get_size() + ).si_then([](auto&& e) { + return get_root_iertr::make_ready_future<CollectionNodeRef>(std::move(e)); + }); +} + +FlatCollectionManager::create_ret +FlatCollectionManager::create(coll_root_t &coll_root, Transaction &t, + coll_t cid, coll_info_t info) +{ + logger().debug("FlatCollectionManager: {}", __func__); + return get_coll_root(coll_root, t + ).si_then([=, this, &coll_root, &t] (auto &&extent) { + return extent->create( + get_coll_context(t), cid, info.split_bits + ).si_then([=, this, &coll_root, &t] (auto ret) { + switch (ret) { + case CollectionNode::create_result_t::OVERFLOW: { + logger().debug("FlatCollectionManager: {} overflow!", __func__); + auto new_size = coll_root.get_size() * 2; // double each time + + // TODO return error probably, but such a nonsensically large number of + // collections would create a ton of other problems as well + assert(new_size < MAX_FLAT_BLOCK_SIZE); + return tm.alloc_extent<CollectionNode>( + t, L_ADDR_MIN, new_size + ).si_then([=, this, &coll_root, &t] (auto &&root_extent) { + coll_root.update(root_extent->get_laddr(), root_extent->get_length()); + + root_extent->decoded = extent->decoded; + return root_extent->create( + get_coll_context(t), cid, info.split_bits + ).si_then([=, this, &t](auto result) { + assert(result == CollectionNode::create_result_t::SUCCESS); + return tm.dec_ref(t, extent->get_laddr()); + }).si_then([] (auto) { + return create_iertr::make_ready_future<>(); + }); + }); + } + case CollectionNode::create_result_t::SUCCESS: { + return create_iertr::make_ready_future<>(); + } + } + __builtin_unreachable(); + }); + }); +} + +FlatCollectionManager::list_ret +FlatCollectionManager::list(const coll_root_t &coll_root, Transaction &t) +{ + logger().debug("FlatCollectionManager: {}", __func__); + return get_coll_root(coll_root, t) + .si_then([] (auto extent) { + return extent->list(); + }); +} + +FlatCollectionManager::update_ret +FlatCollectionManager::update(const coll_root_t &coll_root, Transaction &t, + coll_t cid, coll_info_t info) +{ + logger().debug("FlatCollectionManager: {}", __func__); + return get_coll_root(coll_root, t) + .si_then([this, &t, cid, info] (auto extent) { + return extent->update(get_coll_context(t), cid, info.split_bits); + }); +} + +FlatCollectionManager::remove_ret +FlatCollectionManager::remove(const coll_root_t &coll_root, Transaction &t, + coll_t cid ) +{ + logger().debug("FlatCollectionManager: {}", __func__); + return get_coll_root(coll_root, t).si_then([this, &t, cid] (auto extent) { + return extent->remove(get_coll_context(t), cid); + }); +} + +} diff --git a/src/crimson/os/seastore/collection_manager/flat_collection_manager.h b/src/crimson/os/seastore/collection_manager/flat_collection_manager.h new file mode 100644 index 000000000..1321ec1d8 --- /dev/null +++ b/src/crimson/os/seastore/collection_manager/flat_collection_manager.h @@ -0,0 +1,41 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "include/ceph_assert.h" + +#include "crimson/os/seastore/collection_manager.h" +#include "crimson/os/seastore/collection_manager/collection_flat_node.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/transaction_manager.h" + +namespace crimson::os::seastore::collection_manager { + +class FlatCollectionManager : public CollectionManager { + TransactionManager &tm; + + coll_context_t get_coll_context(Transaction &t) { + return coll_context_t{tm, t}; + } + + using get_root_iertr = base_iertr; + using get_root_ret = get_root_iertr::future<CollectionNodeRef>; + get_root_ret get_coll_root(const coll_root_t &coll_root, Transaction &t); + +public: + explicit FlatCollectionManager(TransactionManager &tm); + + mkfs_ret mkfs(Transaction &t) final; + + create_ret create(coll_root_t &coll_root, Transaction &t, coll_t cid, + coll_info_t info) final; + + list_ret list(const coll_root_t &coll_root, Transaction &t) final; + + remove_ret remove(const coll_root_t &coll_root, Transaction &t, coll_t cid) final; + + update_ret update(const coll_root_t &coll_root, Transaction &t, coll_t cid, coll_info_t info) final; +}; +using FlatCollectionManagerRef = std::unique_ptr<FlatCollectionManager>; +} diff --git a/src/crimson/os/seastore/device.cc b/src/crimson/os/seastore/device.cc new file mode 100644 index 000000000..c3bda82a7 --- /dev/null +++ b/src/crimson/os/seastore/device.cc @@ -0,0 +1,51 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "device.h" + +#include "segment_manager.h" +#include "random_block_manager.h" +#include "random_block_manager/rbm_device.h" + +namespace crimson::os::seastore { + +std::ostream& operator<<(std::ostream& out, const device_spec_t& ds) +{ + return out << "device_spec(" + << "magic=" << ds.magic + << ", dtype=" << ds.dtype + << ", " << device_id_printer_t{ds.id} + << ")"; +} + +std::ostream& operator<<(std::ostream& out, const device_config_t& conf) +{ + out << "device_config_t(" + << "major_dev=" << conf.major_dev + << ", spec=" << conf.spec + << ", meta=" << conf.meta + << ", secondary("; + for (const auto& [k, v] : conf.secondary_devices) { + out << device_id_printer_t{k} + << ": " << v << ", "; + } + return out << "))"; +} + +seastar::future<DeviceRef> +Device::make_device(const std::string& device, device_type_t dtype) +{ + if (get_default_backend_of_device(dtype) == backend_type_t::SEGMENTED) { + return SegmentManager::get_segment_manager(device, dtype + ).then([](DeviceRef ret) { + return ret; + }); + } + assert(get_default_backend_of_device(dtype) == backend_type_t::RANDOM_BLOCK); + return get_rb_device(device + ).then([](DeviceRef ret) { + return ret; + }); +} + +} diff --git a/src/crimson/os/seastore/device.h b/src/crimson/os/seastore/device.h new file mode 100644 index 000000000..ceb1ede64 --- /dev/null +++ b/src/crimson/os/seastore/device.h @@ -0,0 +1,175 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <memory> + +#include "include/buffer_fwd.h" + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/seastore_types.h" + +namespace crimson::os::seastore { + +using magic_t = uint64_t; + +struct device_spec_t { + magic_t magic = 0; + device_type_t dtype = device_type_t::NONE; + device_id_t id = DEVICE_ID_NULL; + DENC(device_spec_t, v, p) { + DENC_START(1, 1, p); + denc(v.magic, p); + denc(v.dtype, p); + denc(v.id, p); + DENC_FINISH(p); + } +}; + +std::ostream& operator<<(std::ostream&, const device_spec_t&); + +using secondary_device_set_t = + std::map<device_id_t, device_spec_t>; + +struct device_config_t { + bool major_dev = false; + device_spec_t spec; + seastore_meta_t meta; + secondary_device_set_t secondary_devices; + DENC(device_config_t, v, p) { + DENC_START(1, 1, p); + denc(v.major_dev, p); + denc(v.spec, p); + denc(v.meta, p); + denc(v.secondary_devices, p); + DENC_FINISH(p); + } + static device_config_t create_primary( + uuid_d new_osd_fsid, + device_id_t id, + device_type_t d_type, + secondary_device_set_t sds) { + return device_config_t{ + true, + device_spec_t{ + (magic_t)std::rand(), + d_type, + id}, + seastore_meta_t{new_osd_fsid}, + sds}; + } + static device_config_t create_secondary( + uuid_d new_osd_fsid, + device_id_t id, + device_type_t d_type, + magic_t magic) { + return device_config_t{ + false, + device_spec_t{ + magic, + d_type, + id}, + seastore_meta_t{new_osd_fsid}, + secondary_device_set_t()}; + } +}; + +std::ostream& operator<<(std::ostream&, const device_config_t&); + +class Device; +using DeviceRef = std::unique_ptr<Device>; + +/** + * Device + * + * Represents a general device regardless of the underlying medium. + */ +class Device { +// interfaces used by device +public: + virtual ~Device() {} + + virtual seastar::future<> start() { + return seastar::now(); + } + + virtual seastar::future<> stop() { + return seastar::now(); + } + // called on the shard to get this shard device; + virtual Device& get_sharded_device() { + return *this; + } + + using access_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::permission_denied, + crimson::ct_error::enoent>; + + using mkfs_ertr = access_ertr; + using mkfs_ret = mkfs_ertr::future<>; + virtual mkfs_ret mkfs(device_config_t) = 0; + + using mount_ertr = access_ertr; + using mount_ret = access_ertr::future<>; + virtual mount_ret mount() = 0; + + static seastar::future<DeviceRef> make_device( + const std::string &device, + device_type_t dtype); + +// interfaces used by each device shard +public: + virtual device_id_t get_device_id() const = 0; + + virtual magic_t get_magic() const = 0; + + virtual device_type_t get_device_type() const = 0; + + virtual backend_type_t get_backend_type() const = 0; + + virtual const seastore_meta_t &get_meta() const = 0; + + virtual extent_len_t get_block_size() const = 0; + + virtual std::size_t get_available_size() const = 0; + + virtual secondary_device_set_t& get_secondary_devices() = 0; + + using close_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + virtual close_ertr::future<> close() = 0; + + using read_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent, + crimson::ct_error::erange>; + virtual read_ertr::future<> read( + paddr_t addr, + size_t len, + ceph::bufferptr &out) = 0; + + read_ertr::future<ceph::bufferptr> read( + paddr_t addr, + size_t len + ) { + auto ptrref = std::make_unique<ceph::bufferptr>( + buffer::create_page_aligned(len)); + return read(addr, len, *ptrref + ).safe_then([ptrref=std::move(ptrref)]() mutable { + return read_ertr::make_ready_future<bufferptr>(std::move(*ptrref)); + }); + } +}; + +} + +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::device_spec_t) +WRITE_CLASS_DENC(crimson::os::seastore::device_config_t) + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::device_config_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::device_spec_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/extent_placement_manager.cc b/src/crimson/os/seastore/extent_placement_manager.cc new file mode 100644 index 000000000..b7aabefc6 --- /dev/null +++ b/src/crimson/os/seastore/extent_placement_manager.cc @@ -0,0 +1,808 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#include "crimson/os/seastore/extent_placement_manager.h" + +#include "crimson/common/config_proxy.h" +#include "crimson/os/seastore/logging.h" + +SET_SUBSYS(seastore_epm); + +namespace crimson::os::seastore { + +SegmentedOolWriter::SegmentedOolWriter( + data_category_t category, + rewrite_gen_t gen, + SegmentProvider& sp, + SegmentSeqAllocator &ssa) + : segment_allocator(nullptr, category, gen, sp, ssa), + record_submitter(crimson::common::get_conf<uint64_t>( + "seastore_journal_iodepth_limit"), + crimson::common::get_conf<uint64_t>( + "seastore_journal_batch_capacity"), + crimson::common::get_conf<Option::size_t>( + "seastore_journal_batch_flush_size"), + crimson::common::get_conf<double>( + "seastore_journal_batch_preferred_fullness"), + segment_allocator) +{ +} + +SegmentedOolWriter::alloc_write_ertr::future<> +SegmentedOolWriter::write_record( + Transaction& t, + record_t&& record, + std::list<LogicalCachedExtentRef>&& extents, + bool with_atomic_roll_segment) +{ + LOG_PREFIX(SegmentedOolWriter::write_record); + assert(extents.size()); + assert(extents.size() == record.extents.size()); + assert(!record.deltas.size()); + + // account transactional ool writes before write() + auto& stats = t.get_ool_write_stats(); + stats.extents.num += extents.size(); + stats.extents.bytes += record.size.dlength; + stats.md_bytes += record.size.get_raw_mdlength(); + stats.num_records += 1; + + return record_submitter.submit( + std::move(record), + with_atomic_roll_segment + ).safe_then([this, FNAME, &t, extents=std::move(extents) + ](record_locator_t ret) mutable { + DEBUGT("{} finish with {} and {} extents", + t, segment_allocator.get_name(), + ret, extents.size()); + paddr_t extent_addr = ret.record_block_base; + for (auto& extent : extents) { + TRACET("{} ool extent written at {} -- {}", + t, segment_allocator.get_name(), + extent_addr, *extent); + t.update_delayed_ool_extent_addr(extent, extent_addr); + extent_addr = extent_addr.as_seg_paddr().add_offset( + extent->get_length()); + } + }); +} + +SegmentedOolWriter::alloc_write_iertr::future<> +SegmentedOolWriter::do_write( + Transaction& t, + std::list<LogicalCachedExtentRef>& extents) +{ + LOG_PREFIX(SegmentedOolWriter::do_write); + assert(!extents.empty()); + if (!record_submitter.is_available()) { + DEBUGT("{} extents={} wait ...", + t, segment_allocator.get_name(), + extents.size()); + return trans_intr::make_interruptible( + record_submitter.wait_available() + ).si_then([this, &t, &extents] { + return do_write(t, extents); + }); + } + record_t record(TRANSACTION_TYPE_NULL); + std::list<LogicalCachedExtentRef> pending_extents; + auto commit_time = seastar::lowres_system_clock::now(); + + for (auto it = extents.begin(); it != extents.end();) { + auto& extent = *it; + record_size_t wouldbe_rsize = record.size; + wouldbe_rsize.account_extent(extent->get_bptr().length()); + using action_t = journal::RecordSubmitter::action_t; + action_t action = record_submitter.check_action(wouldbe_rsize); + if (action == action_t::ROLL) { + auto num_extents = pending_extents.size(); + DEBUGT("{} extents={} submit {} extents and roll, unavailable ...", + t, segment_allocator.get_name(), + extents.size(), num_extents); + auto fut_write = alloc_write_ertr::now(); + if (num_extents > 0) { + assert(record_submitter.check_action(record.size) != + action_t::ROLL); + fut_write = write_record( + t, std::move(record), std::move(pending_extents), + true/* with_atomic_roll_segment */); + } + return trans_intr::make_interruptible( + record_submitter.roll_segment( + ).safe_then([fut_write=std::move(fut_write)]() mutable { + return std::move(fut_write); + }) + ).si_then([this, &t, &extents] { + return do_write(t, extents); + }); + } + + TRACET("{} extents={} add extent to record -- {}", + t, segment_allocator.get_name(), + extents.size(), *extent); + ceph::bufferlist bl; + extent->prepare_write(); + bl.append(extent->get_bptr()); + assert(bl.length() == extent->get_length()); + auto modify_time = extent->get_modify_time(); + if (modify_time == NULL_TIME) { + modify_time = commit_time; + } + record.push_back( + extent_t{ + extent->get_type(), + extent->get_laddr(), + std::move(bl)}, + modify_time); + pending_extents.push_back(extent); + it = extents.erase(it); + + assert(record_submitter.check_action(record.size) == action); + if (action == action_t::SUBMIT_FULL) { + DEBUGT("{} extents={} submit {} extents ...", + t, segment_allocator.get_name(), + extents.size(), pending_extents.size()); + return trans_intr::make_interruptible( + write_record(t, std::move(record), std::move(pending_extents)) + ).si_then([this, &t, &extents] { + if (!extents.empty()) { + return do_write(t, extents); + } else { + return alloc_write_iertr::now(); + } + }); + } + // SUBMIT_NOT_FULL: evaluate the next extent + } + + auto num_extents = pending_extents.size(); + DEBUGT("{} submit the rest {} extents ...", + t, segment_allocator.get_name(), + num_extents); + assert(num_extents > 0); + return trans_intr::make_interruptible( + write_record(t, std::move(record), std::move(pending_extents))); +} + +SegmentedOolWriter::alloc_write_iertr::future<> +SegmentedOolWriter::alloc_write_ool_extents( + Transaction& t, + std::list<LogicalCachedExtentRef>& extents) +{ + if (extents.empty()) { + return alloc_write_iertr::now(); + } + return seastar::with_gate(write_guard, [this, &t, &extents] { + return do_write(t, extents); + }); +} + +void ExtentPlacementManager::init( + JournalTrimmerImplRef &&trimmer, + AsyncCleanerRef &&cleaner, + AsyncCleanerRef &&cold_cleaner) +{ + writer_refs.clear(); + auto cold_segment_cleaner = dynamic_cast<SegmentCleaner*>(cold_cleaner.get()); + dynamic_max_rewrite_generation = MIN_COLD_GENERATION - 1; + if (cold_segment_cleaner) { + dynamic_max_rewrite_generation = MAX_REWRITE_GENERATION; + } + + if (trimmer->get_journal_type() == journal_type_t::SEGMENTED) { + auto segment_cleaner = dynamic_cast<SegmentCleaner*>(cleaner.get()); + ceph_assert(segment_cleaner != nullptr); + auto num_writers = generation_to_writer(dynamic_max_rewrite_generation + 1); + + data_writers_by_gen.resize(num_writers, {}); + for (rewrite_gen_t gen = OOL_GENERATION; gen < MIN_COLD_GENERATION; ++gen) { + writer_refs.emplace_back(std::make_unique<SegmentedOolWriter>( + data_category_t::DATA, gen, *segment_cleaner, + *ool_segment_seq_allocator)); + data_writers_by_gen[generation_to_writer(gen)] = writer_refs.back().get(); + } + + md_writers_by_gen.resize(num_writers, {}); + for (rewrite_gen_t gen = OOL_GENERATION; gen < MIN_COLD_GENERATION; ++gen) { + writer_refs.emplace_back(std::make_unique<SegmentedOolWriter>( + data_category_t::METADATA, gen, *segment_cleaner, + *ool_segment_seq_allocator)); + md_writers_by_gen[generation_to_writer(gen)] = writer_refs.back().get(); + } + + for (auto *device : segment_cleaner->get_segment_manager_group() + ->get_segment_managers()) { + add_device(device); + } + } else { + assert(trimmer->get_journal_type() == journal_type_t::RANDOM_BLOCK); + auto rb_cleaner = dynamic_cast<RBMCleaner*>(cleaner.get()); + ceph_assert(rb_cleaner != nullptr); + auto num_writers = generation_to_writer(dynamic_max_rewrite_generation + 1); + data_writers_by_gen.resize(num_writers, {}); + md_writers_by_gen.resize(num_writers, {}); + writer_refs.emplace_back(std::make_unique<RandomBlockOolWriter>( + rb_cleaner)); + // TODO: implement eviction in RBCleaner and introduce further writers + data_writers_by_gen[generation_to_writer(OOL_GENERATION)] = writer_refs.back().get(); + md_writers_by_gen[generation_to_writer(OOL_GENERATION)] = writer_refs.back().get(); + for (auto *rb : rb_cleaner->get_rb_group()->get_rb_managers()) { + add_device(rb->get_device()); + } + } + + if (cold_segment_cleaner) { + for (rewrite_gen_t gen = MIN_COLD_GENERATION; gen < REWRITE_GENERATIONS; ++gen) { + writer_refs.emplace_back(std::make_unique<SegmentedOolWriter>( + data_category_t::DATA, gen, *cold_segment_cleaner, + *ool_segment_seq_allocator)); + data_writers_by_gen[generation_to_writer(gen)] = writer_refs.back().get(); + } + for (rewrite_gen_t gen = MIN_COLD_GENERATION; gen < REWRITE_GENERATIONS; ++gen) { + writer_refs.emplace_back(std::make_unique<SegmentedOolWriter>( + data_category_t::METADATA, gen, *cold_segment_cleaner, + *ool_segment_seq_allocator)); + md_writers_by_gen[generation_to_writer(gen)] = writer_refs.back().get(); + } + for (auto *device : cold_segment_cleaner->get_segment_manager_group() + ->get_segment_managers()) { + add_device(device); + } + } + + background_process.init(std::move(trimmer), + std::move(cleaner), + std::move(cold_cleaner)); + if (cold_segment_cleaner) { + ceph_assert(get_main_backend_type() == backend_type_t::SEGMENTED); + ceph_assert(background_process.has_cold_tier()); + } else { + ceph_assert(!background_process.has_cold_tier()); + } +} + +void ExtentPlacementManager::set_primary_device(Device *device) +{ + ceph_assert(primary_device == nullptr); + primary_device = device; + ceph_assert(devices_by_id[device->get_device_id()] == device); +} + +ExtentPlacementManager::open_ertr::future<> +ExtentPlacementManager::open_for_write() +{ + LOG_PREFIX(ExtentPlacementManager::open_for_write); + INFO("started with {} devices", num_devices); + ceph_assert(primary_device != nullptr); + return crimson::do_for_each(data_writers_by_gen, [](auto &writer) { + if (writer) { + return writer->open(); + } + return open_ertr::now(); + }).safe_then([this] { + return crimson::do_for_each(md_writers_by_gen, [](auto &writer) { + if (writer) { + return writer->open(); + } + return open_ertr::now(); + }); + }); +} + +ExtentPlacementManager::dispatch_result_t +ExtentPlacementManager::dispatch_delayed_extents(Transaction &t) +{ + dispatch_result_t res; + res.delayed_extents = t.get_delayed_alloc_list(); + + // init projected usage + for (auto &extent : t.get_inline_block_list()) { + if (extent->is_valid()) { + res.usage.inline_usage += extent->get_length(); + res.usage.cleaner_usage.main_usage += extent->get_length(); + } + } + + for (auto &extent : res.delayed_extents) { + if (dispatch_delayed_extent(extent)) { + res.usage.inline_usage += extent->get_length(); + res.usage.cleaner_usage.main_usage += extent->get_length(); + t.mark_delayed_extent_inline(extent); + } else { + if (extent->get_rewrite_generation() < MIN_COLD_GENERATION) { + res.usage.cleaner_usage.main_usage += extent->get_length(); + } else { + assert(background_process.has_cold_tier()); + res.usage.cleaner_usage.cold_ool_usage += extent->get_length(); + } + t.mark_delayed_extent_ool(extent); + auto writer_ptr = get_writer( + extent->get_user_hint(), + get_extent_category(extent->get_type()), + extent->get_rewrite_generation()); + res.alloc_map[writer_ptr].emplace_back(extent); + } + } + return res; +} + +ExtentPlacementManager::alloc_paddr_iertr::future<> +ExtentPlacementManager::write_delayed_ool_extents( + Transaction& t, + extents_by_writer_t& alloc_map) { + return trans_intr::do_for_each(alloc_map, [&t](auto& p) { + auto writer = p.first; + auto& extents = p.second; + return writer->alloc_write_ool_extents(t, extents); + }); +} + +ExtentPlacementManager::alloc_paddr_iertr::future<> +ExtentPlacementManager::write_preallocated_ool_extents( + Transaction &t, + std::list<LogicalCachedExtentRef> extents) +{ + LOG_PREFIX(ExtentPlacementManager::write_preallocated_ool_extents); + DEBUGT("start with {} allocated extents", + t, extents.size()); + assert(writer_refs.size()); + return seastar::do_with( + std::map<ExtentOolWriter*, std::list<LogicalCachedExtentRef>>(), + [this, &t, extents=std::move(extents)](auto& alloc_map) { + for (auto& extent : extents) { + auto writer_ptr = get_writer( + extent->get_user_hint(), + get_extent_category(extent->get_type()), + extent->get_rewrite_generation()); + alloc_map[writer_ptr].emplace_back(extent); + } + return trans_intr::do_for_each(alloc_map, [&t](auto& p) { + auto writer = p.first; + auto& extents = p.second; + return writer->alloc_write_ool_extents(t, extents); + }); + }); +} + +ExtentPlacementManager::close_ertr::future<> +ExtentPlacementManager::close() +{ + LOG_PREFIX(ExtentPlacementManager::close); + INFO("started"); + return crimson::do_for_each(data_writers_by_gen, [](auto &writer) { + if (writer) { + return writer->close(); + } + return close_ertr::now(); + }).safe_then([this] { + return crimson::do_for_each(md_writers_by_gen, [](auto &writer) { + if (writer) { + return writer->close(); + } + return close_ertr::now(); + }); + }); +} + +void ExtentPlacementManager::BackgroundProcess::log_state(const char *caller) const +{ + LOG_PREFIX(BackgroundProcess::log_state); + DEBUG("caller {}, {}, {}", + caller, + JournalTrimmerImpl::stat_printer_t{*trimmer, true}, + AsyncCleaner::stat_printer_t{*main_cleaner, true}); + if (has_cold_tier()) { + DEBUG("caller {}, cold_cleaner: {}", + caller, + AsyncCleaner::stat_printer_t{*cold_cleaner, true}); + } +} + +void ExtentPlacementManager::BackgroundProcess::start_background() +{ + LOG_PREFIX(BackgroundProcess::start_background); + INFO("{}, {}", + JournalTrimmerImpl::stat_printer_t{*trimmer, true}, + AsyncCleaner::stat_printer_t{*main_cleaner, true}); + if (has_cold_tier()) { + INFO("cold_cleaner: {}", + AsyncCleaner::stat_printer_t{*cold_cleaner, true}); + } + ceph_assert(trimmer->check_is_ready()); + ceph_assert(state == state_t::SCAN_SPACE); + assert(!is_running()); + process_join = seastar::now(); + state = state_t::RUNNING; + assert(is_running()); + process_join = run(); +} + +seastar::future<> +ExtentPlacementManager::BackgroundProcess::stop_background() +{ + return seastar::futurize_invoke([this] { + if (!is_running()) { + if (state != state_t::HALT) { + state = state_t::STOP; + } + return seastar::now(); + } + auto ret = std::move(*process_join); + process_join.reset(); + state = state_t::HALT; + assert(!is_running()); + do_wake_background(); + return ret; + }).then([this] { + LOG_PREFIX(BackgroundProcess::stop_background); + INFO("done, {}, {}", + JournalTrimmerImpl::stat_printer_t{*trimmer, true}, + AsyncCleaner::stat_printer_t{*main_cleaner, true}); + if (has_cold_tier()) { + INFO("done, cold_cleaner: {}", + AsyncCleaner::stat_printer_t{*cold_cleaner, true}); + } + // run_until_halt() can be called at HALT + }); +} + +seastar::future<> +ExtentPlacementManager::BackgroundProcess::run_until_halt() +{ + ceph_assert(state == state_t::HALT); + assert(!is_running()); + if (is_running_until_halt) { + return seastar::now(); + } + is_running_until_halt = true; + return seastar::do_until( + [this] { + log_state("run_until_halt"); + assert(is_running_until_halt); + if (background_should_run()) { + return false; + } else { + is_running_until_halt = false; + return true; + } + }, + [this] { + return do_background_cycle(); + } + ); +} + +seastar::future<> +ExtentPlacementManager::BackgroundProcess::reserve_projected_usage( + io_usage_t usage) +{ + if (!is_ready()) { + return seastar::now(); + } + ceph_assert(!blocking_io); + // The pipeline configuration prevents another IO from entering + // prepare until the prior one exits and clears this. + ++stats.io_count; + + auto res = try_reserve_io(usage); + if (res.is_successful()) { + return seastar::now(); + } else { + abort_io_usage(usage, res); + if (!res.reserve_inline_success) { + ++stats.io_blocked_count_trim; + } + if (!res.cleaner_result.is_successful()) { + ++stats.io_blocked_count_clean; + } + ++stats.io_blocking_num; + ++stats.io_blocked_count; + stats.io_blocked_sum += stats.io_blocking_num; + + return seastar::repeat([this, usage] { + blocking_io = seastar::promise<>(); + return blocking_io->get_future( + ).then([this, usage] { + ceph_assert(!blocking_io); + auto res = try_reserve_io(usage); + if (res.is_successful()) { + assert(stats.io_blocking_num == 1); + --stats.io_blocking_num; + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } else { + abort_io_usage(usage, res); + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::no); + } + }); + }); + } +} + +seastar::future<> +ExtentPlacementManager::BackgroundProcess::run() +{ + assert(is_running()); + return seastar::repeat([this] { + if (!is_running()) { + log_state("run(exit)"); + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + return seastar::futurize_invoke([this] { + if (background_should_run()) { + log_state("run(background)"); + return do_background_cycle(); + } else { + log_state("run(block)"); + ceph_assert(!blocking_background); + blocking_background = seastar::promise<>(); + return blocking_background->get_future(); + } + }).then([] { + return seastar::stop_iteration::no; + }); + }); +} + +/** + * Reservation Process + * + * Most of transctions need to reserve its space usage before performing the + * ool writes and committing transactions. If the space reservation is + * unsuccessful, the current transaction is blocked, and waits for new + * background transactions to finish. + * + * The following are the reservation requirements for each transaction type: + * 1. MUTATE transaction: + * (1) inline usage on the trimmer, + * (2) inline usage with OOL usage on the main cleaner, + * (3) cold OOL usage to the cold cleaner(if it exists). + * 2. TRIM_DIRTY/TRIM_ALLOC transaction: + * (1) all extents usage on the main cleaner, + * (2) usage on the cold cleaner(if it exists) + * 3. CLEANER_MAIN: + * (1) cleaned extents size on the cold cleaner(if it exists). + * 4. CLEANER_COLD transction does not require space reservation. + * + * The reserve implementation should satisfy the following conditions: + * 1. The reservation should be atomic. If a reservation involves several reservations, + * such as the MUTATE transaction that needs to reserve space on both the trimmer + * and cleaner at the same time, the successful condition is that all of its + * sub-reservations succeed. If one or more operations fail, the entire reservation + * fails, and the successful operation should be reverted. + * 2. The reserve/block relationship should form a DAG to avoid deadlock. For example, + * TRIM_ALLOC transaction might be blocked by cleaner due to the failure of reserving + * on the cleaner. In such cases, the cleaner must not reserve space on the trimmer + * since the trimmer is already blocked by itself. + * + * Finally the reserve relationship can be represented as follows: + * + * +-------------------------+----------------+ + * | | | + * | v v + * MUTATE ---> TRIM_* ---> CLEANER_MAIN ---> CLEANER_COLD + * | ^ + * | | + * +--------------------------------+ + */ +bool ExtentPlacementManager::BackgroundProcess::try_reserve_cold(std::size_t usage) +{ + if (has_cold_tier()) { + return cold_cleaner->try_reserve_projected_usage(usage); + } else { + assert(usage == 0); + return true; + } +} +void ExtentPlacementManager::BackgroundProcess::abort_cold_usage( + std::size_t usage, bool success) +{ + if (has_cold_tier() && success) { + cold_cleaner->release_projected_usage(usage); + } +} + +reserve_cleaner_result_t +ExtentPlacementManager::BackgroundProcess::try_reserve_cleaner( + const cleaner_usage_t &usage) +{ + return { + main_cleaner->try_reserve_projected_usage(usage.main_usage), + try_reserve_cold(usage.cold_ool_usage) + }; +} + +void ExtentPlacementManager::BackgroundProcess::abort_cleaner_usage( + const cleaner_usage_t &usage, + const reserve_cleaner_result_t &result) +{ + if (result.reserve_main_success) { + main_cleaner->release_projected_usage(usage.main_usage); + } + abort_cold_usage(usage.cold_ool_usage, result.reserve_cold_success); +} + +reserve_io_result_t +ExtentPlacementManager::BackgroundProcess::try_reserve_io( + const io_usage_t &usage) +{ + return { + trimmer->try_reserve_inline_usage(usage.inline_usage), + try_reserve_cleaner(usage.cleaner_usage) + }; +} + +void ExtentPlacementManager::BackgroundProcess::abort_io_usage( + const io_usage_t &usage, + const reserve_io_result_t &result) +{ + if (result.reserve_inline_success) { + trimmer->release_inline_usage(usage.inline_usage); + } + abort_cleaner_usage(usage.cleaner_usage, result.cleaner_result); +} + +seastar::future<> +ExtentPlacementManager::BackgroundProcess::do_background_cycle() +{ + assert(is_ready()); + bool should_trim = trimmer->should_trim(); + bool proceed_trim = false; + auto trim_size = trimmer->get_trim_size_per_cycle(); + cleaner_usage_t trim_usage{ + trim_size, + // We take a cautious policy here that the trimmer also reserves + // the max value on cold cleaner even if no extents will be rewritten + // to the cold tier. Cleaner also takes the same policy. + // The reason is that we don't know the exact value of reservation until + // the construction of trimmer transaction completes after which the reservation + // might fail then the trimmer is possible to be invalidated by cleaner. + // Reserving the max size at first could help us avoid these trouble. + has_cold_tier() ? trim_size : 0 + }; + + reserve_cleaner_result_t trim_reserve_res; + if (should_trim) { + trim_reserve_res = try_reserve_cleaner(trim_usage); + if (trim_reserve_res.is_successful()) { + proceed_trim = true; + } else { + abort_cleaner_usage(trim_usage, trim_reserve_res); + } + } + + if (proceed_trim) { + return trimmer->trim( + ).finally([this, trim_usage] { + abort_cleaner_usage(trim_usage, {true, true}); + }); + } else { + bool should_clean_main = + main_cleaner_should_run() || + // make sure cleaner will start + // when the trimmer should run but + // failed to reserve space. + (should_trim && !proceed_trim && + !trim_reserve_res.reserve_main_success); + bool proceed_clean_main = false; + + auto main_cold_usage = main_cleaner->get_reclaim_size_per_cycle(); + if (should_clean_main) { + if (has_cold_tier()) { + proceed_clean_main = try_reserve_cold(main_cold_usage); + } else { + proceed_clean_main = true; + } + } + + bool proceed_clean_cold = false; + if (has_cold_tier() && + (cold_cleaner->should_clean_space() || + (should_trim && !proceed_trim && + !trim_reserve_res.reserve_cold_success) || + (should_clean_main && !proceed_clean_main))) { + proceed_clean_cold = true; + } + + if (!proceed_clean_main && !proceed_clean_cold) { + ceph_abort("no background process will start"); + } + return seastar::when_all( + [this, proceed_clean_main, main_cold_usage] { + if (!proceed_clean_main) { + return seastar::now(); + } + return main_cleaner->clean_space( + ).handle_error( + crimson::ct_error::assert_all{ + "do_background_cycle encountered invalid error in main clean_space" + } + ).finally([this, main_cold_usage] { + abort_cold_usage(main_cold_usage, true); + }); + }, + [this, proceed_clean_cold] { + if (!proceed_clean_cold) { + return seastar::now(); + } + return cold_cleaner->clean_space( + ).handle_error( + crimson::ct_error::assert_all{ + "do_background_cycle encountered invalid error in cold clean_space" + } + ); + } + ).discard_result(); + } +} + +void ExtentPlacementManager::BackgroundProcess::register_metrics() +{ + namespace sm = seastar::metrics; + metrics.add_group("background_process", { + sm::make_counter("io_count", stats.io_count, + sm::description("the sum of IOs")), + sm::make_counter("io_blocked_count", stats.io_blocked_count, + sm::description("IOs that are blocked by gc")), + sm::make_counter("io_blocked_count_trim", stats.io_blocked_count_trim, + sm::description("IOs that are blocked by trimming")), + sm::make_counter("io_blocked_count_clean", stats.io_blocked_count_clean, + sm::description("IOs that are blocked by cleaning")), + sm::make_counter("io_blocked_sum", stats.io_blocked_sum, + sm::description("the sum of blocking IOs")) + }); +} + +RandomBlockOolWriter::alloc_write_iertr::future<> +RandomBlockOolWriter::alloc_write_ool_extents( + Transaction& t, + std::list<LogicalCachedExtentRef>& extents) +{ + if (extents.empty()) { + return alloc_write_iertr::now(); + } + return seastar::with_gate(write_guard, [this, &t, &extents] { + return do_write(t, extents); + }); +} + +RandomBlockOolWriter::alloc_write_iertr::future<> +RandomBlockOolWriter::do_write( + Transaction& t, + std::list<LogicalCachedExtentRef>& extents) +{ + LOG_PREFIX(RandomBlockOolWriter::do_write); + assert(!extents.empty()); + DEBUGT("start with {} allocated extents", + t, extents.size()); + return trans_intr::do_for_each(extents, + [this, &t, FNAME](auto& ex) { + auto paddr = ex->get_paddr(); + assert(paddr.is_absolute()); + RandomBlockManager * rbm = rb_cleaner->get_rbm(paddr); + assert(rbm); + TRACE("extent {}, allocated addr {}", fmt::ptr(ex.get()), paddr); + auto& stats = t.get_ool_write_stats(); + stats.extents.num += 1; + stats.extents.bytes += ex->get_length(); + stats.num_records += 1; + + ex->prepare_write(); + return rbm->write(paddr, + ex->get_bptr() + ).handle_error( + alloc_write_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error when writing record"} + ).safe_then([&t, &ex, paddr, FNAME]() { + TRACET("ool extent written at {} -- {}", + t, paddr, *ex); + t.mark_allocated_extent_ool(ex); + return alloc_write_iertr::now(); + }); + }); +} + +} diff --git a/src/crimson/os/seastore/extent_placement_manager.h b/src/crimson/os/seastore/extent_placement_manager.h new file mode 100644 index 000000000..b94c03ec3 --- /dev/null +++ b/src/crimson/os/seastore/extent_placement_manager.h @@ -0,0 +1,915 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include "seastar/core/gate.hh" + +#include "crimson/os/seastore/async_cleaner.h" +#include "crimson/os/seastore/cached_extent.h" +#include "crimson/os/seastore/journal/segment_allocator.h" +#include "crimson/os/seastore/journal/record_submitter.h" +#include "crimson/os/seastore/transaction.h" +#include "crimson/os/seastore/random_block_manager.h" +#include "crimson/os/seastore/random_block_manager/block_rb_manager.h" +#include "crimson/os/seastore/randomblock_manager_group.h" + +class transaction_manager_test_t; + +namespace crimson::os::seastore { + +/** + * ExtentOolWriter + * + * Write the extents as out-of-line and allocate the physical addresses. + * Different writers write extents to different locations. + */ +class ExtentOolWriter { + using base_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; +public: + virtual ~ExtentOolWriter() {} + + using open_ertr = base_ertr; + virtual open_ertr::future<> open() = 0; + + virtual paddr_t alloc_paddr(extent_len_t length) = 0; + + using alloc_write_ertr = base_ertr; + using alloc_write_iertr = trans_iertr<alloc_write_ertr>; + virtual alloc_write_iertr::future<> alloc_write_ool_extents( + Transaction &t, + std::list<LogicalCachedExtentRef> &extents) = 0; + + using close_ertr = base_ertr; + virtual close_ertr::future<> close() = 0; +}; +using ExtentOolWriterRef = std::unique_ptr<ExtentOolWriter>; + +/** + * SegmentedOolWriter + * + * Different writers write extents to different out-of-line segments provided + * by the SegmentProvider. + */ +class SegmentedOolWriter : public ExtentOolWriter { +public: + SegmentedOolWriter(data_category_t category, + rewrite_gen_t gen, + SegmentProvider &sp, + SegmentSeqAllocator &ssa); + + open_ertr::future<> open() final { + return record_submitter.open(false).discard_result(); + } + + alloc_write_iertr::future<> alloc_write_ool_extents( + Transaction &t, + std::list<LogicalCachedExtentRef> &extents) final; + + close_ertr::future<> close() final { + return write_guard.close().then([this] { + return record_submitter.close(); + }).safe_then([this] { + write_guard = seastar::gate(); + }); + } + + paddr_t alloc_paddr(extent_len_t length) final { + return make_delayed_temp_paddr(0); + } + +private: + alloc_write_iertr::future<> do_write( + Transaction& t, + std::list<LogicalCachedExtentRef> &extent); + + alloc_write_ertr::future<> write_record( + Transaction& t, + record_t&& record, + std::list<LogicalCachedExtentRef> &&extents, + bool with_atomic_roll_segment=false); + + journal::SegmentAllocator segment_allocator; + journal::RecordSubmitter record_submitter; + seastar::gate write_guard; +}; + + +class RandomBlockOolWriter : public ExtentOolWriter { +public: + RandomBlockOolWriter(RBMCleaner* rb_cleaner) : + rb_cleaner(rb_cleaner) {} + + using open_ertr = ExtentOolWriter::open_ertr; + open_ertr::future<> open() final { + return open_ertr::now(); + } + + alloc_write_iertr::future<> alloc_write_ool_extents( + Transaction &t, + std::list<LogicalCachedExtentRef> &extents) final; + + close_ertr::future<> close() final { + return write_guard.close().then([this] { + write_guard = seastar::gate(); + return close_ertr::now(); + }); + } + + paddr_t alloc_paddr(extent_len_t length) final { + assert(rb_cleaner); + return rb_cleaner->alloc_paddr(length); + } + +private: + alloc_write_iertr::future<> do_write( + Transaction& t, + std::list<LogicalCachedExtentRef> &extent); + + RBMCleaner* rb_cleaner; + seastar::gate write_guard; +}; + +struct cleaner_usage_t { + // The size of all extents write to the main devices, including inline extents + // and out-of-line extents. + std::size_t main_usage = 0; + // The size of extents write to the cold devices + std::size_t cold_ool_usage = 0; +}; + +struct reserve_cleaner_result_t { + bool reserve_main_success = true; + bool reserve_cold_success = true; + + bool is_successful() const { + return reserve_main_success && + reserve_cold_success; + } +}; + +/** + * io_usage_t + * + * io_usage_t describes the space usage consumed by client IO. + */ +struct io_usage_t { + // The total size of all inlined extents, not including deltas and other metadata + // produced by Cache::prepare_record. + std::size_t inline_usage = 0; + cleaner_usage_t cleaner_usage; + friend std::ostream &operator<<(std::ostream &out, const io_usage_t &usage) { + return out << "io_usage_t(" + << "inline_usage=" << usage.inline_usage + << ", main_cleaner_usage=" << usage.cleaner_usage.main_usage + << ", cold_cleaner_usage=" << usage.cleaner_usage.cold_ool_usage + << ")"; + } +}; + +struct reserve_io_result_t { + bool reserve_inline_success = true; + reserve_cleaner_result_t cleaner_result; + + bool is_successful() const { + return reserve_inline_success && + cleaner_result.is_successful(); + } +}; + +class ExtentPlacementManager { +public: + ExtentPlacementManager() + : ool_segment_seq_allocator( + std::make_unique<SegmentSeqAllocator>(segment_type_t::OOL)) + { + devices_by_id.resize(DEVICE_ID_MAX, nullptr); + } + + void init(JournalTrimmerImplRef &&, AsyncCleanerRef &&, AsyncCleanerRef &&); + + SegmentSeqAllocator &get_ool_segment_seq_allocator() const { + return *ool_segment_seq_allocator; + } + + void set_primary_device(Device *device); + + void set_extent_callback(ExtentCallbackInterface *cb) { + background_process.set_extent_callback(cb); + } + + journal_type_t get_journal_type() const { + return background_process.get_journal_type(); + } + + extent_len_t get_block_size() const { + assert(primary_device != nullptr); + // assume all the devices have the same block size + return primary_device->get_block_size(); + } + + Device& get_primary_device() { + assert(primary_device != nullptr); + return *primary_device; + } + + store_statfs_t get_stat() const { + return background_process.get_stat(); + } + + using mount_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + using mount_ret = mount_ertr::future<>; + mount_ret mount() { + return background_process.mount(); + } + + using open_ertr = ExtentOolWriter::open_ertr; + open_ertr::future<> open_for_write(); + + void start_scan_space() { + return background_process.start_scan_space(); + } + + void start_background() { + return background_process.start_background(); + } + + struct alloc_result_t { + paddr_t paddr; + bufferptr bp; + rewrite_gen_t gen; + }; + alloc_result_t alloc_new_extent( + Transaction& t, + extent_types_t type, + extent_len_t length, + placement_hint_t hint, +#ifdef UNIT_TESTS_BUILT + rewrite_gen_t gen, + std::optional<paddr_t> external_paddr = std::nullopt +#else + rewrite_gen_t gen +#endif + ) { + assert(hint < placement_hint_t::NUM_HINTS); + assert(is_target_rewrite_generation(gen)); + assert(gen == INIT_GENERATION || hint == placement_hint_t::REWRITE); + + data_category_t category = get_extent_category(type); + gen = adjust_generation(category, type, hint, gen); + + // XXX: bp might be extended to point to different memory (e.g. PMem) + // according to the allocator. + auto bp = ceph::bufferptr( + buffer::create_page_aligned(length)); + bp.zero(); + paddr_t addr; +#ifdef UNIT_TESTS_BUILT + if (unlikely(external_paddr.has_value())) { + assert(external_paddr->is_fake()); + addr = *external_paddr; + } else if (gen == INLINE_GENERATION) { +#else + if (gen == INLINE_GENERATION) { +#endif + addr = make_record_relative_paddr(0); + } else if (category == data_category_t::DATA) { + assert(data_writers_by_gen[generation_to_writer(gen)]); + addr = data_writers_by_gen[ + generation_to_writer(gen)]->alloc_paddr(length); + } else { + assert(category == data_category_t::METADATA); + assert(md_writers_by_gen[generation_to_writer(gen)]); + addr = md_writers_by_gen[ + generation_to_writer(gen)]->alloc_paddr(length); + } + return {addr, std::move(bp), gen}; + } + + /** + * dispatch_result_t + * + * ool extents are placed in alloc_map and passed to + * EPM::write_delayed_ool_extents, + * delayed_extents is used to update lba mapping. + * usage is used to reserve projected space + */ + using extents_by_writer_t = + std::map<ExtentOolWriter*, std::list<LogicalCachedExtentRef>>; + struct dispatch_result_t { + extents_by_writer_t alloc_map; + std::list<LogicalCachedExtentRef> delayed_extents; + io_usage_t usage; + }; + + /** + * dispatch_delayed_extents + * + * Performs delayed allocation + */ + dispatch_result_t dispatch_delayed_extents(Transaction& t); + + /** + * write_delayed_ool_extents + * + * Do writes for out-of-line extents. + */ + using alloc_paddr_iertr = ExtentOolWriter::alloc_write_iertr; + alloc_paddr_iertr::future<> write_delayed_ool_extents( + Transaction& t, + extents_by_writer_t& alloc_map); + + /** + * write_preallocated_ool_extents + * + * Performs ool writes for extents with pre-allocated addresses. + * See Transaction::pre_alloc_list + */ + alloc_paddr_iertr::future<> write_preallocated_ool_extents( + Transaction &t, + std::list<LogicalCachedExtentRef> extents); + + seastar::future<> stop_background() { + return background_process.stop_background(); + } + + using close_ertr = ExtentOolWriter::close_ertr; + close_ertr::future<> close(); + + using read_ertr = Device::read_ertr; + read_ertr::future<> read( + paddr_t addr, + size_t len, + ceph::bufferptr &out + ) { + assert(devices_by_id[addr.get_device_id()] != nullptr); + return devices_by_id[addr.get_device_id()]->read(addr, len, out); + } + + void mark_space_used(paddr_t addr, extent_len_t len) { + background_process.mark_space_used(addr, len); + } + + void mark_space_free(paddr_t addr, extent_len_t len) { + background_process.mark_space_free(addr, len); + } + + void commit_space_used(paddr_t addr, extent_len_t len) { + return background_process.commit_space_used(addr, len); + } + + seastar::future<> reserve_projected_usage(io_usage_t usage) { + return background_process.reserve_projected_usage(usage); + } + + void release_projected_usage(const io_usage_t &usage) { + background_process.release_projected_usage(usage); + } + + backend_type_t get_main_backend_type() const { + if (!background_process.is_no_background()) { + return background_process.get_main_backend_type(); + } + // for test + assert(primary_device); + return primary_device->get_backend_type(); + } + + // Testing interfaces + + void test_init_no_background(Device *test_device) { + assert(test_device->get_backend_type() == backend_type_t::SEGMENTED); + add_device(test_device); + set_primary_device(test_device); + } + + bool check_usage() { + return background_process.check_usage(); + } + + seastar::future<> run_background_work_until_halt() { + return background_process.run_until_halt(); + } + +private: + rewrite_gen_t adjust_generation( + data_category_t category, + extent_types_t type, + placement_hint_t hint, + rewrite_gen_t gen) { + if (type == extent_types_t::ROOT) { + gen = INLINE_GENERATION; + } else if (get_main_backend_type() == backend_type_t::SEGMENTED && + is_lba_backref_node(type)) { + gen = INLINE_GENERATION; + } else if (hint == placement_hint_t::COLD) { + assert(gen == INIT_GENERATION); + if (background_process.has_cold_tier()) { + gen = MIN_COLD_GENERATION; + } else { + gen = MIN_REWRITE_GENERATION; + } + } else if (gen == INIT_GENERATION) { + if (category == data_category_t::METADATA) { + if (get_main_backend_type() == backend_type_t::SEGMENTED) { + // with SEGMENTED, default not to ool metadata extents to reduce + // padding overhead. + // TODO: improve padding so we can default to the ool path. + gen = INLINE_GENERATION; + } else { + // with RBM, all extents must be OOL + assert(get_main_backend_type() == + backend_type_t::RANDOM_BLOCK); + gen = OOL_GENERATION; + } + } else { + assert(category == data_category_t::DATA); + gen = OOL_GENERATION; + } + } else if (background_process.has_cold_tier()) { + gen = background_process.adjust_generation(gen); + } + + if (gen > dynamic_max_rewrite_generation) { + gen = dynamic_max_rewrite_generation; + } + + return gen; + } + + void add_device(Device *device) { + auto device_id = device->get_device_id(); + ceph_assert(devices_by_id[device_id] == nullptr); + devices_by_id[device_id] = device; + ++num_devices; + } + + /** + * dispatch_delayed_extent + * + * Specify the extent inline or ool + * return true indicates inline otherwise ool + */ + bool dispatch_delayed_extent(LogicalCachedExtentRef& extent) { + // TODO: all delayed extents are ool currently + boost::ignore_unused(extent); + return false; + } + + ExtentOolWriter* get_writer(placement_hint_t hint, + data_category_t category, + rewrite_gen_t gen) { + assert(hint < placement_hint_t::NUM_HINTS); + assert(is_rewrite_generation(gen)); + assert(gen != INLINE_GENERATION); + assert(gen <= dynamic_max_rewrite_generation); + if (category == data_category_t::DATA) { + return data_writers_by_gen[generation_to_writer(gen)]; + } else { + assert(category == data_category_t::METADATA); + return md_writers_by_gen[generation_to_writer(gen)]; + } + } + + /** + * BackgroundProcess + * + * Background process to schedule background transactions. + * + * TODO: device tiering + */ + class BackgroundProcess : public BackgroundListener { + public: + BackgroundProcess() = default; + + void init(JournalTrimmerImplRef &&_trimmer, + AsyncCleanerRef &&_cleaner, + AsyncCleanerRef &&_cold_cleaner) { + trimmer = std::move(_trimmer); + trimmer->set_background_callback(this); + main_cleaner = std::move(_cleaner); + main_cleaner->set_background_callback(this); + if (_cold_cleaner) { + cold_cleaner = std::move(_cold_cleaner); + cold_cleaner->set_background_callback(this); + + cleaners_by_device_id.resize(DEVICE_ID_MAX, nullptr); + for (auto id : main_cleaner->get_device_ids()) { + cleaners_by_device_id[id] = main_cleaner.get(); + } + for (auto id : cold_cleaner->get_device_ids()) { + cleaners_by_device_id[id] = cold_cleaner.get(); + } + + eviction_state.init( + crimson::common::get_conf<double>( + "seastore_multiple_tiers_stop_evict_ratio"), + crimson::common::get_conf<double>( + "seastore_multiple_tiers_default_evict_ratio"), + crimson::common::get_conf<double>( + "seastore_multiple_tiers_fast_evict_ratio")); + } + } + + journal_type_t get_journal_type() const { + return trimmer->get_journal_type(); + } + + bool has_cold_tier() const { + return cold_cleaner.get() != nullptr; + } + + void set_extent_callback(ExtentCallbackInterface *cb) { + trimmer->set_extent_callback(cb); + main_cleaner->set_extent_callback(cb); + if (has_cold_tier()) { + cold_cleaner->set_extent_callback(cb); + } + } + + store_statfs_t get_stat() const { + auto stat = main_cleaner->get_stat(); + if (has_cold_tier()) { + stat.add(cold_cleaner->get_stat()); + } + return stat; + } + + using mount_ret = ExtentPlacementManager::mount_ret; + mount_ret mount() { + ceph_assert(state == state_t::STOP); + state = state_t::MOUNT; + trimmer->reset(); + stats = {}; + register_metrics(); + return main_cleaner->mount( + ).safe_then([this] { + return has_cold_tier() ? cold_cleaner->mount() : mount_ertr::now(); + }); + } + + void start_scan_space() { + ceph_assert(state == state_t::MOUNT); + state = state_t::SCAN_SPACE; + ceph_assert(main_cleaner->check_usage_is_empty()); + ceph_assert(!has_cold_tier() || + cold_cleaner->check_usage_is_empty()); + } + + void start_background(); + + void mark_space_used(paddr_t addr, extent_len_t len) { + if (state < state_t::SCAN_SPACE) { + return; + } + + if (!has_cold_tier()) { + assert(main_cleaner); + main_cleaner->mark_space_used(addr, len); + } else { + auto id = addr.get_device_id(); + assert(id < cleaners_by_device_id.size()); + auto cleaner = cleaners_by_device_id[id]; + assert(cleaner); + cleaner->mark_space_used(addr, len); + } + } + + void mark_space_free(paddr_t addr, extent_len_t len) { + if (state < state_t::SCAN_SPACE) { + return; + } + + if (!has_cold_tier()) { + assert(main_cleaner); + main_cleaner->mark_space_free(addr, len); + } else { + auto id = addr.get_device_id(); + assert(id < cleaners_by_device_id.size()); + auto cleaner = cleaners_by_device_id[id]; + assert(cleaner); + cleaner->mark_space_free(addr, len); + } + } + + void commit_space_used(paddr_t addr, extent_len_t len) { + if (state < state_t::SCAN_SPACE) { + return; + } + + if (!has_cold_tier()) { + assert(main_cleaner); + main_cleaner->commit_space_used(addr, len); + } else { + auto id = addr.get_device_id(); + assert(id < cleaners_by_device_id.size()); + auto cleaner = cleaners_by_device_id[id]; + assert(cleaner); + cleaner->commit_space_used(addr, len); + } + } + + rewrite_gen_t adjust_generation(rewrite_gen_t gen) { + if (has_cold_tier()) { + return eviction_state.adjust_generation_with_eviction(gen); + } else { + return gen; + } + } + + seastar::future<> reserve_projected_usage(io_usage_t usage); + + void release_projected_usage(const io_usage_t &usage) { + if (is_ready()) { + trimmer->release_inline_usage(usage.inline_usage); + main_cleaner->release_projected_usage(usage.cleaner_usage.main_usage); + if (has_cold_tier()) { + cold_cleaner->release_projected_usage(usage.cleaner_usage.cold_ool_usage); + } + } + } + + seastar::future<> stop_background(); + backend_type_t get_main_backend_type() const { + return get_journal_type(); + } + + // Testing interfaces + + bool check_usage() { + return main_cleaner->check_usage() && + (!has_cold_tier() || cold_cleaner->check_usage()); + } + + seastar::future<> run_until_halt(); + + bool is_no_background() const { + return !trimmer || !main_cleaner; + } + + protected: + state_t get_state() const final { + return state; + } + + void maybe_wake_background() final { + if (!is_running()) { + return; + } + if (background_should_run()) { + do_wake_background(); + } + } + + void maybe_wake_blocked_io() final { + if (!is_ready()) { + return; + } + if (!should_block_io() && blocking_io) { + blocking_io->set_value(); + blocking_io = std::nullopt; + } + } + + private: + // reserve helpers + bool try_reserve_cold(std::size_t usage); + void abort_cold_usage(std::size_t usage, bool success); + + reserve_cleaner_result_t try_reserve_cleaner(const cleaner_usage_t &usage); + void abort_cleaner_usage(const cleaner_usage_t &usage, + const reserve_cleaner_result_t &result); + + reserve_io_result_t try_reserve_io(const io_usage_t &usage); + void abort_io_usage(const io_usage_t &usage, + const reserve_io_result_t &result); + + bool is_running() const { + if (state == state_t::RUNNING) { + assert(process_join); + return true; + } else { + assert(!process_join); + return false; + } + } + + void log_state(const char *caller) const; + + seastar::future<> run(); + + void do_wake_background() { + if (blocking_background) { + blocking_background->set_value(); + blocking_background = std::nullopt; + } + } + + // background_should_run() should be atomic with do_background_cycle() + // to make sure the condition is consistent. + bool background_should_run() { + assert(is_ready()); + maybe_update_eviction_mode(); + return main_cleaner_should_run() + || cold_cleaner_should_run() + || trimmer->should_trim(); + } + + bool main_cleaner_should_run() const { + assert(is_ready()); + return main_cleaner->should_clean_space() || + (has_cold_tier() && + main_cleaner->can_clean_space() && + eviction_state.is_fast_mode()); + } + + bool cold_cleaner_should_run() const { + assert(is_ready()); + return has_cold_tier() && + cold_cleaner->should_clean_space(); + } + + bool should_block_io() const { + assert(is_ready()); + return trimmer->should_block_io_on_trim() || + main_cleaner->should_block_io_on_clean() || + (has_cold_tier() && + cold_cleaner->should_block_io_on_clean()); + } + + void maybe_update_eviction_mode() { + if (has_cold_tier()) { + auto main_alive_ratio = main_cleaner->get_stat().get_used_raw_ratio(); + eviction_state.maybe_update_eviction_mode(main_alive_ratio); + } + } + + struct eviction_state_t { + enum class eviction_mode_t { + STOP, // generation greater than or equal to MIN_COLD_GENERATION + // will be set to MIN_COLD_GENERATION - 1, which means + // no extents will be evicted. + DEFAULT, // generation incremented with each rewrite. Extents will + // be evicted when generation reaches MIN_COLD_GENERATION. + FAST, // map all generations located in + // [MIN_REWRITE_GENERATION, MIN_COLD_GENERATIOIN) to + // MIN_COLD_GENERATION. + }; + + eviction_mode_t eviction_mode; + double stop_evict_ratio; + double default_evict_ratio; + double fast_evict_ratio; + + void init(double stop_ratio, + double default_ratio, + double fast_ratio) { + ceph_assert(0 <= stop_ratio); + ceph_assert(stop_ratio < default_ratio); + ceph_assert(default_ratio < fast_ratio); + ceph_assert(fast_ratio <= 1); + eviction_mode = eviction_mode_t::STOP; + stop_evict_ratio = stop_ratio; + default_evict_ratio = default_ratio; + fast_evict_ratio = fast_ratio; + } + + bool is_stop_mode() const { + return eviction_mode == eviction_mode_t::STOP; + } + + bool is_default_mode() const { + return eviction_mode == eviction_mode_t::DEFAULT; + } + + bool is_fast_mode() const { + return eviction_mode == eviction_mode_t::FAST; + } + + rewrite_gen_t adjust_generation_with_eviction(rewrite_gen_t gen) { + rewrite_gen_t ret = gen; + switch(eviction_mode) { + case eviction_mode_t::STOP: + if (gen == MIN_COLD_GENERATION) { + ret = MIN_COLD_GENERATION - 1; + } + break; + case eviction_mode_t::DEFAULT: + break; + case eviction_mode_t::FAST: + if (gen >= MIN_REWRITE_GENERATION && gen < MIN_COLD_GENERATION) { + ret = MIN_COLD_GENERATION; + } + break; + default: + ceph_abort("impossible"); + } + return ret; + } + + // We change the state of eviction_mode according to the alive ratio + // of the main cleaner. + // + // Use A, B, C, D to represent the state of alive ratio: + // A: alive ratio <= stop_evict_ratio + // B: alive ratio <= default_evict_ratio + // C: alive ratio <= fast_evict_ratio + // D: alive ratio > fast_evict_ratio + // + // and use X, Y, Z to shorten the state of eviction_mode_t: + // X: STOP + // Y: DEFAULT + // Z: FAST + // + // Then we can use a form like (A && X) to describe the current state + // of the main cleaner, which indicates the alive ratio is less than or + // equal to stop_evict_ratio and current eviction mode is STOP. + // + // all valid state transitions show as follow: + // (A && X) => (B && X) => (C && Y) => (D && Z) => + // (C && Z) => (B && Y) => (A && X) + // `--> (C && Y) => ... + // + // when the system restarts, the init state is (_ && X), the + // transitions should be: + // (_ && X) -> (A && X) => normal transition + // -> (B && X) => normal transition + // -> (C && X) => (C && Y) => normal transition + // -> (D && X) => (D && Z) => normal transition + void maybe_update_eviction_mode(double main_alive_ratio) { + if (main_alive_ratio <= stop_evict_ratio) { + eviction_mode = eviction_mode_t::STOP; + } else if (main_alive_ratio <= default_evict_ratio) { + if (eviction_mode > eviction_mode_t::DEFAULT) { + eviction_mode = eviction_mode_t::DEFAULT; + } + } else if (main_alive_ratio <= fast_evict_ratio) { + if (eviction_mode < eviction_mode_t::DEFAULT) { + eviction_mode = eviction_mode_t::DEFAULT; + } + } else { + assert(main_alive_ratio > fast_evict_ratio); + eviction_mode = eviction_mode_t::FAST; + } + } + }; + + seastar::future<> do_background_cycle(); + + void register_metrics(); + + struct { + uint64_t io_blocking_num = 0; + uint64_t io_count = 0; + uint64_t io_blocked_count = 0; + uint64_t io_blocked_count_trim = 0; + uint64_t io_blocked_count_clean = 0; + uint64_t io_blocked_sum = 0; + } stats; + seastar::metrics::metric_group metrics; + + JournalTrimmerImplRef trimmer; + AsyncCleanerRef main_cleaner; + + /* + * cold tier (optional, see has_cold_tier()) + */ + AsyncCleanerRef cold_cleaner; + std::vector<AsyncCleaner*> cleaners_by_device_id; + + std::optional<seastar::future<>> process_join; + std::optional<seastar::promise<>> blocking_background; + std::optional<seastar::promise<>> blocking_io; + bool is_running_until_halt = false; + state_t state = state_t::STOP; + eviction_state_t eviction_state; + + friend class ::transaction_manager_test_t; + }; + + std::vector<ExtentOolWriterRef> writer_refs; + std::vector<ExtentOolWriter*> data_writers_by_gen; + // gen 0 METADATA writer is the journal writer + std::vector<ExtentOolWriter*> md_writers_by_gen; + + std::vector<Device*> devices_by_id; + Device* primary_device = nullptr; + std::size_t num_devices = 0; + + rewrite_gen_t dynamic_max_rewrite_generation = REWRITE_GENERATIONS; + BackgroundProcess background_process; + // TODO: drop once paddr->journal_seq_t is introduced + SegmentSeqAllocatorRef ool_segment_seq_allocator; + + friend class ::transaction_manager_test_t; +}; + +using ExtentPlacementManagerRef = std::unique_ptr<ExtentPlacementManager>; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::io_usage_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/extentmap_manager.cc b/src/crimson/os/seastore/extentmap_manager.cc new file mode 100644 index 000000000..b0dc1b8c8 --- /dev/null +++ b/src/crimson/os/seastore/extentmap_manager.cc @@ -0,0 +1,33 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab +#include <experimental/iterator> +#include <iostream> + +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/extentmap_manager.h" +#include "crimson/os/seastore/extentmap_manager/btree/btree_extentmap_manager.h" +namespace crimson::os::seastore::extentmap_manager { + +ExtentMapManagerRef create_extentmap_manager( + TransactionManager &trans_manager) { + return ExtentMapManagerRef(new BtreeExtentMapManager(trans_manager)); +} + +} + +namespace crimson::os::seastore { + +std::ostream &operator<<(std::ostream &out, const extent_mapping_t &rhs) +{ + return out << "extent_mapping_t (" << rhs.logical_offset << "~" << rhs.length + << "->" << rhs.laddr << ")"; +} + +std::ostream &operator<<(std::ostream &out, const extent_map_list_t &rhs) +{ + out << '['; + std::copy(std::begin(rhs), std::end(rhs), std::experimental::make_ostream_joiner(out, ", ")); + return out << ']'; +} + +} diff --git a/src/crimson/os/seastore/journal.cc b/src/crimson/os/seastore/journal.cc new file mode 100644 index 000000000..d4714cf3f --- /dev/null +++ b/src/crimson/os/seastore/journal.cc @@ -0,0 +1,25 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "journal.h" +#include "journal/segmented_journal.h" +#include "journal/circular_bounded_journal.h" + +namespace crimson::os::seastore::journal { + +JournalRef make_segmented( + SegmentProvider &provider, + JournalTrimmer &trimmer) +{ + return std::make_unique<SegmentedJournal>(provider, trimmer); +} + +JournalRef make_circularbounded( + JournalTrimmer &trimmer, + crimson::os::seastore::random_block_device::RBMDevice* device, + std::string path) +{ + return std::make_unique<CircularBoundedJournal>(trimmer, device, path); +} + +} diff --git a/src/crimson/os/seastore/journal.h b/src/crimson/os/seastore/journal.h new file mode 100644 index 000000000..18c0797a8 --- /dev/null +++ b/src/crimson/os/seastore/journal.h @@ -0,0 +1,122 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <memory> + +#include "crimson/os/seastore/ordering_handle.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/segment_seq_allocator.h" + +namespace crimson::os::seastore { + +namespace random_block_device { +class RBMDevice; +} + +class SegmentManagerGroup; +class SegmentProvider; +class JournalTrimmer; + +class Journal { +public: + virtual JournalTrimmer &get_trimmer() = 0; + /** + * initializes journal for mkfs writes -- must run prior to calls + * to submit_record. + */ + using open_for_mkfs_ertr = crimson::errorator< + crimson::ct_error::input_output_error + >; + using open_for_mkfs_ret = open_for_mkfs_ertr::future<journal_seq_t>; + virtual open_for_mkfs_ret open_for_mkfs() = 0; + + /** + * initializes journal for new writes -- must run prior to calls + * to submit_record. Should be called after replay if not a new + * Journal. + */ + using open_for_mount_ertr = open_for_mkfs_ertr; + using open_for_mount_ret = open_for_mkfs_ret; + virtual open_for_mount_ret open_for_mount() = 0; + + /// close journal + using close_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + virtual close_ertr::future<> close() = 0; + + /** + * submit_record + * + * write record with the ordering handle + */ + using submit_record_ertr = crimson::errorator< + crimson::ct_error::erange, + crimson::ct_error::input_output_error + >; + using submit_record_ret = submit_record_ertr::future< + record_locator_t + >; + virtual submit_record_ret submit_record( + record_t &&record, + OrderingHandle &handle + ) = 0; + + /** + * flush + * + * Wait for all outstanding IOs on handle to commit. + * Note, flush() machinery must go through the same pipeline + * stages and locks as submit_record. + */ + virtual seastar::future<> flush(OrderingHandle &handle) = 0; + + /// sets write pipeline reference + virtual void set_write_pipeline(WritePipeline *_write_pipeline) = 0; + + /** + * Read deltas and pass to delta_handler + * + * record_block_start (argument to delta_handler) is the start of the + * of the first block in the record + */ + using replay_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent, + crimson::ct_error::erange>; + using replay_ret = replay_ertr::future<>; + using delta_handler_t = std::function< + replay_ertr::future<bool>( + const record_locator_t&, + const delta_info_t&, + const journal_seq_t&, // dirty_tail + const journal_seq_t&, // alloc_tail + sea_time_point modify_time)>; + virtual replay_ret replay( + delta_handler_t &&delta_handler) = 0; + + virtual seastar::future<> finish_commit( + transaction_type_t type) = 0; + + virtual ~Journal() {} + + virtual journal_type_t get_type() = 0; +}; +using JournalRef = std::unique_ptr<Journal>; + +namespace journal { + +JournalRef make_segmented( + SegmentProvider &provider, + JournalTrimmer &trimmer); + +JournalRef make_circularbounded( + JournalTrimmer &trimmer, + crimson::os::seastore::random_block_device::RBMDevice* device, + std::string path); + +} + +} diff --git a/src/crimson/os/seastore/journal/circular_bounded_journal.cc b/src/crimson/os/seastore/journal/circular_bounded_journal.cc new file mode 100644 index 000000000..ec41bfab1 --- /dev/null +++ b/src/crimson/os/seastore/journal/circular_bounded_journal.cc @@ -0,0 +1,387 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <boost/iterator/counting_iterator.hpp> + +#include "crimson/common/errorator-loop.h" +#include "include/intarith.h" +#include "crimson/os/seastore/async_cleaner.h" +#include "crimson/os/seastore/journal/circular_bounded_journal.h" +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/journal/circular_journal_space.h" + +SET_SUBSYS(seastore_journal); + +namespace crimson::os::seastore::journal { + +CircularBoundedJournal::CircularBoundedJournal( + JournalTrimmer &trimmer, + RBMDevice* device, + const std::string &path) + : trimmer(trimmer), path(path), + cjs(device), + record_submitter(crimson::common::get_conf<uint64_t>( + "seastore_journal_iodepth_limit"), + crimson::common::get_conf<uint64_t>( + "seastore_journal_batch_capacity"), + crimson::common::get_conf<Option::size_t>( + "seastore_journal_batch_flush_size"), + crimson::common::get_conf<double>( + "seastore_journal_batch_preferred_fullness"), + cjs) + {} + +CircularBoundedJournal::open_for_mkfs_ret +CircularBoundedJournal::open_for_mkfs() +{ + return record_submitter.open(true + ).safe_then([this](auto ret) { + return open_for_mkfs_ret( + open_for_mkfs_ertr::ready_future_marker{}, + get_written_to()); + }); +} + +CircularBoundedJournal::open_for_mount_ret +CircularBoundedJournal::open_for_mount() +{ + return record_submitter.open(false + ).safe_then([this](auto ret) { + return open_for_mount_ret( + open_for_mount_ertr::ready_future_marker{}, + get_written_to()); + }); +} + +CircularBoundedJournal::close_ertr::future<> CircularBoundedJournal::close() +{ + return record_submitter.close(); +} + +CircularBoundedJournal::submit_record_ret +CircularBoundedJournal::submit_record( + record_t &&record, + OrderingHandle &handle) +{ + LOG_PREFIX(CircularBoundedJournal::submit_record); + DEBUG("H{} {} start ...", (void*)&handle, record); + assert(write_pipeline); + return do_submit_record(std::move(record), handle); +} + +CircularBoundedJournal::submit_record_ret +CircularBoundedJournal::do_submit_record( + record_t &&record, + OrderingHandle &handle) +{ + LOG_PREFIX(CircularBoundedJournal::do_submit_record); + if (!record_submitter.is_available()) { + DEBUG("H{} wait ...", (void*)&handle); + return record_submitter.wait_available( + ).safe_then([this, record=std::move(record), &handle]() mutable { + return do_submit_record(std::move(record), handle); + }); + } + auto action = record_submitter.check_action(record.size); + if (action == RecordSubmitter::action_t::ROLL) { + return record_submitter.roll_segment( + ).safe_then([this, record=std::move(record), &handle]() mutable { + return do_submit_record(std::move(record), handle); + }); + } + + DEBUG("H{} submit {} ...", + (void*)&handle, + action == RecordSubmitter::action_t::SUBMIT_FULL ? + "FULL" : "NOT_FULL"); + auto submit_fut = record_submitter.submit(std::move(record)); + return handle.enter(write_pipeline->device_submission + ).then([submit_fut=std::move(submit_fut)]() mutable { + return std::move(submit_fut); + }).safe_then([FNAME, this, &handle](record_locator_t result) { + return handle.enter(write_pipeline->finalize + ).then([FNAME, this, result, &handle] { + DEBUG("H{} finish with {}", (void*)&handle, result); + auto new_committed_to = result.write_result.get_end_seq(); + record_submitter.update_committed_to(new_committed_to); + return result; + }); + }); +} + +Journal::replay_ret CircularBoundedJournal::replay_segment( + cbj_delta_handler_t &handler, scan_valid_records_cursor& cursor) +{ + LOG_PREFIX(Journal::replay_segment); + return seastar::do_with( + RecordScanner::found_record_handler_t( + [this, &handler, FNAME]( + record_locator_t locator, + const record_group_header_t& r_header, + const bufferlist& mdbuf) + -> RecordScanner::scan_valid_records_ertr::future<> + { + auto maybe_record_deltas_list = try_decode_deltas( + r_header, mdbuf, locator.record_block_base); + if (!maybe_record_deltas_list) { + // This should be impossible, we did check the crc on the mdbuf + ERROR("unable to decode deltas for record {} at {}", + r_header, locator.record_block_base); + return crimson::ct_error::input_output_error::make(); + } + assert(locator.write_result.start_seq != JOURNAL_SEQ_NULL); + auto cursor_addr = convert_paddr_to_abs_addr(locator.write_result.start_seq.offset); + DEBUG("{} at {}", r_header, cursor_addr); + journal_seq_t start_seq = locator.write_result.start_seq; + auto write_result = write_result_t{ + start_seq, + r_header.mdlength + r_header.dlength + }; + auto expected_seq = locator.write_result.start_seq.segment_seq; + cursor_addr += (r_header.mdlength + r_header.dlength); + if (cursor_addr >= get_journal_end()) { + cursor_addr = get_records_start(); + ++expected_seq; + paddr_t addr = convert_abs_addr_to_paddr( + cursor_addr, + get_device_id()); + write_result.start_seq.offset = addr; + write_result.start_seq.segment_seq = expected_seq; + } + paddr_t addr = convert_abs_addr_to_paddr( + cursor_addr, + get_device_id()); + set_written_to( + journal_seq_t{expected_seq, addr}); + return seastar::do_with( + std::move(*maybe_record_deltas_list), + [write_result, + &handler, + FNAME](auto& record_deltas_list) { + return crimson::do_for_each( + record_deltas_list, + [write_result, + &handler, FNAME](record_deltas_t& record_deltas) { + auto locator = record_locator_t{ + record_deltas.record_block_base, + write_result + }; + DEBUG("processing {} deltas at block_base {}", + record_deltas.deltas.size(), + locator); + return crimson::do_for_each( + record_deltas.deltas, + [locator, + &handler](auto& p) { + auto& modify_time = p.first; + auto& delta = p.second; + return handler( + locator, + delta, + modify_time).discard_result(); + }); + }); + }); + }), + [=, this, &cursor](auto &dhandler) { + return scan_valid_records( + cursor, + cjs.get_cbj_header().magic, + std::numeric_limits<size_t>::max(), + dhandler).safe_then([](auto){} + ).handle_error( + replay_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "shouldn't meet with any other error other replay_ertr" + } + ); + } + ); +} + + +Journal::replay_ret CircularBoundedJournal::scan_valid_record_delta( + cbj_delta_handler_t &&handler, journal_seq_t tail) +{ + LOG_PREFIX(Journal::scan_valid_record_delta); + INFO("starting at {} ", tail); + return seastar::do_with( + scan_valid_records_cursor(tail), + std::move(handler), + bool(false), + [this] (auto &cursor, auto &handler, auto &rolled) { + return crimson::repeat([this, &handler, &cursor, &rolled]() + -> replay_ertr::future<seastar::stop_iteration> + { + return replay_segment(handler, cursor + ).safe_then([this, &cursor, &rolled] { + if (!rolled) { + cursor.last_valid_header_found = false; + } + if (!cursor.is_complete()) { + try_read_rolled_header(cursor); + rolled = true; + return replay_ertr::make_ready_future< + seastar::stop_iteration>(seastar::stop_iteration::no); + } + return replay_ertr::make_ready_future< + seastar::stop_iteration>(seastar::stop_iteration::yes); + }); + }); + }); +} + +RecordScanner::read_ret CircularBoundedJournal::read(paddr_t start, size_t len) +{ + LOG_PREFIX(CircularBoundedJournal::read); + rbm_abs_addr addr = convert_paddr_to_abs_addr(start); + DEBUG("reading data from addr {} read length {}", addr, len); + auto bptr = bufferptr(ceph::buffer::create_page_aligned(len)); + return cjs.read(addr, bptr + ).safe_then([bptr=std::move(bptr)]() { + return read_ret( + RecordScanner::read_ertr::ready_future_marker{}, + std::move(bptr) + ); + }); +} + +bool CircularBoundedJournal::is_record_segment_seq_invalid( + scan_valid_records_cursor &cursor, + record_group_header_t &r_header) +{ + LOG_PREFIX(CircularBoundedJournal::is_record_segment_seq_invalid); + auto print_invalid = [FNAME](auto &r_header) { + DEBUG("invalid header: {}", r_header); + return true; + }; + if (cursor.seq.offset == convert_abs_addr_to_paddr( + get_records_start(), get_device_id())) { + if ((r_header.committed_to.segment_seq == NULL_SEG_SEQ && + cursor.seq.segment_seq != 0) || + r_header.committed_to.segment_seq != cursor.seq.segment_seq - 1) { + return print_invalid(r_header); + } + } else if (r_header.committed_to.segment_seq != cursor.seq.segment_seq) { + /* + * Assuing that seastore issues several records using submit_recods() + * as shown in the following example. + * + * Example ) + * a. submit_record(a); + * b. submit_record(b); + * c. submit_record(c); + * d. roll to begin + * e. submit_record(d); + * f. submit_record(e); + * g. submit_record(f); + * + * In this example, we need to consider the two cases. + * case 1) + * records a - e were issued in a batch manner + * case 2) + * When starts to submit_record(e) at step 6, submit(b) has completed its finalize phase, + * so the header of e's committed_to points to the end of b. + * + * To handle these cases correctly, the following condition is added. + */ + if ((r_header.committed_to.offset >= cursor.last_committed.offset && + r_header.committed_to.segment_seq == cursor.last_committed.segment_seq) && + r_header.committed_to.segment_seq == cursor.seq.segment_seq - 1) { + return false; + } + return print_invalid(r_header); + } + return false; +} + +Journal::replay_ret CircularBoundedJournal::replay( + delta_handler_t &&delta_handler) +{ + /* + * read records from last applied record prior to written_to, and replay + */ + LOG_PREFIX(CircularBoundedJournal::replay); + return cjs.read_header( + ).handle_error( + open_for_mount_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error read_header" + }).safe_then([this, FNAME, delta_handler=std::move(delta_handler)](auto p) + mutable { + auto &[head, bl] = *p; + cjs.set_cbj_header(head); + DEBUG("header : {}", cjs.get_cbj_header()); + cjs.set_initialized(true); + return seastar::do_with( + std::move(delta_handler), + std::map<paddr_t, journal_seq_t>(), + [this](auto &d_handler, auto &map) { + auto build_paddr_seq_map = [&map]( + const auto &offsets, + const auto &e, + sea_time_point modify_time) + { + if (e.type == extent_types_t::ALLOC_INFO) { + alloc_delta_t alloc_delta; + decode(alloc_delta, e.bl); + if (alloc_delta.op == alloc_delta_t::op_types_t::CLEAR) { + for (auto &alloc_blk : alloc_delta.alloc_blk_ranges) { + map[alloc_blk.paddr] = offsets.write_result.start_seq; + } + } + } + return replay_ertr::make_ready_future<bool>(true); + }; + auto tail = get_dirty_tail() <= get_alloc_tail() ? + get_dirty_tail() : get_alloc_tail(); + set_written_to(tail); + // The first pass to build the paddr->journal_seq_t map + // from extent allocations + return scan_valid_record_delta(std::move(build_paddr_seq_map), tail + ).safe_then([this, &map, &d_handler, tail]() { + auto call_d_handler_if_valid = [this, &map, &d_handler]( + const auto &offsets, + const auto &e, + sea_time_point modify_time) + { + if (map.find(e.paddr) == map.end() || + map[e.paddr] <= offsets.write_result.start_seq) { + return d_handler( + offsets, + e, + get_dirty_tail(), + get_alloc_tail(), + modify_time + ); + } + return replay_ertr::make_ready_future<bool>(true); + }; + // The second pass to replay deltas + return scan_valid_record_delta(std::move(call_d_handler_if_valid), tail); + }); + }).safe_then([this]() { + // make sure that committed_to is JOURNAL_SEQ_NULL if jounal is the initial state + if (get_written_to() != + journal_seq_t{0, + convert_abs_addr_to_paddr(get_records_start(), + get_device_id())}) { + record_submitter.update_committed_to(get_written_to()); + } + trimmer.update_journal_tails( + get_dirty_tail(), + get_alloc_tail()); + }); + }); +} + +seastar::future<> CircularBoundedJournal::finish_commit(transaction_type_t type) { + if (is_trim_transaction(type)) { + return update_journal_tail( + trimmer.get_dirty_tail(), + trimmer.get_alloc_tail()); + } + return seastar::now(); +} + +} diff --git a/src/crimson/os/seastore/journal/circular_bounded_journal.h b/src/crimson/os/seastore/journal/circular_bounded_journal.h new file mode 100644 index 000000000..debe535ae --- /dev/null +++ b/src/crimson/os/seastore/journal/circular_bounded_journal.h @@ -0,0 +1,224 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "crimson/common/log.h" + +#include <boost/intrusive_ptr.hpp> + +#include <seastar/core/future.hh> + +#include "include/ceph_assert.h" +#include "include/buffer.h" +#include "include/denc.h" + +#include "crimson/osd/exceptions.h" +#include "crimson/os/seastore/journal.h" +#include "include/uuid.h" +#include "crimson/os/seastore/random_block_manager.h" +#include "crimson/os/seastore/random_block_manager/rbm_device.h" +#include <list> +#include "crimson/os/seastore/journal/record_submitter.h" +#include "crimson/os/seastore/journal/circular_journal_space.h" +#include "crimson/os/seastore/record_scanner.h" + +namespace crimson::os::seastore::journal { + +using RBMDevice = random_block_device::RBMDevice; + +/** + * CircularBoundedJournal + * + * + * CircularBoundedJournal (CBJournal) is the journal that works like circular + * queue. With CBJournal, Seastore will append some of the records if the size + * of the record is small (most likely metadata), at which point the head + * (written_to) will be moved. Then, eventually, Seastore applies the records + * in CBjournal to RBM (TODO). + * + * - Commit time + * After submit_record is done, written_to is increased(this in-memory value) + * ---written_to represents where the new record will be appended. Note that + * applied_to is not changed here. + * + * - Replay time + * At replay time, CBJournal begins to replay records in CBjournal by reading + * records from dirty_tail. Then, CBJournal examines whether the records is valid + * one by one, at which point written_to is recovered + * if the valid record is founded. Note that applied_to is stored + * permanently when the apply work---applying the records in CBJournal to RBM--- + * is done by CBJournal (TODO). + * + * TODO: apply records from CircularBoundedJournal to RandomBlockManager + * + */ + +constexpr uint64_t DEFAULT_BLOCK_SIZE = 4096; + +class CircularBoundedJournal : public Journal, RecordScanner { +public: + CircularBoundedJournal( + JournalTrimmer &trimmer, RBMDevice* device, const std::string &path); + ~CircularBoundedJournal() {} + + JournalTrimmer &get_trimmer() final { + return trimmer; + } + + open_for_mkfs_ret open_for_mkfs() final; + + open_for_mount_ret open_for_mount() final; + + close_ertr::future<> close() final; + + journal_type_t get_type() final { + return journal_type_t::RANDOM_BLOCK; + } + + submit_record_ret submit_record( + record_t &&record, + OrderingHandle &handle + ) final; + + seastar::future<> flush( + OrderingHandle &handle + ) final { + // TODO + return seastar::now(); + } + + replay_ret replay(delta_handler_t &&delta_handler) final; + + rbm_abs_addr get_rbm_addr(journal_seq_t seq) const { + return convert_paddr_to_abs_addr(seq.offset); + } + + /** + * + * CircularBoundedJournal write + * + * NVMe will support a large block write (< 512KB) with atomic write unit command. + * With this command, we expect that the most of incoming data can be stored + * as a single write call, which has lower overhead than existing + * way that uses a combination of system calls such as write() and sync(). + * + */ + + seastar::future<> update_journal_tail( + journal_seq_t dirty, + journal_seq_t alloc) { + return cjs.update_journal_tail(dirty, alloc); + } + journal_seq_t get_dirty_tail() const { + return cjs.get_dirty_tail(); + } + journal_seq_t get_alloc_tail() const { + return cjs.get_alloc_tail(); + } + + void set_write_pipeline(WritePipeline *_write_pipeline) final { + write_pipeline = _write_pipeline; + } + + device_id_t get_device_id() const { + return cjs.get_device_id(); + } + extent_len_t get_block_size() const { + return cjs.get_block_size(); + } + + rbm_abs_addr get_journal_end() const { + return cjs.get_journal_end(); + } + + void set_written_to(journal_seq_t seq) { + cjs.set_written_to(seq); + } + + journal_seq_t get_written_to() { + return cjs.get_written_to(); + } + + rbm_abs_addr get_records_start() const { + return cjs.get_records_start(); + } + + seastar::future<> finish_commit(transaction_type_t type) final; + + using cbj_delta_handler_t = std::function< + replay_ertr::future<bool>( + const record_locator_t&, + const delta_info_t&, + sea_time_point modify_time)>; + + Journal::replay_ret scan_valid_record_delta( + cbj_delta_handler_t &&delta_handler, + journal_seq_t tail); + + submit_record_ret do_submit_record(record_t &&record, OrderingHandle &handle); + + void try_read_rolled_header(scan_valid_records_cursor &cursor) { + paddr_t addr = convert_abs_addr_to_paddr( + get_records_start(), + get_device_id()); + cursor.seq.offset = addr; + cursor.seq.segment_seq += 1; + } + + void initialize_cursor(scan_valid_records_cursor& cursor) final { + cursor.block_size = get_block_size(); + }; + + Journal::replay_ret replay_segment( + cbj_delta_handler_t &handler, scan_valid_records_cursor& cursor); + + read_ret read(paddr_t start, size_t len) final; + + bool is_record_segment_seq_invalid(scan_valid_records_cursor &cursor, + record_group_header_t &h) final; + + int64_t get_segment_end_offset(paddr_t addr) final { + return get_journal_end(); + } + + // Test interfaces + + CircularJournalSpace& get_cjs() { + return cjs; + } + + read_validate_record_metadata_ret test_read_validate_record_metadata( + scan_valid_records_cursor &cursor, + segment_nonce_t nonce) + { + return read_validate_record_metadata(cursor, nonce); + } + + void test_initialize_cursor(scan_valid_records_cursor &cursor) + { + initialize_cursor(cursor); + } + +private: + JournalTrimmer &trimmer; + std::string path; + WritePipeline *write_pipeline = nullptr; + /** + * initialized + * + * true after open_device_read_header, set to false in close(). + * Indicates that device is open and in-memory header is valid. + */ + bool initialized = false; + + // start address where the newest record will be written + // should be in range [get_records_start(), get_journal_end()) + // written_to.segment_seq is circulation seq to track + // the sequence to written records + CircularJournalSpace cjs; + RecordSubmitter record_submitter; +}; + +} + diff --git a/src/crimson/os/seastore/journal/circular_journal_space.cc b/src/crimson/os/seastore/journal/circular_journal_space.cc new file mode 100644 index 000000000..123bb9135 --- /dev/null +++ b/src/crimson/os/seastore/journal/circular_journal_space.cc @@ -0,0 +1,249 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#include "circular_journal_space.h" + +#include <fmt/format.h> +#include <fmt/os.h> + +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/async_cleaner.h" +#include "crimson/os/seastore/journal/circular_bounded_journal.h" + +SET_SUBSYS(seastore_journal); + +namespace crimson::os::seastore::journal { + +std::ostream &operator<<(std::ostream &out, + const CircularJournalSpace::cbj_header_t &header) +{ + return out << "cbj_header_t(" + << "dirty_tail=" << header.dirty_tail + << ", alloc_tail=" << header.alloc_tail + << ", magic=" << header.magic + << ")"; +} + +CircularJournalSpace::CircularJournalSpace(RBMDevice * device) : device(device) {} + +bool CircularJournalSpace::needs_roll(std::size_t length) const { + if (length + get_rbm_addr(get_written_to()) > get_journal_end()) { + return true; + } + return false; +} + +extent_len_t CircularJournalSpace::get_block_size() const { + return device->get_block_size(); +} + +CircularJournalSpace::roll_ertr::future<> CircularJournalSpace::roll() { + paddr_t paddr = convert_abs_addr_to_paddr( + get_records_start(), + get_device_id()); + auto seq = get_written_to(); + seq.segment_seq++; + assert(seq.segment_seq < MAX_SEG_SEQ); + set_written_to( + journal_seq_t{seq.segment_seq, paddr}); + return roll_ertr::now(); +} + +CircularJournalSpace::write_ret +CircularJournalSpace::write(ceph::bufferlist&& to_write) { + LOG_PREFIX(CircularJournalSpace::write); + assert(get_written_to().segment_seq != NULL_SEG_SEQ); + auto encoded_size = to_write.length(); + if (encoded_size > get_records_available_size()) { + ceph_abort("should be impossible with EPM reservation"); + } + assert(encoded_size + get_rbm_addr(get_written_to()) + < get_journal_end()); + + journal_seq_t j_seq = get_written_to(); + auto target = get_rbm_addr(get_written_to()); + auto new_written_to = target + encoded_size; + assert(new_written_to < get_journal_end()); + paddr_t paddr = convert_abs_addr_to_paddr( + new_written_to, + get_device_id()); + set_written_to( + journal_seq_t{get_written_to().segment_seq, paddr}); + DEBUG("{}, target {}", to_write.length(), target); + + auto write_result = write_result_t{ + j_seq, + encoded_size + }; + return device_write_bl(target, to_write + ).safe_then([this, target, + length=encoded_size, + write_result, + FNAME] { + DEBUG("commit target {} used_size {} written length {}", + target, get_records_used_size(), length); + return write_result; + }).handle_error( + base_ertr::pass_further{}, + crimson::ct_error::assert_all{ "Invalid error" } + ); +} + +segment_nonce_t calc_new_nonce( + uint32_t crc, + unsigned char const *data, + unsigned length) +{ + crc &= std::numeric_limits<uint32_t>::max() >> 1; + return ceph_crc32c(crc, data, length); +} + +CircularJournalSpace::open_ret CircularJournalSpace::open(bool is_mkfs) { + std::ostringstream oss; + oss << device_id_printer_t{get_device_id()}; + print_name = oss.str(); + + if (is_mkfs) { + LOG_PREFIX(CircularJournalSpace::open); + assert(device); + ceph::bufferlist bl; + CircularJournalSpace::cbj_header_t head; + assert(device->get_journal_size()); + head.dirty_tail = + journal_seq_t{0, + convert_abs_addr_to_paddr( + get_records_start(), + device->get_device_id())}; + head.alloc_tail = head.dirty_tail; + auto meta = device->get_meta(); + head.magic = calc_new_nonce( + std::rand() % std::numeric_limits<uint32_t>::max(), + reinterpret_cast<const unsigned char *>(meta.seastore_id.bytes()), + sizeof(meta.seastore_id.uuid)); + encode(head, bl); + header = head; + set_written_to(head.dirty_tail); + initialized = true; + DEBUG( + "initialize header block in CircularJournalSpace length {}, head: {}", + bl.length(), header); + return write_header( + ).safe_then([this]() { + return open_ret( + open_ertr::ready_future_marker{}, + get_written_to()); + }).handle_error( + open_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error write_header" + } + ); + } + ceph_assert(initialized); + if (written_to.segment_seq == NULL_SEG_SEQ) { + written_to.segment_seq = 0; + } + return open_ret( + open_ertr::ready_future_marker{}, + get_written_to()); +} + +ceph::bufferlist CircularJournalSpace::encode_header() +{ + bufferlist bl; + encode(header, bl); + auto header_crc_filler = bl.append_hole(sizeof(checksum_t)); + auto bliter = bl.cbegin(); + auto header_crc = bliter.crc32c( + ceph::encoded_sizeof_bounded<cbj_header_t>(), + -1); + ceph_le32 header_crc_le; + header_crc_le = header_crc; + header_crc_filler.copy_in( + sizeof(checksum_t), + reinterpret_cast<const char *>(&header_crc_le)); + return bl; +} + +CircularJournalSpace::write_ertr::future<> CircularJournalSpace::device_write_bl( + rbm_abs_addr offset, bufferlist &bl) +{ + LOG_PREFIX(CircularJournalSpace::device_write_bl); + auto length = bl.length(); + if (offset + length > get_journal_end()) { + return crimson::ct_error::erange::make(); + } + DEBUG( + "overwrite in CircularJournalSpace, offset {}, length {}", + offset, + length); + return device->writev(offset, bl + ).handle_error( + write_ertr::pass_further{}, + crimson::ct_error::assert_all{ "Invalid error device->write" } + ); +} + +CircularJournalSpace::read_header_ret +CircularJournalSpace::read_header() +{ + LOG_PREFIX(CircularJournalSpace::read_header); + assert(device); + auto bptr = bufferptr(ceph::buffer::create_page_aligned( + device->get_block_size())); + DEBUG("reading {}", device->get_shard_journal_start()); + return device->read(device->get_shard_journal_start(), bptr + ).safe_then([bptr, FNAME]() mutable + -> read_header_ret { + bufferlist bl; + bl.append(bptr); + auto bp = bl.cbegin(); + cbj_header_t cbj_header; + try { + decode(cbj_header, bp); + } catch (ceph::buffer::error &e) { + ERROR("unable to read header block"); + return crimson::ct_error::enoent::make(); + } + auto bliter = bl.cbegin(); + auto test_crc = bliter.crc32c( + ceph::encoded_sizeof_bounded<cbj_header_t>(), + -1); + ceph_le32 recorded_crc_le; + decode(recorded_crc_le, bliter); + uint32_t recorded_crc = recorded_crc_le; + if (test_crc != recorded_crc) { + ERROR("error, header crc mismatch."); + return read_header_ret( + read_header_ertr::ready_future_marker{}, + std::nullopt); + } + return read_header_ret( + read_header_ertr::ready_future_marker{}, + std::make_pair(cbj_header, bl) + ); + }); +} + +CircularJournalSpace::write_ertr::future<> +CircularJournalSpace::write_header() +{ + LOG_PREFIX(CircularJournalSpace::write_header); + ceph::bufferlist bl = encode_header(); + ceph_assert(bl.length() <= get_block_size()); + DEBUG( + "sync header of CircularJournalSpace, length {}", + bl.length()); + assert(device); + auto iter = bl.begin(); + assert(bl.length() < get_block_size()); + bufferptr bp = bufferptr(ceph::buffer::create_page_aligned(get_block_size())); + iter.copy(bl.length(), bp.c_str()); + return device->write(device->get_shard_journal_start(), std::move(bp) + ).handle_error( + write_ertr::pass_further{}, + crimson::ct_error::assert_all{ "Invalid error device->write" } + ); +} + +} diff --git a/src/crimson/os/seastore/journal/circular_journal_space.h b/src/crimson/os/seastore/journal/circular_journal_space.h new file mode 100644 index 000000000..c88b65ad5 --- /dev/null +++ b/src/crimson/os/seastore/journal/circular_journal_space.h @@ -0,0 +1,261 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include <optional> +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/metrics.hh> +#include <seastar/core/shared_future.hh> + +#include "include/buffer.h" + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/journal.h" +#include "crimson/os/seastore/random_block_manager.h" +#include "crimson/os/seastore/random_block_manager/rbm_device.h" +#include "crimson/os/seastore/journal/record_submitter.h" +#include "crimson/os/seastore/async_cleaner.h" + +namespace crimson::os::seastore { + class SegmentProvider; + class JournalTrimmer; +} + +namespace crimson::os::seastore::journal { + +class CircularBoundedJournal; +class CircularJournalSpace : public JournalAllocator { + + public: + const std::string& get_name() const final { + return print_name; + } + + extent_len_t get_block_size() const final; + + bool can_write() const final { + return (device != nullptr); + } + + segment_nonce_t get_nonce() const final { + return header.magic; + } + + bool needs_roll(std::size_t length) const final; + + roll_ertr::future<> roll() final; + + write_ret write(ceph::bufferlist&& to_write) final; + + void update_modify_time(record_t& record) final {} + + close_ertr::future<> close() final { + return write_header( + ).safe_then([this]() -> close_ertr::future<> { + initialized = false; + return close_ertr::now(); + }).handle_error( + Journal::open_for_mount_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error write_header" + } + ); + } + + open_ret open(bool is_mkfs) final; + + public: + CircularJournalSpace(RBMDevice * device); + + struct cbj_header_t; + using write_ertr = Journal::submit_record_ertr; + /* + * device_write_bl + * + * @param device address to write + * @param bufferlist to write + * + */ + write_ertr::future<> device_write_bl(rbm_abs_addr offset, ceph::bufferlist &bl); + + using read_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent, + crimson::ct_error::erange>; + using read_header_ertr = read_ertr; + using read_header_ret = read_header_ertr::future< + std::optional<std::pair<cbj_header_t, bufferlist>> + >; + /* + * read_header + * + * read header block from given absolute address + * + * @param absolute address + * + */ + read_header_ret read_header(); + + ceph::bufferlist encode_header(); + + write_ertr::future<> write_header(); + + + /** + * CircularBoundedJournal structure + * + * +-------------------------------------------------------+ + * | header | record | record | record | record | ... | + * +-------------------------------------------------------+ + * ^-----------block aligned-----------------^ + * <----fixed----> + */ + + struct cbj_header_t { + // start offset of CircularBoundedJournal in the device + journal_seq_t dirty_tail; + journal_seq_t alloc_tail; + segment_nonce_t magic; + + DENC(cbj_header_t, v, p) { + DENC_START(1, 1, p); + denc(v.dirty_tail, p); + denc(v.alloc_tail, p); + denc(v.magic, p); + DENC_FINISH(p); + } + }; + + /** + * + * Write position for CircularBoundedJournal + * + * | written to rbm | written length to CircularBoundedJournal | new write | + * ----------------->------------------------------------------------> + * ^ ^ + * applied_to written_to + * + */ + + journal_seq_t get_written_to() const { + return written_to; + } + rbm_abs_addr get_rbm_addr(journal_seq_t seq) const { + return convert_paddr_to_abs_addr(seq.offset); + } + void set_written_to(journal_seq_t seq) { + rbm_abs_addr addr = convert_paddr_to_abs_addr(seq.offset); + assert(addr >= get_records_start()); + assert(addr < get_journal_end()); + written_to = seq; + } + device_id_t get_device_id() const { + return device->get_device_id(); + } + + journal_seq_t get_dirty_tail() const { + return header.dirty_tail; + } + journal_seq_t get_alloc_tail() const { + return header.alloc_tail; + } + + /* + Size-related interfaces + +---------------------------------------------------------+ + | header | record | record | record | record | ... | + +---------------------------------------------------------+ + ^ ^ ^ + | | | + get_journal_start | get_journal_end + get_records_start + <-- get_records_total_size + block_size --> + <--------------- get_journal_size ------------------------> + */ + + size_t get_records_used_size() const { + auto rbm_written_to = get_rbm_addr(get_written_to()); + auto rbm_tail = get_rbm_addr(get_dirty_tail()); + return rbm_written_to >= rbm_tail ? + rbm_written_to - rbm_tail : + rbm_written_to + get_records_total_size() + get_block_size() + - rbm_tail; + } + size_t get_records_total_size() const { + assert(device); + // a block is for header and a block is reserved to denote the end + return device->get_journal_size() - (2 * get_block_size()); + } + rbm_abs_addr get_records_start() const { + assert(device); + return device->get_shard_journal_start() + get_block_size(); + } + size_t get_records_available_size() const { + return get_records_total_size() - get_records_used_size(); + } + bool is_available_size(uint64_t size) { + auto rbm_written_to = get_rbm_addr(get_written_to()); + auto rbm_tail = get_rbm_addr(get_dirty_tail()); + if (rbm_written_to > rbm_tail && + (get_journal_end() - rbm_written_to) < size && + size > (get_records_used_size() - + (get_journal_end() - rbm_written_to))) { + return false; + } + return get_records_available_size() >= size; + } + rbm_abs_addr get_journal_end() const { + assert(device); + return device->get_shard_journal_start() + device->get_journal_size(); + } + + read_ertr::future<> read( + uint64_t offset, + bufferptr &bptr) { + assert(device); + return device->read(offset, bptr); + } + + seastar::future<> update_journal_tail( + journal_seq_t dirty, + journal_seq_t alloc) { + header.dirty_tail = dirty; + header.alloc_tail = alloc; + return write_header( + ).handle_error( + crimson::ct_error::assert_all{ + "encountered invalid error in update_journal_tail" + }); + } + + void set_initialized(bool init) { + initialized = init; + } + + void set_cbj_header(cbj_header_t& head) { + header = head; + } + + cbj_header_t get_cbj_header() { + return header; + } + + private: + std::string print_name; + cbj_header_t header; + RBMDevice* device; + journal_seq_t written_to; + bool initialized = false; +}; + +std::ostream &operator<<(std::ostream &out, const CircularJournalSpace::cbj_header_t &header); + +} + +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::journal::CircularJournalSpace::cbj_header_t) + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::journal::CircularJournalSpace::cbj_header_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/journal/record_submitter.cc b/src/crimson/os/seastore/journal/record_submitter.cc new file mode 100644 index 000000000..5ca53b436 --- /dev/null +++ b/src/crimson/os/seastore/journal/record_submitter.cc @@ -0,0 +1,533 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#include "record_submitter.h" + +#include <fmt/format.h> +#include <fmt/os.h> + +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/async_cleaner.h" + +SET_SUBSYS(seastore_journal); + +namespace crimson::os::seastore::journal { + +RecordBatch::add_pending_ret +RecordBatch::add_pending( + const std::string& name, + record_t&& record, + extent_len_t block_size) +{ + LOG_PREFIX(RecordBatch::add_pending); + auto new_size = get_encoded_length_after(record, block_size); + auto dlength_offset = pending.size.dlength; + TRACE("{} batches={}, write_size={}, dlength_offset={} ...", + name, + pending.get_size() + 1, + new_size.get_encoded_length(), + dlength_offset); + assert(state != state_t::SUBMITTING); + assert(evaluate_submit(record.size, block_size).submit_size == new_size); + + pending.push_back( + std::move(record), block_size); + assert(pending.size == new_size); + if (state == state_t::EMPTY) { + assert(!io_promise.has_value()); + io_promise = seastar::shared_promise<maybe_promise_result_t>(); + } else { + assert(io_promise.has_value()); + } + state = state_t::PENDING; + + return io_promise->get_shared_future( + ).then([dlength_offset, FNAME, &name + ](auto maybe_promise_result) -> add_pending_ret { + if (!maybe_promise_result.has_value()) { + ERROR("{} write failed", name); + return crimson::ct_error::input_output_error::make(); + } + auto write_result = maybe_promise_result->write_result; + auto submit_result = record_locator_t{ + write_result.start_seq.offset.add_offset( + maybe_promise_result->mdlength + dlength_offset), + write_result + }; + TRACE("{} write finish with {}", name, submit_result); + return add_pending_ret( + add_pending_ertr::ready_future_marker{}, + submit_result); + }); +} + +std::pair<ceph::bufferlist, record_group_size_t> +RecordBatch::encode_batch( + const journal_seq_t& committed_to, + segment_nonce_t segment_nonce) +{ + assert(state == state_t::PENDING); + assert(pending.get_size() > 0); + assert(io_promise.has_value()); + + state = state_t::SUBMITTING; + submitting_size = pending.get_size(); + auto gsize = pending.size; + submitting_length = gsize.get_encoded_length(); + submitting_mdlength = gsize.get_mdlength(); + auto bl = encode_records(pending, committed_to, segment_nonce); + // Note: pending is cleared here + assert(bl.length() == submitting_length); + return std::make_pair(bl, gsize); +} + +void RecordBatch::set_result( + maybe_result_t maybe_write_result) +{ + maybe_promise_result_t result; + if (maybe_write_result.has_value()) { + assert(maybe_write_result->length == submitting_length); + result = promise_result_t{ + *maybe_write_result, + submitting_mdlength + }; + } + assert(state == state_t::SUBMITTING); + assert(io_promise.has_value()); + + state = state_t::EMPTY; + submitting_size = 0; + submitting_length = 0; + submitting_mdlength = 0; + io_promise->set_value(result); + io_promise.reset(); +} + +std::pair<ceph::bufferlist, record_group_size_t> +RecordBatch::submit_pending_fast( + record_t&& record, + extent_len_t block_size, + const journal_seq_t& committed_to, + segment_nonce_t segment_nonce) +{ + auto new_size = get_encoded_length_after(record, block_size); + std::ignore = new_size; + assert(state == state_t::EMPTY); + assert(evaluate_submit(record.size, block_size).submit_size == new_size); + + auto group = record_group_t(std::move(record), block_size); + auto size = group.size; + assert(size == new_size); + auto bl = encode_records(group, committed_to, segment_nonce); + assert(bl.length() == size.get_encoded_length()); + return std::make_pair(std::move(bl), size); +} + +RecordSubmitter::RecordSubmitter( + std::size_t io_depth, + std::size_t batch_capacity, + std::size_t batch_flush_size, + double preferred_fullness, + JournalAllocator& ja) + : io_depth_limit{io_depth}, + preferred_fullness{preferred_fullness}, + journal_allocator{ja}, + batches(new RecordBatch[io_depth + 1]) +{ + LOG_PREFIX(RecordSubmitter); + INFO("{} io_depth_limit={}, batch_capacity={}, batch_flush_size={}, " + "preferred_fullness={}", + get_name(), io_depth, batch_capacity, + batch_flush_size, preferred_fullness); + ceph_assert(io_depth > 0); + ceph_assert(batch_capacity > 0); + ceph_assert(preferred_fullness >= 0 && + preferred_fullness <= 1); + free_batch_ptrs.reserve(io_depth + 1); + for (std::size_t i = 0; i <= io_depth; ++i) { + batches[i].initialize(i, batch_capacity, batch_flush_size); + free_batch_ptrs.push_back(&batches[i]); + } + pop_free_batch(); +} + +bool RecordSubmitter::is_available() const +{ + auto ret = !wait_available_promise.has_value() && + !has_io_error; +#ifndef NDEBUG + if (ret) { + // unconditional invariants + ceph_assert(journal_allocator.can_write()); + ceph_assert(p_current_batch != nullptr); + ceph_assert(!p_current_batch->is_submitting()); + // the current batch accepts a further write + ceph_assert(!p_current_batch->needs_flush()); + if (!p_current_batch->is_empty()) { + auto submit_length = + p_current_batch->get_submit_size().get_encoded_length(); + ceph_assert(!journal_allocator.needs_roll(submit_length)); + } + // I'm not rolling + } +#endif + return ret; +} + +RecordSubmitter::wa_ertr::future<> +RecordSubmitter::wait_available() +{ + LOG_PREFIX(RecordSubmitter::wait_available); + assert(!is_available()); + if (has_io_error) { + ERROR("{} I/O is failed before wait", get_name()); + return crimson::ct_error::input_output_error::make(); + } + return wait_available_promise->get_shared_future( + ).then([FNAME, this]() -> wa_ertr::future<> { + if (has_io_error) { + ERROR("{} I/O is failed after wait", get_name()); + return crimson::ct_error::input_output_error::make(); + } + return wa_ertr::now(); + }); +} + +RecordSubmitter::action_t +RecordSubmitter::check_action( + const record_size_t& rsize) const +{ + assert(is_available()); + auto eval = p_current_batch->evaluate_submit( + rsize, journal_allocator.get_block_size()); + if (journal_allocator.needs_roll(eval.submit_size.get_encoded_length())) { + return action_t::ROLL; + } else if (eval.is_full) { + return action_t::SUBMIT_FULL; + } else { + return action_t::SUBMIT_NOT_FULL; + } +} + +RecordSubmitter::roll_segment_ertr::future<> +RecordSubmitter::roll_segment() +{ + LOG_PREFIX(RecordSubmitter::roll_segment); + ceph_assert(p_current_batch->needs_flush() || + is_available()); + // #1 block concurrent submissions due to rolling + wait_available_promise = seastar::shared_promise<>(); + ceph_assert(!wait_unfull_flush_promise.has_value()); + return [FNAME, this] { + if (p_current_batch->is_pending()) { + if (state == state_t::FULL) { + DEBUG("{} wait flush ...", get_name()); + wait_unfull_flush_promise = seastar::promise<>(); + return wait_unfull_flush_promise->get_future(); + } else { // IDLE/PENDING + DEBUG("{} flush", get_name()); + flush_current_batch(); + return seastar::now(); + } + } else { + assert(p_current_batch->is_empty()); + return seastar::now(); + } + }().then_wrapped([FNAME, this](auto fut) { + if (fut.failed()) { + ERROR("{} rolling is skipped unexpectedly, available", get_name()); + has_io_error = true; + wait_available_promise->set_value(); + wait_available_promise.reset(); + return roll_segment_ertr::now(); + } else { + // start rolling in background + std::ignore = journal_allocator.roll( + ).safe_then([FNAME, this] { + // good + DEBUG("{} rolling done, available", get_name()); + assert(!has_io_error); + wait_available_promise->set_value(); + wait_available_promise.reset(); + }).handle_error( + crimson::ct_error::all_same_way([FNAME, this](auto e) { + ERROR("{} got error {}, available", get_name(), e); + has_io_error = true; + wait_available_promise->set_value(); + wait_available_promise.reset(); + }) + ).handle_exception([FNAME, this](auto e) { + ERROR("{} got exception {}, available", get_name(), e); + has_io_error = true; + wait_available_promise->set_value(); + wait_available_promise.reset(); + }); + // wait for background rolling + return wait_available(); + } + }); +} + +RecordSubmitter::submit_ret +RecordSubmitter::submit( + record_t&& record, + bool with_atomic_roll_segment) +{ + LOG_PREFIX(RecordSubmitter::submit); + ceph_assert(is_available()); + assert(check_action(record.size) != action_t::ROLL); + journal_allocator.update_modify_time(record); + auto eval = p_current_batch->evaluate_submit( + record.size, journal_allocator.get_block_size()); + bool needs_flush = ( + state == state_t::IDLE || + eval.submit_size.get_fullness() > preferred_fullness || + // RecordBatch::needs_flush() + eval.is_full || + p_current_batch->get_num_records() + 1 >= + p_current_batch->get_batch_capacity()); + if (p_current_batch->is_empty() && + needs_flush && + state != state_t::FULL) { + // fast path with direct write + increment_io(); + auto [to_write, sizes] = p_current_batch->submit_pending_fast( + std::move(record), + journal_allocator.get_block_size(), + get_committed_to(), + journal_allocator.get_nonce()); + DEBUG("{} fast submit {}, committed_to={}, outstanding_io={} ...", + get_name(), sizes, get_committed_to(), num_outstanding_io); + account_submission(1, sizes); + return journal_allocator.write(std::move(to_write) + ).safe_then([mdlength = sizes.get_mdlength()](auto write_result) { + return record_locator_t{ + write_result.start_seq.offset.add_offset(mdlength), + write_result + }; + }).finally([this] { + decrement_io_with_flush(); + }); + } + // indirect batched write + auto write_fut = p_current_batch->add_pending( + get_name(), + std::move(record), + journal_allocator.get_block_size()); + if (needs_flush) { + if (state == state_t::FULL) { + // #2 block concurrent submissions due to lack of resource + DEBUG("{} added with {} pending, outstanding_io={}, unavailable, wait flush ...", + get_name(), + p_current_batch->get_num_records(), + num_outstanding_io); + if (with_atomic_roll_segment) { + // wait_available_promise and wait_unfull_flush_promise + // need to be delegated to the follow-up atomic roll_segment(); + assert(p_current_batch->is_pending()); + } else { + wait_available_promise = seastar::shared_promise<>(); + ceph_assert(!wait_unfull_flush_promise.has_value()); + wait_unfull_flush_promise = seastar::promise<>(); + // flush and mark available in background + std::ignore = wait_unfull_flush_promise->get_future( + ).finally([FNAME, this] { + DEBUG("{} flush done, available", get_name()); + wait_available_promise->set_value(); + wait_available_promise.reset(); + }); + } + } else { + DEBUG("{} added pending, flush", get_name()); + flush_current_batch(); + } + } else { + // will flush later + DEBUG("{} added with {} pending, outstanding_io={}", + get_name(), + p_current_batch->get_num_records(), + num_outstanding_io); + assert(!p_current_batch->needs_flush()); + } + return write_fut; +} + +RecordSubmitter::open_ret +RecordSubmitter::open(bool is_mkfs) +{ + return journal_allocator.open(is_mkfs + ).safe_then([this](journal_seq_t ret) { + LOG_PREFIX(RecordSubmitter::open); + DEBUG("{} register metrics", get_name()); + stats = {}; + namespace sm = seastar::metrics; + std::vector<sm::label_instance> label_instances; + label_instances.push_back(sm::label_instance("submitter", get_name())); + metrics.add_group( + "journal", + { + sm::make_counter( + "record_num", + stats.record_batch_stats.num_io, + sm::description("total number of records submitted"), + label_instances + ), + sm::make_counter( + "record_batch_num", + stats.record_batch_stats.num_io_grouped, + sm::description("total number of records batched"), + label_instances + ), + sm::make_counter( + "io_num", + stats.io_depth_stats.num_io, + sm::description("total number of io submitted"), + label_instances + ), + sm::make_counter( + "io_depth_num", + stats.io_depth_stats.num_io_grouped, + sm::description("total number of io depth"), + label_instances + ), + sm::make_counter( + "record_group_padding_bytes", + stats.record_group_padding_bytes, + sm::description("bytes of metadata padding when write record groups"), + label_instances + ), + sm::make_counter( + "record_group_metadata_bytes", + stats.record_group_metadata_bytes, + sm::description("bytes of raw metadata when write record groups"), + label_instances + ), + sm::make_counter( + "record_group_data_bytes", + stats.record_group_data_bytes, + sm::description("bytes of data when write record groups"), + label_instances + ), + } + ); + return ret; + }); +} + +RecordSubmitter::close_ertr::future<> +RecordSubmitter::close() +{ + committed_to = JOURNAL_SEQ_NULL; + ceph_assert(state == state_t::IDLE); + ceph_assert(num_outstanding_io == 0); + ceph_assert(p_current_batch != nullptr); + ceph_assert(p_current_batch->is_empty()); + ceph_assert(!wait_available_promise.has_value()); + has_io_error = false; + ceph_assert(!wait_unfull_flush_promise.has_value()); + metrics.clear(); + return journal_allocator.close(); +} + +void RecordSubmitter::update_state() +{ + if (num_outstanding_io == 0) { + state = state_t::IDLE; + } else if (num_outstanding_io < io_depth_limit) { + state = state_t::PENDING; + } else if (num_outstanding_io == io_depth_limit) { + state = state_t::FULL; + } else { + ceph_abort("fatal error: io-depth overflow"); + } +} + +void RecordSubmitter::decrement_io_with_flush() +{ + LOG_PREFIX(RecordSubmitter::decrement_io_with_flush); + assert(num_outstanding_io > 0); + auto prv_state = state; + --num_outstanding_io; + update_state(); + + if (prv_state == state_t::FULL) { + if (wait_unfull_flush_promise.has_value()) { + DEBUG("{} flush, resolve wait_unfull_flush_promise", get_name()); + assert(!p_current_batch->is_empty()); + assert(wait_available_promise.has_value()); + flush_current_batch(); + wait_unfull_flush_promise->set_value(); + wait_unfull_flush_promise.reset(); + return; + } + } else { + ceph_assert(!wait_unfull_flush_promise.has_value()); + } + + auto needs_flush = ( + !p_current_batch->is_empty() && ( + state == state_t::IDLE || + p_current_batch->get_submit_size().get_fullness() > preferred_fullness || + p_current_batch->needs_flush() + )); + if (needs_flush) { + DEBUG("{} flush", get_name()); + flush_current_batch(); + } +} + +void RecordSubmitter::account_submission( + std::size_t num, + const record_group_size_t& size) +{ + stats.record_group_padding_bytes += + (size.get_mdlength() - size.get_raw_mdlength()); + stats.record_group_metadata_bytes += size.get_raw_mdlength(); + stats.record_group_data_bytes += size.dlength; + stats.record_batch_stats.increment(num); +} + +void RecordSubmitter::finish_submit_batch( + RecordBatch* p_batch, + maybe_result_t maybe_result) +{ + assert(p_batch->is_submitting()); + p_batch->set_result(maybe_result); + free_batch_ptrs.push_back(p_batch); + decrement_io_with_flush(); +} + +void RecordSubmitter::flush_current_batch() +{ + LOG_PREFIX(RecordSubmitter::flush_current_batch); + RecordBatch* p_batch = p_current_batch; + assert(p_batch->is_pending()); + p_current_batch = nullptr; + pop_free_batch(); + + increment_io(); + auto num = p_batch->get_num_records(); + auto [to_write, sizes] = p_batch->encode_batch( + get_committed_to(), journal_allocator.get_nonce()); + DEBUG("{} {} records, {}, committed_to={}, outstanding_io={} ...", + get_name(), num, sizes, get_committed_to(), num_outstanding_io); + account_submission(num, sizes); + std::ignore = journal_allocator.write(std::move(to_write) + ).safe_then([this, p_batch, FNAME, num, sizes=sizes](auto write_result) { + TRACE("{} {} records, {}, write done with {}", + get_name(), num, sizes, write_result); + finish_submit_batch(p_batch, write_result); + }).handle_error( + crimson::ct_error::all_same_way([this, p_batch, FNAME, num, sizes=sizes](auto e) { + ERROR("{} {} records, {}, got error {}", + get_name(), num, sizes, e); + finish_submit_batch(p_batch, std::nullopt); + }) + ).handle_exception([this, p_batch, FNAME, num, sizes=sizes](auto e) { + ERROR("{} {} records, {}, got exception {}", + get_name(), num, sizes, e); + finish_submit_batch(p_batch, std::nullopt); + }); +} + +} diff --git a/src/crimson/os/seastore/journal/record_submitter.h b/src/crimson/os/seastore/journal/record_submitter.h new file mode 100644 index 000000000..eedd2dd8c --- /dev/null +++ b/src/crimson/os/seastore/journal/record_submitter.h @@ -0,0 +1,347 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include <optional> +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/metrics.hh> +#include <seastar/core/shared_future.hh> + +#include "include/buffer.h" + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/segment_manager_group.h" +#include "crimson/os/seastore/segment_seq_allocator.h" + +namespace crimson::os::seastore { + class SegmentProvider; + class JournalTrimmer; +} + +namespace crimson::os::seastore::journal { + +class JournalAllocator { +public: + using base_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + virtual const std::string& get_name() const = 0; + + virtual void update_modify_time(record_t& record) = 0; + + virtual extent_len_t get_block_size() const = 0; + + using close_ertr = base_ertr; + virtual close_ertr::future<> close() = 0; + + virtual segment_nonce_t get_nonce() const = 0; + + using write_ertr = base_ertr; + using write_ret = write_ertr::future<write_result_t>; + virtual write_ret write(ceph::bufferlist&& to_write) = 0; + + virtual bool can_write() const = 0; + + using roll_ertr = base_ertr; + virtual roll_ertr::future<> roll() = 0; + + virtual bool needs_roll(std::size_t length) const = 0; + + using open_ertr = base_ertr; + using open_ret = open_ertr::future<journal_seq_t>; + virtual open_ret open(bool is_mkfs) = 0; + +}; + +/** + * RecordBatch + * + * Maintain a batch of records for submit. + */ +class RecordBatch { + enum class state_t { + EMPTY = 0, + PENDING, + SUBMITTING + }; + +public: + RecordBatch() = default; + RecordBatch(RecordBatch&&) = delete; + RecordBatch(const RecordBatch&) = delete; + RecordBatch& operator=(RecordBatch&&) = delete; + RecordBatch& operator=(const RecordBatch&) = delete; + + bool is_empty() const { + return state == state_t::EMPTY; + } + + bool is_pending() const { + return state == state_t::PENDING; + } + + bool is_submitting() const { + return state == state_t::SUBMITTING; + } + + std::size_t get_index() const { + return index; + } + + std::size_t get_num_records() const { + return pending.get_size(); + } + + std::size_t get_batch_capacity() const { + return batch_capacity; + } + + const record_group_size_t& get_submit_size() const { + assert(state != state_t::EMPTY); + return pending.size; + } + + bool needs_flush() const { + assert(state != state_t::SUBMITTING); + assert(pending.get_size() <= batch_capacity); + if (state == state_t::EMPTY) { + return false; + } else { + assert(state == state_t::PENDING); + return (pending.get_size() >= batch_capacity || + pending.size.get_encoded_length() > batch_flush_size); + } + } + + struct evaluation_t { + record_group_size_t submit_size; + bool is_full; + }; + evaluation_t evaluate_submit( + const record_size_t& rsize, + extent_len_t block_size) const { + assert(!needs_flush()); + auto submit_size = pending.size.get_encoded_length_after( + rsize, block_size); + bool is_full = submit_size.get_encoded_length() > batch_flush_size; + return {submit_size, is_full}; + } + + void initialize(std::size_t i, + std::size_t _batch_capacity, + std::size_t _batch_flush_size) { + ceph_assert(_batch_capacity > 0); + index = i; + batch_capacity = _batch_capacity; + batch_flush_size = _batch_flush_size; + pending.reserve(batch_capacity); + } + + // Add to the batch, the future will be resolved after the batch is + // written. + // + // Set write_result_t::write_length to 0 if the record is not the first one + // in the batch. + using add_pending_ertr = JournalAllocator::write_ertr; + using add_pending_ret = add_pending_ertr::future<record_locator_t>; + add_pending_ret add_pending( + const std::string& name, + record_t&&, + extent_len_t block_size); + + // Encode the batched records for write. + std::pair<ceph::bufferlist, record_group_size_t> encode_batch( + const journal_seq_t& committed_to, + segment_nonce_t segment_nonce); + + // Set the write result and reset for reuse + using maybe_result_t = std::optional<write_result_t>; + void set_result(maybe_result_t maybe_write_end_seq); + + // The fast path that is equivalent to submit a single record as a batch. + // + // Essentially, equivalent to the combined logic of: + // add_pending(), encode_batch() and set_result() above without + // the intervention of the shared io_promise. + // + // Note the current RecordBatch can be reused afterwards. + std::pair<ceph::bufferlist, record_group_size_t> submit_pending_fast( + record_t&&, + extent_len_t block_size, + const journal_seq_t& committed_to, + segment_nonce_t segment_nonce); + +private: + record_group_size_t get_encoded_length_after( + const record_t& record, + extent_len_t block_size) const { + return pending.size.get_encoded_length_after( + record.size, block_size); + } + + state_t state = state_t::EMPTY; + std::size_t index = 0; + std::size_t batch_capacity = 0; + std::size_t batch_flush_size = 0; + + record_group_t pending; + std::size_t submitting_size = 0; + extent_len_t submitting_length = 0; + extent_len_t submitting_mdlength = 0; + + struct promise_result_t { + write_result_t write_result; + extent_len_t mdlength; + }; + using maybe_promise_result_t = std::optional<promise_result_t>; + std::optional<seastar::shared_promise<maybe_promise_result_t> > io_promise; +}; + +/** + * RecordSubmitter + * + * Submit records concurrently with RecordBatch with SegmentAllocator. + * + * Configurations and controls: + * - io_depth: the io-depth limit to SegmentAllocator; + * - batch_capacity: the number limit of records in a RecordBatch; + * - batch_flush_size: the bytes threshold to force flush a RecordBatch to + * control the maximum latency; + * - preferred_fullness: the fullness threshold to flush a RecordBatch; + */ +class RecordSubmitter { + enum class state_t { + IDLE = 0, // outstanding_io == 0 + PENDING, // outstanding_io < io_depth_limit + FULL // outstanding_io == io_depth_limit + // OVERFLOW: outstanding_io > io_depth_limit is impossible + }; + + struct grouped_io_stats { + uint64_t num_io = 0; + uint64_t num_io_grouped = 0; + + void increment(uint64_t num_grouped_io) { + ++num_io; + num_io_grouped += num_grouped_io; + } + }; + + using base_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + +public: + RecordSubmitter(std::size_t io_depth, + std::size_t batch_capacity, + std::size_t batch_flush_size, + double preferred_fullness, + JournalAllocator&); + + const std::string& get_name() const { + return journal_allocator.get_name(); + } + + journal_seq_t get_committed_to() const { + return committed_to; + } + + // whether is available to submit a record + bool is_available() const; + + // wait for available if cannot submit, should check is_available() again + // when the future is resolved. + using wa_ertr = base_ertr; + wa_ertr::future<> wait_available(); + + // when available, check for the submit action + // according to the pending record size + enum class action_t { + ROLL, + SUBMIT_FULL, + SUBMIT_NOT_FULL + }; + action_t check_action(const record_size_t&) const; + + // when available, roll the segment if needed + using roll_segment_ertr = base_ertr; + roll_segment_ertr::future<> roll_segment(); + + // when available, submit the record if possible + using submit_ertr = base_ertr; + using submit_ret = submit_ertr::future<record_locator_t>; + submit_ret submit(record_t&&, bool with_atomic_roll_segment=false); + + void update_committed_to(const journal_seq_t& new_committed_to) { + assert(new_committed_to != JOURNAL_SEQ_NULL); + assert(committed_to == JOURNAL_SEQ_NULL || + committed_to <= new_committed_to); + committed_to = new_committed_to; + } + + // open for write, generate the correct print name, and register metrics + using open_ertr = base_ertr; + using open_ret = open_ertr::future<journal_seq_t>; + open_ret open(bool is_mkfs); + + using close_ertr = base_ertr; + close_ertr::future<> close(); + +private: + void update_state(); + + void increment_io() { + ++num_outstanding_io; + stats.io_depth_stats.increment(num_outstanding_io); + update_state(); + } + + void decrement_io_with_flush(); + + void pop_free_batch() { + assert(p_current_batch == nullptr); + assert(!free_batch_ptrs.empty()); + p_current_batch = free_batch_ptrs.front(); + assert(p_current_batch->is_empty()); + assert(p_current_batch == &batches[p_current_batch->get_index()]); + free_batch_ptrs.pop_front(); + } + + void account_submission(std::size_t, const record_group_size_t&); + + using maybe_result_t = RecordBatch::maybe_result_t; + void finish_submit_batch(RecordBatch*, maybe_result_t); + + void flush_current_batch(); + + state_t state = state_t::IDLE; + std::size_t num_outstanding_io = 0; + std::size_t io_depth_limit; + double preferred_fullness; + + JournalAllocator& journal_allocator; + // committed_to may be in a previous journal segment + journal_seq_t committed_to = JOURNAL_SEQ_NULL; + + std::unique_ptr<RecordBatch[]> batches; + // should not be nullptr after constructed + RecordBatch* p_current_batch = nullptr; + seastar::circular_buffer<RecordBatch*> free_batch_ptrs; + + // blocked for rolling or lack of resource + std::optional<seastar::shared_promise<> > wait_available_promise; + bool has_io_error = false; + // when needs flush but io depth is full, + // wait for decrement_io_with_flush() + std::optional<seastar::promise<> > wait_unfull_flush_promise; + + struct { + grouped_io_stats record_batch_stats; + grouped_io_stats io_depth_stats; + uint64_t record_group_padding_bytes = 0; + uint64_t record_group_metadata_bytes = 0; + uint64_t record_group_data_bytes = 0; + } stats; + seastar::metrics::metric_group metrics; +}; + +} diff --git a/src/crimson/os/seastore/journal/segment_allocator.cc b/src/crimson/os/seastore/journal/segment_allocator.cc new file mode 100644 index 000000000..61e1be585 --- /dev/null +++ b/src/crimson/os/seastore/journal/segment_allocator.cc @@ -0,0 +1,283 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#include "segment_allocator.h" + +#include <fmt/format.h> +#include <fmt/os.h> + +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/async_cleaner.h" + +SET_SUBSYS(seastore_journal); + +namespace crimson::os::seastore::journal { + +SegmentAllocator::SegmentAllocator( + JournalTrimmer *trimmer, + data_category_t category, + rewrite_gen_t gen, + SegmentProvider &sp, + SegmentSeqAllocator &ssa) + : print_name{fmt::format("{}_G{}", category, gen)}, + type{trimmer == nullptr ? + segment_type_t::OOL : + segment_type_t::JOURNAL}, + category{category}, + gen{gen}, + segment_provider{sp}, + sm_group{*sp.get_segment_manager_group()}, + segment_seq_allocator(ssa), + trimmer{trimmer} +{ + reset(); +} + +segment_nonce_t calc_new_nonce( + segment_type_t type, + uint32_t crc, + unsigned char const *data, + unsigned length) +{ + crc &= std::numeric_limits<uint32_t>::max() >> 1; + crc |= static_cast<uint32_t>(type) << 31; + return ceph_crc32c(crc, data, length); +} + +SegmentAllocator::open_ret +SegmentAllocator::do_open(bool is_mkfs) +{ + LOG_PREFIX(SegmentAllocator::do_open); + ceph_assert(!current_segment); + segment_seq_t new_segment_seq = + segment_seq_allocator.get_and_inc_next_segment_seq(); + auto meta = sm_group.get_meta(); + current_segment_nonce = calc_new_nonce( + type, + new_segment_seq, + reinterpret_cast<const unsigned char *>(meta.seastore_id.bytes()), + sizeof(meta.seastore_id.uuid)); + auto new_segment_id = segment_provider.allocate_segment( + new_segment_seq, type, category, gen); + ceph_assert(new_segment_id != NULL_SEG_ID); + return sm_group.open(new_segment_id + ).handle_error( + open_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in SegmentAllocator::do_open open" + } + ).safe_then([this, is_mkfs, FNAME, new_segment_seq](auto sref) { + // initialize new segment + segment_id_t segment_id = sref->get_segment_id(); + journal_seq_t dirty_tail; + journal_seq_t alloc_tail; + if (type == segment_type_t::JOURNAL) { + dirty_tail = trimmer->get_dirty_tail(); + alloc_tail = trimmer->get_alloc_tail(); + if (is_mkfs) { + ceph_assert(dirty_tail == JOURNAL_SEQ_NULL); + ceph_assert(alloc_tail == JOURNAL_SEQ_NULL); + auto mkfs_seq = journal_seq_t{ + new_segment_seq, + paddr_t::make_seg_paddr(segment_id, 0) + }; + dirty_tail = mkfs_seq; + alloc_tail = mkfs_seq; + } else { + ceph_assert(dirty_tail != JOURNAL_SEQ_NULL); + ceph_assert(alloc_tail != JOURNAL_SEQ_NULL); + } + } else { // OOL + ceph_assert(!is_mkfs); + dirty_tail = JOURNAL_SEQ_NULL; + alloc_tail = JOURNAL_SEQ_NULL; + } + auto header = segment_header_t{ + new_segment_seq, + segment_id, + dirty_tail, + alloc_tail, + current_segment_nonce, + type, + category, + gen}; + INFO("{} writing header {}", print_name, header); + + auto header_length = get_block_size(); + bufferlist bl; + encode(header, bl); + bufferptr bp(ceph::buffer::create_page_aligned(header_length)); + bp.zero(); + auto iter = bl.cbegin(); + iter.copy(bl.length(), bp.c_str()); + bl.clear(); + bl.append(bp); + + ceph_assert(sref->get_write_ptr() == 0); + assert((unsigned)header_length == bl.length()); + written_to = header_length; + auto new_journal_seq = journal_seq_t{ + new_segment_seq, + paddr_t::make_seg_paddr(segment_id, written_to)}; + segment_provider.update_segment_avail_bytes( + type, new_journal_seq.offset); + return sref->write(0, std::move(bl) + ).handle_error( + open_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in SegmentAllocator::do_open write" + } + ).safe_then([this, + FNAME, + new_journal_seq, + sref=std::move(sref)]() mutable { + ceph_assert(!current_segment); + current_segment = std::move(sref); + DEBUG("{} rolled new segment id={}", + print_name, current_segment->get_segment_id()); + ceph_assert(new_journal_seq.segment_seq == + segment_provider.get_seg_info(current_segment->get_segment_id()).seq); + return new_journal_seq; + }); + }); +} + +SegmentAllocator::open_ret +SegmentAllocator::open(bool is_mkfs) +{ + LOG_PREFIX(SegmentAllocator::open); + auto& device_ids = sm_group.get_device_ids(); + ceph_assert(device_ids.size()); + std::ostringstream oss; + for (auto& device_id : device_ids) { + oss << device_id_printer_t{device_id} << "_"; + } + oss << fmt::format("{}_G{}", category, gen); + print_name = oss.str(); + + DEBUG("{}", print_name); + return do_open(is_mkfs); +} + +SegmentAllocator::roll_ertr::future<> +SegmentAllocator::roll() +{ + ceph_assert(can_write()); + return close_segment().safe_then([this] { + return do_open(false).discard_result(); + }); +} + +SegmentAllocator::write_ret +SegmentAllocator::write(ceph::bufferlist&& to_write) +{ + LOG_PREFIX(SegmentAllocator::write); + assert(can_write()); + auto write_length = to_write.length(); + auto write_start_offset = written_to; + auto write_start_seq = journal_seq_t{ + segment_provider.get_seg_info(current_segment->get_segment_id()).seq, + paddr_t::make_seg_paddr( + current_segment->get_segment_id(), write_start_offset) + }; + TRACE("{} {}~{}", print_name, write_start_seq, write_length); + assert(write_length > 0); + assert((write_length % get_block_size()) == 0); + assert(!needs_roll(write_length)); + + auto write_result = write_result_t{ + write_start_seq, + write_length + }; + written_to += write_length; + segment_provider.update_segment_avail_bytes( + type, + paddr_t::make_seg_paddr( + current_segment->get_segment_id(), written_to) + ); + return current_segment->write( + write_start_offset, std::move(to_write) + ).handle_error( + write_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in SegmentAllocator::write" + } + ).safe_then([write_result, cs=current_segment] { + return write_result; + }); +} + +SegmentAllocator::close_ertr::future<> +SegmentAllocator::close() +{ + return [this] { + LOG_PREFIX(SegmentAllocator::close); + if (current_segment) { + DEBUG("{} close current segment", print_name); + return close_segment(); + } else { + INFO("{} no current segment", print_name); + return close_segment_ertr::now(); + } + }().finally([this] { + reset(); + }); +} + +SegmentAllocator::close_segment_ertr::future<> +SegmentAllocator::close_segment() +{ + LOG_PREFIX(SegmentAllocator::close_segment); + assert(can_write()); + // Note: make sure no one can access the current segment once closing + auto seg_to_close = std::move(current_segment); + auto close_segment_id = seg_to_close->get_segment_id(); + auto close_seg_info = segment_provider.get_seg_info(close_segment_id); + ceph_assert((close_seg_info.modify_time == NULL_TIME && + close_seg_info.num_extents == 0) || + (close_seg_info.modify_time != NULL_TIME && + close_seg_info.num_extents != 0)); + auto tail = segment_tail_t{ + close_seg_info.seq, + close_segment_id, + current_segment_nonce, + type, + timepoint_to_mod(close_seg_info.modify_time), + close_seg_info.num_extents}; + ceph::bufferlist bl; + encode(tail, bl); + INFO("{} close segment {}, written_to={}", + print_name, + tail, + written_to); + + bufferptr bp(ceph::buffer::create_page_aligned(get_block_size())); + bp.zero(); + auto iter = bl.cbegin(); + iter.copy(bl.length(), bp.c_str()); + bl.clear(); + bl.append(bp); + + assert(bl.length() == sm_group.get_rounded_tail_length()); + + auto p_seg_to_close = seg_to_close.get(); + return p_seg_to_close->advance_wp( + sm_group.get_segment_size() - sm_group.get_rounded_tail_length() + ).safe_then([this, FNAME, bl=std::move(bl), p_seg_to_close]() mutable { + DEBUG("Writing tail info to segment {}", p_seg_to_close->get_segment_id()); + return p_seg_to_close->write( + sm_group.get_segment_size() - sm_group.get_rounded_tail_length(), + std::move(bl)); + }).safe_then([p_seg_to_close] { + return p_seg_to_close->close(); + }).safe_then([this, seg_to_close=std::move(seg_to_close)] { + segment_provider.close_segment(seg_to_close->get_segment_id()); + }).handle_error( + close_segment_ertr::pass_further{}, + crimson::ct_error::assert_all { + "Invalid error in SegmentAllocator::close_segment" + }); + +} + +} diff --git a/src/crimson/os/seastore/journal/segment_allocator.h b/src/crimson/os/seastore/journal/segment_allocator.h new file mode 100644 index 000000000..292c23070 --- /dev/null +++ b/src/crimson/os/seastore/journal/segment_allocator.h @@ -0,0 +1,131 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include <optional> +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/metrics.hh> +#include <seastar/core/shared_future.hh> + +#include "include/buffer.h" + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/segment_manager_group.h" +#include "crimson/os/seastore/segment_seq_allocator.h" +#include "crimson/os/seastore/journal/record_submitter.h" +#include "crimson/os/seastore/async_cleaner.h" + +namespace crimson::os::seastore { + class SegmentProvider; + class JournalTrimmer; +} + +namespace crimson::os::seastore::journal { + +/** + * SegmentAllocator + * + * Maintain an available segment for writes. + */ +class SegmentAllocator : public JournalAllocator { + + public: + // SegmentAllocator specific methods + SegmentAllocator(JournalTrimmer *trimmer, + data_category_t category, + rewrite_gen_t gen, + SegmentProvider &sp, + SegmentSeqAllocator &ssa); + + segment_id_t get_segment_id() const { + assert(can_write()); + return current_segment->get_segment_id(); + } + + extent_len_t get_max_write_length() const { + return sm_group.get_segment_size() - + sm_group.get_rounded_header_length() - + sm_group.get_rounded_tail_length(); + } + + public: + // overriding methods + const std::string& get_name() const final { + return print_name; + } + + extent_len_t get_block_size() const final { + return sm_group.get_block_size(); + } + + bool can_write() const final { + return !!current_segment; + } + + segment_nonce_t get_nonce() const final { + assert(can_write()); + return current_segment_nonce; + } + + // returns true iff the current segment has insufficient space + bool needs_roll(std::size_t length) const final { + assert(can_write()); + assert(current_segment->get_write_capacity() == + sm_group.get_segment_size()); + auto write_capacity = current_segment->get_write_capacity() - + sm_group.get_rounded_tail_length(); + return length + written_to > std::size_t(write_capacity); + } + + // open for write and generate the correct print name + open_ret open(bool is_mkfs) final; + + // close the current segment and initialize next one + roll_ertr::future<> roll() final; + + // write the buffer, return the write result + // + // May be called concurrently, but writes may complete in any order. + // If rolling/opening, no write is allowed. + write_ret write(ceph::bufferlist&& to_write) final; + + using close_ertr = base_ertr; + close_ertr::future<> close() final; + + void update_modify_time(record_t& record) final { + segment_provider.update_modify_time( + get_segment_id(), + record.modify_time, + record.extents.size()); + } + + private: + open_ret do_open(bool is_mkfs); + + void reset() { + current_segment.reset(); + written_to = 0; + + current_segment_nonce = 0; + } + + using close_segment_ertr = base_ertr; + close_segment_ertr::future<> close_segment(); + + // device id is not available during construction, + // so generate the print_name later. + std::string print_name; + const segment_type_t type; // JOURNAL or OOL + const data_category_t category; + const rewrite_gen_t gen; + SegmentProvider &segment_provider; + SegmentManagerGroup &sm_group; + SegmentRef current_segment; + segment_off_t written_to; + SegmentSeqAllocator &segment_seq_allocator; + segment_nonce_t current_segment_nonce; + JournalTrimmer *trimmer; +}; + +} diff --git a/src/crimson/os/seastore/journal/segmented_journal.cc b/src/crimson/os/seastore/journal/segmented_journal.cc new file mode 100644 index 000000000..58df91374 --- /dev/null +++ b/src/crimson/os/seastore/journal/segmented_journal.cc @@ -0,0 +1,433 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <iostream> + +#include <boost/iterator/counting_iterator.hpp> + +#include "include/intarith.h" + +#include "segmented_journal.h" + +#include "crimson/common/config_proxy.h" +#include "crimson/os/seastore/logging.h" + +SET_SUBSYS(seastore_journal); + +/* + * format: + * - H<handle-addr> information + * + * levels: + * - INFO: major initiation, closing, rolling and replay operations + * - DEBUG: INFO details, major submit operations + * - TRACE: DEBUG details + */ + +namespace crimson::os::seastore::journal { + +SegmentedJournal::SegmentedJournal( + SegmentProvider &segment_provider, + JournalTrimmer &trimmer) + : segment_seq_allocator( + new SegmentSeqAllocator(segment_type_t::JOURNAL)), + journal_segment_allocator(&trimmer, + data_category_t::METADATA, + INLINE_GENERATION, + segment_provider, + *segment_seq_allocator), + record_submitter(crimson::common::get_conf<uint64_t>( + "seastore_journal_iodepth_limit"), + crimson::common::get_conf<uint64_t>( + "seastore_journal_batch_capacity"), + crimson::common::get_conf<Option::size_t>( + "seastore_journal_batch_flush_size"), + crimson::common::get_conf<double>( + "seastore_journal_batch_preferred_fullness"), + journal_segment_allocator), + sm_group(*segment_provider.get_segment_manager_group()), + trimmer{trimmer} +{ +} + +SegmentedJournal::open_for_mkfs_ret +SegmentedJournal::open_for_mkfs() +{ + return record_submitter.open(true); +} + +SegmentedJournal::open_for_mount_ret +SegmentedJournal::open_for_mount() +{ + return record_submitter.open(false); +} + +SegmentedJournal::close_ertr::future<> SegmentedJournal::close() +{ + LOG_PREFIX(Journal::close); + INFO("closing, committed_to={}", + record_submitter.get_committed_to()); + return record_submitter.close(); +} + +SegmentedJournal::prep_replay_segments_fut +SegmentedJournal::prep_replay_segments( + std::vector<std::pair<segment_id_t, segment_header_t>> segments) +{ + LOG_PREFIX(Journal::prep_replay_segments); + if (segments.empty()) { + ERROR("no journal segments for replay"); + return crimson::ct_error::input_output_error::make(); + } + std::sort( + segments.begin(), + segments.end(), + [](const auto <, const auto &rt) { + return lt.second.segment_seq < + rt.second.segment_seq; + }); + + segment_seq_allocator->set_next_segment_seq( + segments.rbegin()->second.segment_seq + 1); + std::for_each( + segments.begin(), + segments.end(), + [FNAME](auto &seg) + { + if (seg.first != seg.second.physical_segment_id || + seg.second.get_type() != segment_type_t::JOURNAL) { + ERROR("illegal journal segment for replay -- {}", seg.second); + ceph_abort(); + } + }); + + auto last_segment_id = segments.rbegin()->first; + auto last_header = segments.rbegin()->second; + return scan_last_segment(last_segment_id, last_header + ).safe_then([this, FNAME, segments=std::move(segments)] { + INFO("dirty_tail={}, alloc_tail={}", + trimmer.get_dirty_tail(), + trimmer.get_alloc_tail()); + auto journal_tail = trimmer.get_journal_tail(); + auto journal_tail_paddr = journal_tail.offset; + ceph_assert(journal_tail != JOURNAL_SEQ_NULL); + ceph_assert(journal_tail_paddr != P_ADDR_NULL); + auto from = std::find_if( + segments.begin(), + segments.end(), + [&journal_tail_paddr](const auto &seg) -> bool { + auto& seg_addr = journal_tail_paddr.as_seg_paddr(); + return seg.first == seg_addr.get_segment_id(); + }); + if (from->second.segment_seq != journal_tail.segment_seq) { + ERROR("journal_tail {} does not match {}", + journal_tail, from->second); + ceph_abort(); + } + + auto num_segments = segments.end() - from; + INFO("{} segments to replay", num_segments); + auto ret = replay_segments_t(num_segments); + std::transform( + from, segments.end(), ret.begin(), + [this](const auto &p) { + auto ret = journal_seq_t{ + p.second.segment_seq, + paddr_t::make_seg_paddr( + p.first, + sm_group.get_block_size()) + }; + return std::make_pair(ret, p.second); + }); + ret[0].first.offset = journal_tail_paddr; + return prep_replay_segments_fut( + replay_ertr::ready_future_marker{}, + std::move(ret)); + }); +} + +SegmentedJournal::scan_last_segment_ertr::future<> +SegmentedJournal::scan_last_segment( + const segment_id_t &segment_id, + const segment_header_t &segment_header) +{ + LOG_PREFIX(SegmentedJournal::scan_last_segment); + assert(segment_id == segment_header.physical_segment_id); + trimmer.update_journal_tails( + segment_header.dirty_tail, segment_header.alloc_tail); + auto seq = journal_seq_t{ + segment_header.segment_seq, + paddr_t::make_seg_paddr(segment_id, 0) + }; + INFO("scanning journal tail deltas -- {}", segment_header); + return seastar::do_with( + scan_valid_records_cursor(seq), + SegmentManagerGroup::found_record_handler_t( + [FNAME, this]( + record_locator_t locator, + const record_group_header_t& record_group_header, + const bufferlist& mdbuf + ) -> SegmentManagerGroup::scan_valid_records_ertr::future<> + { + DEBUG("decoding {} at {}", record_group_header, locator); + bool has_tail_delta = false; + auto maybe_headers = try_decode_record_headers( + record_group_header, mdbuf); + if (!maybe_headers) { + // This should be impossible, we did check the crc on the mdbuf + ERROR("unable to decode headers from {} at {}", + record_group_header, locator); + ceph_abort(); + } + for (auto &record_header : *maybe_headers) { + ceph_assert(is_valid_transaction(record_header.type)); + if (is_background_transaction(record_header.type)) { + has_tail_delta = true; + } + } + if (has_tail_delta) { + bool found_delta = false; + auto maybe_record_deltas_list = try_decode_deltas( + record_group_header, mdbuf, locator.record_block_base); + if (!maybe_record_deltas_list) { + ERROR("unable to decode deltas from {} at {}", + record_group_header, locator); + ceph_abort(); + } + for (auto &record_deltas : *maybe_record_deltas_list) { + for (auto &[ctime, delta] : record_deltas.deltas) { + if (delta.type == extent_types_t::JOURNAL_TAIL) { + found_delta = true; + journal_tail_delta_t tail_delta; + decode(tail_delta, delta.bl); + auto start_seq = locator.write_result.start_seq; + DEBUG("got {}, at {}", tail_delta, start_seq); + ceph_assert(tail_delta.dirty_tail != JOURNAL_SEQ_NULL); + ceph_assert(tail_delta.alloc_tail != JOURNAL_SEQ_NULL); + trimmer.update_journal_tails( + tail_delta.dirty_tail, tail_delta.alloc_tail); + } + } + } + ceph_assert(found_delta); + } + return seastar::now(); + }), + [this, nonce=segment_header.segment_nonce](auto &cursor, auto &handler) + { + return sm_group.scan_valid_records( + cursor, + nonce, + std::numeric_limits<std::size_t>::max(), + handler).discard_result(); + }); +} + +SegmentedJournal::replay_ertr::future<> +SegmentedJournal::replay_segment( + journal_seq_t seq, + segment_header_t header, + delta_handler_t &handler, + replay_stats_t &stats) +{ + LOG_PREFIX(Journal::replay_segment); + INFO("starting at {} -- {}", seq, header); + return seastar::do_with( + scan_valid_records_cursor(seq), + SegmentManagerGroup::found_record_handler_t( + [&handler, this, &stats]( + record_locator_t locator, + const record_group_header_t& header, + const bufferlist& mdbuf) + -> SegmentManagerGroup::scan_valid_records_ertr::future<> + { + LOG_PREFIX(Journal::replay_segment); + ++stats.num_record_groups; + auto maybe_record_deltas_list = try_decode_deltas( + header, mdbuf, locator.record_block_base); + if (!maybe_record_deltas_list) { + // This should be impossible, we did check the crc on the mdbuf + ERROR("unable to decode deltas for record {} at {}", + header, locator); + return crimson::ct_error::input_output_error::make(); + } + + return seastar::do_with( + std::move(*maybe_record_deltas_list), + [write_result=locator.write_result, + this, + FNAME, + &handler, + &stats](auto& record_deltas_list) + { + return crimson::do_for_each( + record_deltas_list, + [write_result, + this, + FNAME, + &handler, + &stats](record_deltas_t& record_deltas) + { + ++stats.num_records; + auto locator = record_locator_t{ + record_deltas.record_block_base, + write_result + }; + DEBUG("processing {} deltas at block_base {}", + record_deltas.deltas.size(), + locator); + return crimson::do_for_each( + record_deltas.deltas, + [locator, + this, + &handler, + &stats](auto &p) + { + auto& modify_time = p.first; + auto& delta = p.second; + return handler( + locator, + delta, + trimmer.get_dirty_tail(), + trimmer.get_alloc_tail(), + modify_time + ).safe_then([&stats, delta_type=delta.type](bool is_applied) { + if (is_applied) { + // see Cache::replay_delta() + assert(delta_type != extent_types_t::JOURNAL_TAIL); + if (delta_type == extent_types_t::ALLOC_INFO) { + ++stats.num_alloc_deltas; + } else { + ++stats.num_dirty_deltas; + } + } + }); + }); + }); + }); + }), + [=, this](auto &cursor, auto &dhandler) { + return sm_group.scan_valid_records( + cursor, + header.segment_nonce, + std::numeric_limits<size_t>::max(), + dhandler).safe_then([](auto){} + ).handle_error( + replay_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "shouldn't meet with any other error other replay_ertr" + } + ); + } + ); +} + +SegmentedJournal::replay_ret SegmentedJournal::replay( + delta_handler_t &&delta_handler) +{ + LOG_PREFIX(Journal::replay); + return sm_group.find_journal_segment_headers( + ).safe_then([this, FNAME, delta_handler=std::move(delta_handler)] + (auto &&segment_headers) mutable -> replay_ret { + INFO("got {} segments", segment_headers.size()); + return seastar::do_with( + std::move(delta_handler), + replay_segments_t(), + replay_stats_t(), + [this, segment_headers=std::move(segment_headers), FNAME] + (auto &handler, auto &segments, auto &stats) mutable -> replay_ret { + return prep_replay_segments(std::move(segment_headers) + ).safe_then([this, &handler, &segments, &stats](auto replay_segs) mutable { + segments = std::move(replay_segs); + return crimson::do_for_each(segments,[this, &handler, &stats](auto i) mutable { + return replay_segment(i.first, i.second, handler, stats); + }); + }).safe_then([&stats, FNAME] { + INFO("replay done, record_groups={}, records={}, " + "alloc_deltas={}, dirty_deltas={}", + stats.num_record_groups, + stats.num_records, + stats.num_alloc_deltas, + stats.num_dirty_deltas); + }); + }); + }); +} + +seastar::future<> SegmentedJournal::flush(OrderingHandle &handle) +{ + LOG_PREFIX(SegmentedJournal::flush); + DEBUG("H{} flush ...", (void*)&handle); + assert(write_pipeline); + return handle.enter(write_pipeline->device_submission + ).then([this, &handle] { + return handle.enter(write_pipeline->finalize); + }).then([FNAME, &handle] { + DEBUG("H{} flush done", (void*)&handle); + }); +} + +SegmentedJournal::submit_record_ret +SegmentedJournal::do_submit_record( + record_t &&record, + OrderingHandle &handle) +{ + LOG_PREFIX(SegmentedJournal::do_submit_record); + if (!record_submitter.is_available()) { + DEBUG("H{} wait ...", (void*)&handle); + return record_submitter.wait_available( + ).safe_then([this, record=std::move(record), &handle]() mutable { + return do_submit_record(std::move(record), handle); + }); + } + auto action = record_submitter.check_action(record.size); + if (action == RecordSubmitter::action_t::ROLL) { + DEBUG("H{} roll, unavailable ...", (void*)&handle); + return record_submitter.roll_segment( + ).safe_then([this, record=std::move(record), &handle]() mutable { + return do_submit_record(std::move(record), handle); + }); + } else { // SUBMIT_FULL/NOT_FULL + DEBUG("H{} submit {} ...", + (void*)&handle, + action == RecordSubmitter::action_t::SUBMIT_FULL ? + "FULL" : "NOT_FULL"); + auto submit_fut = record_submitter.submit(std::move(record)); + return handle.enter(write_pipeline->device_submission + ).then([submit_fut=std::move(submit_fut)]() mutable { + return std::move(submit_fut); + }).safe_then([FNAME, this, &handle](record_locator_t result) { + return handle.enter(write_pipeline->finalize + ).then([FNAME, this, result, &handle] { + DEBUG("H{} finish with {}", (void*)&handle, result); + auto new_committed_to = result.write_result.get_end_seq(); + record_submitter.update_committed_to(new_committed_to); + return result; + }); + }); + } +} + +SegmentedJournal::submit_record_ret +SegmentedJournal::submit_record( + record_t &&record, + OrderingHandle &handle) +{ + LOG_PREFIX(SegmentedJournal::submit_record); + DEBUG("H{} {} start ...", (void*)&handle, record); + assert(write_pipeline); + auto expected_size = record_group_size_t( + record.size, + journal_segment_allocator.get_block_size() + ).get_encoded_length(); + auto max_record_length = journal_segment_allocator.get_max_write_length(); + if (expected_size > max_record_length) { + ERROR("H{} {} exceeds max record size {}", + (void*)&handle, record, max_record_length); + return crimson::ct_error::erange::make(); + } + + return do_submit_record(std::move(record), handle); +} + +} diff --git a/src/crimson/os/seastore/journal/segmented_journal.h b/src/crimson/os/seastore/journal/segmented_journal.h new file mode 100644 index 000000000..3d580817c --- /dev/null +++ b/src/crimson/os/seastore/journal/segmented_journal.h @@ -0,0 +1,105 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <seastar/core/future.hh> + +#include "include/ceph_assert.h" +#include "include/buffer.h" +#include "include/denc.h" + +#include "crimson/os/seastore/async_cleaner.h" +#include "crimson/os/seastore/journal.h" +#include "crimson/os/seastore/segment_manager_group.h" +#include "crimson/os/seastore/ordering_handle.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/osd/exceptions.h" +#include "segment_allocator.h" +#include "crimson/os/seastore/segment_seq_allocator.h" +#include "record_submitter.h" + +namespace crimson::os::seastore::journal { +/** + * Manages stream of atomically written records to a SegmentManager. + */ +class SegmentedJournal : public Journal { +public: + SegmentedJournal( + SegmentProvider &segment_provider, + JournalTrimmer &trimmer); + ~SegmentedJournal() {} + + JournalTrimmer &get_trimmer() final { + return trimmer; + } + + open_for_mkfs_ret open_for_mkfs() final; + + open_for_mount_ret open_for_mount() final; + + close_ertr::future<> close() final; + + submit_record_ret submit_record( + record_t &&record, + OrderingHandle &handle) final; + + seastar::future<> flush(OrderingHandle &handle) final; + + replay_ret replay(delta_handler_t &&delta_handler) final; + + void set_write_pipeline(WritePipeline *_write_pipeline) final { + write_pipeline = _write_pipeline; + } + + journal_type_t get_type() final { + return journal_type_t::SEGMENTED; + } + seastar::future<> finish_commit(transaction_type_t type) { + return seastar::now(); + } + +private: + submit_record_ret do_submit_record( + record_t &&record, + OrderingHandle &handle + ); + + SegmentSeqAllocatorRef segment_seq_allocator; + SegmentAllocator journal_segment_allocator; + RecordSubmitter record_submitter; + SegmentManagerGroup &sm_group; + JournalTrimmer &trimmer; + WritePipeline* write_pipeline = nullptr; + + /// return ordered vector of segments to replay + using replay_segments_t = std::vector< + std::pair<journal_seq_t, segment_header_t>>; + using prep_replay_segments_fut = replay_ertr::future< + replay_segments_t>; + prep_replay_segments_fut prep_replay_segments( + std::vector<std::pair<segment_id_t, segment_header_t>> segments); + + /// scan the last segment for tail deltas + using scan_last_segment_ertr = replay_ertr; + scan_last_segment_ertr::future<> scan_last_segment( + const segment_id_t&, const segment_header_t&); + + struct replay_stats_t { + std::size_t num_record_groups = 0; + std::size_t num_records = 0; + std::size_t num_alloc_deltas = 0; + std::size_t num_dirty_deltas = 0; + }; + + /// replays records starting at start through end of segment + replay_ertr::future<> + replay_segment( + journal_seq_t start, ///< [in] starting addr, seq + segment_header_t header, ///< [in] segment header + delta_handler_t &delta_handler, ///< [in] processes deltas in order + replay_stats_t &stats ///< [out] replay stats + ); +}; + +} diff --git a/src/crimson/os/seastore/lba_manager.cc b/src/crimson/os/seastore/lba_manager.cc new file mode 100644 index 000000000..d113bbd1e --- /dev/null +++ b/src/crimson/os/seastore/lba_manager.cc @@ -0,0 +1,31 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/lba_manager.h" +#include "crimson/os/seastore/lba_manager/btree/btree_lba_manager.h" + +namespace crimson::os::seastore { + +LBAManager::update_mappings_ret +LBAManager::update_mappings( + Transaction& t, + const std::list<LogicalCachedExtentRef>& extents) +{ + return trans_intr::do_for_each(extents, + [this, &t](auto &extent) { + return update_mapping( + t, + extent->get_laddr(), + extent->get_prior_paddr_and_reset(), + extent->get_paddr(), + nullptr // all the extents should have already been + // added to the fixed_kv_btree + ); + }); +} + +LBAManagerRef lba_manager::create_lba_manager(Cache &cache) { + return LBAManagerRef(new btree::BtreeLBAManager(cache)); +} + +} diff --git a/src/crimson/os/seastore/lba_manager.h b/src/crimson/os/seastore/lba_manager.h new file mode 100644 index 000000000..d7adf2304 --- /dev/null +++ b/src/crimson/os/seastore/lba_manager.h @@ -0,0 +1,237 @@ +// -*- 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_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include <seastar/core/future.hh> + +#include "include/ceph_assert.h" +#include "include/buffer_fwd.h" +#include "include/interval_set.h" +#include "common/interval_map.h" + +#include "crimson/osd/exceptions.h" + +#include "crimson/os/seastore/cache.h" +#include "crimson/os/seastore/seastore_types.h" + +namespace crimson::os::seastore { + +/** + * Abstract interface for managing the logical to physical mapping + */ +class LBAManager { +public: + using base_iertr = Cache::base_iertr; + + using mkfs_iertr = base_iertr; + using mkfs_ret = mkfs_iertr::future<>; + virtual mkfs_ret mkfs( + Transaction &t + ) = 0; + + /** + * Fetches mappings for laddr_t in range [offset, offset + len) + * + * Future will not resolve until all pins have resolved (set_paddr called) + * For indirect lba mappings, get_mappings will always retrieve the original + * lba value. + */ + using get_mappings_iertr = base_iertr; + using get_mappings_ret = get_mappings_iertr::future<lba_pin_list_t>; + virtual get_mappings_ret get_mappings( + Transaction &t, + laddr_t offset, extent_len_t length) = 0; + + /** + * Fetches mappings for a list of laddr_t in range [offset, offset + len) + * + * Future will not resolve until all pins have resolved (set_paddr called) + * For indirect lba mappings, get_mappings will always retrieve the original + * lba value. + */ + virtual get_mappings_ret get_mappings( + Transaction &t, + laddr_list_t &&extent_lisk) = 0; + + /** + * Fetches the mapping for laddr_t + * + * Future will not resolve until the pin has resolved (set_paddr called) + * For indirect lba mappings, get_mapping will always retrieve the original + * lba value. + */ + using get_mapping_iertr = base_iertr::extend< + crimson::ct_error::enoent>; + using get_mapping_ret = get_mapping_iertr::future<LBAMappingRef>; + virtual get_mapping_ret get_mapping( + Transaction &t, + laddr_t offset) = 0; + + /** + * Allocates a new mapping referenced by LBARef + * + * Offset will be relative to the block offset of the record + * This mapping will block from transaction submission until set_paddr + * is called on the LBAMapping. + */ + using alloc_extent_iertr = base_iertr; + using alloc_extent_ret = alloc_extent_iertr::future<LBAMappingRef>; + virtual alloc_extent_ret alloc_extent( + Transaction &t, + laddr_t hint, + extent_len_t len, + paddr_t addr, + LogicalCachedExtent &nextent) = 0; + + virtual alloc_extent_ret clone_extent( + Transaction &t, + laddr_t hint, + extent_len_t len, + laddr_t intermediate_key, + paddr_t actual_addr, + laddr_t intermediate_base) = 0; + + virtual alloc_extent_ret reserve_region( + Transaction &t, + laddr_t hint, + extent_len_t len) = 0; + + struct ref_update_result_t { + unsigned refcount = 0; + pladdr_t addr; + extent_len_t length = 0; + }; + using ref_iertr = base_iertr::extend< + crimson::ct_error::enoent>; + using ref_ret = ref_iertr::future<ref_update_result_t>; + + /** + * Decrements ref count on extent + * + * @return returns resulting refcount + */ + virtual ref_ret decref_extent( + Transaction &t, + laddr_t addr, + bool cascade_remove) = 0; + + /** + * Increments ref count on extent + * + * @return returns resulting refcount + */ + virtual ref_ret incref_extent( + Transaction &t, + laddr_t addr) = 0; + + /** + * Increments ref count on extent + * + * @return returns resulting refcount + */ + virtual ref_ret incref_extent( + Transaction &t, + laddr_t addr, + int delta) = 0; + + /** + * Should be called after replay on each cached extent. + * Implementation must initialize the LBAMapping on any + * LogicalCachedExtent's and may also read in any dependent + * structures, etc. + * + * @return returns whether the extent is alive + */ + using init_cached_extent_iertr = base_iertr; + using init_cached_extent_ret = init_cached_extent_iertr::future<bool>; + virtual init_cached_extent_ret init_cached_extent( + Transaction &t, + CachedExtentRef e) = 0; + + using check_child_trackers_ret = base_iertr::future<>; + virtual check_child_trackers_ret check_child_trackers(Transaction &t) = 0; + + /** + * Calls f for each mapping in [begin, end) + */ + using scan_mappings_iertr = base_iertr; + using scan_mappings_ret = scan_mappings_iertr::future<>; + using scan_mappings_func_t = std::function< + void(laddr_t, paddr_t, extent_len_t)>; + virtual scan_mappings_ret scan_mappings( + Transaction &t, + laddr_t begin, + laddr_t end, + scan_mappings_func_t &&f) = 0; + + /** + * rewrite_extent + * + * rewrite extent into passed transaction + */ + using rewrite_extent_iertr = base_iertr; + using rewrite_extent_ret = rewrite_extent_iertr::future<>; + virtual rewrite_extent_ret rewrite_extent( + Transaction &t, + CachedExtentRef extent) = 0; + + /** + * update_mapping + * + * update lba mapping for a delayed allocated extent + */ + using update_mapping_iertr = base_iertr; + using update_mapping_ret = base_iertr::future<>; + virtual update_mapping_ret update_mapping( + Transaction& t, + laddr_t laddr, + paddr_t prev_addr, + paddr_t paddr, + LogicalCachedExtent *nextent) = 0; + + /** + * update_mappings + * + * update lba mappings for delayed allocated extents + */ + using update_mappings_iertr = update_mapping_iertr; + using update_mappings_ret = update_mapping_ret; + update_mappings_ret update_mappings( + Transaction& t, + const std::list<LogicalCachedExtentRef>& extents); + + /** + * get_physical_extent_if_live + * + * Returns extent at addr/laddr if still live (if laddr + * still points at addr). Extent must be an internal, physical + * extent. + * + * Returns a null CachedExtentRef if extent is not live. + */ + using get_physical_extent_if_live_iertr = base_iertr; + using get_physical_extent_if_live_ret = + get_physical_extent_if_live_iertr::future<CachedExtentRef>; + virtual get_physical_extent_if_live_ret get_physical_extent_if_live( + Transaction &t, + extent_types_t type, + paddr_t addr, + laddr_t laddr, + extent_len_t len) = 0; + + virtual ~LBAManager() {} +}; +using LBAManagerRef = std::unique_ptr<LBAManager>; + +class Cache; +namespace lba_manager { +LBAManagerRef create_lba_manager(Cache &cache); +} + +} diff --git a/src/crimson/os/seastore/lba_manager/btree/btree_lba_manager.cc b/src/crimson/os/seastore/lba_manager/btree/btree_lba_manager.cc new file mode 100644 index 000000000..a607cd612 --- /dev/null +++ b/src/crimson/os/seastore/lba_manager/btree/btree_lba_manager.cc @@ -0,0 +1,761 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <sys/mman.h> +#include <string.h> + +#include <seastar/core/metrics.hh> + +#include "include/buffer.h" +#include "crimson/os/seastore/lba_manager/btree/btree_lba_manager.h" +#include "crimson/os/seastore/lba_manager/btree/lba_btree_node.h" +#include "crimson/os/seastore/logging.h" + +SET_SUBSYS(seastore_lba); +/* + * levels: + * - INFO: mkfs + * - DEBUG: modification operations + * - TRACE: read operations, DEBUG details + */ + +namespace crimson::os::seastore { + +template <typename T> +Transaction::tree_stats_t& get_tree_stats(Transaction &t) +{ + return t.get_lba_tree_stats(); +} + +template Transaction::tree_stats_t& +get_tree_stats< + crimson::os::seastore::lba_manager::btree::LBABtree>( + Transaction &t); + +template <typename T> +phy_tree_root_t& get_phy_tree_root(root_t &r) +{ + return r.lba_root; +} + +template phy_tree_root_t& +get_phy_tree_root< + crimson::os::seastore::lba_manager::btree::LBABtree>(root_t &r); + +template <> +const get_phy_tree_root_node_ret get_phy_tree_root_node< + crimson::os::seastore::lba_manager::btree::LBABtree>( + const RootBlockRef &root_block, op_context_t<laddr_t> c) +{ + auto lba_root = root_block->lba_root_node; + if (lba_root) { + ceph_assert(lba_root->is_initial_pending() + == root_block->is_pending()); + return {true, + trans_intr::make_interruptible( + c.cache.get_extent_viewable_by_trans(c.trans, lba_root))}; + } else if (root_block->is_pending()) { + auto &prior = static_cast<RootBlock&>(*root_block->get_prior_instance()); + lba_root = prior.lba_root_node; + if (lba_root) { + return {true, + trans_intr::make_interruptible( + c.cache.get_extent_viewable_by_trans(c.trans, lba_root))}; + } else { + return {false, + trans_intr::make_interruptible( + Cache::get_extent_ertr::make_ready_future< + CachedExtentRef>())}; + } + } else { + return {false, + trans_intr::make_interruptible( + Cache::get_extent_ertr::make_ready_future< + CachedExtentRef>())}; + } +} + +template <typename ROOT> +void link_phy_tree_root_node(RootBlockRef &root_block, ROOT* lba_root) { + root_block->lba_root_node = lba_root; + ceph_assert(lba_root != nullptr); + lba_root->root_block = root_block; +} + +template void link_phy_tree_root_node( + RootBlockRef &root_block, lba_manager::btree::LBAInternalNode* lba_root); +template void link_phy_tree_root_node( + RootBlockRef &root_block, lba_manager::btree::LBALeafNode* lba_root); +template void link_phy_tree_root_node( + RootBlockRef &root_block, lba_manager::btree::LBANode* lba_root); + +template <> +void unlink_phy_tree_root_node<laddr_t>(RootBlockRef &root_block) { + root_block->lba_root_node = nullptr; +} + +} + +namespace crimson::os::seastore::lba_manager::btree { + +BtreeLBAManager::mkfs_ret +BtreeLBAManager::mkfs( + Transaction &t) +{ + LOG_PREFIX(BtreeLBAManager::mkfs); + INFOT("start", t); + return cache.get_root(t).si_then([this, &t](auto croot) { + assert(croot->is_mutation_pending()); + croot->get_root().lba_root = LBABtree::mkfs(croot, get_context(t)); + return mkfs_iertr::now(); + }).handle_error_interruptible( + mkfs_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in BtreeLBAManager::mkfs" + } + ); +} + +BtreeLBAManager::get_mappings_ret +BtreeLBAManager::get_mappings( + Transaction &t, + laddr_t offset, extent_len_t length) +{ + LOG_PREFIX(BtreeLBAManager::get_mappings); + TRACET("{}~{}", t, offset, length); + auto c = get_context(t); + return with_btree_state<LBABtree, lba_pin_list_t>( + cache, + c, + [c, offset, length, FNAME, this](auto &btree, auto &ret) { + return seastar::do_with( + std::list<BtreeLBAMappingRef>(), + [offset, length, c, FNAME, this, &ret, &btree](auto &pin_list) { + return LBABtree::iterate_repeat( + c, + btree.upper_bound_right(c, offset), + [&pin_list, offset, length, c, FNAME](auto &pos) { + if (pos.is_end() || pos.get_key() >= (offset + length)) { + TRACET("{}~{} done with {} results", + c.trans, offset, length, pin_list.size()); + return LBABtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } + TRACET("{}~{} got {}, {}, repeat ...", + c.trans, offset, length, pos.get_key(), pos.get_val()); + ceph_assert((pos.get_key() + pos.get_val().len) > offset); + pin_list.push_back(pos.get_pin(c)); + return LBABtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::no); + }).si_then([this, &ret, c, &pin_list] { + return _get_original_mappings(c, pin_list + ).si_then([&ret](auto _ret) { + ret = std::move(_ret); + }); + }); + }); + }); +} + +BtreeLBAManager::_get_original_mappings_ret +BtreeLBAManager::_get_original_mappings( + op_context_t<laddr_t> c, + std::list<BtreeLBAMappingRef> &pin_list) +{ + return seastar::do_with( + lba_pin_list_t(), + [this, c, &pin_list](auto &ret) { + return trans_intr::do_for_each( + pin_list, + [this, c, &ret](auto &pin) { + LOG_PREFIX(BtreeLBAManager::get_mappings); + if (pin->get_raw_val().is_paddr()) { + ret.emplace_back(std::move(pin)); + return get_mappings_iertr::now(); + } + TRACET( + "getting original mapping for indirect mapping {}~{}", + c.trans, pin->get_key(), pin->get_length()); + return this->get_mappings( + c.trans, pin->get_raw_val().get_laddr(), pin->get_length() + ).si_then([&pin, &ret, c](auto new_pin_list) { + LOG_PREFIX(BtreeLBAManager::get_mappings); + assert(new_pin_list.size() == 1); + auto &new_pin = new_pin_list.front(); + auto intermediate_key = pin->get_raw_val().get_laddr(); + assert(!new_pin->is_indirect()); + assert(new_pin->get_key() <= intermediate_key); + assert(new_pin->get_key() + new_pin->get_length() >= + intermediate_key + pin->get_length()); + + TRACET("Got mapping {}~{} for indirect mapping {}~{}, " + "intermediate_key {}", + c.trans, + new_pin->get_key(), new_pin->get_length(), + pin->get_key(), pin->get_length(), + pin->get_raw_val().get_laddr()); + auto &btree_new_pin = static_cast<BtreeLBAMapping&>(*new_pin); + btree_new_pin.set_key_for_indirect( + pin->get_key(), + pin->get_length(), + pin->get_raw_val().get_laddr()); + ret.emplace_back(std::move(new_pin)); + return seastar::now(); + }).handle_error_interruptible( + crimson::ct_error::input_output_error::pass_further{}, + crimson::ct_error::assert_all("unexpected enoent") + ); + } + ).si_then([&ret] { + return std::move(ret); + }); + }); +} + + +BtreeLBAManager::get_mappings_ret +BtreeLBAManager::get_mappings( + Transaction &t, + laddr_list_t &&list) +{ + LOG_PREFIX(BtreeLBAManager::get_mappings); + TRACET("{}", t, list); + auto l = std::make_unique<laddr_list_t>(std::move(list)); + auto retptr = std::make_unique<lba_pin_list_t>(); + auto &ret = *retptr; + return trans_intr::do_for_each( + l->begin(), + l->end(), + [this, &t, &ret](const auto &p) { + return this->get_mappings(t, p.first, p.second).si_then( + [&ret](auto res) { + ret.splice(ret.end(), res, res.begin(), res.end()); + return get_mappings_iertr::now(); + }); + }).si_then([l=std::move(l), retptr=std::move(retptr)]() mutable { + return std::move(*retptr); + }); +} + +BtreeLBAManager::get_mapping_ret +BtreeLBAManager::get_mapping( + Transaction &t, + laddr_t offset) +{ + LOG_PREFIX(BtreeLBAManager::get_mapping); + TRACET("{}", t, offset); + return _get_mapping(t, offset + ).si_then([](auto pin) { + return get_mapping_iertr::make_ready_future<LBAMappingRef>(std::move(pin)); + }); +} + +BtreeLBAManager::_get_mapping_ret +BtreeLBAManager::_get_mapping( + Transaction &t, + laddr_t offset) +{ + LOG_PREFIX(BtreeLBAManager::_get_mapping); + TRACET("{}", t, offset); + auto c = get_context(t); + return with_btree_ret<LBABtree, BtreeLBAMappingRef>( + cache, + c, + [FNAME, c, offset, this](auto &btree) { + return btree.lower_bound( + c, offset + ).si_then([FNAME, offset, c](auto iter) -> _get_mapping_ret { + if (iter.is_end() || iter.get_key() != offset) { + ERRORT("laddr={} doesn't exist", c.trans, offset); + return crimson::ct_error::enoent::make(); + } else { + TRACET("{} got {}, {}", + c.trans, offset, iter.get_key(), iter.get_val()); + auto e = iter.get_pin(c); + return _get_mapping_ret( + interruptible::ready_future_marker{}, + std::move(e)); + } + }).si_then([this, c](auto pin) -> _get_mapping_ret { + if (pin->get_raw_val().is_laddr()) { + return seastar::do_with( + std::move(pin), + [this, c](auto &pin) { + return _get_mapping( + c.trans, pin->get_raw_val().get_laddr() + ).si_then([&pin](auto new_pin) { + ceph_assert(pin->get_length() == new_pin->get_length()); + new_pin->set_key_for_indirect( + pin->get_key(), + pin->get_length()); + return new_pin; + }); + }); + } else { + return get_mapping_iertr::make_ready_future<BtreeLBAMappingRef>(std::move(pin)); + } + }); + }); +} + +BtreeLBAManager::alloc_extent_ret +BtreeLBAManager::_alloc_extent( + Transaction &t, + laddr_t hint, + extent_len_t len, + pladdr_t addr, + paddr_t actual_addr, + laddr_t intermediate_base, + LogicalCachedExtent* nextent) +{ + struct state_t { + laddr_t last_end; + + std::optional<typename LBABtree::iterator> insert_iter; + std::optional<typename LBABtree::iterator> ret; + + state_t(laddr_t hint) : last_end(hint) {} + }; + + LOG_PREFIX(BtreeLBAManager::_alloc_extent); + TRACET("{}~{}, hint={}", t, addr, len, hint); + auto c = get_context(t); + ++stats.num_alloc_extents; + auto lookup_attempts = stats.num_alloc_extents_iter_nexts; + return crimson::os::seastore::with_btree_state<LBABtree, state_t>( + cache, + c, + hint, + [this, FNAME, c, hint, len, addr, lookup_attempts, + &t, nextent](auto &btree, auto &state) { + return LBABtree::iterate_repeat( + c, + btree.upper_bound_right(c, hint), + [this, &state, len, addr, &t, hint, FNAME, lookup_attempts](auto &pos) { + ++stats.num_alloc_extents_iter_nexts; + if (pos.is_end()) { + DEBUGT("{}~{}, hint={}, state: end, done with {} attempts, insert at {}", + t, addr, len, hint, + stats.num_alloc_extents_iter_nexts - lookup_attempts, + state.last_end); + state.insert_iter = pos; + return typename LBABtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } else if (pos.get_key() >= (state.last_end + len)) { + DEBUGT("{}~{}, hint={}, state: {}~{}, done with {} attempts, insert at {} -- {}", + t, addr, len, hint, + pos.get_key(), pos.get_val().len, + stats.num_alloc_extents_iter_nexts - lookup_attempts, + state.last_end, + pos.get_val()); + state.insert_iter = pos; + return typename LBABtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } else { + state.last_end = pos.get_key() + pos.get_val().len; + TRACET("{}~{}, hint={}, state: {}~{}, repeat ... -- {}", + t, addr, len, hint, + pos.get_key(), pos.get_val().len, + pos.get_val()); + return typename LBABtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::no); + } + }).si_then([FNAME, c, addr, len, hint, &btree, &state, nextent] { + return btree.insert( + c, + *state.insert_iter, + state.last_end, + lba_map_val_t{len, pladdr_t(addr), 1, 0}, + nextent + ).si_then([&state, FNAME, c, addr, len, hint, nextent](auto &&p) { + auto [iter, inserted] = std::move(p); + TRACET("{}~{}, hint={}, inserted at {}", + c.trans, addr, len, hint, state.last_end); + if (nextent) { + ceph_assert(addr.is_paddr()); + nextent->set_laddr(iter.get_key()); + } + ceph_assert(inserted); + state.ret = iter; + }); + }); + }).si_then([c, actual_addr, addr, intermediate_base](auto &&state) { + auto ret_pin = state.ret->get_pin(c); + if (actual_addr != P_ADDR_NULL) { + ceph_assert(addr.is_laddr()); + ret_pin->set_paddr(actual_addr); + ret_pin->set_intermediate_base(intermediate_base); + } else { + ceph_assert(addr.is_paddr()); + } + return alloc_extent_iertr::make_ready_future<LBAMappingRef>( + std::move(ret_pin)); + }); +} + +static bool is_lba_node(const CachedExtent &e) +{ + return is_lba_node(e.get_type()); +} + +BtreeLBAManager::base_iertr::template future<> +_init_cached_extent( + op_context_t<laddr_t> c, + const CachedExtentRef &e, + LBABtree &btree, + bool &ret) +{ + if (e->is_logical()) { + auto logn = e->cast<LogicalCachedExtent>(); + return btree.lower_bound( + c, + logn->get_laddr() + ).si_then([e, c, logn, &ret](auto iter) { + LOG_PREFIX(BtreeLBAManager::init_cached_extent); + if (!iter.is_end() && + iter.get_key() == logn->get_laddr() && + iter.get_val().pladdr.is_paddr() && + iter.get_val().pladdr.get_paddr() == logn->get_paddr()) { + assert(!iter.get_leaf_node()->is_pending()); + iter.get_leaf_node()->link_child(logn.get(), iter.get_leaf_pos()); + logn->set_laddr(iter.get_pin(c)->get_key()); + ceph_assert(iter.get_val().len == e->get_length()); + DEBUGT("logical extent {} live", c.trans, *logn); + ret = true; + } else { + DEBUGT("logical extent {} not live", c.trans, *logn); + ret = false; + } + }); + } else { + return btree.init_cached_extent(c, e + ).si_then([&ret](bool is_alive) { + ret = is_alive; + }); + } +} + +BtreeLBAManager::init_cached_extent_ret +BtreeLBAManager::init_cached_extent( + Transaction &t, + CachedExtentRef e) +{ + LOG_PREFIX(BtreeLBAManager::init_cached_extent); + TRACET("{}", t, *e); + return seastar::do_with(bool(), [this, e, &t](bool &ret) { + auto c = get_context(t); + return with_btree<LBABtree>( + cache, c, + [c, e, &ret](auto &btree) -> base_iertr::future<> { + LOG_PREFIX(BtreeLBAManager::init_cached_extent); + DEBUGT("extent {}", c.trans, *e); + return _init_cached_extent(c, e, btree, ret); + } + ).si_then([&ret] { return ret; }); + }); +} + +BtreeLBAManager::check_child_trackers_ret +BtreeLBAManager::check_child_trackers( + Transaction &t) { + auto c = get_context(t); + return with_btree<LBABtree>( + cache, c, + [c](auto &btree) { + return btree.check_child_trackers(c); + }); +} + +BtreeLBAManager::scan_mappings_ret +BtreeLBAManager::scan_mappings( + Transaction &t, + laddr_t begin, + laddr_t end, + scan_mappings_func_t &&f) +{ + LOG_PREFIX(BtreeLBAManager::scan_mappings); + DEBUGT("begin: {}, end: {}", t, begin, end); + + auto c = get_context(t); + return with_btree<LBABtree>( + cache, + c, + [c, f=std::move(f), begin, end](auto &btree) mutable { + return LBABtree::iterate_repeat( + c, + btree.upper_bound_right(c, begin), + [f=std::move(f), begin, end](auto &pos) { + if (pos.is_end() || pos.get_key() >= end) { + return typename LBABtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::yes); + } + ceph_assert((pos.get_key() + pos.get_val().len) > begin); + f(pos.get_key(), pos.get_val().pladdr.get_paddr(), pos.get_val().len); + return LBABtree::iterate_repeat_ret_inner( + interruptible::ready_future_marker{}, + seastar::stop_iteration::no); + }); + }); +} + +BtreeLBAManager::rewrite_extent_ret +BtreeLBAManager::rewrite_extent( + Transaction &t, + CachedExtentRef extent) +{ + LOG_PREFIX(BtreeLBAManager::rewrite_extent); + if (extent->has_been_invalidated()) { + ERRORT("extent has been invalidated -- {}", t, *extent); + ceph_abort(); + } + assert(!extent->is_logical()); + + if (is_lba_node(*extent)) { + DEBUGT("rewriting lba extent -- {}", t, *extent); + auto c = get_context(t); + return with_btree<LBABtree>( + cache, + c, + [c, extent](auto &btree) mutable { + return btree.rewrite_extent(c, extent); + }); + } else { + DEBUGT("skip non lba extent -- {}", t, *extent); + return rewrite_extent_iertr::now(); + } +} + +BtreeLBAManager::update_mapping_ret +BtreeLBAManager::update_mapping( + Transaction& t, + laddr_t laddr, + paddr_t prev_addr, + paddr_t addr, + LogicalCachedExtent *nextent) +{ + LOG_PREFIX(BtreeLBAManager::update_mapping); + TRACET("laddr={}, paddr {} => {}", t, laddr, prev_addr, addr); + return _update_mapping( + t, + laddr, + [prev_addr, addr]( + const lba_map_val_t &in) { + assert(!addr.is_null()); + lba_map_val_t ret = in; + ceph_assert(in.pladdr.is_paddr()); + ceph_assert(in.pladdr.get_paddr() == prev_addr); + ret.pladdr = addr; + return ret; + }, + nextent + ).si_then([&t, laddr, prev_addr, addr, FNAME](auto result) { + DEBUGT("laddr={}, paddr {} => {} done -- {}", + t, laddr, prev_addr, addr, result); + }, + update_mapping_iertr::pass_further{}, + /* ENOENT in particular should be impossible */ + crimson::ct_error::assert_all{ + "Invalid error in BtreeLBAManager::update_mapping" + } + ); +} + +BtreeLBAManager::get_physical_extent_if_live_ret +BtreeLBAManager::get_physical_extent_if_live( + Transaction &t, + extent_types_t type, + paddr_t addr, + laddr_t laddr, + extent_len_t len) +{ + LOG_PREFIX(BtreeLBAManager::get_physical_extent_if_live); + DEBUGT("{}, laddr={}, paddr={}, length={}", + t, type, laddr, addr, len); + ceph_assert(is_lba_node(type)); + auto c = get_context(t); + return with_btree_ret<LBABtree, CachedExtentRef>( + cache, + c, + [c, type, addr, laddr, len](auto &btree) { + if (type == extent_types_t::LADDR_INTERNAL) { + return btree.get_internal_if_live(c, addr, laddr, len); + } else { + assert(type == extent_types_t::LADDR_LEAF || + type == extent_types_t::DINK_LADDR_LEAF); + return btree.get_leaf_if_live(c, addr, laddr, len); + } + }); +} + +void BtreeLBAManager::register_metrics() +{ + LOG_PREFIX(BtreeLBAManager::register_metrics); + DEBUG("start"); + stats = {}; + namespace sm = seastar::metrics; + metrics.add_group( + "LBA", + { + sm::make_counter( + "alloc_extents", + stats.num_alloc_extents, + sm::description("total number of lba alloc_extent operations") + ), + sm::make_counter( + "alloc_extents_iter_nexts", + stats.num_alloc_extents_iter_nexts, + sm::description("total number of iterator next operations during extent allocation") + ), + } + ); +} + +BtreeLBAManager::ref_iertr::future<std::optional<std::pair<paddr_t, extent_len_t>>> +BtreeLBAManager::_decref_intermediate( + Transaction &t, + laddr_t addr, + extent_len_t len) +{ + auto c = get_context(t); + return with_btree<LBABtree>( + cache, + c, + [c, addr, len](auto &btree) mutable { + return btree.upper_bound_right( + c, addr + ).si_then([&btree, addr, len, c](auto iter) { + return seastar::do_with( + std::move(iter), + [&btree, addr, len, c](auto &iter) { + ceph_assert(!iter.is_end()); + ceph_assert(iter.get_key() <= addr); + auto val = iter.get_val(); + ceph_assert(iter.get_key() + val.len >= addr + len); + ceph_assert(val.pladdr.is_paddr()); + ceph_assert(val.refcount >= 1); + val.refcount -= 1; + + LOG_PREFIX(BtreeLBAManager::_decref_intermediate); + TRACET("decreased refcount of intermediate key {} -- {}", + c.trans, + iter.get_key(), + val); + + if (!val.refcount) { + return btree.remove(c, iter + ).si_then([val] { + return std::make_optional< + std::pair<paddr_t, extent_len_t>>( + val.pladdr.get_paddr(), val.len); + }); + } else { + return btree.update(c, iter, val, nullptr + ).si_then([](auto) { + return seastar::make_ready_future< + std::optional<std::pair<paddr_t, extent_len_t>>>(std::nullopt); + }); + } + }); + }); + }); +} + +BtreeLBAManager::update_refcount_ret +BtreeLBAManager::update_refcount( + Transaction &t, + laddr_t addr, + int delta, + bool cascade_remove) +{ + LOG_PREFIX(BtreeLBAManager::update_refcount); + TRACET("laddr={}, delta={}", t, addr, delta); + return _update_mapping( + t, + addr, + [delta](const lba_map_val_t &in) { + lba_map_val_t out = in; + ceph_assert((int)out.refcount + delta >= 0); + out.refcount += delta; + return out; + }, + nullptr + ).si_then([&t, addr, delta, FNAME, this, cascade_remove](auto result) { + DEBUGT("laddr={}, delta={} done -- {}", t, addr, delta, result); + auto fut = ref_iertr::make_ready_future< + std::optional<std::pair<paddr_t, extent_len_t>>>(); + if (!result.refcount && result.pladdr.is_laddr() && cascade_remove) { + fut = _decref_intermediate( + t, + result.pladdr.get_laddr(), + result.len + ); + } + return fut.si_then([result](auto removed) { + if (result.pladdr.is_laddr() + && removed) { + return ref_update_result_t{ + result.refcount, + removed->first, + removed->second}; + } else { + return ref_update_result_t{ + result.refcount, + result.pladdr, + result.len + }; + } + }); + }); +} + +BtreeLBAManager::_update_mapping_ret +BtreeLBAManager::_update_mapping( + Transaction &t, + laddr_t addr, + update_func_t &&f, + LogicalCachedExtent* nextent) +{ + auto c = get_context(t); + return with_btree_ret<LBABtree, lba_map_val_t>( + cache, + c, + [f=std::move(f), c, addr, nextent](auto &btree) mutable { + return btree.lower_bound( + c, addr + ).si_then([&btree, f=std::move(f), c, addr, nextent](auto iter) + -> _update_mapping_ret { + if (iter.is_end() || iter.get_key() != addr) { + LOG_PREFIX(BtreeLBAManager::_update_mapping); + ERRORT("laddr={} doesn't exist", c.trans, addr); + return crimson::ct_error::enoent::make(); + } + + auto ret = f(iter.get_val()); + if (ret.refcount == 0) { + return btree.remove( + c, + iter + ).si_then([ret] { + return ret; + }); + } else { + return btree.update( + c, + iter, + ret, + nextent + ).si_then([ret](auto) { + return ret; + }); + } + }); + }); +} + +} diff --git a/src/crimson/os/seastore/lba_manager/btree/btree_lba_manager.h b/src/crimson/os/seastore/lba_manager/btree/btree_lba_manager.h new file mode 100644 index 000000000..892600ed0 --- /dev/null +++ b/src/crimson/os/seastore/lba_manager/btree/btree_lba_manager.h @@ -0,0 +1,396 @@ +// -*- 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_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <seastar/core/future.hh> + +#include "include/ceph_assert.h" +#include "include/buffer_fwd.h" +#include "include/interval_set.h" +#include "common/interval_map.h" +#include "crimson/osd/exceptions.h" + +#include "crimson/os/seastore/btree/fixed_kv_btree.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/lba_manager.h" +#include "crimson/os/seastore/cache.h" + +#include "crimson/os/seastore/lba_manager/btree/lba_btree_node.h" +#include "crimson/os/seastore/btree/btree_range_pin.h" + +namespace crimson::os::seastore::lba_manager::btree { + +class BtreeLBAMapping : public BtreeNodeMapping<laddr_t, paddr_t> { +// To support cloning, there are two kinds of lba mappings: +// 1. physical lba mapping: the pladdr in the value of which is the paddr of +// the corresponding extent; +// 2. indirect lba mapping: the pladdr in the value of which is an laddr pointing +// to the physical lba mapping that's pointing to the actual paddr of the +// extent being searched; +// +// Accordingly, BtreeLBAMapping may also work under two modes: indirect or direct +// 1. BtreeLBAMappings that come from quering an indirect lba mapping in the lba tree +// are indirect; +// 2. BtreeLBAMappings that come from quering a physical lba mapping in the lba tree +// are direct. +// +// For direct BtreeLBAMappings, there are two important fields: +// 1. key: the laddr of the lba mapping being queried; +// 2. paddr: the paddr recorded in the value of the lba mapping being queried. +// For indirect BtreeLBAMappings, BtreeLBAMapping has three important fields: +// 1. key: the laddr key of the lba entry being queried; +// 2. intermediate_key: the laddr within the scope of the physical lba mapping +// that the current indirect lba mapping points to; although an indirect mapping +// points to the start of the physical lba mapping, it may change to other +// laddr after remap +// 3. intermediate_base: the laddr key of the physical lba mapping, intermediate_key +// and intermediate_base should be the same when doing cloning +// 4. intermediate_offset: intermediate_key - intermediate_base +// 5. paddr: the paddr recorded in the physical lba mapping pointed to by the +// indirect lba mapping being queried; +// +// NOTE THAT, for direct BtreeLBAMappings, their intermediate_keys are the same as +// their keys. +public: + BtreeLBAMapping(op_context_t<laddr_t> ctx) + : BtreeNodeMapping(ctx) {} + BtreeLBAMapping( + op_context_t<laddr_t> c, + CachedExtentRef parent, + uint16_t pos, + lba_map_val_t &val, + lba_node_meta_t meta) + : BtreeNodeMapping( + c, + parent, + pos, + val.pladdr.is_paddr() ? val.pladdr.get_paddr() : P_ADDR_NULL, + val.len, + meta), + key(meta.begin), + indirect(val.pladdr.is_laddr() ? true : false), + intermediate_key(indirect ? val.pladdr.get_laddr() : L_ADDR_NULL), + intermediate_length(indirect ? val.len : 0), + raw_val(val.pladdr), + map_val(val) + {} + + lba_map_val_t get_map_val() const { + return map_val; + } + + bool is_indirect() const final { + return indirect; + } + + void set_key_for_indirect( + laddr_t new_key, + extent_len_t length, + laddr_t interkey = L_ADDR_NULL) + { + turn_indirect(interkey); + key = new_key; + intermediate_length = len; + len = length; + } + + laddr_t get_key() const final { + return key; + } + + pladdr_t get_raw_val() const { + return raw_val; + } + + void set_paddr(paddr_t addr) { + value = addr; + } + + laddr_t get_intermediate_key() const final { + assert(is_indirect()); + assert(intermediate_key != L_ADDR_NULL); + return intermediate_key; + } + + laddr_t get_intermediate_base() const final { + assert(is_indirect()); + assert(intermediate_base != L_ADDR_NULL); + return intermediate_base; + } + + extent_len_t get_intermediate_offset() const final { + assert(intermediate_key >= intermediate_base); + assert((intermediate_key == L_ADDR_NULL) + == (intermediate_base == L_ADDR_NULL)); + return intermediate_key - intermediate_base; + } + + extent_len_t get_intermediate_length() const final { + assert(is_indirect()); + assert(intermediate_length); + return intermediate_length; + } + + void set_intermediate_base(laddr_t base) { + intermediate_base = base; + } + +protected: + std::unique_ptr<BtreeNodeMapping<laddr_t, paddr_t>> _duplicate( + op_context_t<laddr_t> ctx) const final { + auto pin = std::unique_ptr<BtreeLBAMapping>(new BtreeLBAMapping(ctx)); + pin->key = key; + pin->intermediate_base = intermediate_base; + pin->intermediate_key = intermediate_key; + pin->indirect = indirect; + pin->raw_val = raw_val; + pin->map_val = map_val; + return pin; + } +private: + void turn_indirect(laddr_t interkey) { + assert(value.is_paddr()); + intermediate_base = key; + intermediate_key = (interkey == L_ADDR_NULL ? key : interkey); + indirect = true; + } + laddr_t key = L_ADDR_NULL; + bool indirect = false; + laddr_t intermediate_key = L_ADDR_NULL; + laddr_t intermediate_base = L_ADDR_NULL; + extent_len_t intermediate_length = 0; + pladdr_t raw_val; + lba_map_val_t map_val; +}; + +using BtreeLBAMappingRef = std::unique_ptr<BtreeLBAMapping>; + +using LBABtree = FixedKVBtree< + laddr_t, lba_map_val_t, LBAInternalNode, + LBALeafNode, BtreeLBAMapping, LBA_BLOCK_SIZE, true>; + +/** + * BtreeLBAManager + * + * Uses a wandering btree to track two things: + * 1) lba state including laddr_t -> paddr_t mapping + * 2) reverse paddr_t -> laddr_t mapping for gc (TODO) + * + * Generally, any transaction will involve + * 1) deltas against lba tree nodes + * 2) new lba tree nodes + * - Note, there must necessarily be a delta linking + * these new nodes into the tree -- might be a + * bootstrap_state_t delta if new root + * + * get_mappings, alloc_extent_*, etc populate a Transaction + * which then gets submitted + */ +class BtreeLBAManager : public LBAManager { +public: + BtreeLBAManager(Cache &cache) + : cache(cache) + { + register_metrics(); + } + + mkfs_ret mkfs( + Transaction &t) final; + + get_mappings_ret get_mappings( + Transaction &t, + laddr_t offset, extent_len_t length) final; + + get_mappings_ret get_mappings( + Transaction &t, + laddr_list_t &&list) final; + + get_mapping_ret get_mapping( + Transaction &t, + laddr_t offset) final; + + alloc_extent_ret reserve_region( + Transaction &t, + laddr_t hint, + extent_len_t len) + { + return _alloc_extent( + t, + hint, + len, + P_ADDR_ZERO, + P_ADDR_NULL, + L_ADDR_NULL, + nullptr); + } + + alloc_extent_ret clone_extent( + Transaction &t, + laddr_t hint, + extent_len_t len, + laddr_t intermediate_key, + paddr_t actual_addr, + laddr_t intermediate_base) + { + return _alloc_extent( + t, + hint, + len, + intermediate_key, + actual_addr, + intermediate_base, + nullptr); + } + + alloc_extent_ret alloc_extent( + Transaction &t, + laddr_t hint, + extent_len_t len, + paddr_t addr, + LogicalCachedExtent &ext) final + { + return _alloc_extent( + t, + hint, + len, + addr, + P_ADDR_NULL, + L_ADDR_NULL, + &ext); + } + + ref_ret decref_extent( + Transaction &t, + laddr_t addr, + bool cascade_remove) final { + return update_refcount(t, addr, -1, cascade_remove); + } + + ref_ret incref_extent( + Transaction &t, + laddr_t addr) final { + return update_refcount(t, addr, 1, false); + } + + ref_ret incref_extent( + Transaction &t, + laddr_t addr, + int delta) final { + ceph_assert(delta > 0); + return update_refcount(t, addr, delta, false); + } + + /** + * init_cached_extent + * + * Checks whether e is live (reachable from lba tree) and drops or initializes + * accordingly. + * + * Returns if e is live. + */ + init_cached_extent_ret init_cached_extent( + Transaction &t, + CachedExtentRef e) final; + + check_child_trackers_ret check_child_trackers(Transaction &t) final; + + scan_mappings_ret scan_mappings( + Transaction &t, + laddr_t begin, + laddr_t end, + scan_mappings_func_t &&f) final; + + rewrite_extent_ret rewrite_extent( + Transaction &t, + CachedExtentRef extent) final; + + update_mapping_ret update_mapping( + Transaction& t, + laddr_t laddr, + paddr_t prev_addr, + paddr_t paddr, + LogicalCachedExtent*) final; + + get_physical_extent_if_live_ret get_physical_extent_if_live( + Transaction &t, + extent_types_t type, + paddr_t addr, + laddr_t laddr, + extent_len_t len) final; +private: + Cache &cache; + + + struct { + uint64_t num_alloc_extents = 0; + uint64_t num_alloc_extents_iter_nexts = 0; + } stats; + + op_context_t<laddr_t> get_context(Transaction &t) { + return op_context_t<laddr_t>{cache, t}; + } + + seastar::metrics::metric_group metrics; + void register_metrics(); + + /** + * update_refcount + * + * Updates refcount, returns resulting refcount + */ + using update_refcount_ret = ref_ret; + update_refcount_ret update_refcount( + Transaction &t, + laddr_t addr, + int delta, + bool cascade_remove); + + /** + * _update_mapping + * + * Updates mapping, removes if f returns nullopt + */ + using _update_mapping_iertr = ref_iertr; + using _update_mapping_ret = ref_iertr::future<lba_map_val_t>; + using update_func_t = std::function< + lba_map_val_t(const lba_map_val_t &v) + >; + _update_mapping_ret _update_mapping( + Transaction &t, + laddr_t addr, + update_func_t &&f, + LogicalCachedExtent*); + + alloc_extent_ret _alloc_extent( + Transaction &t, + laddr_t hint, + extent_len_t len, + pladdr_t addr, + paddr_t actual_addr, + laddr_t intermediate_base, + LogicalCachedExtent*); + + using _get_mapping_ret = get_mapping_iertr::future<BtreeLBAMappingRef>; + _get_mapping_ret _get_mapping( + Transaction &t, + laddr_t offset); + + using _get_original_mappings_ret = get_mappings_ret; + _get_original_mappings_ret _get_original_mappings( + op_context_t<laddr_t> c, + std::list<BtreeLBAMappingRef> &pin_list); + + ref_iertr::future<std::optional<std::pair<paddr_t, extent_len_t>>> + _decref_intermediate( + Transaction &t, + laddr_t addr, + extent_len_t len); +}; +using BtreeLBAManagerRef = std::unique_ptr<BtreeLBAManager>; + +} diff --git a/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.cc b/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.cc new file mode 100644 index 000000000..66dc94394 --- /dev/null +++ b/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.cc @@ -0,0 +1,55 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <sys/mman.h> +#include <string.h> + +#include <memory> +#include <string.h> + +#include "include/buffer.h" +#include "include/byteorder.h" + +#include "crimson/os/seastore/lba_manager/btree/lba_btree_node.h" +#include "crimson/os/seastore/logging.h" + +SET_SUBSYS(seastore_lba); + +namespace crimson::os::seastore::lba_manager::btree { + +std::ostream& operator<<(std::ostream& out, const lba_map_val_t& v) +{ + return out << "lba_map_val_t(" + << v.pladdr + << "~" << v.len + << ", refcount=" << v.refcount + << ", checksum=" << v.checksum + << ")"; +} + +std::ostream &LBALeafNode::_print_detail(std::ostream &out) const +{ + out << ", size=" << this->get_size() + << ", meta=" << this->get_meta() + << ", my_tracker=" << (void*)this->my_tracker; + if (this->my_tracker) { + out << ", my_tracker->parent=" << (void*)this->my_tracker->get_parent().get(); + } + return out << ", root_block=" << (void*)this->root_block.get(); +} + +void LBALeafNode::resolve_relative_addrs(paddr_t base) +{ + LOG_PREFIX(LBALeafNode::resolve_relative_addrs); + for (auto i: *this) { + auto val = i->get_val(); + if (val.pladdr.is_paddr() && + val.pladdr.get_paddr().is_relative()) { + val.pladdr = base.add_relative(val.pladdr.get_paddr()); + TRACE("{} -> {}", i->get_val().pladdr, val.pladdr); + i->set_val(val); + } + } +} + +} diff --git a/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.h b/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.h new file mode 100644 index 000000000..ffce2c1b5 --- /dev/null +++ b/src/crimson/os/seastore/lba_manager/btree/lba_btree_node.h @@ -0,0 +1,294 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <sys/mman.h> +#include <memory> +#include <string.h> + + +#include "include/buffer.h" + +#include "crimson/common/fixed_kv_node_layout.h" +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/lba_manager.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/cache.h" +#include "crimson/os/seastore/cached_extent.h" + +#include "crimson/os/seastore/btree/btree_range_pin.h" +#include "crimson/os/seastore/btree/fixed_kv_btree.h" +#include "crimson/os/seastore/btree/fixed_kv_node.h" + +namespace crimson::os::seastore::lba_manager::btree { + +using base_iertr = LBAManager::base_iertr; +using LBANode = FixedKVNode<laddr_t>; + +/** + * lba_map_val_t + * + * struct representing a single lba mapping + */ +struct lba_map_val_t { + extent_len_t len = 0; ///< length of mapping + pladdr_t pladdr; ///< physical addr of mapping or + // laddr of a physical lba mapping(see btree_lba_manager.h) + uint32_t refcount = 0; ///< refcount + uint32_t checksum = 0; ///< checksum of original block written at paddr (TODO) + + lba_map_val_t() = default; + lba_map_val_t( + extent_len_t len, + pladdr_t pladdr, + uint32_t refcount, + uint32_t checksum) + : len(len), pladdr(pladdr), refcount(refcount), checksum(checksum) {} + bool operator==(const lba_map_val_t&) const = default; +}; + +std::ostream& operator<<(std::ostream& out, const lba_map_val_t&); + +constexpr size_t LBA_BLOCK_SIZE = 4096; + +using lba_node_meta_t = fixed_kv_node_meta_t<laddr_t>; + +using lba_node_meta_le_t = fixed_kv_node_meta_le_t<laddr_le_t>; + +/** + * LBAInternalNode + * + * Abstracts operations on and layout of internal nodes for the + * LBA Tree. + * + * Layout (4k): + * size : uint32_t[1] 4b + * (padding) : 4b + * meta : lba_node_meta_le_t[3] (1*24)b + * keys : laddr_t[255] (254*8)b + * values : paddr_t[255] (254*8)b + * = 4096 + + * TODO: make the above capacity calculation part of FixedKVNodeLayout + * TODO: the above alignment probably isn't portable without further work + */ +constexpr size_t INTERNAL_NODE_CAPACITY = 254; +struct LBAInternalNode + : FixedKVInternalNode< + INTERNAL_NODE_CAPACITY, + laddr_t, laddr_le_t, + LBA_BLOCK_SIZE, + LBAInternalNode> { + using Ref = TCachedExtentRef<LBAInternalNode>; + using internal_iterator_t = const_iterator; + template <typename... T> + LBAInternalNode(T&&... t) : + FixedKVInternalNode(std::forward<T>(t)...) {} + + static constexpr extent_types_t TYPE = extent_types_t::LADDR_INTERNAL; + + extent_types_t get_type() const final { + return TYPE; + } +}; +using LBAInternalNodeRef = LBAInternalNode::Ref; + +/** + * LBALeafNode + * + * Abstracts operations on and layout of leaf nodes for the + * LBA Tree. + * + * Layout (4k): + * size : uint32_t[1] 4b + * (padding) : 4b + * meta : lba_node_meta_le_t[3] (1*24)b + * keys : laddr_t[170] (140*8)b + * values : lba_map_val_t[170] (140*21)b + * = 4092 + * + * TODO: update FixedKVNodeLayout to handle the above calculation + * TODO: the above alignment probably isn't portable without further work + */ +constexpr size_t LEAF_NODE_CAPACITY = 140; + +/** + * lba_map_val_le_t + * + * On disk layout for lba_map_val_t. + */ +struct lba_map_val_le_t { + extent_len_le_t len = init_extent_len_le(0); + pladdr_le_t pladdr; + ceph_le32 refcount{0}; + ceph_le32 checksum{0}; + + lba_map_val_le_t() = default; + lba_map_val_le_t(const lba_map_val_le_t &) = default; + explicit lba_map_val_le_t(const lba_map_val_t &val) + : len(init_extent_len_le(val.len)), + pladdr(pladdr_le_t(val.pladdr)), + refcount(val.refcount), + checksum(val.checksum) {} + + operator lba_map_val_t() const { + return lba_map_val_t{ len, pladdr, refcount, checksum }; + } +}; + +struct LBALeafNode + : FixedKVLeafNode< + LEAF_NODE_CAPACITY, + laddr_t, laddr_le_t, + lba_map_val_t, lba_map_val_le_t, + LBA_BLOCK_SIZE, + LBALeafNode, + true> { + using Ref = TCachedExtentRef<LBALeafNode>; + using parent_type_t = FixedKVLeafNode< + LEAF_NODE_CAPACITY, + laddr_t, laddr_le_t, + lba_map_val_t, lba_map_val_le_t, + LBA_BLOCK_SIZE, + LBALeafNode, + true>; + using internal_const_iterator_t = + typename parent_type_t::node_layout_t::const_iterator; + using internal_iterator_t = + typename parent_type_t::node_layout_t::iterator; + template <typename... T> + LBALeafNode(T&&... t) : + parent_type_t(std::forward<T>(t)...) {} + + static constexpr extent_types_t TYPE = extent_types_t::LADDR_LEAF; + + bool validate_stable_children() final { + LOG_PREFIX(LBALeafNode::validate_stable_children); + if (this->children.empty()) { + return false; + } + + for (auto i : *this) { + auto child = (LogicalCachedExtent*)this->children[i.get_offset()]; + if (is_valid_child_ptr(child) && child->get_laddr() != i.get_key()) { + SUBERROR(seastore_fixedkv_tree, + "stable child not valid: child {}, key {}", + *child, + i.get_key()); + ceph_abort(); + return false; + } + } + return true; + } + + void update( + internal_const_iterator_t iter, + lba_map_val_t val, + LogicalCachedExtent* nextent) final { + LOG_PREFIX(LBALeafNode::update); + if (nextent) { + SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, {}", + this->pending_for_transaction, + iter.get_offset(), + *nextent); + // child-ptr may already be correct, see LBAManager::update_mappings() + this->update_child_ptr(iter, nextent); + } + if (val.pladdr.is_paddr()) { + val.pladdr = maybe_generate_relative(val.pladdr.get_paddr()); + } + return this->journal_update( + iter, + val, + this->maybe_get_delta_buffer()); + } + + internal_const_iterator_t insert( + internal_const_iterator_t iter, + laddr_t addr, + lba_map_val_t val, + LogicalCachedExtent* nextent) final { + LOG_PREFIX(LBALeafNode::insert); + SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, key {}, extent {}", + this->pending_for_transaction, + iter.get_offset(), + addr, + (void*)nextent); + this->insert_child_ptr(iter, nextent); + if (val.pladdr.is_paddr()) { + val.pladdr = maybe_generate_relative(val.pladdr.get_paddr()); + } + this->journal_insert( + iter, + addr, + val, + this->maybe_get_delta_buffer()); + return iter; + } + + void remove(internal_const_iterator_t iter) final { + LOG_PREFIX(LBALeafNode::remove); + SUBTRACE(seastore_fixedkv_tree, "trans.{}, pos {}, key {}", + this->pending_for_transaction, + iter.get_offset(), + iter.get_key()); + assert(iter != this->end()); + this->remove_child_ptr(iter); + return this->journal_remove( + iter, + this->maybe_get_delta_buffer()); + } + + // See LBAInternalNode, same concept + void resolve_relative_addrs(paddr_t base); + void node_resolve_vals( + internal_iterator_t from, + internal_iterator_t to) const final + { + if (this->is_initial_pending()) { + for (auto i = from; i != to; ++i) { + auto val = i->get_val(); + if (val.pladdr.is_paddr() + && val.pladdr.get_paddr().is_relative()) { + assert(val.pladdr.get_paddr().is_block_relative()); + val.pladdr = this->get_paddr().add_relative(val.pladdr.get_paddr()); + i->set_val(val); + } + } + } + } + void node_unresolve_vals( + internal_iterator_t from, + internal_iterator_t to) const final + { + if (this->is_initial_pending()) { + for (auto i = from; i != to; ++i) { + auto val = i->get_val(); + if (val.pladdr.is_paddr() + && val.pladdr.get_paddr().is_relative()) { + assert(val.pladdr.get_paddr().is_record_relative()); + val.pladdr = val.pladdr.get_paddr().block_relative_to(this->get_paddr()); + i->set_val(val); + } + } + } + } + + extent_types_t get_type() const final { + return TYPE; + } + + std::ostream &_print_detail(std::ostream &out) const final; +}; +using LBALeafNodeRef = TCachedExtentRef<LBALeafNode>; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::lba_manager::btree::lba_node_meta_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::lba_manager::btree::lba_map_val_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::lba_manager::btree::LBAInternalNode> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::lba_manager::btree::LBALeafNode> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/logging.h b/src/crimson/os/seastore/logging.h new file mode 100644 index 000000000..3f12ee72c --- /dev/null +++ b/src/crimson/os/seastore/logging.h @@ -0,0 +1,30 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <fmt/format.h> + +#include "crimson/common/log.h" + +#define LOGT(level_, MSG, t, ...) \ + LOCAL_LOGGER.log(level_, "{} trans.{} {}: " MSG, (void*)&t, \ + (t).get_trans_id(), FNAME , ##__VA_ARGS__) +#define SUBLOGT(subname_, level_, MSG, t, ...) \ + LOGGER(subname_).log(level_, "{} trans.{} {}: " MSG, (void*)&t, \ + (t).get_trans_id(), FNAME , ##__VA_ARGS__) + +#define TRACET(...) LOGT(seastar::log_level::trace, __VA_ARGS__) +#define SUBTRACET(subname_, ...) SUBLOGT(subname_, seastar::log_level::trace, __VA_ARGS__) + +#define DEBUGT(...) LOGT(seastar::log_level::debug, __VA_ARGS__) +#define SUBDEBUGT(subname_, ...) SUBLOGT(subname_, seastar::log_level::debug, __VA_ARGS__) + +#define INFOT(...) LOGT(seastar::log_level::info, __VA_ARGS__) +#define SUBINFOT(subname_, ...) SUBLOGT(subname_, seastar::log_level::info, __VA_ARGS__) + +#define WARNT(...) LOGT(seastar::log_level::warn, __VA_ARGS__) +#define SUBWARNT(subname_, ...) SUBLOGT(subname_, seastar::log_level::warn, __VA_ARGS__) + +#define ERRORT(...) LOGT(seastar::log_level::error, __VA_ARGS__) +#define SUBERRORT(subname_, ...) SUBLOGT(subname_, seastar::log_level::error, __VA_ARGS__) diff --git a/src/crimson/os/seastore/object_data_handler.cc b/src/crimson/os/seastore/object_data_handler.cc new file mode 100644 index 000000000..0d852696b --- /dev/null +++ b/src/crimson/os/seastore/object_data_handler.cc @@ -0,0 +1,1638 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <utility> +#include <functional> + +#include "crimson/common/log.h" + +#include "crimson/os/seastore/object_data_handler.h" + +namespace { + seastar::logger& logger() { + return crimson::get_logger(ceph_subsys_seastore_odata); + } +} + +SET_SUBSYS(seastore_odata); + +namespace crimson::os::seastore { +#define assert_aligned(x) ceph_assert(((x)%ctx.tm.get_block_size()) == 0) + +using context_t = ObjectDataHandler::context_t; +using get_iertr = ObjectDataHandler::write_iertr; + +/** + * extent_to_write_t + * + * Encapsulates smallest write operations in overwrite. + * Indicates a zero/existing extent or a data extent based on whether + * to_write is populate. + * Should be handled by prepare_ops_list. + */ +struct extent_to_write_t { + enum class type_t { + DATA, + ZERO, + EXISTING, + }; + type_t type; + + /// pin of original extent, not nullptr if type == EXISTING + LBAMappingRef pin; + + laddr_t addr; + extent_len_t len; + + /// non-nullopt if and only if type == DATA + std::optional<bufferlist> to_write; + + extent_to_write_t(const extent_to_write_t &) = delete; + extent_to_write_t(extent_to_write_t &&) = default; + + bool is_data() const { + return type == type_t::DATA; + } + + bool is_zero() const { + return type == type_t::ZERO; + } + + bool is_existing() const { + return type == type_t::EXISTING; + } + + laddr_t get_end_addr() const { + return addr + len; + } + + static extent_to_write_t create_data( + laddr_t addr, bufferlist to_write) { + return extent_to_write_t(addr, to_write); + } + + static extent_to_write_t create_zero( + laddr_t addr, extent_len_t len) { + return extent_to_write_t(addr, len); + } + + static extent_to_write_t create_existing( + LBAMappingRef &&pin, laddr_t addr, extent_len_t len) { + assert(pin); + return extent_to_write_t(std::move(pin), addr, len); + } + +private: + extent_to_write_t(laddr_t addr, bufferlist to_write) + : type(type_t::DATA), addr(addr), len(to_write.length()), + to_write(to_write) {} + + extent_to_write_t(laddr_t addr, extent_len_t len) + : type(type_t::ZERO), addr(addr), len(len) {} + + extent_to_write_t(LBAMappingRef &&pin, laddr_t addr, extent_len_t len) + : type(type_t::EXISTING), pin(std::move(pin)), addr(addr), len(len) {} +}; +using extent_to_write_list_t = std::list<extent_to_write_t>; + +// Encapsulates extents to be written out using do_remappings. +struct extent_to_remap_t { + enum class type_t { + REMAP, + OVERWRITE + }; + type_t type; + /// pin of original extent + LBAMappingRef pin; + /// offset of remapped extent or overwrite part of overwrite extent. + /// overwrite part of overwrite extent might correspond to mutiple + /// fresh write extent. + extent_len_t new_offset; + /// length of remapped extent or overwrite part of overwrite extent + extent_len_t new_len; + + extent_to_remap_t(const extent_to_remap_t &) = delete; + extent_to_remap_t(extent_to_remap_t &&) = default; + + bool is_remap() const { + return type == type_t::REMAP; + } + + bool is_overwrite() const { + assert((new_offset != 0) && (pin->get_length() != new_offset + new_len)); + return type == type_t::OVERWRITE; + } + + using remap_entry = TransactionManager::remap_entry; + remap_entry create_remap_entry() { + assert(is_remap()); + return remap_entry( + new_offset, + new_len); + } + + remap_entry create_left_remap_entry() { + assert(is_overwrite()); + return remap_entry( + 0, + new_offset); + } + + remap_entry create_right_remap_entry() { + assert(is_overwrite()); + return remap_entry( + new_offset + new_len, + pin->get_length() - new_offset - new_len); + } + + static extent_to_remap_t create_remap( + LBAMappingRef &&pin, extent_len_t new_offset, extent_len_t new_len) { + return extent_to_remap_t(type_t::REMAP, + std::move(pin), new_offset, new_len); + } + + static extent_to_remap_t create_overwrite( + LBAMappingRef &&pin, extent_len_t new_offset, extent_len_t new_len) { + return extent_to_remap_t(type_t::OVERWRITE, + std::move(pin), new_offset, new_len); + } + +private: + extent_to_remap_t(type_t type, + LBAMappingRef &&pin, extent_len_t new_offset, extent_len_t new_len) + : type(type), + pin(std::move(pin)), new_offset(new_offset), new_len(new_len) {} +}; +using extent_to_remap_list_t = std::list<extent_to_remap_t>; + +// Encapsulates extents to be written out using do_insertions. +struct extent_to_insert_t { + enum class type_t { + DATA, + ZERO + }; + type_t type; + /// laddr of new extent + laddr_t addr; + /// length of new extent + extent_len_t len; + /// non-nullopt if type == DATA + std::optional<bufferlist> bl; + + extent_to_insert_t(const extent_to_insert_t &) = default; + extent_to_insert_t(extent_to_insert_t &&) = default; + + bool is_data() const { + return type == type_t::DATA; + } + + bool is_zero() const { + return type == type_t::ZERO; + } + + static extent_to_insert_t create_data( + laddr_t addr, extent_len_t len, std::optional<bufferlist> bl) { + return extent_to_insert_t(addr, len, bl); + } + + static extent_to_insert_t create_zero( + laddr_t addr, extent_len_t len) { + return extent_to_insert_t(addr, len); + } + +private: + extent_to_insert_t(laddr_t addr, extent_len_t len, + std::optional<bufferlist> bl) + :type(type_t::DATA), addr(addr), len(len), bl(bl) {} + + extent_to_insert_t(laddr_t addr, extent_len_t len) + :type(type_t::ZERO), addr(addr), len(len) {} +}; +using extent_to_insert_list_t = std::list<extent_to_insert_t>; + +// Encapsulates extents to be retired in do_removals. +using extent_to_remove_list_t = std::list<LBAMappingRef>; + +struct overwrite_ops_t { + extent_to_remap_list_t to_remap; + extent_to_insert_list_t to_insert; + extent_to_remove_list_t to_remove; +}; + +// prepare to_remap, to_retire, to_insert list +overwrite_ops_t prepare_ops_list( + lba_pin_list_t &pins_to_remove, + extent_to_write_list_t &to_write) { + assert(pins_to_remove.size() != 0); + overwrite_ops_t ops; + ops.to_remove.swap(pins_to_remove); + if (to_write.empty()) { + logger().debug("empty to_write"); + return ops; + } + long unsigned int visitted = 0; + auto& front = to_write.front(); + auto& back = to_write.back(); + + // prepare overwrite, happens in one original extent. + if (ops.to_remove.size() == 1 && + front.is_existing() && back.is_existing()) { + visitted += 2; + assert(to_write.size() > 2); + assert(front.addr == front.pin->get_key()); + assert(back.addr > back.pin->get_key()); + ops.to_remap.push_back(extent_to_remap_t::create_overwrite( + std::move(front.pin), + front.len, + back.addr - front.addr - front.len)); + ops.to_remove.pop_front(); + } else { + // prepare to_remap, happens in one or multiple extents + if (front.is_existing()) { + visitted++; + assert(to_write.size() > 1); + assert(front.addr == front.pin->get_key()); + ops.to_remap.push_back(extent_to_remap_t::create_remap( + std::move(front.pin), + 0, + front.len)); + ops.to_remove.pop_front(); + } + if (back.is_existing()) { + visitted++; + assert(to_write.size() > 1); + assert(back.addr + back.len == + back.pin->get_key() + back.pin->get_length()); + ops.to_remap.push_back(extent_to_remap_t::create_remap( + std::move(back.pin), + back.addr - back.pin->get_key(), + back.len)); + ops.to_remove.pop_back(); + } + } + + // prepare to_insert + for (auto ®ion : to_write) { + if (region.is_data()) { + visitted++; + assert(region.to_write.has_value()); + ops.to_insert.push_back(extent_to_insert_t::create_data( + region.addr, region.len, region.to_write)); + } else if (region.is_zero()) { + visitted++; + assert(!(region.to_write.has_value())); + ops.to_insert.push_back(extent_to_insert_t::create_zero( + region.addr, region.len)); + } + } + + logger().debug( + "to_remap list size: {}" + " to_insert list size: {}" + " to_remove list size: {}", + ops.to_remap.size(), ops.to_insert.size(), ops.to_remove.size()); + assert(visitted == to_write.size()); + return ops; +} + +/** + * append_extent_to_write + * + * Appends passed extent_to_write_t maintaining invariant that the + * list may not contain consecutive zero elements by checking and + * combining them. + */ +void append_extent_to_write( + extent_to_write_list_t &to_write, extent_to_write_t &&to_append) +{ + assert(to_write.empty() || + to_write.back().get_end_addr() == to_append.addr); + if (to_write.empty() || + to_write.back().is_data() || + to_append.is_data() || + to_write.back().type != to_append.type) { + to_write.push_back(std::move(to_append)); + } else { + to_write.back().len += to_append.len; + } +} + +/** + * splice_extent_to_write + * + * splices passed extent_to_write_list_t maintaining invariant that the + * list may not contain consecutive zero elements by checking and + * combining them. + */ +void splice_extent_to_write( + extent_to_write_list_t &to_write, extent_to_write_list_t &&to_splice) +{ + if (!to_splice.empty()) { + append_extent_to_write(to_write, std::move(to_splice.front())); + to_splice.pop_front(); + to_write.splice(to_write.end(), std::move(to_splice)); + } +} + +/// Creates remap extents in to_remap +ObjectDataHandler::write_ret do_remappings( + context_t ctx, + extent_to_remap_list_t &to_remap) +{ + return trans_intr::do_for_each( + to_remap, + [ctx](auto ®ion) { + if (region.is_remap()) { + return ctx.tm.remap_pin<ObjectDataBlock, 1>( + ctx.t, + std::move(region.pin), + std::array{ + region.create_remap_entry() + } + ).si_then([®ion](auto pins) { + ceph_assert(pins.size() == 1); + ceph_assert(region.new_len == pins[0]->get_length()); + return ObjectDataHandler::write_iertr::now(); + }); + } else if (region.is_overwrite()) { + return ctx.tm.remap_pin<ObjectDataBlock, 2>( + ctx.t, + std::move(region.pin), + std::array{ + region.create_left_remap_entry(), + region.create_right_remap_entry() + } + ).si_then([®ion](auto pins) { + ceph_assert(pins.size() == 2); + ceph_assert(region.pin->get_key() == pins[0]->get_key()); + ceph_assert(region.pin->get_key() + pins[0]->get_length() + + region.new_len == pins[1]->get_key()); + return ObjectDataHandler::write_iertr::now(); + }); + } else { + ceph_abort("impossible"); + return ObjectDataHandler::write_iertr::now(); + } + }); +} + +ObjectDataHandler::write_ret do_removals( + context_t ctx, + lba_pin_list_t &to_remove) +{ + return trans_intr::do_for_each( + to_remove, + [ctx](auto &pin) { + LOG_PREFIX(object_data_handler.cc::do_removals); + DEBUGT("decreasing ref: {}", + ctx.t, + pin->get_key()); + return ctx.tm.dec_ref( + ctx.t, + pin->get_key() + ).si_then( + [](auto){}, + ObjectDataHandler::write_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "object_data_handler::do_removals invalid error" + } + ); + }); +} + +/// Creates zero/data extents in to_insert +ObjectDataHandler::write_ret do_insertions( + context_t ctx, + extent_to_insert_list_t &to_insert) +{ + return trans_intr::do_for_each( + to_insert, + [ctx](auto ®ion) { + LOG_PREFIX(object_data_handler.cc::do_insertions); + if (region.is_data()) { + assert_aligned(region.addr); + assert_aligned(region.len); + ceph_assert(region.len == region.bl->length()); + DEBUGT("allocating extent: {}~{}", + ctx.t, + region.addr, + region.len); + return ctx.tm.alloc_extent<ObjectDataBlock>( + ctx.t, + region.addr, + region.len + ).si_then([®ion](auto extent) { + if (extent->get_laddr() != region.addr) { + logger().debug( + "object_data_handler::do_insertions alloc got addr {}," + " should have been {}", + extent->get_laddr(), + region.addr); + } + ceph_assert(extent->get_laddr() == region.addr); + ceph_assert(extent->get_length() == region.len); + auto iter = region.bl->cbegin(); + iter.copy(region.len, extent->get_bptr().c_str()); + return ObjectDataHandler::write_iertr::now(); + }); + } else if (region.is_zero()) { + DEBUGT("reserving: {}~{}", + ctx.t, + region.addr, + region.len); + return ctx.tm.reserve_region( + ctx.t, + region.addr, + region.len + ).si_then([FNAME, ctx, ®ion](auto pin) { + ceph_assert(pin->get_length() == region.len); + if (pin->get_key() != region.addr) { + ERRORT( + "inconsistent laddr: pin: {} region {}", + ctx.t, + pin->get_key(), + region.addr); + } + ceph_assert(pin->get_key() == region.addr); + return ObjectDataHandler::write_iertr::now(); + }); + } else { + ceph_abort("impossible"); + return ObjectDataHandler::write_iertr::now(); + } + }); +} + +enum class overwrite_operation_t { + UNKNOWN, + OVERWRITE_ZERO, // fill unaligned data with zero + MERGE_EXISTING, // if present, merge data with the clean/pending extent + SPLIT_EXISTING, // split the existing extent, and fill unaligned data +}; + +std::ostream& operator<<( + std::ostream &out, + const overwrite_operation_t &operation) +{ + switch (operation) { + case overwrite_operation_t::UNKNOWN: + return out << "UNKNOWN"; + case overwrite_operation_t::OVERWRITE_ZERO: + return out << "OVERWRITE_ZERO"; + case overwrite_operation_t::MERGE_EXISTING: + return out << "MERGE_EXISTING"; + case overwrite_operation_t::SPLIT_EXISTING: + return out << "SPLIT_EXISTING"; + default: + return out << "!IMPOSSIBLE_OPERATION"; + } +} + +/** + * overwrite_plan_t + * + * |<--------------------------pins_size---------------------------------------------->| + * pin_begin(aligned) pin_end(aligned) + * |<------aligned_data_size-------------------------->| (aligned-bl) + * aligned_data_begin aligned_data_end + * |<-data_size->| (bl) + * data_begin end + * left(l) right(r) + * |<l_extent_size>|<l_alignment_size>| |<r_alignment_size>|<r_extent_size>| + * |<-----------left_size------------>| |<-----------right_size----------->| + * + * |<-----(existing left extent/pin)----->| |<-----(existing right extent/pin)----->| + * left_paddr right_paddr + */ +struct overwrite_plan_t { + // addresses + laddr_t pin_begin; + laddr_t pin_end; + paddr_t left_paddr; + paddr_t right_paddr; + laddr_t data_begin; + laddr_t data_end; + laddr_t aligned_data_begin; + laddr_t aligned_data_end; + + // operations + overwrite_operation_t left_operation; + overwrite_operation_t right_operation; + + // helper member + extent_len_t block_size; + +public: + extent_len_t get_left_size() const { + return data_begin - pin_begin; + } + + extent_len_t get_left_extent_size() const { + return aligned_data_begin - pin_begin; + } + + extent_len_t get_left_alignment_size() const { + return data_begin - aligned_data_begin; + } + + extent_len_t get_right_size() const { + return pin_end - data_end; + } + + extent_len_t get_right_extent_size() const { + return pin_end - aligned_data_end; + } + + extent_len_t get_right_alignment_size() const { + return aligned_data_end - data_end; + } + + extent_len_t get_aligned_data_size() const { + return aligned_data_end - aligned_data_begin; + } + + extent_len_t get_pins_size() const { + return pin_end - pin_begin; + } + + friend std::ostream& operator<<( + std::ostream& out, + const overwrite_plan_t& overwrite_plan) { + return out << "overwrite_plan_t(" + << "pin_begin=" << overwrite_plan.pin_begin + << ", pin_end=" << overwrite_plan.pin_end + << ", left_paddr=" << overwrite_plan.left_paddr + << ", right_paddr=" << overwrite_plan.right_paddr + << ", data_begin=" << overwrite_plan.data_begin + << ", data_end=" << overwrite_plan.data_end + << ", aligned_data_begin=" << overwrite_plan.aligned_data_begin + << ", aligned_data_end=" << overwrite_plan.aligned_data_end + << ", left_operation=" << overwrite_plan.left_operation + << ", right_operation=" << overwrite_plan.right_operation + << ", block_size=" << overwrite_plan.block_size + << ")"; + } + + overwrite_plan_t(laddr_t offset, + extent_len_t len, + const lba_pin_list_t& pins, + extent_len_t block_size, + Transaction& t) : + pin_begin(pins.front()->get_key()), + pin_end(pins.back()->get_key() + pins.back()->get_length()), + left_paddr(pins.front()->get_val()), + right_paddr(pins.back()->get_val()), + data_begin(offset), + data_end(offset + len), + aligned_data_begin(p2align((uint64_t)data_begin, (uint64_t)block_size)), + aligned_data_end(p2roundup((uint64_t)data_end, (uint64_t)block_size)), + left_operation(overwrite_operation_t::UNKNOWN), + right_operation(overwrite_operation_t::UNKNOWN), + block_size(block_size) { + validate(); + evaluate_operations(t); + assert(left_operation != overwrite_operation_t::UNKNOWN); + assert(right_operation != overwrite_operation_t::UNKNOWN); + } + +private: + // refer to overwrite_plan_t description + void validate() const { + ceph_assert(pin_begin % block_size == 0); + ceph_assert(pin_end % block_size == 0); + ceph_assert(aligned_data_begin % block_size == 0); + ceph_assert(aligned_data_end % block_size == 0); + + ceph_assert(pin_begin <= aligned_data_begin); + ceph_assert(aligned_data_begin <= data_begin); + ceph_assert(data_begin <= data_end); + ceph_assert(data_end <= aligned_data_end); + ceph_assert(aligned_data_end <= pin_end); + } + + /* + * When trying to modify a portion of an object data block, follow + * the read-full-extent-then-merge-new-data strategy, if the write + * amplification caused by it is not greater than + * seastore_obj_data_write_amplification; otherwise, split the + * original extent into at most three parts: origin-left, part-to-be-modified + * and origin-right. + */ + void evaluate_operations(Transaction& t) { + auto actual_write_size = get_pins_size(); + auto aligned_data_size = get_aligned_data_size(); + auto left_ext_size = get_left_extent_size(); + auto right_ext_size = get_right_extent_size(); + + auto can_merge = [](Transaction& t, paddr_t paddr) { + CachedExtentRef ext; + if (paddr.is_relative() || paddr.is_delayed()) { + return true; + } else if (t.get_extent(paddr, &ext) == + Transaction::get_extent_ret::PRESENT) { + // FIXME: there is no need to lookup the cache if the pin can + // be associated with the extent state + if (ext->is_mutable()) { + return true; + } + } + return false; + }; + if (left_paddr.is_zero()) { + actual_write_size -= left_ext_size; + left_ext_size = 0; + left_operation = overwrite_operation_t::OVERWRITE_ZERO; + } else if (can_merge(t, left_paddr)) { + aligned_data_size += left_ext_size; + left_ext_size = 0; + left_operation = overwrite_operation_t::MERGE_EXISTING; + } + + if (right_paddr.is_zero()) { + actual_write_size -= right_ext_size; + right_ext_size = 0; + right_operation = overwrite_operation_t::OVERWRITE_ZERO; + } else if (can_merge(t, right_paddr)) { + aligned_data_size += right_ext_size; + right_ext_size = 0; + right_operation = overwrite_operation_t::MERGE_EXISTING; + } + + while (left_operation == overwrite_operation_t::UNKNOWN || + right_operation == overwrite_operation_t::UNKNOWN) { + if (((double)actual_write_size / (double)aligned_data_size) <= + crimson::common::get_conf<double>("seastore_obj_data_write_amplification")) { + break; + } + if (left_ext_size == 0 && right_ext_size == 0) { + break; + } + if (left_ext_size >= right_ext_size) { + // split left + assert(left_operation == overwrite_operation_t::UNKNOWN); + actual_write_size -= left_ext_size; + left_ext_size = 0; + left_operation = overwrite_operation_t::SPLIT_EXISTING; + } else { // left_ext_size < right_ext_size + // split right + assert(right_operation == overwrite_operation_t::UNKNOWN); + actual_write_size -= right_ext_size; + right_ext_size = 0; + right_operation = overwrite_operation_t::SPLIT_EXISTING; + } + } + + if (left_operation == overwrite_operation_t::UNKNOWN) { + // no split left, so merge with left + left_operation = overwrite_operation_t::MERGE_EXISTING; + } + + if (right_operation == overwrite_operation_t::UNKNOWN) { + // no split right, so merge with right + right_operation = overwrite_operation_t::MERGE_EXISTING; + } + } +}; + +} // namespace crimson::os::seastore + +#if FMT_VERSION >= 90000 +template<> struct fmt::formatter<crimson::os::seastore::overwrite_plan_t> : fmt::ostream_formatter {}; +#endif + +namespace crimson::os::seastore { + +/** + * operate_left + * + * Proceed overwrite_plan.left_operation. + */ +using operate_ret_bare = std::pair< + std::optional<extent_to_write_t>, + std::optional<bufferptr>>; +using operate_ret = get_iertr::future<operate_ret_bare>; +operate_ret operate_left(context_t ctx, LBAMappingRef &pin, const overwrite_plan_t &overwrite_plan) +{ + if (overwrite_plan.get_left_size() == 0) { + return get_iertr::make_ready_future<operate_ret_bare>( + std::nullopt, + std::nullopt); + } + + if (overwrite_plan.left_operation == overwrite_operation_t::OVERWRITE_ZERO) { + assert(pin->get_val().is_zero()); + auto zero_extent_len = overwrite_plan.get_left_extent_size(); + assert_aligned(zero_extent_len); + auto zero_prepend_len = overwrite_plan.get_left_alignment_size(); + return get_iertr::make_ready_future<operate_ret_bare>( + (zero_extent_len == 0 + ? std::nullopt + : std::make_optional(extent_to_write_t::create_zero( + overwrite_plan.pin_begin, zero_extent_len))), + (zero_prepend_len == 0 + ? std::nullopt + : std::make_optional(bufferptr( + ceph::buffer::create(zero_prepend_len, 0)))) + ); + } else if (overwrite_plan.left_operation == overwrite_operation_t::MERGE_EXISTING) { + auto prepend_len = overwrite_plan.get_left_size(); + if (prepend_len == 0) { + return get_iertr::make_ready_future<operate_ret_bare>( + std::nullopt, + std::nullopt); + } else { + extent_len_t off = pin->get_intermediate_offset(); + return ctx.tm.read_pin<ObjectDataBlock>( + ctx.t, pin->duplicate() + ).si_then([prepend_len, off](auto left_extent) { + return get_iertr::make_ready_future<operate_ret_bare>( + std::nullopt, + std::make_optional(bufferptr( + left_extent->get_bptr(), + off, + prepend_len))); + }); + } + } else { + assert(overwrite_plan.left_operation == overwrite_operation_t::SPLIT_EXISTING); + + auto extent_len = overwrite_plan.get_left_extent_size(); + assert(extent_len); + std::optional<extent_to_write_t> left_to_write_extent = + std::make_optional(extent_to_write_t::create_existing( + pin->duplicate(), + pin->get_key(), + extent_len)); + + auto prepend_len = overwrite_plan.get_left_alignment_size(); + if (prepend_len == 0) { + return get_iertr::make_ready_future<operate_ret_bare>( + std::move(left_to_write_extent), + std::nullopt); + } else { + extent_len_t off = pin->get_intermediate_offset(); + return ctx.tm.read_pin<ObjectDataBlock>( + ctx.t, pin->duplicate() + ).si_then([prepend_offset=extent_len + off, prepend_len, + left_to_write_extent=std::move(left_to_write_extent)] + (auto left_extent) mutable { + return get_iertr::make_ready_future<operate_ret_bare>( + std::move(left_to_write_extent), + std::make_optional(bufferptr( + left_extent->get_bptr(), + prepend_offset, + prepend_len))); + }); + } + } +}; + +/** + * operate_right + * + * Proceed overwrite_plan.right_operation. + */ +operate_ret operate_right(context_t ctx, LBAMappingRef &pin, const overwrite_plan_t &overwrite_plan) +{ + if (overwrite_plan.get_right_size() == 0) { + return get_iertr::make_ready_future<operate_ret_bare>( + std::nullopt, + std::nullopt); + } + + auto right_pin_begin = pin->get_key(); + assert(overwrite_plan.data_end >= right_pin_begin); + if (overwrite_plan.right_operation == overwrite_operation_t::OVERWRITE_ZERO) { + assert(pin->get_val().is_zero()); + auto zero_suffix_len = overwrite_plan.get_right_alignment_size(); + auto zero_extent_len = overwrite_plan.get_right_extent_size(); + assert_aligned(zero_extent_len); + return get_iertr::make_ready_future<operate_ret_bare>( + (zero_extent_len == 0 + ? std::nullopt + : std::make_optional(extent_to_write_t::create_zero( + overwrite_plan.aligned_data_end, zero_extent_len))), + (zero_suffix_len == 0 + ? std::nullopt + : std::make_optional(bufferptr( + ceph::buffer::create(zero_suffix_len, 0)))) + ); + } else if (overwrite_plan.right_operation == overwrite_operation_t::MERGE_EXISTING) { + auto append_len = overwrite_plan.get_right_size(); + if (append_len == 0) { + return get_iertr::make_ready_future<operate_ret_bare>( + std::nullopt, + std::nullopt); + } else { + auto append_offset = + overwrite_plan.data_end + - right_pin_begin + + pin->get_intermediate_offset(); + return ctx.tm.read_pin<ObjectDataBlock>( + ctx.t, pin->duplicate() + ).si_then([append_offset, append_len](auto right_extent) { + return get_iertr::make_ready_future<operate_ret_bare>( + std::nullopt, + std::make_optional(bufferptr( + right_extent->get_bptr(), + append_offset, + append_len))); + }); + } + } else { + assert(overwrite_plan.right_operation == overwrite_operation_t::SPLIT_EXISTING); + + auto extent_len = overwrite_plan.get_right_extent_size(); + assert(extent_len); + std::optional<extent_to_write_t> right_to_write_extent = + std::make_optional(extent_to_write_t::create_existing( + pin->duplicate(), + overwrite_plan.aligned_data_end, + extent_len)); + + auto append_len = overwrite_plan.get_right_alignment_size(); + if (append_len == 0) { + return get_iertr::make_ready_future<operate_ret_bare>( + std::move(right_to_write_extent), + std::nullopt); + } else { + auto append_offset = + overwrite_plan.data_end + - right_pin_begin + + pin->get_intermediate_offset(); + return ctx.tm.read_pin<ObjectDataBlock>( + ctx.t, pin->duplicate() + ).si_then([append_offset, append_len, + right_to_write_extent=std::move(right_to_write_extent)] + (auto right_extent) mutable { + return get_iertr::make_ready_future<operate_ret_bare>( + std::move(right_to_write_extent), + std::make_optional(bufferptr( + right_extent->get_bptr(), + append_offset, + append_len))); + }); + } + } +}; + +template <typename F> +auto with_object_data( + ObjectDataHandler::context_t ctx, + F &&f) +{ + return seastar::do_with( + ctx.onode.get_layout().object_data.get(), + std::forward<F>(f), + [ctx](auto &object_data, auto &f) { + return std::invoke(f, object_data + ).si_then([ctx, &object_data] { + if (object_data.must_update()) { + ctx.onode.get_mutable_layout(ctx.t).object_data.update(object_data); + } + return seastar::now(); + }); + }); +} + +template <typename F> +auto with_objects_data( + ObjectDataHandler::context_t ctx, + F &&f) +{ + ceph_assert(ctx.d_onode); + return seastar::do_with( + ctx.onode.get_layout().object_data.get(), + ctx.d_onode->get_layout().object_data.get(), + std::forward<F>(f), + [ctx](auto &object_data, auto &d_object_data, auto &f) { + return std::invoke(f, object_data, d_object_data + ).si_then([ctx, &object_data, &d_object_data] { + if (object_data.must_update()) { + ctx.onode.get_mutable_layout(ctx.t).object_data.update(object_data); + } + if (d_object_data.must_update()) { + ctx.d_onode->get_mutable_layout( + ctx.t).object_data.update(d_object_data); + } + return seastar::now(); + }); + }); +} + +ObjectDataHandler::write_ret ObjectDataHandler::prepare_data_reservation( + context_t ctx, + object_data_t &object_data, + extent_len_t size) +{ + LOG_PREFIX(ObjectDataHandler::prepare_data_reservation); + ceph_assert(size <= max_object_size); + if (!object_data.is_null()) { + ceph_assert(object_data.get_reserved_data_len() == max_object_size); + DEBUGT("reservation present: {}~{}", + ctx.t, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len()); + return write_iertr::now(); + } else { + DEBUGT("reserving: {}~{}", + ctx.t, + ctx.onode.get_data_hint(), + max_object_size); + return ctx.tm.reserve_region( + ctx.t, + ctx.onode.get_data_hint(), + max_object_size + ).si_then([max_object_size=max_object_size, &object_data](auto pin) { + ceph_assert(pin->get_length() == max_object_size); + object_data.update_reserved( + pin->get_key(), + pin->get_length()); + return write_iertr::now(); + }); + } +} + +ObjectDataHandler::clear_ret ObjectDataHandler::trim_data_reservation( + context_t ctx, object_data_t &object_data, extent_len_t size) +{ + ceph_assert(!object_data.is_null()); + ceph_assert(size <= object_data.get_reserved_data_len()); + return seastar::do_with( + lba_pin_list_t(), + extent_to_write_list_t(), + [ctx, size, &object_data](auto &pins, auto &to_write) { + LOG_PREFIX(ObjectDataHandler::trim_data_reservation); + DEBUGT("object_data: {}~{}", + ctx.t, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len()); + return ctx.tm.get_pins( + ctx.t, + object_data.get_reserved_data_base() + size, + object_data.get_reserved_data_len() - size + ).si_then([ctx, size, &pins, &object_data, &to_write](auto _pins) { + _pins.swap(pins); + ceph_assert(pins.size()); + if (!size) { + // no need to reserve region if we are truncating the object's + // size to 0 + return clear_iertr::now(); + } + auto &pin = *pins.front(); + ceph_assert(pin.get_key() >= object_data.get_reserved_data_base()); + ceph_assert( + pin.get_key() <= object_data.get_reserved_data_base() + size); + auto pin_offset = pin.get_key() - + object_data.get_reserved_data_base(); + if ((pin.get_key() == (object_data.get_reserved_data_base() + size)) || + (pin.get_val().is_zero())) { + /* First pin is exactly at the boundary or is a zero pin. Either way, + * remove all pins and add a single zero pin to the end. */ + to_write.push_back(extent_to_write_t::create_zero( + pin.get_key(), + object_data.get_reserved_data_len() - pin_offset)); + return clear_iertr::now(); + } else { + /* First pin overlaps the boundary and has data, remap it + * if aligned or rewrite it if not aligned to size */ + auto roundup_size = p2roundup(size, ctx.tm.get_block_size()); + auto append_len = roundup_size - size; + if (append_len == 0) { + LOG_PREFIX(ObjectDataHandler::trim_data_reservation); + TRACET("First pin overlaps the boundary and has aligned data" + "create existing at addr:{}, len:{}", + ctx.t, pin.get_key(), size - pin_offset); + to_write.push_back(extent_to_write_t::create_existing( + pin.duplicate(), + pin.get_key(), + size - pin_offset)); + to_write.push_back(extent_to_write_t::create_zero( + object_data.get_reserved_data_base() + roundup_size, + object_data.get_reserved_data_len() - roundup_size)); + return clear_iertr::now(); + } else { + return ctx.tm.read_pin<ObjectDataBlock>( + ctx.t, + pin.duplicate() + ).si_then([ctx, size, pin_offset, append_len, roundup_size, + &pin, &object_data, &to_write](auto extent) { + bufferlist bl; + bl.append( + bufferptr( + extent->get_bptr(), + pin.get_intermediate_offset(), + size - pin_offset + )); + bl.append_zero(append_len); + LOG_PREFIX(ObjectDataHandler::trim_data_reservation); + TRACET("First pin overlaps the boundary and has unaligned data" + "create data at addr:{}, len:{}", + ctx.t, pin.get_key(), bl.length()); + to_write.push_back(extent_to_write_t::create_data( + pin.get_key(), + bl)); + to_write.push_back(extent_to_write_t::create_zero( + object_data.get_reserved_data_base() + roundup_size, + object_data.get_reserved_data_len() - roundup_size)); + return clear_iertr::now(); + }); + } + } + }).si_then([ctx, size, &to_write, &object_data, &pins] { + return seastar::do_with( + prepare_ops_list(pins, to_write), + [ctx, size, &object_data](auto &ops) { + return do_remappings(ctx, ops.to_remap + ).si_then([ctx, &ops] { + return do_removals(ctx, ops.to_remove); + }).si_then([ctx, &ops] { + return do_insertions(ctx, ops.to_insert); + }).si_then([size, &object_data] { + if (size == 0) { + object_data.clear(); + } + return ObjectDataHandler::clear_iertr::now(); + }); + }); + }); + }); +} + +/** + * get_to_writes_with_zero_buffer + * + * Returns extent_to_write_t's reflecting a zero region extending + * from offset~len with headptr optionally on the left and tailptr + * optionally on the right. + */ +extent_to_write_list_t get_to_writes_with_zero_buffer( + const extent_len_t block_size, + laddr_t offset, extent_len_t len, + std::optional<bufferptr> &&headptr, std::optional<bufferptr> &&tailptr) +{ + auto zero_left = p2roundup(offset, (laddr_t)block_size); + auto zero_right = p2align(offset + len, (laddr_t)block_size); + auto left = headptr ? (offset - headptr->length()) : offset; + auto right = tailptr ? + (offset + len + tailptr->length()) : + (offset + len); + + assert( + (headptr && ((zero_left - left) == + p2roundup(headptr->length(), block_size))) ^ + (!headptr && (zero_left == left))); + assert( + (tailptr && ((right - zero_right) == + p2roundup(tailptr->length(), block_size))) ^ + (!tailptr && (right == zero_right))); + + assert(right > left); + assert((left % block_size) == 0); + assert((right % block_size) == 0); + + // zero region too small for a reserved section, + // headptr and tailptr in same extent + if (zero_right <= zero_left) { + bufferlist bl; + if (headptr) { + bl.append(*headptr); + } + bl.append_zero( + right - left - bl.length() - (tailptr ? tailptr->length() : 0)); + if (tailptr) { + bl.append(*tailptr); + } + assert(bl.length() % block_size == 0); + assert(bl.length() == (right - left)); + extent_to_write_list_t ret; + ret.push_back(extent_to_write_t::create_data(left, bl)); + return ret; + } else { + // reserved section between ends, headptr and tailptr in different extents + extent_to_write_list_t ret; + if (headptr) { + bufferlist headbl; + headbl.append(*headptr); + headbl.append_zero(zero_left - left - headbl.length()); + assert(headbl.length() % block_size == 0); + assert(headbl.length() > 0); + ret.push_back(extent_to_write_t::create_data(left, headbl)); + } + // reserved zero region + ret.push_back(extent_to_write_t::create_zero(zero_left, zero_right - zero_left)); + assert(ret.back().len % block_size == 0); + assert(ret.back().len > 0); + if (tailptr) { + bufferlist tailbl; + tailbl.append(*tailptr); + tailbl.append_zero(right - zero_right - tailbl.length()); + assert(tailbl.length() % block_size == 0); + assert(tailbl.length() > 0); + ret.push_back(extent_to_write_t::create_data(zero_right, tailbl)); + } + return ret; + } +} + +/** + * get_to_writes + * + * Returns extent_to_write_t's from bl. + * + * TODO: probably add some kind of upper limit on extent size. + */ +extent_to_write_list_t get_to_writes(laddr_t offset, bufferlist &bl) +{ + auto ret = extent_to_write_list_t(); + ret.push_back(extent_to_write_t::create_data(offset, bl)); + return ret; +}; + +ObjectDataHandler::write_ret ObjectDataHandler::overwrite( + context_t ctx, + laddr_t offset, + extent_len_t len, + std::optional<bufferlist> &&bl, + lba_pin_list_t &&_pins) +{ + if (bl.has_value()) { + assert(bl->length() == len); + } + overwrite_plan_t overwrite_plan(offset, len, _pins, ctx.tm.get_block_size(), ctx.t); + return seastar::do_with( + std::move(_pins), + extent_to_write_list_t(), + [ctx, len, offset, overwrite_plan, bl=std::move(bl)] + (auto &pins, auto &to_write) mutable + { + LOG_PREFIX(ObjectDataHandler::overwrite); + DEBUGT("overwrite: {}~{}", + ctx.t, + offset, + len); + ceph_assert(pins.size() >= 1); + DEBUGT("overwrite: split overwrite_plan {}", ctx.t, overwrite_plan); + + return operate_left( + ctx, + pins.front(), + overwrite_plan + ).si_then([ctx, len, offset, overwrite_plan, bl=std::move(bl), + &to_write, &pins](auto p) mutable { + auto &[left_extent, headptr] = p; + if (left_extent) { + ceph_assert(left_extent->addr == overwrite_plan.pin_begin); + append_extent_to_write(to_write, std::move(*left_extent)); + } + if (headptr) { + assert(headptr->length() > 0); + } + return operate_right( + ctx, + pins.back(), + overwrite_plan + ).si_then([ctx, len, offset, + pin_begin=overwrite_plan.pin_begin, + pin_end=overwrite_plan.pin_end, + bl=std::move(bl), headptr=std::move(headptr), + &to_write, &pins](auto p) mutable { + auto &[right_extent, tailptr] = p; + if (bl.has_value()) { + auto write_offset = offset; + bufferlist write_bl; + if (headptr) { + write_bl.append(*headptr); + write_offset -= headptr->length(); + assert_aligned(write_offset); + } + write_bl.claim_append(*bl); + if (tailptr) { + write_bl.append(*tailptr); + assert_aligned(write_bl.length()); + } + splice_extent_to_write( + to_write, + get_to_writes(write_offset, write_bl)); + } else { + splice_extent_to_write( + to_write, + get_to_writes_with_zero_buffer( + ctx.tm.get_block_size(), + offset, + len, + std::move(headptr), + std::move(tailptr))); + } + if (right_extent) { + ceph_assert(right_extent->get_end_addr() == pin_end); + append_extent_to_write(to_write, std::move(*right_extent)); + } + assert(to_write.size()); + assert(pin_begin == to_write.front().addr); + assert(pin_end == to_write.back().get_end_addr()); + + return seastar::do_with( + prepare_ops_list(pins, to_write), + [ctx](auto &ops) { + return do_remappings(ctx, ops.to_remap + ).si_then([ctx, &ops] { + return do_removals(ctx, ops.to_remove); + }).si_then([ctx, &ops] { + return do_insertions(ctx, ops.to_insert); + }); + }); + }); + }); + }); +} + +ObjectDataHandler::zero_ret ObjectDataHandler::zero( + context_t ctx, + objaddr_t offset, + extent_len_t len) +{ + return with_object_data( + ctx, + [this, ctx, offset, len](auto &object_data) { + LOG_PREFIX(ObjectDataHandler::zero); + DEBUGT("zero to {}~{}, object_data: {}~{}, is_null {}", + ctx.t, + offset, + len, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len(), + object_data.is_null()); + return prepare_data_reservation( + ctx, + object_data, + p2roundup(offset + len, ctx.tm.get_block_size()) + ).si_then([this, ctx, offset, len, &object_data] { + auto logical_offset = object_data.get_reserved_data_base() + offset; + return ctx.tm.get_pins( + ctx.t, + logical_offset, + len + ).si_then([this, ctx, logical_offset, len](auto pins) { + return overwrite( + ctx, logical_offset, len, + std::nullopt, std::move(pins)); + }); + }); + }); +} + +ObjectDataHandler::write_ret ObjectDataHandler::write( + context_t ctx, + objaddr_t offset, + const bufferlist &bl) +{ + return with_object_data( + ctx, + [this, ctx, offset, &bl](auto &object_data) { + LOG_PREFIX(ObjectDataHandler::write); + DEBUGT("writing to {}~{}, object_data: {}~{}, is_null {}", + ctx.t, + offset, + bl.length(), + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len(), + object_data.is_null()); + return prepare_data_reservation( + ctx, + object_data, + p2roundup(offset + bl.length(), ctx.tm.get_block_size()) + ).si_then([this, ctx, offset, &object_data, &bl] { + auto logical_offset = object_data.get_reserved_data_base() + offset; + return ctx.tm.get_pins( + ctx.t, + logical_offset, + bl.length() + ).si_then([this, ctx,logical_offset, &bl]( + auto pins) { + return overwrite( + ctx, logical_offset, bl.length(), + bufferlist(bl), std::move(pins)); + }); + }); + }); +} + +ObjectDataHandler::read_ret ObjectDataHandler::read( + context_t ctx, + objaddr_t obj_offset, + extent_len_t len) +{ + return seastar::do_with( + bufferlist(), + [ctx, obj_offset, len](auto &ret) { + return with_object_data( + ctx, + [ctx, obj_offset, len, &ret](const auto &object_data) { + LOG_PREFIX(ObjectDataHandler::read); + DEBUGT("reading {}~{}", + ctx.t, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len()); + /* Assumption: callers ensure that onode size is <= reserved + * size and that len is adjusted here prior to call */ + ceph_assert(!object_data.is_null()); + ceph_assert((obj_offset + len) <= object_data.get_reserved_data_len()); + ceph_assert(len > 0); + laddr_t loffset = + object_data.get_reserved_data_base() + obj_offset; + return ctx.tm.get_pins( + ctx.t, + loffset, + len + ).si_then([ctx, loffset, len, &ret](auto _pins) { + // offset~len falls within reserved region and len > 0 + ceph_assert(_pins.size() >= 1); + ceph_assert((*_pins.begin())->get_key() <= loffset); + return seastar::do_with( + std::move(_pins), + loffset, + [ctx, loffset, len, &ret](auto &pins, auto ¤t) { + return trans_intr::do_for_each( + pins, + [ctx, loffset, len, ¤t, &ret](auto &pin) + -> read_iertr::future<> { + ceph_assert(current <= (loffset + len)); + ceph_assert( + (loffset + len) > pin->get_key()); + laddr_t end = std::min( + pin->get_key() + pin->get_length(), + loffset + len); + if (pin->get_val().is_zero()) { + ceph_assert(end > current); // See LBAManager::get_mappings + ret.append_zero(end - current); + current = end; + return seastar::now(); + } else { + LOG_PREFIX(ObjectDataHandler::read); + auto key = pin->get_key(); + bool is_indirect = pin->is_indirect(); + extent_len_t off = pin->get_intermediate_offset(); + DEBUGT("reading {}~{}, indirect: {}, " + "intermediate offset: {}, current: {}, end: {}", + ctx.t, + key, + pin->get_length(), + is_indirect, + off, + current, + end); + return ctx.tm.read_pin<ObjectDataBlock>( + ctx.t, + std::move(pin) + ).si_then([&ret, ¤t, end, key, off, + is_indirect](auto extent) { + ceph_assert( + is_indirect + ? (key - off + extent->get_length()) >= end + : (extent->get_laddr() + extent->get_length()) >= end); + ceph_assert(end > current); + ret.append( + bufferptr( + extent->get_bptr(), + off + current - (is_indirect ? key : extent->get_laddr()), + end - current)); + current = end; + return seastar::now(); + }).handle_error_interruptible( + read_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "ObjectDataHandler::read hit invalid error" + } + ); + } + }); + }); + }); + }).si_then([&ret] { + return std::move(ret); + }); + }); +} + +ObjectDataHandler::fiemap_ret ObjectDataHandler::fiemap( + context_t ctx, + objaddr_t obj_offset, + extent_len_t len) +{ + return seastar::do_with( + std::map<uint64_t, uint64_t>(), + [ctx, obj_offset, len](auto &ret) { + return with_object_data( + ctx, + [ctx, obj_offset, len, &ret](const auto &object_data) { + LOG_PREFIX(ObjectDataHandler::fiemap); + DEBUGT( + "{}~{}, reservation {}~{}", + ctx.t, + obj_offset, + len, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len()); + /* Assumption: callers ensure that onode size is <= reserved + * size and that len is adjusted here prior to call */ + ceph_assert(!object_data.is_null()); + ceph_assert((obj_offset + len) <= object_data.get_reserved_data_len()); + ceph_assert(len > 0); + laddr_t loffset = + object_data.get_reserved_data_base() + obj_offset; + return ctx.tm.get_pins( + ctx.t, + loffset, + len + ).si_then([loffset, len, &object_data, &ret](auto &&pins) { + ceph_assert(pins.size() >= 1); + ceph_assert((*pins.begin())->get_key() <= loffset); + for (auto &&i: pins) { + if (!(i->get_val().is_zero())) { + auto ret_left = std::max(i->get_key(), loffset); + auto ret_right = std::min( + i->get_key() + i->get_length(), + loffset + len); + assert(ret_right > ret_left); + ret.emplace( + std::make_pair( + ret_left - object_data.get_reserved_data_base(), + ret_right - ret_left + )); + } + } + }); + }).si_then([&ret] { + return std::move(ret); + }); + }); +} + +ObjectDataHandler::truncate_ret ObjectDataHandler::truncate( + context_t ctx, + objaddr_t offset) +{ + return with_object_data( + ctx, + [this, ctx, offset](auto &object_data) { + LOG_PREFIX(ObjectDataHandler::truncate); + DEBUGT("truncating {}~{} offset: {}", + ctx.t, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len(), + offset); + if (offset < object_data.get_reserved_data_len()) { + return trim_data_reservation(ctx, object_data, offset); + } else if (offset > object_data.get_reserved_data_len()) { + return prepare_data_reservation( + ctx, + object_data, + p2roundup(offset, ctx.tm.get_block_size())); + } else { + return truncate_iertr::now(); + } + }); +} + +ObjectDataHandler::clear_ret ObjectDataHandler::clear( + context_t ctx) +{ + return with_object_data( + ctx, + [this, ctx](auto &object_data) { + LOG_PREFIX(ObjectDataHandler::clear); + DEBUGT("clearing: {}~{}", + ctx.t, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len()); + if (object_data.is_null()) { + return clear_iertr::now(); + } + return trim_data_reservation(ctx, object_data, 0); + }); +} + +ObjectDataHandler::clone_ret ObjectDataHandler::clone_extents( + context_t ctx, + object_data_t &object_data, + lba_pin_list_t &pins, + laddr_t data_base) +{ + LOG_PREFIX(ObjectDataHandler::clone_extents); + TRACET(" object_data: {}~{}, data_base: {}", + ctx.t, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len(), + data_base); + return ctx.tm.dec_ref( + ctx.t, + object_data.get_reserved_data_base() + ).si_then( + [&pins, &object_data, ctx, data_base](auto) mutable { + return seastar::do_with( + (extent_len_t)0, + [&object_data, ctx, data_base, &pins](auto &last_pos) { + return trans_intr::do_for_each( + pins, + [&last_pos, &object_data, ctx, data_base](auto &pin) { + auto offset = pin->get_key() - data_base; + ceph_assert(offset == last_pos); + auto fut = TransactionManager::alloc_extent_iertr + ::make_ready_future<LBAMappingRef>(); + auto addr = object_data.get_reserved_data_base() + offset; + if (pin->get_val().is_zero()) { + fut = ctx.tm.reserve_region(ctx.t, addr, pin->get_length()); + } else { + fut = ctx.tm.clone_pin(ctx.t, addr, *pin); + } + return fut.si_then( + [&pin, &last_pos, offset](auto) { + last_pos = offset + pin->get_length(); + return seastar::now(); + }).handle_error_interruptible( + crimson::ct_error::input_output_error::pass_further(), + crimson::ct_error::assert_all("not possible") + ); + }).si_then([&last_pos, &object_data, ctx] { + if (last_pos != object_data.get_reserved_data_len()) { + return ctx.tm.reserve_region( + ctx.t, + object_data.get_reserved_data_base() + last_pos, + object_data.get_reserved_data_len() - last_pos + ).si_then([](auto) { + return seastar::now(); + }); + } + return TransactionManager::reserve_extent_iertr::now(); + }); + }); + }, + ObjectDataHandler::write_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "object_data_handler::clone invalid error" + } + ); +} + +ObjectDataHandler::clone_ret ObjectDataHandler::clone( + context_t ctx) +{ + // the whole clone procedure can be seperated into the following steps: + // 1. let clone onode(d_object_data) take the head onode's + // object data base; + // 2. reserve a new region in lba tree for the head onode; + // 3. clone all extents of the clone onode, see transaction_manager.h + // for the details of clone_pin; + // 4. reserve the space between the head onode's size and its reservation + // length. + return with_objects_data( + ctx, + [ctx, this](auto &object_data, auto &d_object_data) { + ceph_assert(d_object_data.is_null()); + if (object_data.is_null()) { + return clone_iertr::now(); + } + return prepare_data_reservation( + ctx, + d_object_data, + object_data.get_reserved_data_len() + ).si_then([&object_data, &d_object_data, ctx, this] { + assert(!object_data.is_null()); + auto base = object_data.get_reserved_data_base(); + auto len = object_data.get_reserved_data_len(); + object_data.clear(); + LOG_PREFIX(ObjectDataHandler::clone); + DEBUGT("cloned obj reserve_data_base: {}, len {}", + ctx.t, + d_object_data.get_reserved_data_base(), + d_object_data.get_reserved_data_len()); + return prepare_data_reservation( + ctx, + object_data, + d_object_data.get_reserved_data_len() + ).si_then([&d_object_data, ctx, &object_data, base, len, this] { + LOG_PREFIX("ObjectDataHandler::clone"); + DEBUGT("head obj reserve_data_base: {}, len {}", + ctx.t, + object_data.get_reserved_data_base(), + object_data.get_reserved_data_len()); + return ctx.tm.get_pins(ctx.t, base, len + ).si_then([ctx, &object_data, &d_object_data, base, this](auto pins) { + return seastar::do_with( + std::move(pins), + [ctx, &object_data, &d_object_data, base, this](auto &pins) { + return clone_extents(ctx, object_data, pins, base + ).si_then([ctx, &d_object_data, base, &pins, this] { + return clone_extents(ctx, d_object_data, pins, base); + }).si_then([&pins, ctx] { + return do_removals(ctx, pins); + }); + }); + }); + }); + }); + }); +} + +} // namespace crimson::os::seastore diff --git a/src/crimson/os/seastore/object_data_handler.h b/src/crimson/os/seastore/object_data_handler.h new file mode 100644 index 000000000..b5f432d5a --- /dev/null +++ b/src/crimson/os/seastore/object_data_handler.h @@ -0,0 +1,156 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <iostream> +#include <limits> + +#include "include/buffer.h" + +#include "test/crimson/seastore/test_block.h" // TODO + +#include "crimson/os/seastore/onode.h" +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/transaction.h" + +namespace crimson::os::seastore { + +struct ObjectDataBlock : crimson::os::seastore::LogicalCachedExtent { + using Ref = TCachedExtentRef<ObjectDataBlock>; + + explicit ObjectDataBlock(ceph::bufferptr &&ptr) + : LogicalCachedExtent(std::move(ptr)) {} + explicit ObjectDataBlock(const ObjectDataBlock &other) + : LogicalCachedExtent(other) {} + explicit ObjectDataBlock(extent_len_t length) + : LogicalCachedExtent(length) {} + + CachedExtentRef duplicate_for_write(Transaction&) final { + return CachedExtentRef(new ObjectDataBlock(*this)); + }; + + static constexpr extent_types_t TYPE = extent_types_t::OBJECT_DATA_BLOCK; + extent_types_t get_type() const final { + return TYPE; + } + + ceph::bufferlist get_delta() final { + /* Currently, we always allocate fresh ObjectDataBlock's rather than + * mutating existing ones. */ + ceph_assert(0 == "Should be impossible"); + } + + void apply_delta(const ceph::bufferlist &bl) final { + // See get_delta() + ceph_assert(0 == "Should be impossible"); + } +}; +using ObjectDataBlockRef = TCachedExtentRef<ObjectDataBlock>; + +class ObjectDataHandler { +public: + using base_iertr = TransactionManager::base_iertr; + + ObjectDataHandler(uint32_t mos) : max_object_size(mos) {} + + struct context_t { + TransactionManager &tm; + Transaction &t; + Onode &onode; + Onode *d_onode = nullptr; // The desination node in case of clone + }; + + /// Writes bl to [offset, offset + bl.length()) + using write_iertr = base_iertr; + using write_ret = write_iertr::future<>; + write_ret write( + context_t ctx, + objaddr_t offset, + const bufferlist &bl); + + using zero_iertr = base_iertr; + using zero_ret = zero_iertr::future<>; + zero_ret zero( + context_t ctx, + objaddr_t offset, + extent_len_t len); + + /// Reads data in [offset, offset + len) + using read_iertr = base_iertr; + using read_ret = read_iertr::future<bufferlist>; + read_ret read( + context_t ctx, + objaddr_t offset, + extent_len_t len); + + /// sparse read data, get range interval in [offset, offset + len) + using fiemap_iertr = base_iertr; + using fiemap_ret = fiemap_iertr::future<std::map<uint64_t, uint64_t>>; + fiemap_ret fiemap( + context_t ctx, + objaddr_t offset, + extent_len_t len); + + /// Clears data past offset + using truncate_iertr = base_iertr; + using truncate_ret = truncate_iertr::future<>; + truncate_ret truncate( + context_t ctx, + objaddr_t offset); + + /// Clears data and reservation + using clear_iertr = base_iertr; + using clear_ret = clear_iertr::future<>; + clear_ret clear(context_t ctx); + + /// Clone data of an Onode + using clone_iertr = base_iertr; + using clone_ret = clone_iertr::future<>; + clone_ret clone(context_t ctx); + +private: + /// Updates region [_offset, _offset + bl.length) to bl + write_ret overwrite( + context_t ctx, ///< [in] ctx + laddr_t offset, ///< [in] write offset + extent_len_t len, ///< [in] len to write, len == bl->length() if bl + std::optional<bufferlist> &&bl, ///< [in] buffer to write, empty for zeros + lba_pin_list_t &&pins ///< [in] set of pins overlapping above region + ); + + /// Ensures object_data reserved region is prepared + write_ret prepare_data_reservation( + context_t ctx, + object_data_t &object_data, + extent_len_t size); + + /// Trims data past size + clear_ret trim_data_reservation( + context_t ctx, + object_data_t &object_data, + extent_len_t size); + + clone_ret clone_extents( + context_t ctx, + object_data_t &object_data, + lba_pin_list_t &pins, + laddr_t data_base); + +private: + /** + * max_object_size + * + * For now, we allocate a fixed region of laddr space of size max_object_size + * for any object. In the future, once we have the ability to remap logical + * mappings (necessary for clone), we'll add the ability to grow and shrink + * these regions and remove this assumption. + */ + const uint32_t max_object_size = 0; +}; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::ObjectDataBlock> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/omap_manager.cc b/src/crimson/os/seastore/omap_manager.cc new file mode 100644 index 000000000..7ad37a2e9 --- /dev/null +++ b/src/crimson/os/seastore/omap_manager.cc @@ -0,0 +1,42 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab +#include <experimental/iterator> +#include <iostream> + +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/omap_manager.h" +#include "crimson/os/seastore/omap_manager/btree/btree_omap_manager.h" + +namespace crimson::os::seastore::omap_manager { + +OMapManagerRef create_omap_manager(TransactionManager &trans_manager) { + return OMapManagerRef(new BtreeOMapManager(trans_manager)); +} + +} + +namespace std { +std::ostream &operator<<(std::ostream &out, const std::pair<std::string, std::string> &rhs) +{ + return out << "key_value_map (" << rhs.first<< "->" << rhs.second << ")"; +} +} + +namespace crimson::os::seastore { + +std::ostream &operator<<(std::ostream &out, const std::list<std::string> &rhs) +{ + out << '['; + std::copy(std::begin(rhs), std::end(rhs), std::experimental::make_ostream_joiner(out, ", ")); + return out << ']'; +} + +std::ostream &operator<<(std::ostream &out, const std::vector<std::pair<std::string, std::string>> &rhs) +{ + out << '['; + std::ostream_iterator<std::pair<std::string, std::string>> out_it(out, ", "); + std::copy(rhs.begin(), rhs.end(), out_it); + return out << ']'; +} + +} diff --git a/src/crimson/os/seastore/omap_manager.h b/src/crimson/os/seastore/omap_manager.h new file mode 100644 index 000000000..fc4e03e2b --- /dev/null +++ b/src/crimson/os/seastore/omap_manager.h @@ -0,0 +1,210 @@ +// -*- 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_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include <seastar/core/future.hh> + +#include "crimson/osd/exceptions.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/transaction_manager.h" + +#define OMAP_INNER_BLOCK_SIZE 4096 +#define OMAP_LEAF_BLOCK_SIZE 8192 + +namespace crimson::os::seastore { + +std::ostream &operator<<(std::ostream &out, const std::list<std::string> &rhs); +std::ostream &operator<<(std::ostream &out, const std::map<std::string, std::string> &rhs); + +class OMapManager { + /* all OMapManager API use reference to transfer input string parameters, + * the upper caller should guarantee the referenced string values alive (not freed) + * until these functions future resolved. + */ +public: + using base_iertr = TransactionManager::base_iertr; + + /** + * allocate omap tree root node + * + * @param Transaction &t, current transaction + * @retval return the omap_root_t structure. + */ + using initialize_omap_iertr = base_iertr; + using initialize_omap_ret = initialize_omap_iertr::future<omap_root_t>; + virtual initialize_omap_ret initialize_omap(Transaction &t, laddr_t hint) = 0; + + /** + * get value(string) by key(string) + * + * @param omap_root_t &omap_root, omap btree root information + * @param Transaction &t, current transaction + * @param string &key, omap string key + * @retval return string key->string value mapping pair. + */ + using omap_get_value_iertr = base_iertr; + using omap_get_value_ret = omap_get_value_iertr::future< + std::optional<bufferlist>>; + virtual omap_get_value_ret omap_get_value( + const omap_root_t &omap_root, + Transaction &t, + const std::string &key) = 0; + + /** + * set key value mapping in omap + * + * @param omap_root_t &omap_root, omap btree root information + * @param Transaction &t, current transaction + * @param string &key, omap string key + * @param string &value, mapped value corresponding key + */ + using omap_set_key_iertr = base_iertr; + using omap_set_key_ret = omap_set_key_iertr::future<>; + virtual omap_set_key_ret omap_set_key( + omap_root_t &omap_root, + Transaction &t, + const std::string &key, + const ceph::bufferlist &value) = 0; + + using omap_set_keys_iertr = base_iertr; + using omap_set_keys_ret = omap_set_keys_iertr::future<>; + virtual omap_set_keys_ret omap_set_keys( + omap_root_t &omap_root, + Transaction &t, + std::map<std::string, ceph::bufferlist>&& keys) = 0; + + /** + * remove key value mapping in omap tree + * + * @param omap_root_t &omap_root, omap btree root information + * @param Transaction &t, current transaction + * @param string &key, omap string key + */ + using omap_rm_key_iertr = base_iertr; + using omap_rm_key_ret = omap_rm_key_iertr::future<>; + virtual omap_rm_key_ret omap_rm_key( + omap_root_t &omap_root, + Transaction &t, + const std::string &key) = 0; + + /** + * omap_list + * + * Scans key/value pairs in order. + * + * @param omap_root: omap btree root information + * @param t: current transaction + * @param first: range start, nullopt sorts before any string, + * behavior based on config.inclusive, + * must alive during the call + * @param last: range end, nullopt sorts after any string, + * behavior based on config.inclusive, + * must alive during the call + * @param config: see below for params + * @retval listed key->value and bool indicating complete + */ + struct omap_list_config_t { + /// max results to return + size_t max_result_size = 128; + + /// true denotes behavior like lower_bound, upper_bound otherwise + /// range start behavior + bool first_inclusive = false; + /// range end behavior + bool last_inclusive = false; + + omap_list_config_t( + size_t max_result_size, + bool first_inclusive, + bool last_inclusive) + : max_result_size(max_result_size), + first_inclusive(first_inclusive), + last_inclusive(last_inclusive) {} + omap_list_config_t() {} + omap_list_config_t(const omap_list_config_t &) = default; + omap_list_config_t(omap_list_config_t &&) = default; + omap_list_config_t &operator=(const omap_list_config_t &) = default; + omap_list_config_t &operator=(omap_list_config_t &&) = default; + + auto with_max(size_t max) { + this->max_result_size = max; + return *this; + } + + auto without_max() { + this->max_result_size = std::numeric_limits<size_t>::max(); + return *this; + } + + auto with_inclusive( + bool first_inclusive, + bool last_inclusive) { + this->first_inclusive = first_inclusive; + this->last_inclusive = last_inclusive; + return *this; + } + + auto with_reduced_max(size_t reduced_by) const { + assert(reduced_by <= max_result_size); + return omap_list_config_t( + max_result_size - reduced_by, + first_inclusive, + last_inclusive); + } + }; + using omap_list_iertr = base_iertr; + using omap_list_bare_ret = std::tuple< + bool, + std::map<std::string, bufferlist, std::less<>>>; + using omap_list_ret = omap_list_iertr::future<omap_list_bare_ret>; + virtual omap_list_ret omap_list( + const omap_root_t &omap_root, + Transaction &t, + const std::optional<std::string> &first, + const std::optional<std::string> &last, + omap_list_config_t config = omap_list_config_t()) = 0; + + /** + * remove key value mappings in a key range from omap tree + * + * @param omap_root_t &omap_root, omap btree root information + * @param Transaction &t, current transaction + * @param string &first, range start + * @param string &last, range end + */ + using omap_rm_key_range_iertr = base_iertr; + using omap_rm_key_range_ret = omap_rm_key_range_iertr::future<>; + virtual omap_rm_key_range_ret omap_rm_key_range( + omap_root_t &omap_root, + Transaction &t, + const std::string &first, + const std::string &last, + omap_list_config_t config) = 0; + + /** + * clear all omap tree key->value mapping + * + * @param omap_root_t &omap_root, omap btree root information + * @param Transaction &t, current transaction + */ + using omap_clear_iertr = base_iertr; + using omap_clear_ret = omap_clear_iertr::future<>; + virtual omap_clear_ret omap_clear(omap_root_t &omap_root, Transaction &t) = 0; + + virtual ~OMapManager() {} +}; +using OMapManagerRef = std::unique_ptr<OMapManager>; + +namespace omap_manager { + +OMapManagerRef create_omap_manager ( + TransactionManager &trans_manager); +} + +} diff --git a/src/crimson/os/seastore/omap_manager/btree/btree_omap_manager.cc b/src/crimson/os/seastore/omap_manager/btree/btree_omap_manager.cc new file mode 100644 index 000000000..1782d7ee6 --- /dev/null +++ b/src/crimson/os/seastore/omap_manager/btree/btree_omap_manager.cc @@ -0,0 +1,293 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <string.h> + +#include "crimson/common/log.h" + +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/omap_manager/btree/btree_omap_manager.h" +#include "crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.h" + +SET_SUBSYS(seastore_omap); + +namespace crimson::os::seastore::omap_manager { + +BtreeOMapManager::BtreeOMapManager( + TransactionManager &tm) + : tm(tm) {} + +BtreeOMapManager::initialize_omap_ret +BtreeOMapManager::initialize_omap(Transaction &t, laddr_t hint) +{ + LOG_PREFIX(BtreeOMapManager::initialize_omap); + DEBUGT("hint: {}", t, hint); + return tm.alloc_extent<OMapLeafNode>(t, hint, OMAP_LEAF_BLOCK_SIZE) + .si_then([hint, &t](auto&& root_extent) { + root_extent->set_size(0); + omap_node_meta_t meta{1}; + root_extent->set_meta(meta); + omap_root_t omap_root; + omap_root.update(root_extent->get_laddr(), 1, hint); + t.get_omap_tree_stats().depth = 1u; + t.get_omap_tree_stats().extents_num_delta++; + return initialize_omap_iertr::make_ready_future<omap_root_t>(omap_root); + }); +} + +BtreeOMapManager::get_root_ret +BtreeOMapManager::get_omap_root(omap_context_t oc, const omap_root_t &omap_root) +{ + assert(omap_root.get_location() != L_ADDR_NULL); + laddr_t laddr = omap_root.get_location(); + return omap_load_extent(oc, laddr, omap_root.get_depth()); +} + +BtreeOMapManager::handle_root_split_ret +BtreeOMapManager::handle_root_split( + omap_context_t oc, + omap_root_t &omap_root, + const OMapNode::mutation_result_t& mresult) +{ + LOG_PREFIX(BtreeOMapManager::handle_root_split); + DEBUGT("{}", oc.t, omap_root); + return oc.tm.alloc_extent<OMapInnerNode>(oc.t, omap_root.hint, + OMAP_INNER_BLOCK_SIZE) + .si_then([&omap_root, mresult, oc](auto&& nroot) -> handle_root_split_ret { + auto [left, right, pivot] = *(mresult.split_tuple); + omap_node_meta_t meta{omap_root.depth + 1}; + nroot->set_meta(meta); + nroot->journal_inner_insert(nroot->iter_begin(), left->get_laddr(), + "", nroot->maybe_get_delta_buffer()); + nroot->journal_inner_insert(nroot->iter_begin() + 1, right->get_laddr(), + pivot, nroot->maybe_get_delta_buffer()); + omap_root.update(nroot->get_laddr(), omap_root.get_depth() + 1, omap_root.hint); + oc.t.get_omap_tree_stats().depth = omap_root.depth; + ++(oc.t.get_omap_tree_stats().extents_num_delta); + return seastar::now(); + }); +} + +BtreeOMapManager::handle_root_merge_ret +BtreeOMapManager::handle_root_merge( + omap_context_t oc, + omap_root_t &omap_root, + OMapNode::mutation_result_t mresult) +{ + LOG_PREFIX(BtreeOMapManager::handle_root_merge); + DEBUGT("{}", oc.t, omap_root); + auto root = *(mresult.need_merge); + auto iter = root->cast<OMapInnerNode>()->iter_begin(); + omap_root.update( + iter->get_val(), + omap_root.depth -= 1, + omap_root.hint); + oc.t.get_omap_tree_stats().depth = omap_root.depth; + oc.t.get_omap_tree_stats().extents_num_delta--; + return oc.tm.dec_ref(oc.t, root->get_laddr() + ).si_then([](auto &&ret) -> handle_root_merge_ret { + return seastar::now(); + }).handle_error_interruptible( + handle_root_merge_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in handle_root_merge" + } + ); +} + +BtreeOMapManager::omap_get_value_ret +BtreeOMapManager::omap_get_value( + const omap_root_t &omap_root, + Transaction &t, + const std::string &key) +{ + LOG_PREFIX(BtreeOMapManager::omap_get_value); + DEBUGT("key={}", t, key); + return get_omap_root( + get_omap_context(t, omap_root.hint), + omap_root + ).si_then([this, &t, &key, &omap_root](auto&& extent) { + return extent->get_value(get_omap_context(t, omap_root.hint), key); + }).si_then([](auto &&e) { + return omap_get_value_ret( + interruptible::ready_future_marker{}, + std::move(e)); + }); +} + +BtreeOMapManager::omap_set_keys_ret +BtreeOMapManager::omap_set_keys( + omap_root_t &omap_root, + Transaction &t, + std::map<std::string, ceph::bufferlist>&& keys) +{ + return seastar::do_with(std::move(keys), [&, this](auto& keys) { + return trans_intr::do_for_each( + keys.begin(), + keys.end(), + [&, this](auto &p) { + return omap_set_key(omap_root, t, p.first, p.second); + }); + }); +} + +BtreeOMapManager::omap_set_key_ret +BtreeOMapManager::omap_set_key( + omap_root_t &omap_root, + Transaction &t, + const std::string &key, + const ceph::bufferlist &value) +{ + LOG_PREFIX(BtreeOMapManager::omap_set_key); + DEBUGT("{} -> {}", t, key, value); + return get_omap_root( + get_omap_context(t, omap_root.hint), + omap_root + ).si_then([this, &t, &key, &value, &omap_root](auto root) { + return root->insert(get_omap_context(t, omap_root.hint), key, value); + }).si_then([this, &omap_root, &t](auto mresult) -> omap_set_key_ret { + if (mresult.status == mutation_status_t::SUCCESS) + return seastar::now(); + else if (mresult.status == mutation_status_t::WAS_SPLIT) + return handle_root_split(get_omap_context(t, omap_root.hint), omap_root, mresult); + else + return seastar::now(); + }); +} + +BtreeOMapManager::omap_rm_key_ret +BtreeOMapManager::omap_rm_key( + omap_root_t &omap_root, + Transaction &t, + const std::string &key) +{ + LOG_PREFIX(BtreeOMapManager::omap_rm_key); + DEBUGT("{}", t, key); + return get_omap_root( + get_omap_context(t, omap_root.hint), + omap_root + ).si_then([this, &t, &key, &omap_root](auto root) { + return root->rm_key(get_omap_context(t, omap_root.hint), key); + }).si_then([this, &omap_root, &t](auto mresult) -> omap_rm_key_ret { + if (mresult.status == mutation_status_t::SUCCESS) { + return seastar::now(); + } else if (mresult.status == mutation_status_t::WAS_SPLIT) { + return handle_root_split(get_omap_context(t, omap_root.hint), omap_root, mresult); + } else if (mresult.status == mutation_status_t::NEED_MERGE) { + auto root = *(mresult.need_merge); + if (root->get_node_size() == 1 && omap_root.depth != 1) { + return handle_root_merge(get_omap_context(t, omap_root.hint), omap_root, mresult); + } else { + return seastar::now(); + } + } else { + return seastar::now(); + } + }); + +} + +BtreeOMapManager::omap_rm_key_range_ret +BtreeOMapManager::omap_rm_key_range( + omap_root_t &omap_root, + Transaction &t, + const std::string &first, + const std::string &last, + omap_list_config_t config) +{ + LOG_PREFIX(BtreeOMapManager::omap_rm_key_range); + DEBUGT("{} ~ {}", t, first, last); + assert(first <= last); + return seastar::do_with( + std::make_optional<std::string>(first), + std::make_optional<std::string>(last), + [this, &omap_root, &t, config](auto &first, auto &last) { + return omap_list( + omap_root, + t, + first, + last, + config); + }).si_then([this, &omap_root, &t](auto results) { + LOG_PREFIX(BtreeOMapManager::omap_rm_key_range); + auto &[complete, kvs] = results; + std::vector<std::string> keys; + for (const auto& [k, _] : kvs) { + keys.push_back(k); + } + DEBUGT("total {} keys to remove", t, keys.size()); + return seastar::do_with( + std::move(keys), + [this, &omap_root, &t](auto& keys) { + return trans_intr::do_for_each( + keys.begin(), + keys.end(), + [this, &omap_root, &t](auto& key) { + return omap_rm_key(omap_root, t, key); + }); + }); + }); +} + +BtreeOMapManager::omap_list_ret +BtreeOMapManager::omap_list( + const omap_root_t &omap_root, + Transaction &t, + const std::optional<std::string> &first, + const std::optional<std::string> &last, + omap_list_config_t config) +{ + LOG_PREFIX(BtreeOMapManager::omap_list); + if (first && last) { + DEBUGT("{}, first: {}, last: {}", t, omap_root, *first, *last); + assert(last >= first); + } else if (first) { + DEBUGT("{}, first: {}", t, omap_root, *first); + } else if (last) { + DEBUGT("{}, last: {}", t, omap_root, *last); + } else { + DEBUGT("{}", t, omap_root); + } + + return get_omap_root( + get_omap_context(t, omap_root.hint), + omap_root + ).si_then([this, config, &t, &first, &last, &omap_root](auto extent) { + return extent->list( + get_omap_context(t, omap_root.hint), + first, + last, + config); + }); +} + +BtreeOMapManager::omap_clear_ret +BtreeOMapManager::omap_clear( + omap_root_t &omap_root, + Transaction &t) +{ + LOG_PREFIX(BtreeOMapManager::omap_clear); + DEBUGT("{}", t, omap_root); + return get_omap_root( + get_omap_context(t, omap_root.hint), + omap_root + ).si_then([this, &t, &omap_root](auto extent) { + return extent->clear(get_omap_context(t, omap_root.hint)); + }).si_then([this, &omap_root, &t] { + return tm.dec_ref( + t, omap_root.get_location() + ).si_then([&omap_root] (auto ret) { + omap_root.update( + L_ADDR_NULL, + 0, L_ADDR_MIN); + return omap_clear_iertr::now(); + }); + }).handle_error_interruptible( + omap_clear_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in BtreeOMapManager::omap_clear" + } + ); +} + +} diff --git a/src/crimson/os/seastore/omap_manager/btree/btree_omap_manager.h b/src/crimson/os/seastore/omap_manager/btree/btree_omap_manager.h new file mode 100644 index 000000000..7fcba64c0 --- /dev/null +++ b/src/crimson/os/seastore/omap_manager/btree/btree_omap_manager.h @@ -0,0 +1,111 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <seastar/core/future.hh> + +#include "include/ceph_assert.h" +#include "crimson/osd/exceptions.h" + +#include "crimson/os/seastore/omap_manager.h" +#include "crimson/os/seastore/omap_manager/btree/omap_btree_node.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/transaction_manager.h" + +namespace crimson::os::seastore::omap_manager { +/** + * BtreeOMapManager + * + * Uses a btree to track : + * string -> string mapping for each onode omap + */ + +class BtreeOMapManager : public OMapManager { + TransactionManager &tm; + + omap_context_t get_omap_context( + Transaction &t, laddr_t addr_min) { + return omap_context_t{tm, t, addr_min}; + } + + /* get_omap_root + * + * load omap tree root node + */ + using get_root_iertr = base_iertr; + using get_root_ret = get_root_iertr::future<OMapNodeRef>; + static get_root_ret get_omap_root( + omap_context_t c, + const omap_root_t &omap_root); + + /* handle_root_split + * + * root has been split and needs to update omap_root_t + */ + using handle_root_split_iertr = base_iertr; + using handle_root_split_ret = handle_root_split_iertr::future<>; + handle_root_split_ret handle_root_split( + omap_context_t c, + omap_root_t &omap_root, + const OMapNode::mutation_result_t& mresult); + + /* handle_root_merge + * + * root node has only one item and it is not leaf node, need remove a layer + */ + using handle_root_merge_iertr = base_iertr; + using handle_root_merge_ret = handle_root_merge_iertr::future<>; + handle_root_merge_ret handle_root_merge( + omap_context_t oc, + omap_root_t &omap_root, + OMapNode:: mutation_result_t mresult); + +public: + explicit BtreeOMapManager(TransactionManager &tm); + + initialize_omap_ret initialize_omap(Transaction &t, laddr_t hint) final; + + omap_get_value_ret omap_get_value( + const omap_root_t &omap_root, + Transaction &t, + const std::string &key) final; + + omap_set_key_ret omap_set_key( + omap_root_t &omap_root, + Transaction &t, + const std::string &key, const ceph::bufferlist &value) final; + + omap_set_keys_ret omap_set_keys( + omap_root_t &omap_root, + Transaction &t, + std::map<std::string, ceph::bufferlist>&& keys) final; + + omap_rm_key_ret omap_rm_key( + omap_root_t &omap_root, + Transaction &t, + const std::string &key) final; + + omap_rm_key_range_ret omap_rm_key_range( + omap_root_t &omap_root, + Transaction &t, + const std::string &first, + const std::string &last, + omap_list_config_t config) final; + + omap_list_ret omap_list( + const omap_root_t &omap_root, + Transaction &t, + const std::optional<std::string> &first, + const std::optional<std::string> &last, + omap_list_config_t config = omap_list_config_t()) final; + + omap_clear_ret omap_clear( + omap_root_t &omap_root, + Transaction &t) final; + +}; +using BtreeOMapManagerRef = std::unique_ptr<BtreeOMapManager>; + +} diff --git a/src/crimson/os/seastore/omap_manager/btree/omap_btree_node.h b/src/crimson/os/seastore/omap_manager/btree/omap_btree_node.h new file mode 100644 index 000000000..795daeddb --- /dev/null +++ b/src/crimson/os/seastore/omap_manager/btree/omap_btree_node.h @@ -0,0 +1,122 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab +#pragma once + +#include <string> +#include <vector> + +//#include <boost/iterator/counting_iterator.hpp> + +#include "crimson/common/log.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/omap_manager.h" +#include "crimson/os/seastore/omap_manager/btree/omap_types.h" + +namespace crimson::os::seastore::omap_manager{ + +struct omap_context_t { + TransactionManager &tm; + Transaction &t; + laddr_t hint; +}; + +enum class mutation_status_t : uint8_t { + SUCCESS = 0, + WAS_SPLIT = 1, + NEED_MERGE = 2, + FAIL = 3 +}; + +struct OMapNode : LogicalCachedExtent { + using base_iertr = OMapManager::base_iertr; + + using OMapNodeRef = TCachedExtentRef<OMapNode>; + + struct mutation_result_t { + mutation_status_t status; + /// Only populated if WAS_SPLIT, indicates the newly created left and right nodes + /// from splitting the target entry during insertion. + std::optional<std::tuple<OMapNodeRef, OMapNodeRef, std::string>> split_tuple; + /// only sopulated if need merged, indicate which entry need be doing merge in upper layer. + std::optional<OMapNodeRef> need_merge; + + mutation_result_t(mutation_status_t s, std::optional<std::tuple<OMapNodeRef, + OMapNodeRef, std::string>> tuple, std::optional<OMapNodeRef> n_merge) + : status(s), + split_tuple(tuple), + need_merge(n_merge) {} + }; + + OMapNode(ceph::bufferptr &&ptr) : LogicalCachedExtent(std::move(ptr)) {} + OMapNode(const OMapNode &other) + : LogicalCachedExtent(other) {} + + using get_value_iertr = base_iertr; + using get_value_ret = OMapManager::omap_get_value_ret; + virtual get_value_ret get_value( + omap_context_t oc, + const std::string &key) = 0; + + using insert_iertr = base_iertr; + using insert_ret = insert_iertr::future<mutation_result_t>; + virtual insert_ret insert( + omap_context_t oc, + const std::string &key, + const ceph::bufferlist &value) = 0; + + using rm_key_iertr = base_iertr; + using rm_key_ret = rm_key_iertr::future<mutation_result_t>; + virtual rm_key_ret rm_key( + omap_context_t oc, + const std::string &key) = 0; + + using omap_list_config_t = OMapManager::omap_list_config_t; + using list_iertr = base_iertr; + using list_bare_ret = OMapManager::omap_list_bare_ret; + using list_ret = OMapManager::omap_list_ret; + virtual list_ret list( + omap_context_t oc, + const std::optional<std::string> &first, + const std::optional<std::string> &last, + omap_list_config_t config) = 0; + + using clear_iertr = base_iertr; + using clear_ret = clear_iertr::future<>; + virtual clear_ret clear(omap_context_t oc) = 0; + + using full_merge_iertr = base_iertr; + using full_merge_ret = full_merge_iertr::future<OMapNodeRef>; + virtual full_merge_ret make_full_merge( + omap_context_t oc, + OMapNodeRef right) = 0; + + using make_balanced_iertr = base_iertr; + using make_balanced_ret = make_balanced_iertr::future + <std::tuple<OMapNodeRef, OMapNodeRef, std::string>>; + virtual make_balanced_ret make_balanced( + omap_context_t oc, + OMapNodeRef _right) = 0; + + virtual omap_node_meta_t get_node_meta() const = 0; + virtual bool extent_will_overflow( + size_t ksize, + std::optional<size_t> vsize) const = 0; + virtual bool can_merge(OMapNodeRef right) const = 0; + virtual bool extent_is_below_min() const = 0; + virtual uint32_t get_node_size() = 0; + + virtual ~OMapNode() = default; +}; + +using OMapNodeRef = OMapNode::OMapNodeRef; + +using omap_load_extent_iertr = OMapNode::base_iertr; +omap_load_extent_iertr::future<OMapNodeRef> +omap_load_extent(omap_context_t oc, laddr_t laddr, depth_t depth); + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::omap_manager::OMapNode> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.cc b/src/crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.cc new file mode 100644 index 000000000..4db58414a --- /dev/null +++ b/src/crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.cc @@ -0,0 +1,738 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <algorithm> +#include <string.h> +#include "include/buffer.h" +#include "include/byteorder.h" +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/omap_manager/btree/omap_btree_node.h" +#include "crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.h" +#include "seastar/core/thread.hh" + +SET_SUBSYS(seastore_omap); + +namespace crimson::os::seastore::omap_manager { + +std::ostream &operator<<(std::ostream &out, const omap_inner_key_t &rhs) +{ + return out << "omap_inner_key (" << rhs.key_off<< " - " << rhs.key_len + << " - " << rhs.laddr << ")"; +} + +std::ostream &operator<<(std::ostream &out, const omap_leaf_key_t &rhs) +{ + return out << "omap_leaf_key_t (" << rhs.key_off<< " - " << rhs.key_len + << " - " << rhs.val_len << ")"; +} + +std::ostream &OMapInnerNode::print_detail_l(std::ostream &out) const +{ + return out << ", size=" << get_size() + << ", depth=" << get_meta().depth; +} + +using dec_ref_iertr = OMapInnerNode::base_iertr; +using dec_ref_ret = dec_ref_iertr::future<>; +template <typename T> +dec_ref_ret dec_ref(omap_context_t oc, T&& addr) { + return oc.tm.dec_ref(oc.t, std::forward<T>(addr)).handle_error_interruptible( + dec_ref_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in OMapInnerNode helper dec_ref" + } + ).si_then([](auto &&e) {}); +} + +/** + * make_split_insert + * + * insert an entry at iter, with the address of key. + * will result in a split outcome encoded in the returned mutation_result_t + */ +OMapInnerNode::make_split_insert_ret +OMapInnerNode::make_split_insert( + omap_context_t oc, + internal_iterator_t iter, + std::string key, + laddr_t laddr) +{ + LOG_PREFIX(OMapInnerNode::make_split_insert); + DEBUGT("this: {}, key: {}", oc.t, *this, key); + return make_split_children(oc).si_then([=] (auto tuple) { + auto [left, right, pivot] = tuple; + if (pivot > key) { + auto liter = left->iter_idx(iter.get_index()); + left->journal_inner_insert(liter, laddr, key, + left->maybe_get_delta_buffer()); + } else { //right + auto riter = right->iter_idx(iter.get_index() - left->get_node_size()); + right->journal_inner_insert(riter, laddr, key, + right->maybe_get_delta_buffer()); + } + ++(oc.t.get_omap_tree_stats().extents_num_delta); + return make_split_insert_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::WAS_SPLIT, tuple, std::nullopt)); + }); + +} + + +OMapInnerNode::handle_split_ret +OMapInnerNode::handle_split( + omap_context_t oc, + internal_iterator_t iter, + mutation_result_t mresult) +{ + LOG_PREFIX(OMapInnerNode::handle_split); + DEBUGT("this: {}", oc.t, *this); + if (!is_mutable()) { + auto mut = oc.tm.get_mutable_extent(oc.t, this)->cast<OMapInnerNode>(); + auto mut_iter = mut->iter_idx(iter.get_index()); + return mut->handle_split(oc, mut_iter, mresult); + } + auto [left, right, pivot] = *(mresult.split_tuple); + //update operation will not cause node overflow, so we can do it first. + journal_inner_update(iter, left->get_laddr(), maybe_get_delta_buffer()); + bool overflow = extent_will_overflow(pivot.size(), std::nullopt); + if (!overflow) { + journal_inner_insert(iter + 1, right->get_laddr(), pivot, + maybe_get_delta_buffer()); + return insert_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::SUCCESS, std::nullopt, std::nullopt)); + } else { + return make_split_insert(oc, iter + 1, pivot, right->get_laddr()) + .si_then([this, oc] (auto m_result) { + return dec_ref(oc, get_laddr()) + .si_then([m_result = std::move(m_result)] { + return insert_ret( + interruptible::ready_future_marker{}, + m_result); + }); + }); + } +} + +OMapInnerNode::get_value_ret +OMapInnerNode::get_value( + omap_context_t oc, + const std::string &key) +{ + LOG_PREFIX(OMapInnerNode::get_value); + DEBUGT("key = {}, this: {}", oc.t, key, *this); + auto child_pt = get_containing_child(key); + assert(child_pt != iter_cend()); + auto laddr = child_pt->get_val(); + return omap_load_extent(oc, laddr, get_meta().depth - 1).si_then( + [oc, &key] (auto extent) { + return extent->get_value(oc, key); + }).finally([ref = OMapNodeRef(this)] {}); +} + +OMapInnerNode::insert_ret +OMapInnerNode::insert( + omap_context_t oc, + const std::string &key, + const ceph::bufferlist &value) +{ + LOG_PREFIX(OMapInnerNode::insert); + DEBUGT("{}->{}, this: {}", oc.t, key, value, *this); + auto child_pt = get_containing_child(key); + assert(child_pt != iter_cend()); + auto laddr = child_pt->get_val(); + return omap_load_extent(oc, laddr, get_meta().depth - 1).si_then( + [oc, &key, &value] (auto extent) { + return extent->insert(oc, key, value); + }).si_then([this, oc, child_pt] (auto mresult) { + if (mresult.status == mutation_status_t::SUCCESS) { + return insert_iertr::make_ready_future<mutation_result_t>(mresult); + } else if (mresult.status == mutation_status_t::WAS_SPLIT) { + return handle_split(oc, child_pt, mresult); + } else { + return insert_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::SUCCESS, std::nullopt, std::nullopt)); + } + }); +} + +OMapInnerNode::rm_key_ret +OMapInnerNode::rm_key(omap_context_t oc, const std::string &key) +{ + LOG_PREFIX(OMapInnerNode::rm_key); + DEBUGT("key={}, this: {}", oc.t, key, *this); + auto child_pt = get_containing_child(key); + assert(child_pt != iter_cend()); + auto laddr = child_pt->get_val(); + return omap_load_extent(oc, laddr, get_meta().depth - 1).si_then( + [this, oc, &key, child_pt] (auto extent) { + return extent->rm_key(oc, key) + .si_then([this, oc, child_pt, extent = std::move(extent)] (auto mresult) { + switch (mresult.status) { + case mutation_status_t::SUCCESS: + case mutation_status_t::FAIL: + return rm_key_iertr::make_ready_future<mutation_result_t>(mresult); + case mutation_status_t::NEED_MERGE: { + if (get_node_size() >1) + return merge_entry(oc, child_pt, *(mresult.need_merge)); + else + return rm_key_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::SUCCESS, + std::nullopt, std::nullopt)); + } + case mutation_status_t::WAS_SPLIT: + return handle_split(oc, child_pt, mresult); + default: + return rm_key_iertr::make_ready_future<mutation_result_t>(mresult); + } + }); + }); +} + +OMapInnerNode::list_ret +OMapInnerNode::list( + omap_context_t oc, + const std::optional<std::string> &first, + const std::optional<std::string> &last, + omap_list_config_t config) +{ + LOG_PREFIX(OMapInnerNode::list); + if (first && last) { + DEBUGT("first: {}, last: {}, this: {}", oc.t, *first, *last, *this); + assert(*first <= *last); + } else if (first) { + DEBUGT("first: {}, this: {}", oc.t, *first, *this); + } else if (last) { + DEBUGT("last: {}, this: {}", oc.t, *last, *this); + } else { + DEBUGT("this: {}", oc.t, *this); + } + + auto first_iter = first ? + get_containing_child(*first) : + iter_cbegin(); + auto last_iter = last ? + get_containing_child(*last) + 1: + iter_cend(); + assert(first_iter != iter_cend()); + + return seastar::do_with( + first_iter, + last_iter, + iter_t(first_iter), + list_bare_ret(false, {}), + [this, &first, &last, oc, config]( + auto &fiter, + auto &liter, + auto &iter, + auto &ret) + { + auto &complete = std::get<0>(ret); + auto &result = std::get<1>(ret); + return trans_intr::repeat( + [&, config, oc, this]() -> list_iertr::future<seastar::stop_iteration> + { + if (iter == liter) { + complete = true; + return list_iertr::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + assert(result.size() < config.max_result_size); + auto laddr = iter->get_val(); + return omap_load_extent( + oc, laddr, + get_meta().depth - 1 + ).si_then([&, config, oc](auto &&extent) { + return seastar::do_with( + iter == fiter ? first : std::optional<std::string>(std::nullopt), + iter == liter - 1 ? last : std::optional<std::string>(std::nullopt), + [&result, extent = std::move(extent), config, oc]( + auto &nfirst, + auto &nlast) { + return extent->list( + oc, + nfirst, + nlast, + config.with_reduced_max(result.size())); + }).si_then([&, config](auto &&child_ret) mutable { + boost::ignore_unused(config); // avoid clang warning; + auto &[child_complete, child_result] = child_ret; + if (result.size() && child_result.size()) { + assert(child_result.begin()->first > result.rbegin()->first); + } + if (child_result.size() && first && iter == fiter) { + if (config.first_inclusive) { + assert(child_result.begin()->first >= *first); + } else { + assert(child_result.begin()->first > *first); + } + } + if (child_result.size() && last && iter == liter - 1) { + auto biter = --(child_result.end()); + if (config.last_inclusive) { + assert(biter->first <= *last); + } else { + assert(biter->first < *last); + } + } + result.merge(std::move(child_result)); + if (result.size() == config.max_result_size) { + return list_iertr::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + ++iter; + assert(child_complete); + return list_iertr::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::no); + }); + }); + }).si_then([&ret, ref = OMapNodeRef(this)] { + return list_iertr::make_ready_future<list_bare_ret>(std::move(ret)); + }); + }); +} + +OMapInnerNode::clear_ret +OMapInnerNode::clear(omap_context_t oc) +{ + LOG_PREFIX(OMapInnerNode::clear); + DEBUGT("this: {}", oc.t, *this); + return trans_intr::do_for_each(iter_begin(), iter_end(), + [oc, this](auto iter) { + auto laddr = iter->get_val(); + auto ndepth = get_meta().depth - 1; + if (ndepth > 1) { + return omap_load_extent(oc, laddr, ndepth + ).si_then([oc](auto &&extent) { + return extent->clear(oc); + }).si_then([oc, laddr] { + return dec_ref(oc, laddr); + }).si_then([ref = OMapNodeRef(this)] { + return clear_iertr::now(); + }); + } else { + assert(ndepth == 1); + return dec_ref(oc, laddr + ).si_then([ref = OMapNodeRef(this)] { + return clear_iertr::now(); + }); + } + }); +} + +OMapInnerNode::split_children_ret +OMapInnerNode:: make_split_children(omap_context_t oc) +{ + LOG_PREFIX(OMapInnerNode::make_split_children); + DEBUGT("this: {}", oc.t, *this); + return oc.tm.alloc_extents<OMapInnerNode>(oc.t, oc.hint, + OMAP_INNER_BLOCK_SIZE, 2) + .si_then([this, oc] (auto &&ext_pair) { + LOG_PREFIX(OMapInnerNode::make_split_children); + auto left = ext_pair.front(); + auto right = ext_pair.back(); + DEBUGT("this: {}, split into: l {} r {}", oc.t, *this, *left, *right); + return split_children_ret( + interruptible::ready_future_marker{}, + std::make_tuple(left, right, split_into(*left, *right))); + }); +} + +OMapInnerNode::full_merge_ret +OMapInnerNode::make_full_merge(omap_context_t oc, OMapNodeRef right) +{ + LOG_PREFIX(OMapInnerNode::make_full_merge); + DEBUGT("", oc.t); + return oc.tm.alloc_extent<OMapInnerNode>(oc.t, oc.hint, + OMAP_INNER_BLOCK_SIZE) + .si_then([this, right] (auto &&replacement) { + replacement->merge_from(*this, *right->cast<OMapInnerNode>()); + return full_merge_ret( + interruptible::ready_future_marker{}, + std::move(replacement)); + }); +} + +OMapInnerNode::make_balanced_ret +OMapInnerNode::make_balanced(omap_context_t oc, OMapNodeRef _right) +{ + LOG_PREFIX(OMapInnerNode::make_balanced); + DEBUGT("l: {}, r: {}", oc.t, *this, *_right); + ceph_assert(_right->get_type() == TYPE); + return oc.tm.alloc_extents<OMapInnerNode>(oc.t, oc.hint, + OMAP_INNER_BLOCK_SIZE, 2) + .si_then([this, _right] (auto &&replacement_pair){ + auto replacement_left = replacement_pair.front(); + auto replacement_right = replacement_pair.back(); + auto &right = *_right->cast<OMapInnerNode>(); + return make_balanced_ret( + interruptible::ready_future_marker{}, + std::make_tuple(replacement_left, replacement_right, + balance_into_new_nodes(*this, right, + *replacement_left, *replacement_right))); + }); +} + +OMapInnerNode::merge_entry_ret +OMapInnerNode::merge_entry( + omap_context_t oc, + internal_iterator_t iter, + OMapNodeRef entry) +{ + LOG_PREFIX(OMapInnerNode::merge_entry); + DEBUGT("{}, parent: {}", oc.t, *entry, *this); + if (!is_mutable()) { + auto mut = oc.tm.get_mutable_extent(oc.t, this)->cast<OMapInnerNode>(); + auto mut_iter = mut->iter_idx(iter->get_index()); + return mut->merge_entry(oc, mut_iter, entry); + } + auto is_left = (iter + 1) == iter_cend(); + auto donor_iter = is_left ? iter - 1 : iter + 1; + return omap_load_extent(oc, donor_iter->get_val(), get_meta().depth - 1 + ).si_then([=, this](auto &&donor) mutable { + LOG_PREFIX(OMapInnerNode::merge_entry); + auto [l, r] = is_left ? + std::make_pair(donor, entry) : std::make_pair(entry, donor); + auto [liter, riter] = is_left ? + std::make_pair(donor_iter, iter) : std::make_pair(iter, donor_iter); + if (l->can_merge(r)) { + DEBUGT("make_full_merge l {} r {}", oc.t, *l, *r); + assert(entry->extent_is_below_min()); + return l->make_full_merge(oc, r + ).si_then([liter=liter, riter=riter, l=l, r=r, oc, this] + (auto &&replacement) { + LOG_PREFIX(OMapInnerNode::merge_entry); + DEBUGT("to update parent: {}", oc.t, *this); + journal_inner_update( + liter, + replacement->get_laddr(), + maybe_get_delta_buffer()); + journal_inner_remove(riter, maybe_get_delta_buffer()); + //retire extent + std::vector<laddr_t> dec_laddrs {l->get_laddr(), r->get_laddr()}; + return dec_ref(oc, dec_laddrs + ).si_then([this, oc] { + --(oc.t.get_omap_tree_stats().extents_num_delta); + if (extent_is_below_min()) { + return merge_entry_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::NEED_MERGE, + std::nullopt, this)); + } else { + return merge_entry_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::SUCCESS, + std::nullopt, std::nullopt)); + } + }); + }); + } else { + DEBUGT("balanced l {} r {}", oc.t, *l, *r); + return l->make_balanced(oc, r + ).si_then([liter=liter, riter=riter, l=l, r=r, oc, this](auto tuple) { + LOG_PREFIX(OMapInnerNode::merge_entry); + DEBUGT("to update parent: {}", oc.t, *this); + auto [replacement_l, replacement_r, replacement_pivot] = tuple; + //update operation will not cuase node overflow, so we can do it first + journal_inner_update( + liter, + replacement_l->get_laddr(), + maybe_get_delta_buffer()); + bool overflow = extent_will_overflow(replacement_pivot.size(), + std::nullopt); + if (!overflow) { + journal_inner_remove(riter, maybe_get_delta_buffer()); + journal_inner_insert( + riter, + replacement_r->get_laddr(), + replacement_pivot, + maybe_get_delta_buffer()); + std::vector<laddr_t> dec_laddrs{l->get_laddr(), r->get_laddr()}; + return dec_ref(oc, dec_laddrs + ).si_then([] { + return merge_entry_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::SUCCESS, + std::nullopt, std::nullopt)); + }); + } else { + DEBUGT("balanced and split {} r {}", oc.t, *l, *r); + //use remove and insert to instead of replace, + //remove operation will not cause node split, so we can do it first + journal_inner_remove(riter, maybe_get_delta_buffer()); + return make_split_insert(oc, riter, replacement_pivot, + replacement_r->get_laddr() + ).si_then([this, oc, l = l, r = r](auto mresult) { + std::vector<laddr_t> dec_laddrs{ + l->get_laddr(), + r->get_laddr(), + get_laddr()}; + return dec_ref(oc, dec_laddrs + ).si_then([mresult = std::move(mresult)] { + return merge_entry_ret( + interruptible::ready_future_marker{}, mresult); + }); + }); + } + }); + } + }); + +} + +OMapInnerNode::internal_iterator_t +OMapInnerNode::get_containing_child(const std::string &key) +{ + auto iter = std::find_if(iter_begin(), iter_end(), + [&key](auto it) { return it.contains(key); }); + return iter; +} + +std::ostream &OMapLeafNode::print_detail_l(std::ostream &out) const +{ + return out << ", size=" << get_size() + << ", depth=" << get_meta().depth; +} + +OMapLeafNode::get_value_ret +OMapLeafNode::get_value(omap_context_t oc, const std::string &key) +{ + LOG_PREFIX(OMapLeafNode::get_value); + DEBUGT("key = {}, this: {}", oc.t, *this, key); + auto ite = find_string_key(key); + if (ite != iter_end()) { + auto value = ite->get_val(); + return get_value_ret( + interruptible::ready_future_marker{}, + value); + } else { + return get_value_ret( + interruptible::ready_future_marker{}, + std::nullopt); + } +} + +OMapLeafNode::insert_ret +OMapLeafNode::insert( + omap_context_t oc, + const std::string &key, + const ceph::bufferlist &value) +{ + LOG_PREFIX(OMapLeafNode::insert); + DEBUGT("{} -> {}, this: {}", oc.t, key, value, *this); + bool overflow = extent_will_overflow(key.size(), value.length()); + if (!overflow) { + if (!is_mutable()) { + auto mut = oc.tm.get_mutable_extent(oc.t, this)->cast<OMapLeafNode>(); + return mut->insert(oc, key, value); + } + auto replace_pt = find_string_key(key); + if (replace_pt != iter_end()) { + ++(oc.t.get_omap_tree_stats().num_updates); + journal_leaf_update(replace_pt, key, value, maybe_get_delta_buffer()); + } else { + ++(oc.t.get_omap_tree_stats().num_inserts); + auto insert_pt = string_lower_bound(key); + journal_leaf_insert(insert_pt, key, value, maybe_get_delta_buffer()); + + DEBUGT("inserted {}, this: {}", oc.t, insert_pt.get_key(), *this); + } + return insert_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::SUCCESS, std::nullopt, std::nullopt)); + } else { + return make_split_children(oc).si_then([this, oc, &key, &value] (auto tuple) { + auto [left, right, pivot] = tuple; + auto replace_pt = find_string_key(key); + if (replace_pt != iter_end()) { + ++(oc.t.get_omap_tree_stats().num_updates); + if (key < pivot) { //left + auto mut_iter = left->iter_idx(replace_pt->get_index()); + left->journal_leaf_update(mut_iter, key, value, left->maybe_get_delta_buffer()); + } else if (key >= pivot) { //right + auto mut_iter = right->iter_idx(replace_pt->get_index() - left->get_node_size()); + right->journal_leaf_update(mut_iter, key, value, right->maybe_get_delta_buffer()); + } + } else { + ++(oc.t.get_omap_tree_stats().num_inserts); + auto insert_pt = string_lower_bound(key); + if (key < pivot) { //left + auto mut_iter = left->iter_idx(insert_pt->get_index()); + left->journal_leaf_insert(mut_iter, key, value, left->maybe_get_delta_buffer()); + } else { + auto mut_iter = right->iter_idx(insert_pt->get_index() - left->get_node_size()); + right->journal_leaf_insert(mut_iter, key, value, right->maybe_get_delta_buffer()); + } + } + ++(oc.t.get_omap_tree_stats().extents_num_delta); + return dec_ref(oc, get_laddr()) + .si_then([tuple = std::move(tuple)] { + return insert_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::WAS_SPLIT, tuple, std::nullopt)); + }); + }); + } +} + +OMapLeafNode::rm_key_ret +OMapLeafNode::rm_key(omap_context_t oc, const std::string &key) +{ + LOG_PREFIX(OMapLeafNode::rm_key); + DEBUGT("{}, this: {}", oc.t, key, *this); + auto rm_pt = find_string_key(key); + if (!is_mutable() && rm_pt != iter_end()) { + auto mut = oc.tm.get_mutable_extent(oc.t, this)->cast<OMapLeafNode>(); + return mut->rm_key(oc, key); + } + + if (rm_pt != iter_end()) { + ++(oc.t.get_omap_tree_stats().num_erases); + journal_leaf_remove(rm_pt, maybe_get_delta_buffer()); + if (extent_is_below_min()) { + return rm_key_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::NEED_MERGE, std::nullopt, + this->cast<OMapNode>())); + } else { + return rm_key_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::SUCCESS, std::nullopt, std::nullopt)); + } + } else { + return rm_key_ret( + interruptible::ready_future_marker{}, + mutation_result_t(mutation_status_t::FAIL, std::nullopt, std::nullopt)); + } + +} + +OMapLeafNode::list_ret +OMapLeafNode::list( + omap_context_t oc, + const std::optional<std::string> &first, + const std::optional<std::string> &last, + omap_list_config_t config) +{ + LOG_PREFIX(OMapLeafNode::list); + DEBUGT( + "first {} last {} max_result_size {} first_inclusive {} \ + last_inclusive {}, this: {}", + oc.t, + first ? first->c_str() : "", + last ? last->c_str() : "", + config.max_result_size, + config.first_inclusive, + config.last_inclusive, + *this + ); + auto ret = list_bare_ret(false, {}); + auto &[complete, result] = ret; + auto iter = first ? + (config.first_inclusive ? + string_lower_bound(*first) : + string_upper_bound(*first)) : + iter_begin(); + auto liter = last ? + (config.last_inclusive ? + string_upper_bound(*last) : + string_lower_bound(*last)) : + iter_end(); + + for (; iter != liter && result.size() < config.max_result_size; iter++) { + result.emplace(std::make_pair(iter->get_key(), iter->get_val())); + } + + complete = (iter == liter); + + return list_iertr::make_ready_future<list_bare_ret>( + std::move(ret)); +} + +OMapLeafNode::clear_ret +OMapLeafNode::clear(omap_context_t oc) +{ + return clear_iertr::now(); +} + +OMapLeafNode::split_children_ret +OMapLeafNode::make_split_children(omap_context_t oc) +{ + LOG_PREFIX(OMapLeafNode::make_split_children); + DEBUGT("this: {}", oc.t, *this); + return oc.tm.alloc_extents<OMapLeafNode>(oc.t, oc.hint, OMAP_LEAF_BLOCK_SIZE, 2) + .si_then([this] (auto &&ext_pair) { + auto left = ext_pair.front(); + auto right = ext_pair.back(); + return split_children_ret( + interruptible::ready_future_marker{}, + std::make_tuple(left, right, split_into(*left, *right))); + }); +} + +OMapLeafNode::full_merge_ret +OMapLeafNode::make_full_merge(omap_context_t oc, OMapNodeRef right) +{ + ceph_assert(right->get_type() == TYPE); + LOG_PREFIX(OMapLeafNode::make_full_merge); + DEBUGT("this: {}", oc.t, *this); + return oc.tm.alloc_extent<OMapLeafNode>(oc.t, oc.hint, OMAP_LEAF_BLOCK_SIZE) + .si_then([this, right] (auto &&replacement) { + replacement->merge_from(*this, *right->cast<OMapLeafNode>()); + return full_merge_ret( + interruptible::ready_future_marker{}, + std::move(replacement)); + }); +} + +OMapLeafNode::make_balanced_ret +OMapLeafNode::make_balanced(omap_context_t oc, OMapNodeRef _right) +{ + ceph_assert(_right->get_type() == TYPE); + LOG_PREFIX(OMapLeafNode::make_balanced); + DEBUGT("this: {}", oc.t, *this); + return oc.tm.alloc_extents<OMapLeafNode>(oc.t, oc.hint, OMAP_LEAF_BLOCK_SIZE, 2) + .si_then([this, _right] (auto &&replacement_pair) { + auto replacement_left = replacement_pair.front(); + auto replacement_right = replacement_pair.back(); + auto &right = *_right->cast<OMapLeafNode>(); + return make_balanced_ret( + interruptible::ready_future_marker{}, + std::make_tuple( + replacement_left, replacement_right, + balance_into_new_nodes( + *this, right, + *replacement_left, *replacement_right))); + }); +} + + +omap_load_extent_iertr::future<OMapNodeRef> +omap_load_extent(omap_context_t oc, laddr_t laddr, depth_t depth) +{ + ceph_assert(depth > 0); + if (depth > 1) { + return oc.tm.read_extent<OMapInnerNode>(oc.t, laddr, + OMAP_INNER_BLOCK_SIZE) + .handle_error_interruptible( + omap_load_extent_iertr::pass_further{}, + crimson::ct_error::assert_all{ "Invalid error in omap_load_extent" } + ).si_then( + [](auto&& e) { + return seastar::make_ready_future<OMapNodeRef>(std::move(e)); + }); + } else { + return oc.tm.read_extent<OMapLeafNode>(oc.t, laddr, OMAP_LEAF_BLOCK_SIZE + ).handle_error_interruptible( + omap_load_extent_iertr::pass_further{}, + crimson::ct_error::assert_all{ "Invalid error in omap_load_extent" } + ).si_then( + [](auto&& e) { + return seastar::make_ready_future<OMapNodeRef>(std::move(e)); + }); + } +} +} diff --git a/src/crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.h b/src/crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.h new file mode 100644 index 000000000..a2b51bbb0 --- /dev/null +++ b/src/crimson/os/seastore/omap_manager/btree/omap_btree_node_impl.h @@ -0,0 +1,250 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <string.h> + +#include "include/buffer.h" + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/omap_manager.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/omap_manager/btree/string_kv_node_layout.h" +#include "crimson/os/seastore/omap_manager/btree/omap_types.h" +#include "crimson/os/seastore/omap_manager/btree/omap_btree_node.h" + +namespace crimson::os::seastore::omap_manager { + +/** + * OMapInnerNode + * + * Abstracts operations on and layout of internal nodes for the + * omap Tree. + * + * Layout (4k): + * num_entries: meta : keys : values : + */ + +struct OMapInnerNode + : OMapNode, + StringKVInnerNodeLayout { + using OMapInnerNodeRef = TCachedExtentRef<OMapInnerNode>; + using internal_iterator_t = const_iterator; + template <typename... T> + OMapInnerNode(T&&... t) : + OMapNode(std::forward<T>(t)...), + StringKVInnerNodeLayout(get_bptr().c_str()) {} + + omap_node_meta_t get_node_meta() const final { return get_meta(); } + bool extent_will_overflow(size_t ksize, std::optional<size_t> vsize) const { + return is_overflow(ksize); + } + bool can_merge(OMapNodeRef right) const { + return !is_overflow(*right->cast<OMapInnerNode>()); + } + bool extent_is_below_min() const { return below_min(); } + uint32_t get_node_size() { return get_size(); } + + CachedExtentRef duplicate_for_write(Transaction&) final { + assert(delta_buffer.empty()); + return CachedExtentRef(new OMapInnerNode(*this)); + } + + delta_inner_buffer_t delta_buffer; + delta_inner_buffer_t *maybe_get_delta_buffer() { + return is_mutation_pending() ? &delta_buffer : nullptr; + } + + get_value_ret get_value(omap_context_t oc, const std::string &key) final; + + insert_ret insert( + omap_context_t oc, + const std::string &key, + const ceph::bufferlist &value) final; + + rm_key_ret rm_key( + omap_context_t oc, + const std::string &key) final; + + list_ret list( + omap_context_t oc, + const std::optional<std::string> &first, + const std::optional<std::string> &last, + omap_list_config_t config) final; + + clear_ret clear(omap_context_t oc) final; + + using split_children_iertr = base_iertr; + using split_children_ret = split_children_iertr::future + <std::tuple<OMapInnerNodeRef, OMapInnerNodeRef, std::string>>; + split_children_ret make_split_children(omap_context_t oc); + + full_merge_ret make_full_merge( + omap_context_t oc, OMapNodeRef right) final; + + make_balanced_ret make_balanced( + omap_context_t oc, OMapNodeRef right) final; + + using make_split_insert_iertr = base_iertr; + using make_split_insert_ret = make_split_insert_iertr::future<mutation_result_t>; + make_split_insert_ret make_split_insert( + omap_context_t oc, internal_iterator_t iter, + std::string key, laddr_t laddr); + + using merge_entry_iertr = base_iertr; + using merge_entry_ret = merge_entry_iertr::future<mutation_result_t>; + merge_entry_ret merge_entry( + omap_context_t oc, + internal_iterator_t iter, OMapNodeRef entry); + + using handle_split_iertr = base_iertr; + using handle_split_ret = handle_split_iertr::future<mutation_result_t>; + handle_split_ret handle_split( + omap_context_t oc, internal_iterator_t iter, + mutation_result_t mresult); + + std::ostream &print_detail_l(std::ostream &out) const final; + + static constexpr extent_types_t TYPE = extent_types_t::OMAP_INNER; + extent_types_t get_type() const final { + return TYPE; + } + + ceph::bufferlist get_delta() final { + ceph::bufferlist bl; + if (!delta_buffer.empty()) { + encode(delta_buffer, bl); + delta_buffer.clear(); + } + return bl; + } + + void apply_delta(const ceph::bufferlist &bl) final { + assert(bl.length()); + delta_inner_buffer_t buffer; + auto bptr = bl.cbegin(); + decode(buffer, bptr); + buffer.replay(*this); + } + + internal_iterator_t get_containing_child(const std::string &key); +}; +using OMapInnerNodeRef = OMapInnerNode::OMapInnerNodeRef; + +/** + * OMapLeafNode + * + * Abstracts operations on and layout of leaf nodes for the + * OMap Tree. + * + * Layout (4k): + * num_entries: meta : keys : values : + */ + +struct OMapLeafNode + : OMapNode, + StringKVLeafNodeLayout { + + using OMapLeafNodeRef = TCachedExtentRef<OMapLeafNode>; + using internal_iterator_t = const_iterator; + template <typename... T> + OMapLeafNode(T&&... t) : + OMapNode(std::forward<T>(t)...), + StringKVLeafNodeLayout(get_bptr().c_str()) {} + + omap_node_meta_t get_node_meta() const final { return get_meta(); } + bool extent_will_overflow( + size_t ksize, std::optional<size_t> vsize) const { + return is_overflow(ksize, *vsize); + } + bool can_merge(OMapNodeRef right) const { + return !is_overflow(*right->cast<OMapLeafNode>()); + } + bool extent_is_below_min() const { return below_min(); } + uint32_t get_node_size() { return get_size(); } + + CachedExtentRef duplicate_for_write(Transaction&) final { + assert(delta_buffer.empty()); + return CachedExtentRef(new OMapLeafNode(*this)); + } + + delta_leaf_buffer_t delta_buffer; + delta_leaf_buffer_t *maybe_get_delta_buffer() { + return is_mutation_pending() ? &delta_buffer : nullptr; + } + + get_value_ret get_value( + omap_context_t oc, const std::string &key) final; + + insert_ret insert( + omap_context_t oc, + const std::string &key, + const ceph::bufferlist &value) final; + + rm_key_ret rm_key( + omap_context_t oc, const std::string &key) final; + + list_ret list( + omap_context_t oc, + const std::optional<std::string> &first, + const std::optional<std::string> &last, + omap_list_config_t config) final; + + clear_ret clear( + omap_context_t oc) final; + + using split_children_iertr = base_iertr; + using split_children_ret = split_children_iertr::future + <std::tuple<OMapLeafNodeRef, OMapLeafNodeRef, std::string>>; + split_children_ret make_split_children( + omap_context_t oc); + + full_merge_ret make_full_merge( + omap_context_t oc, + OMapNodeRef right) final; + + make_balanced_ret make_balanced( + omap_context_t oc, + OMapNodeRef _right) final; + + static constexpr extent_types_t TYPE = extent_types_t::OMAP_LEAF; + extent_types_t get_type() const final { + return TYPE; + } + + ceph::bufferlist get_delta() final { + ceph::bufferlist bl; + if (!delta_buffer.empty()) { + encode(delta_buffer, bl); + delta_buffer.clear(); + } + return bl; + } + + void apply_delta(const ceph::bufferlist &_bl) final { + assert(_bl.length()); + ceph::bufferlist bl = _bl; + bl.rebuild(); + delta_leaf_buffer_t buffer; + auto bptr = bl.cbegin(); + decode(buffer, bptr); + buffer.replay(*this); + } + + std::ostream &print_detail_l(std::ostream &out) const final; + + std::pair<internal_iterator_t, internal_iterator_t> + get_leaf_entries(std::string &key); + +}; +using OMapLeafNodeRef = OMapLeafNode::OMapLeafNodeRef; + +std::ostream &operator<<(std::ostream &out, const omap_inner_key_t &rhs); +std::ostream &operator<<(std::ostream &out, const omap_leaf_key_t &rhs); +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::omap_manager::OMapInnerNode> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::omap_manager::OMapLeafNode> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/omap_manager/btree/omap_types.h b/src/crimson/os/seastore/omap_manager/btree/omap_types.h new file mode 100644 index 000000000..9e0d10e03 --- /dev/null +++ b/src/crimson/os/seastore/omap_manager/btree/omap_types.h @@ -0,0 +1,157 @@ +// -*- 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/seastore_types.h" + +namespace crimson::os::seastore::omap_manager { + +struct omap_node_meta_t { + depth_t depth = 0; + + std::pair<omap_node_meta_t, omap_node_meta_t> split_into() const { + return std::make_pair( + omap_node_meta_t{depth}, + omap_node_meta_t{depth}); + } + + static omap_node_meta_t merge_from( + const omap_node_meta_t &lhs, const omap_node_meta_t &rhs) { + assert(lhs.depth == rhs.depth); + return omap_node_meta_t{lhs.depth}; + } + + static std::pair<omap_node_meta_t, omap_node_meta_t> + rebalance(const omap_node_meta_t &lhs, const omap_node_meta_t &rhs) { + assert(lhs.depth == rhs.depth); + return std::make_pair( + omap_node_meta_t{lhs.depth}, + omap_node_meta_t{lhs.depth}); + } +}; + +struct omap_node_meta_le_t { + depth_le_t depth = init_depth_le(0); + + omap_node_meta_le_t() = default; + omap_node_meta_le_t(const omap_node_meta_le_t &) = default; + explicit omap_node_meta_le_t(const omap_node_meta_t &val) + : depth(init_depth_le(val.depth)) {} + + operator omap_node_meta_t() const { + return omap_node_meta_t{ depth }; + } +}; + +struct omap_inner_key_t { + uint16_t key_off = 0; + uint16_t key_len = 0; + laddr_t laddr = 0; + + omap_inner_key_t() = default; + omap_inner_key_t(uint16_t off, uint16_t len, laddr_t addr) + : key_off(off), key_len(len), laddr(addr) {} + + inline bool operator==(const omap_inner_key_t b) const { + return key_off == b.key_off && key_len == b.key_len && laddr == b.laddr; + } + inline bool operator!=(const omap_inner_key_t b) const { + return key_off != b.key_off || key_len != b.key_len || laddr != b.laddr; + } + DENC(omap_inner_key_t, v, p) { + DENC_START(1, 1, p); + denc(v.key_off, p); + denc(v.key_len, p); + denc(v.laddr, p); + DENC_FINISH(p); + } +}; + +struct omap_inner_key_le_t { + ceph_le16 key_off{0}; + ceph_le16 key_len{0}; + laddr_le_t laddr{0}; + + omap_inner_key_le_t() = default; + omap_inner_key_le_t(const omap_inner_key_le_t &) = default; + explicit omap_inner_key_le_t(const omap_inner_key_t &key) + : key_off(key.key_off), + key_len(key.key_len), + laddr(key.laddr) {} + + operator omap_inner_key_t() const { + return omap_inner_key_t{uint16_t(key_off), uint16_t(key_len), laddr_t(laddr)}; + } + + omap_inner_key_le_t& operator=(omap_inner_key_t key) { + key_off = key.key_off; + key_len = key.key_len; + laddr = laddr_le_t(key.laddr); + return *this; + } + + inline bool operator==(const omap_inner_key_le_t b) const { + return key_off == b.key_off && key_len == b.key_len && laddr == b.laddr; + } +}; + +struct omap_leaf_key_t { + uint16_t key_off = 0; + uint16_t key_len = 0; + uint16_t val_len = 0; + + omap_leaf_key_t() = default; + omap_leaf_key_t(uint16_t k_off, uint16_t k_len, uint16_t v_len) + : key_off(k_off), key_len(k_len), val_len(v_len) {} + + inline bool operator==(const omap_leaf_key_t b) const { + return key_off == b.key_off && key_len == b.key_len && + val_len == b.val_len; + } + inline bool operator!=(const omap_leaf_key_t b) const { + return key_off != b.key_off || key_len != b.key_len || + val_len != b.val_len; + } + + DENC(omap_leaf_key_t, v, p) { + DENC_START(1, 1, p); + denc(v.key_off, p); + denc(v.key_len, p); + denc(v.val_len, p); + DENC_FINISH(p); + } +}; + +struct omap_leaf_key_le_t { + ceph_le16 key_off{0}; + ceph_le16 key_len{0}; + ceph_le16 val_len{0}; + + omap_leaf_key_le_t() = default; + omap_leaf_key_le_t(const omap_leaf_key_le_t &) = default; + explicit omap_leaf_key_le_t(const omap_leaf_key_t &key) + : key_off(key.key_off), + key_len(key.key_len), + val_len(key.val_len) {} + + operator omap_leaf_key_t() const { + return omap_leaf_key_t{uint16_t(key_off), uint16_t(key_len), + uint16_t(val_len)}; + } + + omap_leaf_key_le_t& operator=(omap_leaf_key_t key) { + key_off = key.key_off; + key_len = key.key_len; + val_len = key.val_len; + return *this; + } + + inline bool operator==(const omap_leaf_key_le_t b) const { + return key_off == b.key_off && key_len == b.key_len && + val_len == b.val_len; + } +}; + +} +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::omap_manager::omap_inner_key_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::omap_manager::omap_leaf_key_t) diff --git a/src/crimson/os/seastore/omap_manager/btree/string_kv_node_layout.h b/src/crimson/os/seastore/omap_manager/btree/string_kv_node_layout.h new file mode 100644 index 000000000..72b13fedf --- /dev/null +++ b/src/crimson/os/seastore/omap_manager/btree/string_kv_node_layout.h @@ -0,0 +1,1550 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <iostream> +#include <string> + +#include "include/byteorder.h" +#include "include/denc.h" +#include "include/encoding.h" + +#include "crimson/common/layout.h" +#include "crimson/common/fixed_kv_node_layout.h" +#include "crimson/os/seastore/omap_manager.h" +#include "crimson/os/seastore/omap_manager/btree/omap_types.h" + +namespace crimson::os::seastore::omap_manager { +class StringKVInnerNodeLayout; +class StringKVLeafNodeLayout; + +/** + * copy_from_foreign + * + * Copy from another node entries to this node. + * [from_src, to_src) is another node entry range. + * tgt is this node entry to copy to. + * tgt and from_src must be from different nodes. + * from_src and to_src must be in the same node. + */ +template <typename iterator, typename const_iterator> +static void copy_from_foreign( + iterator tgt, + const_iterator from_src, + const_iterator to_src) { + assert(tgt->node != from_src->node); + assert(to_src->node == from_src->node); + if (from_src == to_src) + return; + + auto to_copy = from_src->get_right_ptr_end() - to_src->get_right_ptr_end(); + assert(to_copy > 0); + memcpy( + tgt->get_right_ptr_end() - to_copy, + to_src->get_right_ptr_end(), + to_copy); + memcpy( + tgt->get_node_key_ptr(), + from_src->get_node_key_ptr(), + to_src->get_node_key_ptr() - from_src->get_node_key_ptr()); + + auto offset_diff = tgt->get_right_offset_end() - from_src->get_right_offset_end(); + for (auto i = tgt; i != tgt + (to_src - from_src); ++i) { + i->update_offset(offset_diff); + } +} + +/** + * copy_from_local + * + * Copies entries from [from_src, to_src) to tgt. + * tgt, from_src, and to_src must be from the same node. + */ +template <typename iterator> +static void copy_from_local( + unsigned len, + iterator tgt, + iterator from_src, + iterator to_src) { + assert(tgt->node == from_src->node); + assert(to_src->node == from_src->node); + + auto to_copy = from_src->get_right_ptr_end() - to_src->get_right_ptr_end(); + assert(to_copy > 0); + int adjust_offset = tgt > from_src? -len : len; + memmove(to_src->get_right_ptr_end() + adjust_offset, + to_src->get_right_ptr_end(), + to_copy); + + for ( auto ite = from_src; ite < to_src; ite++) { + ite->update_offset(-adjust_offset); + } + memmove(tgt->get_node_key_ptr(), from_src->get_node_key_ptr(), + to_src->get_node_key_ptr() - from_src->get_node_key_ptr()); +} + +struct delta_inner_t { + enum class op_t : uint_fast8_t { + INSERT, + UPDATE, + REMOVE, + } op; + std::string key; + laddr_t addr; + + DENC(delta_inner_t, v, p) { + DENC_START(1, 1, p); + denc(v.op, p); + denc(v.key, p); + denc(v.addr, p); + DENC_FINISH(p); + } + + void replay(StringKVInnerNodeLayout &l); + bool operator==(const delta_inner_t &rhs) const { + return op == rhs.op && + key == rhs.key && + addr == rhs.addr; + } +}; +} +WRITE_CLASS_DENC(crimson::os::seastore::omap_manager::delta_inner_t) + +namespace crimson::os::seastore::omap_manager { +struct delta_leaf_t { + enum class op_t : uint_fast8_t { + INSERT, + UPDATE, + REMOVE, + } op; + std::string key; + ceph::bufferlist val; + + DENC(delta_leaf_t, v, p) { + DENC_START(1, 1, p); + denc(v.op, p); + denc(v.key, p); + denc(v.val, p); + DENC_FINISH(p); + } + + void replay(StringKVLeafNodeLayout &l); + bool operator==(const delta_leaf_t &rhs) const { + return op == rhs.op && + key == rhs.key && + val == rhs.val; + } +}; +} +WRITE_CLASS_DENC(crimson::os::seastore::omap_manager::delta_leaf_t) + +namespace crimson::os::seastore::omap_manager { +class delta_inner_buffer_t { + std::vector<delta_inner_t> buffer; +public: + bool empty() const { + return buffer.empty(); + } + void insert( + const std::string &key, + laddr_t addr) { + buffer.push_back( + delta_inner_t{ + delta_inner_t::op_t::INSERT, + key, + addr + }); + } + void update( + const std::string &key, + laddr_t addr) { + buffer.push_back( + delta_inner_t{ + delta_inner_t::op_t::UPDATE, + key, + addr + }); + } + void remove(const std::string &key) { + buffer.push_back( + delta_inner_t{ + delta_inner_t::op_t::REMOVE, + key, + L_ADDR_NULL + }); + } + + void replay(StringKVInnerNodeLayout &node) { + for (auto &i: buffer) { + i.replay(node); + } + } + + void clear() { + buffer.clear(); + } + + DENC(delta_inner_buffer_t, v, p) { + DENC_START(1, 1, p); + denc(v.buffer, p); + DENC_FINISH(p); + } + + bool operator==(const delta_inner_buffer_t &rhs) const { + return buffer == rhs.buffer; + } +}; +} +WRITE_CLASS_DENC(crimson::os::seastore::omap_manager::delta_inner_buffer_t) + +namespace crimson::os::seastore::omap_manager { +class delta_leaf_buffer_t { + std::vector<delta_leaf_t> buffer; +public: + bool empty() const { + return buffer.empty(); + } + void insert( + const std::string &key, + const ceph::bufferlist &val) { + buffer.push_back( + delta_leaf_t{ + delta_leaf_t::op_t::INSERT, + key, + val + }); + } + void update( + const std::string &key, + const ceph::bufferlist &val) { + buffer.push_back( + delta_leaf_t{ + delta_leaf_t::op_t::UPDATE, + key, + val + }); + } + void remove(const std::string &key) { + buffer.push_back( + delta_leaf_t{ + delta_leaf_t::op_t::REMOVE, + key, + bufferlist() + }); + } + + void replay(StringKVLeafNodeLayout &node) { + for (auto &i: buffer) { + i.replay(node); + } + } + + void clear() { + buffer.clear(); + } + + DENC(delta_leaf_buffer_t, v, p) { + DENC_START(1, 1, p); + denc(v.buffer, p); + DENC_FINISH(p); + } + + bool operator==(const delta_leaf_buffer_t &rhs) const { + return buffer == rhs.buffer; + } +}; +} +WRITE_CLASS_DENC(crimson::os::seastore::omap_manager::delta_leaf_buffer_t) + +namespace crimson::os::seastore::omap_manager { +/** + * StringKVInnerNodeLayout + * + * Uses absl::container_internal::Layout for the actual key memory layout. + * + * The primary interface exposed is centered on the iterator + * and related methods. + * + * Also included are helpers for doing splits and merges as for a btree. + * + * layout diagram: + * + * # <----------------------------- node range --------------------------------------------> # + * # #<~># free space # + * # <------------- left part -----------------------------> # <~# <----- right keys -----> # + * # # <------------ left keys --------------> #~> # # + * # # keys [2, n) |<~># #<~>| right keys [2, n) # + * # # <--- key 0 ----> | <--- key 1 ----> | # # | <- k1 -> | <-- k0 --> # + * # # | | # # | | # + * # num_ | meta # key | key | val | key | key | val | # # | key | key # + * # keys | depth # off | len | laddr| off | len | laddr| # # | buff | buff # + * # | # 0 | 0 | 0 | 1 | 1 | 1 |...#...#...| key 1 | key 0 # + * # | | | <- off --+----------> # + * # | | ^ | <- off --> # + * | | | ^ + * | +----------------------------------+ | + * +----------------------------------------------------------------+ + */ +class StringKVInnerNodeLayout { + char *buf = nullptr; + + using L = absl::container_internal::Layout<ceph_le32, omap_node_meta_le_t, omap_inner_key_le_t>; + static constexpr L layout{1, 1, 1}; // = L::Partial(1, 1, 1); + friend class delta_inner_t; +public: + template <bool is_const> + class iter_t { + friend class StringKVInnerNodeLayout; + + template <typename iterator, typename const_iterator> + friend void copy_from_foreign(iterator, const_iterator, const_iterator); + template <typename iterator> + friend void copy_from_local(unsigned, iterator, iterator, iterator); + + using parent_t = typename crimson::common::maybe_const_t<StringKVInnerNodeLayout, is_const>::type; + + mutable parent_t node; + uint16_t index; + + iter_t( + parent_t parent, + uint16_t index) : node(parent), index(index) {} + + public: + using iterator_category = std::input_iterator_tag; + using value_type = StringKVInnerNodeLayout; + using difference_type = std::ptrdiff_t; + using pointer = StringKVInnerNodeLayout*; + using reference = iter_t&; + + iter_t(const iter_t &) = default; + iter_t(iter_t &&) = default; + iter_t &operator=(const iter_t &) = default; + iter_t &operator=(iter_t &&) = default; + + operator iter_t<!is_const>() const { + static_assert(!is_const); + return iter_t<!is_const>(node, index); + } + + iter_t &operator*() { return *this; } + iter_t *operator->() { return this; } + + iter_t operator++(int) { + auto ret = *this; + ++index; + return ret; + } + + iter_t &operator++() { + ++index; + return *this; + } + + iter_t operator--(int) { + auto ret = *this; + assert(index > 0); + --index; + return ret; + } + + iter_t &operator--() { + assert(index > 0); + --index; + return *this; + } + + uint16_t operator-(const iter_t &rhs) const { + assert(rhs.node == node); + return index - rhs.index; + } + + iter_t operator+(uint16_t off) const { + return iter_t(node, index + off); + } + iter_t operator-(uint16_t off) const { + return iter_t(node, index - off); + } + + uint16_t operator<(const iter_t &rhs) const { + assert(rhs.node == node); + return index < rhs.index; + } + + uint16_t operator>(const iter_t &rhs) const { + assert(rhs.node == node); + return index > rhs.index; + } + + friend bool operator==(const iter_t &lhs, const iter_t &rhs) { + assert(lhs.node == rhs.node); + return lhs.index == rhs.index; + } + + private: + omap_inner_key_t get_node_key() const { + omap_inner_key_le_t kint = node->get_node_key_ptr()[index]; + return omap_inner_key_t(kint); + } + auto get_node_key_ptr() const { + return reinterpret_cast< + typename crimson::common::maybe_const_t<char, is_const>::type>( + node->get_node_key_ptr() + index); + } + + uint32_t get_node_val_offset() const { + return get_node_key().key_off; + } + auto get_node_val_ptr() const { + auto tail = node->buf + OMAP_INNER_BLOCK_SIZE; + if (*this == node->iter_end()) + return tail; + else { + return tail - get_node_val_offset(); + } + } + + int get_right_offset_end() const { + if (index == 0) + return 0; + else + return (*this - 1)->get_node_val_offset(); + } + auto get_right_ptr_end() const { + return node->buf + OMAP_INNER_BLOCK_SIZE - get_right_offset_end(); + } + + void update_offset(int offset) { + static_assert(!is_const); + auto key = get_node_key(); + assert(offset + key.key_off >= 0); + key.key_off += offset; + set_node_key(key); + } + + void set_node_key(omap_inner_key_t _lb) { + static_assert(!is_const); + omap_inner_key_le_t lb; + lb = _lb; + node->get_node_key_ptr()[index] = lb; + } + + void set_node_val(const std::string &str) { + static_assert(!is_const); + assert(str.size() == get_node_key().key_len); + assert(get_node_key().key_off >= str.size()); + assert(get_node_key().key_off < OMAP_INNER_BLOCK_SIZE); + assert(str.size() < OMAP_INNER_BLOCK_SIZE); + ::memcpy(get_node_val_ptr(), str.data(), str.size()); + } + + public: + uint16_t get_index() const { + return index; + } + + std::string get_key() const { + return std::string( + get_node_val_ptr(), + get_node_key().key_len); + } + + laddr_t get_val() const { + return get_node_key().laddr; + } + + bool contains(std::string_view key) const { + assert(*this != node->iter_end()); + auto next = *this + 1; + if (next == node->iter_end()) { + return get_key() <= key; + } else { + return (get_key() <= key) && (next->get_key() > key); + } + } + }; + using const_iterator = iter_t<true>; + using iterator = iter_t<false>; + +public: + void journal_inner_insert( + const_iterator _iter, + const laddr_t laddr, + const std::string &key, + delta_inner_buffer_t *recorder) { + auto iter = iterator(this, _iter.index); + if (recorder) { + recorder->insert( + key, + laddr); + } + inner_insert(iter, key, laddr); + } + + void journal_inner_update( + const_iterator _iter, + const laddr_t laddr, + delta_inner_buffer_t *recorder) { + auto iter = iterator(this, _iter.index); + auto key = iter->get_key(); + if (recorder) { + recorder->update(key, laddr); + } + inner_update(iter, laddr); + } + + void journal_inner_remove( + const_iterator _iter, + delta_inner_buffer_t *recorder) { + auto iter = iterator(this, _iter.index); + if (recorder) { + recorder->remove(iter->get_key()); + } + inner_remove(iter); + } + + StringKVInnerNodeLayout(char *buf) : + buf(buf) {} + + uint32_t get_size() const { + ceph_le32 &size = *layout.template Pointer<0>(buf); + return uint32_t(size); + } + + /** + * set_size + * + * Set size representation to match size + */ + void set_size(uint32_t size) { + ceph_le32 s; + s = size; + *layout.template Pointer<0>(buf) = s; + } + + const_iterator iter_cbegin() const { + return const_iterator( + this, + 0); + } + const_iterator iter_begin() const { + return iter_cbegin(); + } + + const_iterator iter_cend() const { + return const_iterator( + this, + get_size()); + } + const_iterator iter_end() const { + return iter_cend(); + } + + iterator iter_begin() { + return iterator( + this, + 0); + } + + iterator iter_end() { + return iterator( + this, + get_size()); + } + + const_iterator iter_idx(uint16_t off) const { + return const_iterator( + this, + off); + } + + const_iterator string_lower_bound(std::string_view str) const { + auto it = std::lower_bound(boost::make_counting_iterator<uint16_t>(0), + boost::make_counting_iterator<uint16_t>(get_size()), + str, + [this](uint16_t i, std::string_view str) { + const_iterator iter(this, i); + return iter->get_key() < str; + }); + return const_iterator(this, *it); + } + + iterator string_lower_bound(std::string_view str) { + const auto &tref = *this; + return iterator(this, tref.string_lower_bound(str).index); + } + + const_iterator string_upper_bound(std::string_view str) const { + auto it = std::upper_bound(boost::make_counting_iterator<uint16_t>(0), + boost::make_counting_iterator<uint16_t>(get_size()), + str, + [this](std::string_view str, uint16_t i) { + const_iterator iter(this, i); + return str < iter->get_key(); + }); + return const_iterator(this, *it); + } + + iterator string_upper_bound(std::string_view str) { + const auto &tref = *this; + return iterator(this, tref.string_upper_bound(str).index); + } + + const_iterator find_string_key(std::string_view str) const { + auto ret = iter_begin(); + for (; ret != iter_end(); ++ret) { + std::string s = ret->get_key(); + if (s == str) + break; + } + return ret; + } + + iterator find_string_key(std::string_view str) { + const auto &tref = *this; + return iterator(this, tref.find_string_key(str).index); + } + + const_iterator get_split_pivot() const { + uint32_t total_size = omap_inner_key_t( + get_node_key_ptr()[get_size()-1]).key_off; + uint32_t pivot_size = total_size / 2; + uint32_t size = 0; + for (auto ite = iter_begin(); ite < iter_end(); ite++) { + auto node_key = ite->get_node_key(); + size += node_key.key_len; + if (size >= pivot_size){ + return ite; + } + } + return iter_end(); + } + + + /** + * get_meta/set_meta + * + * Enables stashing a templated type within the layout. + * Cannot be modified after initial write as it is not represented + * in delta_t + */ + omap_node_meta_t get_meta() const { + omap_node_meta_le_t &metaint = *layout.template Pointer<1>(buf); + return omap_node_meta_t(metaint); + } + void set_meta(const omap_node_meta_t &meta) { + *layout.template Pointer<1>(buf) = omap_node_meta_le_t(meta); + } + + uint32_t used_space() const { + uint32_t count = get_size(); + if (count) { + omap_inner_key_t last_key = omap_inner_key_t(get_node_key_ptr()[count-1]); + return last_key.key_off + count * sizeof(omap_inner_key_le_t); + } else { + return 0; + } + } + + uint32_t free_space() const { + return capacity() - used_space(); + } + + uint16_t capacity() const { + return OMAP_INNER_BLOCK_SIZE + - (reinterpret_cast<char*>(layout.template Pointer<2>(buf)) + - reinterpret_cast<char*>(layout.template Pointer<0>(buf))); + } + + bool is_overflow(size_t ksize) const { + return free_space() < (sizeof(omap_inner_key_le_t) + ksize); + } + + bool is_overflow(const StringKVInnerNodeLayout &rhs) const { + return free_space() < rhs.used_space(); + } + + bool below_min() const { + return free_space() > (capacity() / 2); + } + + bool operator==(const StringKVInnerNodeLayout &rhs) const { + if (get_size() != rhs.get_size()) { + return false; + } + + auto iter = iter_begin(); + auto iter2 = rhs.iter_begin(); + while (iter != iter_end()) { + if (iter->get_key() != iter2->get_key() || + iter->get_val() != iter2->get_val()) { + return false; + } + iter++; + iter2++; + } + return true; + } + + /** + * split_into + * + * Takes *this and splits its contents into left and right. + */ + std::string split_into( + StringKVInnerNodeLayout &left, + StringKVInnerNodeLayout &right) const { + auto piviter = get_split_pivot(); + assert(piviter != iter_end()); + + copy_from_foreign(left.iter_begin(), iter_begin(), piviter); + left.set_size(piviter - iter_begin()); + + copy_from_foreign(right.iter_begin(), piviter, iter_end()); + right.set_size(iter_end() - piviter); + + auto [lmeta, rmeta] = get_meta().split_into(); + left.set_meta(lmeta); + right.set_meta(rmeta); + + return piviter->get_key(); + } + + /** + * merge_from + * + * Takes two nodes and copies their contents into *this. + * + * precondition: left.size() + right.size() < CAPACITY + */ + void merge_from( + const StringKVInnerNodeLayout &left, + const StringKVInnerNodeLayout &right) { + copy_from_foreign( + iter_end(), + left.iter_begin(), + left.iter_end()); + set_size(left.get_size()); + + copy_from_foreign( + iter_end(), + right.iter_begin(), + right.iter_end()); + set_size(left.get_size() + right.get_size()); + set_meta(omap_node_meta_t::merge_from(left.get_meta(), right.get_meta())); + } + + /** + * balance_into_new_nodes + * + * Takes the contents of left and right and copies them into + * replacement_left and replacement_right such that + * the size of replacement_left just >= 1/2 of (left + right) + */ + static std::string balance_into_new_nodes( + const StringKVInnerNodeLayout &left, + const StringKVInnerNodeLayout &right, + StringKVInnerNodeLayout &replacement_left, + StringKVInnerNodeLayout &replacement_right) + { + uint32_t left_size = omap_inner_key_t(left.get_node_key_ptr()[left.get_size()-1]).key_off; + uint32_t right_size = omap_inner_key_t(right.get_node_key_ptr()[right.get_size()-1]).key_off; + uint32_t total = left_size + right_size; + uint32_t pivot_size = total / 2; + uint32_t pivot_idx = 0; + if (pivot_size < left_size) { + uint32_t size = 0; + for (auto ite = left.iter_begin(); ite < left.iter_end(); ite++) { + auto node_key = ite->get_node_key(); + size += node_key.key_len; + if (size >= pivot_size){ + pivot_idx = ite.get_index(); + break; + } + } + } else { + uint32_t more_size = pivot_size - left_size; + uint32_t size = 0; + for (auto ite = right.iter_begin(); ite < right.iter_end(); ite++) { + auto node_key = ite->get_node_key(); + size += node_key.key_len; + if (size >= more_size){ + pivot_idx = ite.get_index() + left.get_size(); + break; + } + } + } + + auto replacement_pivot = pivot_idx >= left.get_size() ? + right.iter_idx(pivot_idx - left.get_size())->get_key() : + left.iter_idx(pivot_idx)->get_key(); + + if (pivot_size < left_size) { + copy_from_foreign( + replacement_left.iter_end(), + left.iter_begin(), + left.iter_idx(pivot_idx)); + replacement_left.set_size(pivot_idx); + + copy_from_foreign( + replacement_right.iter_end(), + left.iter_idx(pivot_idx), + left.iter_end()); + replacement_right.set_size(left.get_size() - pivot_idx); + + copy_from_foreign( + replacement_right.iter_end(), + right.iter_begin(), + right.iter_end()); + replacement_right.set_size(right.get_size() + left.get_size()- pivot_idx); + } else { + copy_from_foreign( + replacement_left.iter_end(), + left.iter_begin(), + left.iter_end()); + replacement_left.set_size(left.get_size()); + + copy_from_foreign( + replacement_left.iter_end(), + right.iter_begin(), + right.iter_idx(pivot_idx - left.get_size())); + replacement_left.set_size(pivot_idx); + + copy_from_foreign( + replacement_right.iter_end(), + right.iter_idx(pivot_idx - left.get_size()), + right.iter_end()); + replacement_right.set_size(right.get_size() + left.get_size() - pivot_idx); + } + + auto [lmeta, rmeta] = omap_node_meta_t::rebalance( + left.get_meta(), right.get_meta()); + replacement_left.set_meta(lmeta); + replacement_right.set_meta(rmeta); + return replacement_pivot; + } + +private: + void inner_insert( + iterator iter, + const std::string &key, + laddr_t val) { + if (iter != iter_begin()) { + assert((iter - 1)->get_key() < key); + } + if (iter != iter_end()) { + assert(iter->get_key() > key); + } + assert(!is_overflow(key.size())); + + if (iter != iter_end()) { + copy_from_local(key.size(), iter + 1, iter, iter_end()); + } + + omap_inner_key_t nkey; + nkey.key_len = key.size(); + nkey.laddr = val; + if (iter != iter_begin()) { + auto pkey = (iter - 1).get_node_key(); + nkey.key_off = nkey.key_len + pkey.key_off; + } else { + nkey.key_off = nkey.key_len; + } + + iter->set_node_key(nkey); + set_size(get_size() + 1); + iter->set_node_val(key); + } + + void inner_update( + iterator iter, + laddr_t addr) { + assert(iter != iter_end()); + auto node_key = iter->get_node_key(); + node_key.laddr = addr; + iter->set_node_key(node_key); + } + + void inner_remove(iterator iter) { + assert(iter != iter_end()); + if ((iter + 1) != iter_end()) + copy_from_local(iter->get_node_key().key_len, iter, iter + 1, iter_end()); + set_size(get_size() - 1); + } + + /** + * get_key_ptr + * + * Get pointer to start of key array + */ + omap_inner_key_le_t *get_node_key_ptr() { + return L::Partial(1, 1, get_size()).template Pointer<2>(buf); + } + const omap_inner_key_le_t *get_node_key_ptr() const { + return L::Partial(1, 1, get_size()).template Pointer<2>(buf); + } + +}; + +/** + * StringKVLeafNodeLayout + * + * layout diagram: + * + * # <----------------------------- node range -------------------------------------------------> # + * # #<~># free space # + * # <------------- left part ---------------------------> # <~# <----- right key-value pairs --> # + * # # <------------ left keys ------------> #~> # # + * # # keys [2, n) |<~># #<~>| right kvs [2, n) # + * # # <--- key 0 ---> | <--- key 1 ---> | # # | <-- kv 1 --> | <-- kv 0 --> # + * # # | | # # | | # + * # num_ | meta # key | key | val | key | key | val | # # | key | val | key | val # + * # keys | depth # off | len | len | off | len | len | # # | buff | buff | buff | buff # + * # # 0 | 0 | 0 | 1 | 1 | 1 |...#...#...| key 1 | val 1| key 0 | val 0 # + * # | | | <--- off ----+-------------> # + * # | | ^ | <--- off ---> # + * | | | ^ + * | +-----------------------------------+ | + * +-------------------------------------------------------------------+ + */ +class StringKVLeafNodeLayout { + char *buf = nullptr; + + using L = absl::container_internal::Layout<ceph_le32, omap_node_meta_le_t, omap_leaf_key_le_t>; + static constexpr L layout{1, 1, 1}; // = L::Partial(1, 1, 1); + friend class delta_leaf_t; + +public: + template <bool is_const> + class iter_t { + friend class StringKVLeafNodeLayout; + using parent_t = typename crimson::common::maybe_const_t<StringKVLeafNodeLayout, is_const>::type; + + template <typename iterator, typename const_iterator> + friend void copy_from_foreign(iterator, const_iterator, const_iterator); + template <typename iterator> + friend void copy_from_local(unsigned, iterator, iterator, iterator); + + parent_t node; + uint16_t index; + + iter_t( + parent_t parent, + uint16_t index) : node(parent), index(index) {} + + public: + iter_t(const iter_t &) = default; + iter_t(iter_t &&) = default; + iter_t &operator=(const iter_t &) = default; + iter_t &operator=(iter_t &&) = default; + + operator iter_t<!is_const>() const { + static_assert(!is_const); + return iter_t<!is_const>(node, index); + } + + iter_t &operator*() { return *this; } + iter_t *operator->() { return this; } + + iter_t operator++(int) { + auto ret = *this; + ++index; + return ret; + } + + iter_t &operator++() { + ++index; + return *this; + } + + uint16_t operator-(const iter_t &rhs) const { + assert(rhs.node == node); + return index - rhs.index; + } + + iter_t operator+(uint16_t off) const { + return iter_t( + node, + index + off); + } + iter_t operator-(uint16_t off) const { + return iter_t( + node, + index - off); + } + + uint16_t operator<(const iter_t &rhs) const { + assert(rhs.node == node); + return index < rhs.index; + } + + uint16_t operator>(const iter_t &rhs) const { + assert(rhs.node == node); + return index > rhs.index; + } + + bool operator==(const iter_t &rhs) const { + assert(node == rhs.node); + return rhs.index == index; + } + + bool operator!=(const iter_t &rhs) const { + assert(node == rhs.node); + return index != rhs.index; + } + + private: + omap_leaf_key_t get_node_key() const { + omap_leaf_key_le_t kint = node->get_node_key_ptr()[index]; + return omap_leaf_key_t(kint); + } + auto get_node_key_ptr() const { + return reinterpret_cast< + typename crimson::common::maybe_const_t<char, is_const>::type>( + node->get_node_key_ptr() + index); + } + + uint32_t get_node_val_offset() const { + return get_node_key().key_off; + } + auto get_node_val_ptr() const { + auto tail = node->buf + OMAP_LEAF_BLOCK_SIZE; + if (*this == node->iter_end()) + return tail; + else { + return tail - get_node_val_offset(); + } + } + + int get_right_offset_end() const { + if (index == 0) + return 0; + else + return (*this - 1)->get_node_val_offset(); + } + auto get_right_ptr_end() const { + return node->buf + OMAP_LEAF_BLOCK_SIZE - get_right_offset_end(); + } + + void update_offset(int offset) { + auto key = get_node_key(); + assert(offset + key.key_off >= 0); + key.key_off += offset; + set_node_key(key); + } + + void set_node_key(omap_leaf_key_t _lb) const { + static_assert(!is_const); + omap_leaf_key_le_t lb; + lb = _lb; + node->get_node_key_ptr()[index] = lb; + } + + void set_node_val(const std::string &key, const ceph::bufferlist &val) { + static_assert(!is_const); + auto node_key = get_node_key(); + assert(key.size() == node_key.key_len); + assert(val.length() == node_key.val_len); + ::memcpy(get_node_val_ptr(), key.data(), key.size()); + auto bliter = val.begin(); + bliter.copy(node_key.val_len, get_node_val_ptr() + node_key.key_len); + } + + public: + uint16_t get_index() const { + return index; + } + + std::string get_key() const { + return std::string( + get_node_val_ptr(), + get_node_key().key_len); + } + + std::string get_str_val() const { + auto node_key = get_node_key(); + return std::string( + get_node_val_ptr() + node_key.key_len, + get_node_key().val_len); + } + + ceph::bufferlist get_val() const { + auto node_key = get_node_key(); + ceph::bufferlist bl; + ceph::bufferptr bptr( + get_node_val_ptr() + node_key.key_len, + get_node_key().val_len); + bl.append(bptr); + return bl; + } + }; + using const_iterator = iter_t<true>; + using iterator = iter_t<false>; + +public: + void journal_leaf_insert( + const_iterator _iter, + const std::string &key, + const ceph::bufferlist &val, + delta_leaf_buffer_t *recorder) { + auto iter = iterator(this, _iter.index); + if (recorder) { + recorder->insert( + key, + val); + } + leaf_insert(iter, key, val); + } + + void journal_leaf_update( + const_iterator _iter, + const std::string &key, + const ceph::bufferlist &val, + delta_leaf_buffer_t *recorder) { + auto iter = iterator(this, _iter.index); + if (recorder) { + recorder->remove(iter->get_key()); + recorder->insert(key, val); + } + leaf_update(iter, key, val); + } + + void journal_leaf_remove( + const_iterator _iter, + delta_leaf_buffer_t *recorder) { + auto iter = iterator(this, _iter.index); + if (recorder) { + recorder->remove(iter->get_key()); + } + leaf_remove(iter); + } + + StringKVLeafNodeLayout(char *buf) : + buf(buf) {} + + const_iterator iter_begin() const { + return const_iterator( + this, + 0); + } + + const_iterator iter_end() const { + return const_iterator( + this, + get_size()); + } + + iterator iter_begin() { + return iterator( + this, + 0); + } + + iterator iter_end() { + return iterator( + this, + get_size()); + } + + const_iterator iter_idx(uint16_t off) const { + return const_iterator( + this, + off); + } + + const_iterator string_lower_bound(std::string_view str) const { + uint16_t start = 0, end = get_size(); + while (start != end) { + unsigned mid = (start + end) / 2; + const_iterator iter(this, mid); + std::string s = iter->get_key(); + if (s < str) { + start = ++mid; + } else if (s > str) { + end = mid; + } else { + return iter; + } + } + return const_iterator(this, start); + } + + iterator string_lower_bound(std::string_view str) { + const auto &tref = *this; + return iterator(this, tref.string_lower_bound(str).index); + } + + const_iterator string_upper_bound(std::string_view str) const { + auto ret = iter_begin(); + for (; ret != iter_end(); ++ret) { + std::string s = ret->get_key(); + if (s > str) + break; + } + return ret; + } + + iterator string_upper_bound(std::string_view str) { + const auto &tref = *this; + return iterator(this, tref.string_upper_bound(str).index); + } + + const_iterator find_string_key(std::string_view str) const { + auto ret = iter_begin(); + for (; ret != iter_end(); ++ret) { + std::string s = ret->get_key(); + if (s == str) + break; + } + return ret; + } + iterator find_string_key(std::string_view str) { + const auto &tref = *this; + return iterator(this, tref.find_string_key(str).index); + } + + const_iterator get_split_pivot() const { + uint32_t total_size = omap_leaf_key_t(get_node_key_ptr()[get_size()-1]).key_off; + uint32_t pivot_size = total_size / 2; + uint32_t size = 0; + for (auto ite = iter_begin(); ite < iter_end(); ite++) { + auto node_key = ite->get_node_key(); + size += node_key.key_len + node_key.val_len; + if (size >= pivot_size){ + return ite; + } + } + return iter_end(); + } + + uint32_t get_size() const { + ceph_le32 &size = *layout.template Pointer<0>(buf); + return uint32_t(size); + } + + /** + * set_size + * + * Set size representation to match size + */ + void set_size(uint32_t size) { + ceph_le32 s; + s = size; + *layout.template Pointer<0>(buf) = s; + } + + /** + * get_meta/set_meta + * + * Enables stashing a templated type within the layout. + * Cannot be modified after initial write as it is not represented + * in delta_t + */ + omap_node_meta_t get_meta() const { + omap_node_meta_le_t &metaint = *layout.template Pointer<1>(buf); + return omap_node_meta_t(metaint); + } + void set_meta(const omap_node_meta_t &meta) { + *layout.template Pointer<1>(buf) = omap_node_meta_le_t(meta); + } + + uint32_t used_space() const { + uint32_t count = get_size(); + if (count) { + omap_leaf_key_t last_key = omap_leaf_key_t(get_node_key_ptr()[count-1]); + return last_key.key_off + count * sizeof(omap_leaf_key_le_t); + } else { + return 0; + } + } + + uint32_t free_space() const { + return capacity() - used_space(); + } + + uint32_t capacity() const { + return OMAP_LEAF_BLOCK_SIZE + - (reinterpret_cast<char*>(layout.template Pointer<2>(buf)) + - reinterpret_cast<char*>(layout.template Pointer<0>(buf))); + } + + bool is_overflow(size_t ksize, size_t vsize) const { + return free_space() < (sizeof(omap_leaf_key_le_t) + ksize + vsize); + } + + bool is_overflow(const StringKVLeafNodeLayout &rhs) const { + return free_space() < rhs.used_space(); + } + + bool below_min() const { + return free_space() > (capacity() / 2); + } + + bool operator==(const StringKVLeafNodeLayout &rhs) const { + if (get_size() != rhs.get_size()) { + return false; + } + + auto iter = iter_begin(); + auto iter2 = rhs.iter_begin(); + while (iter != iter_end()) { + if(iter->get_key() != iter2->get_key() || + iter->get_val() != iter2->get_val()) { + return false; + } + iter++; + iter2++; + } + return true; + } + + /** + * split_into + * + * Takes *this and splits its contents into left and right. + */ + std::string split_into( + StringKVLeafNodeLayout &left, + StringKVLeafNodeLayout &right) const { + auto piviter = get_split_pivot(); + assert (piviter != iter_end()); + + copy_from_foreign(left.iter_begin(), iter_begin(), piviter); + left.set_size(piviter - iter_begin()); + + copy_from_foreign(right.iter_begin(), piviter, iter_end()); + right.set_size(iter_end() - piviter); + + auto [lmeta, rmeta] = get_meta().split_into(); + left.set_meta(lmeta); + right.set_meta(rmeta); + + return piviter->get_key(); + } + + /** + * merge_from + * + * Takes two nodes and copies their contents into *this. + * + * precondition: left.size() + right.size() < CAPACITY + */ + void merge_from( + const StringKVLeafNodeLayout &left, + const StringKVLeafNodeLayout &right) + { + copy_from_foreign( + iter_end(), + left.iter_begin(), + left.iter_end()); + set_size(left.get_size()); + copy_from_foreign( + iter_end(), + right.iter_begin(), + right.iter_end()); + set_size(left.get_size() + right.get_size()); + set_meta(omap_node_meta_t::merge_from(left.get_meta(), right.get_meta())); + } + + /** + * balance_into_new_nodes + * + * Takes the contents of left and right and copies them into + * replacement_left and replacement_right such that + * the size of replacement_left side just >= 1/2 of the total size (left + right). + */ + static std::string balance_into_new_nodes( + const StringKVLeafNodeLayout &left, + const StringKVLeafNodeLayout &right, + StringKVLeafNodeLayout &replacement_left, + StringKVLeafNodeLayout &replacement_right) + { + uint32_t left_size = omap_leaf_key_t(left.get_node_key_ptr()[left.get_size()-1]).key_off; + uint32_t right_size = omap_leaf_key_t(right.get_node_key_ptr()[right.get_size()-1]).key_off; + uint32_t total = left_size + right_size; + uint32_t pivot_size = total / 2; + uint32_t pivot_idx = 0; + if (pivot_size < left_size) { + uint32_t size = 0; + for (auto ite = left.iter_begin(); ite < left.iter_end(); ite++) { + auto node_key = ite->get_node_key(); + size += node_key.key_len + node_key.val_len; + if (size >= pivot_size){ + pivot_idx = ite.get_index(); + break; + } + } + } else { + uint32_t more_size = pivot_size - left_size; + uint32_t size = 0; + for (auto ite = right.iter_begin(); ite < right.iter_end(); ite++) { + auto node_key = ite->get_node_key(); + size += node_key.key_len + node_key.val_len; + if (size >= more_size){ + pivot_idx = ite.get_index() + left.get_size(); + break; + } + } + } + + auto replacement_pivot = pivot_idx >= left.get_size() ? + right.iter_idx(pivot_idx - left.get_size())->get_key() : + left.iter_idx(pivot_idx)->get_key(); + + if (pivot_size < left_size) { + copy_from_foreign( + replacement_left.iter_end(), + left.iter_begin(), + left.iter_idx(pivot_idx)); + replacement_left.set_size(pivot_idx); + + copy_from_foreign( + replacement_right.iter_end(), + left.iter_idx(pivot_idx), + left.iter_end()); + replacement_right.set_size(left.get_size() - pivot_idx); + + copy_from_foreign( + replacement_right.iter_end(), + right.iter_begin(), + right.iter_end()); + replacement_right.set_size(right.get_size() + left.get_size() - pivot_idx); + } else { + copy_from_foreign( + replacement_left.iter_end(), + left.iter_begin(), + left.iter_end()); + replacement_left.set_size(left.get_size()); + + copy_from_foreign( + replacement_left.iter_end(), + right.iter_begin(), + right.iter_idx(pivot_idx - left.get_size())); + replacement_left.set_size(pivot_idx); + + copy_from_foreign( + replacement_right.iter_end(), + right.iter_idx(pivot_idx - left.get_size()), + right.iter_end()); + replacement_right.set_size(right.get_size() + left.get_size() - pivot_idx); + } + + auto [lmeta, rmeta] = omap_node_meta_t::rebalance( + left.get_meta(), right.get_meta()); + replacement_left.set_meta(lmeta); + replacement_right.set_meta(rmeta); + return replacement_pivot; + } + +private: + void leaf_insert( + iterator iter, + const std::string &key, + const bufferlist &val) { + if (iter != iter_begin()) { + assert((iter - 1)->get_key() < key); + } + if (iter != iter_end()) { + assert(iter->get_key() > key); + } + assert(!is_overflow(key.size(), val.length())); + omap_leaf_key_t node_key; + if (iter == iter_begin()) { + node_key.key_off = key.size() + val.length(); + node_key.key_len = key.size(); + node_key.val_len = val.length(); + } else { + node_key.key_off = (iter - 1)->get_node_key().key_off + + (key.size() + val.length()); + node_key.key_len = key.size(); + node_key.val_len = val.length(); + } + if (get_size() != 0 && iter != iter_end()) + copy_from_local(node_key.key_len + node_key.val_len, iter + 1, iter, iter_end()); + + iter->set_node_key(node_key); + set_size(get_size() + 1); + iter->set_node_val(key, val); + } + + void leaf_update( + iterator iter, + const std::string &key, + const ceph::bufferlist &val) { + assert(iter != iter_end()); + leaf_remove(iter); + assert(!is_overflow(key.size(), val.length())); + leaf_insert(iter, key, val); + } + + void leaf_remove(iterator iter) { + assert(iter != iter_end()); + if ((iter + 1) != iter_end()) { + omap_leaf_key_t key = iter->get_node_key(); + copy_from_local(key.key_len + key.val_len, iter, iter + 1, iter_end()); + } + set_size(get_size() - 1); + } + + /** + * get_key_ptr + * + * Get pointer to start of key array + */ + omap_leaf_key_le_t *get_node_key_ptr() { + return L::Partial(1, 1, get_size()).template Pointer<2>(buf); + } + const omap_leaf_key_le_t *get_node_key_ptr() const { + return L::Partial(1, 1, get_size()).template Pointer<2>(buf); + } + +}; + +inline void delta_inner_t::replay(StringKVInnerNodeLayout &l) { + switch (op) { + case op_t::INSERT: { + l.inner_insert(l.string_lower_bound(key), key, addr); + break; + } + case op_t::UPDATE: { + auto iter = l.find_string_key(key); + assert(iter != l.iter_end()); + l.inner_update(iter, addr); + break; + } + case op_t::REMOVE: { + auto iter = l.find_string_key(key); + assert(iter != l.iter_end()); + l.inner_remove(iter); + break; + } + default: + assert(0 == "Impossible"); + } +} + +inline void delta_leaf_t::replay(StringKVLeafNodeLayout &l) { + switch (op) { + case op_t::INSERT: { + l.leaf_insert(l.string_lower_bound(key), key, val); + break; + } + case op_t::UPDATE: { + auto iter = l.find_string_key(key); + assert(iter != l.iter_end()); + l.leaf_update(iter, key, val); + break; + } + case op_t::REMOVE: { + auto iter = l.find_string_key(key); + assert(iter != l.iter_end()); + l.leaf_remove(iter); + break; + } + default: + assert(0 == "Impossible"); + } +} + +} diff --git a/src/crimson/os/seastore/onode.cc b/src/crimson/os/seastore/onode.cc new file mode 100644 index 000000000..96b69fb7c --- /dev/null +++ b/src/crimson/os/seastore/onode.cc @@ -0,0 +1,18 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "onode.h" +#include <iostream> + +namespace crimson::os::seastore { + +std::ostream& operator<<(std::ostream &out, const Onode &rhs) +{ + auto &layout = rhs.get_layout(); + return out << "Onode(" + << "size=" << static_cast<uint32_t>(layout.size) + << ")"; +} + +} + diff --git a/src/crimson/os/seastore/onode.h b/src/crimson/os/seastore/onode.h new file mode 100644 index 000000000..069daa3df --- /dev/null +++ b/src/crimson/os/seastore/onode.h @@ -0,0 +1,89 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <iosfwd> + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include "include/byteorder.h" +#include "seastore_types.h" + +namespace crimson::os::seastore { + +struct onode_layout_t { + // The expected decode size of object_info_t without oid. + static constexpr int MAX_OI_LENGTH = 232; + // We might want to move the ss field out of onode_layout_t. + // The reason is that ss_attr may grow to relative large, as + // its clone_overlap may grow to a large size, if applications + // set objects to a relative large size(for the purpose of reducing + // the number of objects per OSD, so that all objects' metadata + // can be cached in memory) and do many modifications between + // snapshots. + // TODO: implement flexible-sized onode value to store inline ss_attr + // effectively. + static constexpr int MAX_SS_LENGTH = 1; + + ceph_le32 size{0}; + ceph_le32 oi_size{0}; + ceph_le32 ss_size{0}; + omap_root_le_t omap_root; + omap_root_le_t xattr_root; + + object_data_le_t object_data; + + char oi[MAX_OI_LENGTH]; + char ss[MAX_SS_LENGTH]; +} __attribute__((packed)); + +class Transaction; + +/** + * Onode + * + * Interface manipulated by seastore. OnodeManager implementations should + * return objects derived from this interface with layout referencing + * internal representation of onode_layout_t. + */ +class Onode : public boost::intrusive_ref_counter< + Onode, + boost::thread_unsafe_counter> +{ +protected: + virtual laddr_t get_hint() const = 0; + const uint32_t default_metadata_offset = 0; + const uint32_t default_metadata_range = 0; +public: + Onode(uint32_t ddr, uint32_t dmr) + : default_metadata_offset(ddr), + default_metadata_range(dmr) + {} + + virtual bool is_alive() const = 0; + virtual const onode_layout_t &get_layout() const = 0; + virtual onode_layout_t &get_mutable_layout(Transaction &t) = 0; + virtual ~Onode() = default; + + laddr_t get_metadata_hint(uint64_t block_size) const { + assert(default_metadata_offset); + assert(default_metadata_range); + uint64_t range_blocks = default_metadata_range / block_size; + return get_hint() + default_metadata_offset + + (((uint32_t)std::rand() % range_blocks) * block_size); + } + laddr_t get_data_hint() const { + return get_hint(); + } +}; + + +std::ostream& operator<<(std::ostream &out, const Onode &rhs); +using OnodeRef = boost::intrusive_ptr<Onode>; +} + +#if FMT_VERSION >= 90000 +template<> struct fmt::formatter<crimson::os::seastore::Onode> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/onode_manager.h b/src/crimson/os/seastore/onode_manager.h new file mode 100644 index 000000000..123c9e4f8 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager.h @@ -0,0 +1,86 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <iostream> + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <seastar/core/future.hh> + +#include "include/buffer_fwd.h" +#include "include/ceph_assert.h" +#include "common/hobject.h" + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/onode.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/osd/exceptions.h" + +namespace crimson::os::seastore { + +class OnodeManager { + using base_iertr = TransactionManager::base_iertr; +public: + using mkfs_iertr = base_iertr; + using mkfs_ret = mkfs_iertr::future<>; + virtual mkfs_ret mkfs(Transaction &t) = 0; + + using contains_onode_iertr = base_iertr; + using contains_onode_ret = contains_onode_iertr::future<bool>; + virtual contains_onode_ret contains_onode( + Transaction &trans, + const ghobject_t &hoid) = 0; + + using get_onode_iertr = base_iertr::extend< + crimson::ct_error::enoent>; + using get_onode_ret = get_onode_iertr::future< + OnodeRef>; + virtual get_onode_ret get_onode( + Transaction &trans, + const ghobject_t &hoid) = 0; + + using get_or_create_onode_iertr = base_iertr::extend< + crimson::ct_error::value_too_large>; + using get_or_create_onode_ret = get_or_create_onode_iertr::future< + OnodeRef>; + virtual get_or_create_onode_ret get_or_create_onode( + Transaction &trans, + const ghobject_t &hoid) = 0; + + using get_or_create_onodes_iertr = base_iertr::extend< + crimson::ct_error::value_too_large>; + using get_or_create_onodes_ret = get_or_create_onodes_iertr::future< + std::vector<OnodeRef>>; + virtual get_or_create_onodes_ret get_or_create_onodes( + Transaction &trans, + const std::vector<ghobject_t> &hoids) = 0; + + using write_dirty_iertr = base_iertr; + using write_dirty_ret = write_dirty_iertr::future<>; + virtual write_dirty_ret write_dirty( + Transaction &trans, + const std::vector<OnodeRef> &onodes) = 0; + + using erase_onode_iertr = base_iertr; + using erase_onode_ret = erase_onode_iertr::future<>; + virtual erase_onode_ret erase_onode( + Transaction &trans, + OnodeRef &onode) = 0; + + using list_onodes_iertr = base_iertr; + using list_onodes_bare_ret = std::tuple<std::vector<ghobject_t>, ghobject_t>; + using list_onodes_ret = list_onodes_iertr::future<list_onodes_bare_ret>; + virtual list_onodes_ret list_onodes( + Transaction &trans, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) = 0; + + virtual ~OnodeManager() {} +}; +using OnodeManagerRef = std::unique_ptr<OnodeManager>; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.cc new file mode 100644 index 000000000..bff27ab65 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.cc @@ -0,0 +1,183 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/logging.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.h" + +SET_SUBSYS(seastore_onode); + +namespace crimson::os::seastore::onode { + +FLTreeOnodeManager::contains_onode_ret FLTreeOnodeManager::contains_onode( + Transaction &trans, + const ghobject_t &hoid) +{ + return tree.contains(trans, hoid); +} + +FLTreeOnodeManager::get_onode_ret FLTreeOnodeManager::get_onode( + Transaction &trans, + const ghobject_t &hoid) +{ + LOG_PREFIX(FLTreeOnodeManager::get_onode); + return tree.find( + trans, hoid + ).si_then([this, &hoid, &trans, FNAME](auto cursor) + -> get_onode_ret { + if (cursor == tree.end()) { + DEBUGT("no entry for {}", trans, hoid); + return crimson::ct_error::enoent::make(); + } + auto val = OnodeRef(new FLTreeOnode( + default_data_reservation, + default_metadata_range, + cursor.value())); + return get_onode_iertr::make_ready_future<OnodeRef>( + val + ); + }); +} + +FLTreeOnodeManager::get_or_create_onode_ret +FLTreeOnodeManager::get_or_create_onode( + Transaction &trans, + const ghobject_t &hoid) +{ + LOG_PREFIX(FLTreeOnodeManager::get_or_create_onode); + return tree.insert( + trans, hoid, + OnodeTree::tree_value_config_t{sizeof(onode_layout_t)} + ).si_then([this, &trans, &hoid, FNAME](auto p) + -> get_or_create_onode_ret { + auto [cursor, created] = std::move(p); + auto val = OnodeRef(new FLTreeOnode( + default_data_reservation, + default_metadata_range, + cursor.value())); + if (created) { + DEBUGT("created onode for entry for {}", trans, hoid); + val->get_mutable_layout(trans) = onode_layout_t{}; + } + return get_or_create_onode_iertr::make_ready_future<OnodeRef>( + val + ); + }); +} + +FLTreeOnodeManager::get_or_create_onodes_ret +FLTreeOnodeManager::get_or_create_onodes( + Transaction &trans, + const std::vector<ghobject_t> &hoids) +{ + return seastar::do_with( + std::vector<OnodeRef>(), + [this, &hoids, &trans](auto &ret) { + ret.reserve(hoids.size()); + return trans_intr::do_for_each( + hoids, + [this, &trans, &ret](auto &hoid) { + return get_or_create_onode(trans, hoid + ).si_then([&ret](auto &&onoderef) { + ret.push_back(std::move(onoderef)); + }); + }).si_then([&ret] { + return std::move(ret); + }); + }); +} + +FLTreeOnodeManager::write_dirty_ret FLTreeOnodeManager::write_dirty( + Transaction &trans, + const std::vector<OnodeRef> &onodes) +{ + return trans_intr::do_for_each( + onodes, + [&trans](auto &onode) -> eagain_ifuture<> { + if (!onode) { + return eagain_iertr::make_ready_future<>(); + } + auto &flonode = static_cast<FLTreeOnode&>(*onode); + if (!flonode.is_alive()) { + return eagain_iertr::make_ready_future<>(); + } + switch (flonode.status) { + case FLTreeOnode::status_t::MUTATED: { + flonode.populate_recorder(trans); + return eagain_iertr::make_ready_future<>(); + } + case FLTreeOnode::status_t::STABLE: { + return eagain_iertr::make_ready_future<>(); + } + default: + __builtin_unreachable(); + } + }); +} + +FLTreeOnodeManager::erase_onode_ret FLTreeOnodeManager::erase_onode( + Transaction &trans, + OnodeRef &onode) +{ + auto &flonode = static_cast<FLTreeOnode&>(*onode); + assert(flonode.is_alive()); + if (flonode.status == FLTreeOnode::status_t::MUTATED) { + flonode.populate_recorder(trans); + } + flonode.mark_delete(); + return tree.erase(trans, flonode); +} + +FLTreeOnodeManager::list_onodes_ret FLTreeOnodeManager::list_onodes( + Transaction &trans, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) +{ + return tree.lower_bound(trans, start + ).si_then([this, &trans, end, limit] (auto&& cursor) { + using crimson::os::seastore::onode::full_key_t; + return seastar::do_with( + limit, + std::move(cursor), + list_onodes_bare_ret(), + [this, &trans, end] (auto& to_list, auto& current_cursor, auto& ret) { + return trans_intr::repeat( + [this, &trans, end, &to_list, ¤t_cursor, &ret] () + -> eagain_ifuture<seastar::stop_iteration> { + if (current_cursor.is_end()) { + std::get<1>(ret) = ghobject_t::get_max(); + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } else if (current_cursor.get_ghobj() >= end) { + std::get<1>(ret) = end; + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + if (to_list == 0) { + std::get<1>(ret) = current_cursor.get_ghobj(); + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + std::get<0>(ret).emplace_back(current_cursor.get_ghobj()); + return tree.get_next(trans, current_cursor + ).si_then([&to_list, ¤t_cursor] (auto&& next_cursor) mutable { + // we intentionally hold the current_cursor during get_next() to + // accelerate tree lookup. + --to_list; + current_cursor = next_cursor; + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::no); + }); + }).si_then([&ret] () mutable { + return seastar::make_ready_future<list_onodes_bare_ret>( + std::move(ret)); + // return ret; + }); + }); + }); +} + +FLTreeOnodeManager::~FLTreeOnodeManager() {} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.h b/src/crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.h new file mode 100644 index 000000000..09998fbfa --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.h @@ -0,0 +1,171 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "crimson/os/seastore/onode_manager.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/value.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/tree.h" + +namespace crimson::os::seastore::onode { + +struct FLTreeOnode final : Onode, Value { + static constexpr tree_conf_t TREE_CONF = { + value_magic_t::ONODE, + 256, // max_ns_size + // same to option osd_max_object_namespace_len + 2048, // max_oid_size + // same to option osd_max_object_name_len + 1200, // max_value_payload_size + // see crimson::os::seastore::onode_layout_t + 8192, // internal_node_size + // see the formula in validate_tree_config + 16384 // leaf_node_size + // see the formula in validate_tree_config + }; + + enum class status_t { + STABLE, + MUTATED, + DELETED + } status = status_t::STABLE; + + FLTreeOnode(FLTreeOnode&&) = default; + FLTreeOnode& operator=(FLTreeOnode&&) = delete; + + FLTreeOnode(const FLTreeOnode&) = default; + FLTreeOnode& operator=(const FLTreeOnode&) = delete; + + template <typename... T> + FLTreeOnode(uint32_t ddr, uint32_t dmr, T&&... args) + : Onode(ddr, dmr), + Value(std::forward<T>(args)...) {} + + template <typename... T> + FLTreeOnode(T&&... args) + : Onode(0, 0), + Value(std::forward<T>(args)...) {} + + struct Recorder : public ValueDeltaRecorder { + Recorder(bufferlist &bl) : ValueDeltaRecorder(bl) {} + + value_magic_t get_header_magic() const final { + return TREE_CONF.value_magic; + } + + void apply_value_delta( + ceph::bufferlist::const_iterator &bliter, + NodeExtentMutable &value, + laddr_t) final { + assert(value.get_length() == sizeof(onode_layout_t)); + bliter.copy(value.get_length(), value.get_write()); + } + + void record_delta(NodeExtentMutable &value) { + // TODO: probably could use versioning, etc + assert(value.get_length() == sizeof(onode_layout_t)); + ceph::buffer::ptr bptr(value.get_length()); + memcpy(bptr.c_str(), value.get_read(), value.get_length()); + get_encoded(value).append(bptr); + } + }; + + bool is_alive() const { + return status != status_t::DELETED; + } + const onode_layout_t &get_layout() const final { + assert(status != status_t::DELETED); + return *read_payload<onode_layout_t>(); + } + + onode_layout_t &get_mutable_layout(Transaction &t) final { + assert(status != status_t::DELETED); + auto p = prepare_mutate_payload< + onode_layout_t, + Recorder>(t); + status = status_t::MUTATED; + return *reinterpret_cast<onode_layout_t*>(p.first.get_write()); + }; + + void populate_recorder(Transaction &t) { + assert(status == status_t::MUTATED); + auto p = prepare_mutate_payload< + onode_layout_t, + Recorder>(t); + if (p.second) { + p.second->record_delta( + p.first); + } + status = status_t::STABLE; + } + + void mark_delete() { + assert(status != status_t::DELETED); + status = status_t::DELETED; + } + + laddr_t get_hint() const final { + return Value::get_hint(); + } + ~FLTreeOnode() final {} +}; + +using OnodeTree = Btree<FLTreeOnode>; + +using crimson::common::get_conf; + +class FLTreeOnodeManager : public crimson::os::seastore::OnodeManager { + OnodeTree tree; + + uint32_t default_data_reservation = 0; + uint32_t default_metadata_offset = 0; + uint32_t default_metadata_range = 0; +public: + FLTreeOnodeManager(TransactionManager &tm) : + tree(NodeExtentManager::create_seastore(tm)), + default_data_reservation( + get_conf<uint64_t>("seastore_default_max_object_size")), + default_metadata_offset(default_data_reservation), + default_metadata_range( + get_conf<uint64_t>("seastore_default_object_metadata_reservation")) + {} + + mkfs_ret mkfs(Transaction &t) { + return tree.mkfs(t); + } + + contains_onode_ret contains_onode( + Transaction &trans, + const ghobject_t &hoid) final; + + get_onode_ret get_onode( + Transaction &trans, + const ghobject_t &hoid) final; + + get_or_create_onode_ret get_or_create_onode( + Transaction &trans, + const ghobject_t &hoid) final; + + get_or_create_onodes_ret get_or_create_onodes( + Transaction &trans, + const std::vector<ghobject_t> &hoids) final; + + write_dirty_ret write_dirty( + Transaction &trans, + const std::vector<OnodeRef> &onodes) final; + + erase_onode_ret erase_onode( + Transaction &trans, + OnodeRef &onode) final; + + list_onodes_ret list_onodes( + Transaction &trans, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) final; + + ~FLTreeOnodeManager(); +}; +using FLTreeOnodeManagerRef = std::unique_ptr<FLTreeOnodeManager>; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/fwd.h b/src/crimson/os/seastore/onode_manager/staged-fltree/fwd.h new file mode 100644 index 000000000..43f8b87ed --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/fwd.h @@ -0,0 +1,196 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <algorithm> +#include <cstring> +#include <limits> +#include <memory> +#include <ostream> +#include <string> + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/cached_extent.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/transaction.h" + +namespace crimson::os::seastore::onode { + +using eagain_iertr = trans_iertr< + crimson::errorator<crimson::ct_error::input_output_error> >; +template <class ValueT=void> +using eagain_ifuture = eagain_iertr::future<ValueT>; + +using crimson::os::seastore::Transaction; +using crimson::os::seastore::TransactionRef; +using crimson::os::seastore::laddr_t; +using crimson::os::seastore::L_ADDR_MIN; +using crimson::os::seastore::L_ADDR_NULL; +using crimson::os::seastore::extent_len_t; + +class DeltaRecorder; +class NodeExtent; +class NodeExtentManager; +class RootNodeTracker; +struct ValueBuilder; +using DeltaRecorderURef = std::unique_ptr<DeltaRecorder>; +using NodeExtentRef = crimson::os::seastore::TCachedExtentRef<NodeExtent>; +using NodeExtentManagerURef = std::unique_ptr<NodeExtentManager>; +using RootNodeTrackerURef = std::unique_ptr<RootNodeTracker>; +struct context_t { + NodeExtentManager& nm; + const ValueBuilder& vb; + Transaction& t; +}; + +class LeafNodeImpl; +class InternalNodeImpl; +class NodeImpl; +using LeafNodeImplURef = std::unique_ptr<LeafNodeImpl>; +using InternalNodeImplURef = std::unique_ptr<InternalNodeImpl>; +using NodeImplURef = std::unique_ptr<NodeImpl>; + +using level_t = uint8_t; +constexpr auto MAX_LEVEL = std::numeric_limits<level_t>::max(); + +// a type only to index within a node, 32 bits should be enough +using index_t = uint32_t; +constexpr auto INDEX_END = std::numeric_limits<index_t>::max(); +constexpr auto INDEX_LAST = INDEX_END - 0x4; +constexpr auto INDEX_UPPER_BOUND = INDEX_END - 0x8; +inline bool is_valid_index(index_t index) { return index < INDEX_UPPER_BOUND; } + +// we support up to 64 KiB tree nodes +using node_offset_t = uint16_t; +constexpr node_offset_t DISK_BLOCK_SIZE = 1u << 12; +constexpr auto MAX_NODE_SIZE = + (extent_len_t)std::numeric_limits<node_offset_t>::max() + 1; +inline bool is_valid_node_size(extent_len_t node_size) { + return (node_size > 0 && + node_size <= MAX_NODE_SIZE && + node_size % DISK_BLOCK_SIZE == 0); +} + +using string_size_t = uint16_t; + +enum class MatchKindBS : int8_t { NE = -1, EQ = 0 }; + +enum class MatchKindCMP : int8_t { LT = -1, EQ = 0, GT }; +inline MatchKindCMP toMatchKindCMP(int value) { + if (value > 0) { + return MatchKindCMP::GT; + } else if (value < 0) { + return MatchKindCMP::LT; + } else { + return MatchKindCMP::EQ; + } +} +template <typename Type> +MatchKindCMP toMatchKindCMP(const Type& l, const Type& r) { + if (l > r) { + return MatchKindCMP::GT; + } else if (l < r) { + return MatchKindCMP::LT; + } else { + return MatchKindCMP::EQ; + } +} + +inline MatchKindCMP toMatchKindCMP( + std::string_view l, std::string_view r) { + return toMatchKindCMP(l.compare(r)); +} + +inline MatchKindCMP reverse(MatchKindCMP cmp) { + if (cmp == MatchKindCMP::LT) { + return MatchKindCMP::GT; + } else if (cmp == MatchKindCMP::GT) { + return MatchKindCMP::LT; + } else { + return cmp; + } +} + +struct tree_stats_t { + size_t size_persistent_leaf = 0; + size_t size_persistent_internal = 0; + size_t size_filled_leaf = 0; + size_t size_filled_internal = 0; + size_t size_logical_leaf = 0; + size_t size_logical_internal = 0; + size_t size_overhead_leaf = 0; + size_t size_overhead_internal = 0; + size_t size_value_leaf = 0; + size_t size_value_internal = 0; + unsigned num_kvs_leaf = 0; + unsigned num_kvs_internal = 0; + unsigned num_nodes_leaf = 0; + unsigned num_nodes_internal = 0; + unsigned height = 0; + + size_t size_persistent() const { + return size_persistent_leaf + size_persistent_internal; } + size_t size_filled() const { + return size_filled_leaf + size_filled_internal; } + size_t size_logical() const { + return size_logical_leaf + size_logical_internal; } + size_t size_overhead() const { + return size_overhead_leaf + size_overhead_internal; } + size_t size_value() const { + return size_value_leaf + size_value_internal; } + unsigned num_kvs() const { + return num_kvs_leaf + num_kvs_internal; } + unsigned num_nodes() const { + return num_nodes_leaf + num_nodes_internal; } + + double ratio_fullness() const { + return (double)size_filled() / size_persistent(); } + double ratio_key_compression() const { + return (double)(size_filled() - size_value()) / (size_logical() - size_value()); } + double ratio_overhead() const { + return (double)size_overhead() / size_filled(); } + double ratio_keys_leaf() const { + return (double)num_kvs_leaf / num_kvs(); } + double ratio_nodes_leaf() const { + return (double)num_nodes_leaf / num_nodes(); } + double ratio_filled_leaf() const { + return (double)size_filled_leaf / size_filled(); } +}; +inline std::ostream& operator<<(std::ostream& os, const tree_stats_t& stats) { + os << "Tree stats:" + << "\n height = " << stats.height + << "\n num values = " << stats.num_kvs_leaf + << "\n num nodes = " << stats.num_nodes() + << " (leaf=" << stats.num_nodes_leaf + << ", internal=" << stats.num_nodes_internal << ")" + << "\n size persistent = " << stats.size_persistent() << "B" + << "\n size filled = " << stats.size_filled() << "B" + << " (value=" << stats.size_value_leaf << "B" + << ", rest=" << stats.size_filled() - stats.size_value_leaf << "B)" + << "\n size logical = " << stats.size_logical() << "B" + << "\n size overhead = " << stats.size_overhead() << "B" + << "\n ratio fullness = " << stats.ratio_fullness() + << "\n ratio keys leaf = " << stats.ratio_keys_leaf() + << "\n ratio nodes leaf = " << stats.ratio_nodes_leaf() + << "\n ratio filled leaf = " << stats.ratio_filled_leaf() + << "\n ratio key compression = " << stats.ratio_key_compression(); + assert(stats.num_kvs_internal + 1 == stats.num_nodes()); + return os; +} + +template <typename PtrType> +void reset_ptr(PtrType& ptr, const char* origin_base, + const char* new_base, extent_len_t node_size) { + assert((const char*)ptr > origin_base); + assert((const char*)ptr - origin_base < (int)node_size); + ptr = reinterpret_cast<PtrType>( + (const char*)ptr - origin_base + new_base); +} + +} + +#if FMT_VERSION >= 90000 +template<> +struct fmt::formatter<crimson::os::seastore::onode::tree_stats_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/node.cc new file mode 100644 index 000000000..6f08f4d3c --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node.cc @@ -0,0 +1,2282 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "node.h" + +#include <cassert> +#include <exception> +#include <sstream> + +#include "common/likely.h" +#include "crimson/common/utility.h" +#include "crimson/os/seastore/logging.h" + +#include "node_extent_manager.h" +#include "node_impl.h" +#include "stages/node_stage_layout.h" + +SET_SUBSYS(seastore_onode); + +namespace fmt { +template <typename T> +const void* ptr(const ::boost::intrusive_ptr<T>& p) { + return p.get(); +} +} + +namespace crimson::os::seastore::onode { +/* + * tree_cursor_t + */ + +// create from insert +tree_cursor_t::tree_cursor_t(Ref<LeafNode> node, const search_position_t& pos) + : ref_leaf_node{node}, position{pos}, cache{ref_leaf_node} +{ + assert(is_tracked()); + ref_leaf_node->do_track_cursor<true>(*this); + // do not account updates for the inserted values + is_mutated = true; +} + +// create from lookup +tree_cursor_t::tree_cursor_t( + Ref<LeafNode> node, const search_position_t& pos, + const key_view_t& key_view, const value_header_t* p_value_header) + : ref_leaf_node{node}, position{pos}, cache{ref_leaf_node} +{ + assert(is_tracked()); + update_cache_same_node(key_view, p_value_header); + ref_leaf_node->do_track_cursor<true>(*this); +} + +// lookup reaches the end, contain leaf node for further insert +tree_cursor_t::tree_cursor_t(Ref<LeafNode> node) + : ref_leaf_node{node}, position{search_position_t::end()}, cache{ref_leaf_node} +{ + assert(is_end()); + assert(ref_leaf_node->is_level_tail()); +} + +// create an invalid tree_cursor_t +tree_cursor_t::~tree_cursor_t() +{ + if (is_tracked()) { + ref_leaf_node->do_untrack_cursor(*this); + } +} + +eagain_ifuture<Ref<tree_cursor_t>> +tree_cursor_t::get_next(context_t c) +{ + assert(is_tracked()); + return ref_leaf_node->get_next_cursor(c, position); +} + +void tree_cursor_t::assert_next_to( + const tree_cursor_t& prv, value_magic_t magic) const +{ +#ifndef NDEBUG + assert(!prv.is_end()); + if (is_end()) { + assert(ref_leaf_node == prv.ref_leaf_node); + assert(ref_leaf_node->is_level_tail()); + } else if (is_tracked()) { + auto key = get_key_view(magic); + auto prv_key = prv.get_key_view(magic); + assert(key > prv_key); + if (ref_leaf_node == prv.ref_leaf_node) { + position.assert_next_to(prv.position); + } else { + assert(!prv.ref_leaf_node->is_level_tail()); + assert(position == search_position_t::begin()); + } + } else { + assert(is_invalid()); + ceph_abort("impossible"); + } +#endif +} + +template <bool FORCE_MERGE> +eagain_ifuture<Ref<tree_cursor_t>> +tree_cursor_t::erase(context_t c, bool get_next) +{ + assert(is_tracked()); + return ref_leaf_node->erase<FORCE_MERGE>(c, position, get_next); +} +template eagain_ifuture<Ref<tree_cursor_t>> +tree_cursor_t::erase<true>(context_t, bool); +template eagain_ifuture<Ref<tree_cursor_t>> +tree_cursor_t::erase<false>(context_t, bool); + +std::strong_ordering tree_cursor_t::compare_to( + const tree_cursor_t& o, value_magic_t magic) const +{ + if (!is_tracked() && !o.is_tracked()) { + return std::strong_ordering::equal; + } else if (!is_tracked()) { + return std::strong_ordering::greater; + } else if (!o.is_tracked()) { + return std::strong_ordering::less; + } + + assert(is_tracked() && o.is_tracked()); + // all tracked cursors are singletons + if (this == &o) { + return std::strong_ordering::equal; + } + + std::strong_ordering ret = std::strong_ordering::equal; + if (ref_leaf_node == o.ref_leaf_node) { + ret = position <=> o.position; + } else { + auto key = get_key_view(magic); + auto o_key = o.get_key_view(magic); + ret = key <=> o_key; + } + assert(ret != 0); + return ret; +} + +eagain_ifuture<> +tree_cursor_t::extend_value(context_t c, value_size_t extend_size) +{ + assert(is_tracked()); + return ref_leaf_node->extend_value(c, position, extend_size); +} + +eagain_ifuture<> +tree_cursor_t::trim_value(context_t c, value_size_t trim_size) +{ + assert(is_tracked()); + return ref_leaf_node->trim_value(c, position, trim_size); +} + +template <bool VALIDATE> +void tree_cursor_t::update_track( + Ref<LeafNode> node, const search_position_t& pos) +{ + // I must be already untracked + assert(is_tracked()); + assert(!ref_leaf_node->check_is_tracking(*this)); + // track the new node and new pos + assert(!pos.is_end()); + ref_leaf_node = node; + position = pos; + // we lazy update the key/value information until user asked + cache.invalidate(); + ref_leaf_node->do_track_cursor<VALIDATE>(*this); +} +template void tree_cursor_t::update_track<true>(Ref<LeafNode>, const search_position_t&); +template void tree_cursor_t::update_track<false>(Ref<LeafNode>, const search_position_t&); + +void tree_cursor_t::update_cache_same_node(const key_view_t& key_view, + const value_header_t* p_value_header) const +{ + assert(is_tracked()); + cache.update_all(ref_leaf_node->get_version(), key_view, p_value_header); + cache.validate_is_latest(position); +} + +void tree_cursor_t::invalidate() +{ + assert(is_tracked()); + ref_leaf_node.reset(); + assert(is_invalid()); + // I must be removed from LeafNode +} + +/* + * tree_cursor_t::Cache + */ + +tree_cursor_t::Cache::Cache(Ref<LeafNode>& ref_leaf_node) + : ref_leaf_node{ref_leaf_node} {} + +void tree_cursor_t::Cache::update_all(const node_version_t& current_version, + const key_view_t& _key_view, + const value_header_t* _p_value_header) +{ + assert(_p_value_header); + + needs_update_all = false; + version = current_version; + + p_node_base = ref_leaf_node->read(); + key_view = _key_view; + p_value_header = _p_value_header; + assert((const char*)p_value_header > p_node_base); + assert((const char*)p_value_header - p_node_base < + (int)ref_leaf_node->get_node_size()); + + value_payload_mut.reset(); + p_value_recorder = nullptr; +} + +void tree_cursor_t::Cache::maybe_duplicate(const node_version_t& current_version) +{ + assert(!needs_update_all); + assert(version.layout == current_version.layout); + if (version.state == current_version.state) { + // cache is already latest. + } else if (version.state < current_version.state) { + // the extent has been copied but the layout has not been changed. + assert(p_node_base != nullptr); + assert(key_view.has_value()); + assert(p_value_header != nullptr); + + auto current_p_node_base = ref_leaf_node->read(); + assert(current_p_node_base != p_node_base); + auto node_size = ref_leaf_node->get_node_size(); + + version.state = current_version.state; + reset_ptr(p_value_header, p_node_base, + current_p_node_base, node_size); + key_view->reset_to(p_node_base, current_p_node_base, node_size); + value_payload_mut.reset(); + p_value_recorder = nullptr; + + p_node_base = current_p_node_base; + } else { + // It is impossible to change state backwards, see node_types.h. + ceph_abort("impossible"); + } +} + +void tree_cursor_t::Cache::make_latest( + value_magic_t magic, const search_position_t& pos) +{ + auto current_version = ref_leaf_node->get_version(); + if (needs_update_all || version.layout != current_version.layout) { + auto [_key_view, _p_value_header] = ref_leaf_node->get_kv(pos); + update_all(current_version, _key_view, _p_value_header); + } else { + maybe_duplicate(current_version); + } + assert(p_value_header->magic == magic); + validate_is_latest(pos); +} + +void tree_cursor_t::Cache::validate_is_latest(const search_position_t& pos) const +{ +#ifndef NDEBUG + assert(!needs_update_all); + assert(version == ref_leaf_node->get_version()); + + auto [_key_view, _p_value_header] = ref_leaf_node->get_kv(pos); + assert(p_node_base == ref_leaf_node->read()); + assert(key_view ==_key_view); + assert(p_value_header == _p_value_header); +#endif +} + +std::pair<NodeExtentMutable&, ValueDeltaRecorder*> +tree_cursor_t::Cache::prepare_mutate_value_payload( + context_t c, const search_position_t& pos) +{ + make_latest(c.vb.get_header_magic(), pos); + if (!value_payload_mut.has_value()) { + assert(!p_value_recorder); + auto value_mutable = ref_leaf_node->prepare_mutate_value_payload(c); + auto current_version = ref_leaf_node->get_version(); + maybe_duplicate(current_version); + value_payload_mut = p_value_header->get_payload_mutable(value_mutable.first); + p_value_recorder = value_mutable.second; + validate_is_latest(pos); + } + return {*value_payload_mut, p_value_recorder}; +} + +/* + * Node + */ + +Node::Node(NodeImplURef&& impl) : impl{std::move(impl)} {} + +Node::~Node() +{ + if (!is_tracked()) { + // possible scenarios: + // a. I'm erased; + // b. Eagain happened after the node extent is allocated/loaded + // and before the node is initialized correctly; + } else { + assert(!impl->is_extent_retired()); + if (is_root()) { + super->do_untrack_root(*this); + } else { + _parent_info->ptr->do_untrack_child(*this); + } + } +} + +level_t Node::level() const +{ + return impl->level(); +} + +eagain_ifuture<Node::search_result_t> Node::lower_bound( + context_t c, const key_hobj_t& key) +{ + return seastar::do_with( + MatchHistory(), [this, c, &key](auto& history) { + return lower_bound_tracked(c, key, history); + } + ); +} + +eagain_ifuture<std::pair<Ref<tree_cursor_t>, bool>> Node::insert( + context_t c, + const key_hobj_t& key, + value_config_t vconf, + Ref<Node>&& this_ref) +{ + return seastar::do_with( + MatchHistory(), [this, c, &key, vconf, + this_ref = std::move(this_ref)] (auto& history) mutable { + return lower_bound_tracked(c, key, history + ).si_then([c, &key, vconf, &history, + this_ref = std::move(this_ref)] (auto result) mutable { + // the cursor in the result should already hold the root node upwards + this_ref.reset(); + if (result.match() == MatchKindBS::EQ) { + return eagain_iertr::make_ready_future<std::pair<Ref<tree_cursor_t>, bool>>( + std::make_pair(result.p_cursor, false)); + } else { + auto leaf_node = result.p_cursor->get_leaf_node(); + return leaf_node->insert_value( + c, key, vconf, result.p_cursor->get_position(), history, result.mstat + ).si_then([](auto p_cursor) { + return seastar::make_ready_future<std::pair<Ref<tree_cursor_t>, bool>>( + std::make_pair(p_cursor, true)); + }); + } + }); + } + ); +} + +eagain_ifuture<std::size_t> Node::erase( + context_t c, + const key_hobj_t& key, + Ref<Node>&& this_ref) +{ + return lower_bound(c, key + ).si_then([c, this_ref = std::move(this_ref)] (auto result) mutable { + // the cursor in the result should already hold the root node upwards + this_ref.reset(); + if (result.match() != MatchKindBS::EQ) { + return eagain_iertr::make_ready_future<std::size_t>(0); + } + auto ref_cursor = result.p_cursor; + return ref_cursor->erase(c, false + ).si_then([ref_cursor] (auto next_cursor) { + assert(ref_cursor->is_invalid()); + assert(!next_cursor); + return std::size_t(1); + }); + }); +} + +eagain_ifuture<tree_stats_t> Node::get_tree_stats(context_t c) +{ + return seastar::do_with( + tree_stats_t(), [this, c](auto& stats) { + return do_get_tree_stats(c, stats).si_then([&stats] { + return stats; + }); + } + ); +} + +std::ostream& Node::dump(std::ostream& os) const +{ + return impl->dump(os); +} + +std::ostream& Node::dump_brief(std::ostream& os) const +{ + return impl->dump_brief(os); +} + +const std::string& Node::get_name() const +{ + return impl->get_name(); +} + +void Node::test_make_destructable( + context_t c, NodeExtentMutable& mut, Super::URef&& _super) +{ + impl->test_set_tail(mut); + make_root(c, std::move(_super)); +} + +eagain_ifuture<> Node::mkfs(context_t c, RootNodeTracker& root_tracker) +{ + LOG_PREFIX(OTree::Node::mkfs); + return LeafNode::allocate_root(c, root_tracker + ).si_then([c, FNAME](auto ret) { + c.t.get_onode_tree_stats().extents_num_delta++; + INFOT("allocated root {}", c.t, ret->get_name()); + }); +} + +eagain_ifuture<Ref<Node>> Node::load_root(context_t c, RootNodeTracker& root_tracker) +{ + LOG_PREFIX(OTree::Node::load_root); + return c.nm.get_super(c.t, root_tracker + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::input_output_error::handle([FNAME, c] { + ERRORT("EIO during get_super()", c.t); + ceph_abort("fatal error"); + }) + ).si_then([c, &root_tracker, FNAME](auto&& _super) { + assert(_super); + auto root_addr = _super->get_root_laddr(); + assert(root_addr != L_ADDR_NULL); + TRACET("loading root_addr={:x} ...", c.t, root_addr); + return Node::load(c, root_addr, true + ).si_then([c, _super = std::move(_super), + &root_tracker, FNAME](auto root) mutable { + TRACET("loaded {}", c.t, root->get_name()); + assert(root->impl->field_type() == field_type_t::N0); + root->as_root(std::move(_super)); + std::ignore = c; // as only used in an assert + std::ignore = root_tracker; + assert(root == root_tracker.get_root(c.t)); + return seastar::make_ready_future<Ref<Node>>(root); + }); + }); +} + +void Node::make_root(context_t c, Super::URef&& _super) +{ + _super->write_root_laddr(c, impl->laddr()); + as_root(std::move(_super)); + c.t.get_onode_tree_stats().depth = static_cast<uint64_t>(level()) + 1; +} + +void Node::as_root(Super::URef&& _super) +{ + assert(!is_tracked()); + assert(_super->get_root_laddr() == impl->laddr()); + assert(impl->is_level_tail()); + super = std::move(_super); + super->do_track_root(*this); + assert(is_root()); +} + +Super::URef Node::deref_super() +{ + assert(is_root()); + assert(super->get_root_laddr() == impl->laddr()); + assert(impl->is_level_tail()); + super->do_untrack_root(*this); + auto ret = std::move(super); + assert(!is_tracked()); + return ret; +} + +eagain_ifuture<> Node::upgrade_root(context_t c, laddr_t hint) +{ + LOG_PREFIX(OTree::Node::upgrade_root); + assert(impl->field_type() == field_type_t::N0); + auto super_to_move = deref_super(); + return InternalNode::allocate_root( + c, hint, impl->level(), impl->laddr(), std::move(super_to_move) + ).si_then([this, c, FNAME](auto new_root) { + as_child(search_position_t::end(), new_root); + INFOT("upgraded from {} to {}", + c.t, get_name(), new_root->get_name()); + }); +} + +template <bool VALIDATE> +void Node::as_child(const search_position_t& pos, Ref<InternalNode> parent_node) +{ + assert(!is_tracked() || !is_root()); +#ifndef NDEBUG + // Although I might have an outdated _parent_info during fixing, + // I must be already untracked. + if (_parent_info.has_value()) { + assert(!_parent_info->ptr->check_is_tracking(*this)); + } +#endif + _parent_info = parent_info_t{pos, parent_node}; + parent_info().ptr->do_track_child<VALIDATE>(*this); + assert(!is_root()); +} +template void Node::as_child<true>(const search_position_t&, Ref<InternalNode>); +template void Node::as_child<false>(const search_position_t&, Ref<InternalNode>); + +Ref<InternalNode> Node::deref_parent() +{ + assert(!is_root()); + auto parent_ref = std::move(parent_info().ptr); + parent_ref->do_untrack_child(*this); + _parent_info.reset(); + assert(!is_tracked()); + return parent_ref; +} + +eagain_ifuture<> Node::apply_split_to_parent( + context_t c, + Ref<Node>&& this_ref, + Ref<Node>&& split_right, + bool update_right_index) +{ + assert(!is_root()); + assert(this == this_ref.get()); + // TODO(cross-node string dedup) + return parent_info().ptr->apply_child_split( + c, std::move(this_ref), std::move(split_right), update_right_index); +} + +eagain_ifuture<Ref<tree_cursor_t>> +Node::get_next_cursor_from_parent(context_t c) +{ + assert(!impl->is_level_tail()); + assert(!is_root()); + return parent_info().ptr->get_next_cursor(c, parent_info().position); +} + +template <bool FORCE_MERGE> +eagain_ifuture<> +Node::try_merge_adjacent( + context_t c, bool update_parent_index, Ref<Node>&& this_ref) +{ + LOG_PREFIX(OTree::Node::try_merge_adjacent); + assert(this == this_ref.get()); + impl->validate_non_empty(); + assert(!is_root()); + if constexpr (!FORCE_MERGE) { + if (!impl->is_size_underflow() && + !impl->has_single_value()) { + // skip merge + if (update_parent_index) { + return fix_parent_index(c, std::move(this_ref), false); + } else { + parent_info().ptr->validate_child_tracked(*this); + return eagain_iertr::now(); + } + } + } + + return parent_info().ptr->get_child_peers(c, parent_info().position + ).si_then([c, this_ref = std::move(this_ref), this, FNAME, + update_parent_index] (auto lr_nodes) mutable -> eagain_ifuture<> { + auto& [lnode, rnode] = lr_nodes; + Ref<Node> left_for_merge; + Ref<Node> right_for_merge; + Ref<Node>* p_this_ref; + bool is_left; + if (!lnode && !rnode) { + // XXX: this is possible before node rebalance is implemented, + // when its parent cannot merge with its peers and has only one child + // (this node). + p_this_ref = &this_ref; + } else if (!lnode) { + left_for_merge = std::move(this_ref); + p_this_ref = &left_for_merge; + right_for_merge = std::move(rnode); + is_left = true; + } else if (!rnode) { + left_for_merge = std::move(lnode); + right_for_merge = std::move(this_ref); + p_this_ref = &right_for_merge; + is_left = false; + } else { // lnode && rnode + if (lnode->impl->free_size() > rnode->impl->free_size()) { + left_for_merge = std::move(lnode); + right_for_merge = std::move(this_ref); + p_this_ref = &right_for_merge; + is_left = false; + } else { // lnode free size <= rnode free size + left_for_merge = std::move(this_ref); + p_this_ref = &left_for_merge; + right_for_merge = std::move(rnode); + is_left = true; + } + } + + if (left_for_merge) { + assert(right_for_merge); + auto [merge_stage, merge_size] = left_for_merge->impl->evaluate_merge( + *right_for_merge->impl); + if (merge_size <= left_for_merge->impl->total_size()) { + // proceed merge + bool update_index_after_merge; + if (is_left) { + update_index_after_merge = false; + } else { + update_index_after_merge = update_parent_index; + } + DEBUGT("merge {} and {} at merge_stage={}, merge_size={}B, " + "update_index={}, is_left={} ...", + c.t, left_for_merge->get_name(), right_for_merge->get_name(), + merge_stage, merge_size, update_index_after_merge, is_left); + // we currently cannot generate delta depends on another extent content, + // so use rebuild_extent() as a workaround to rebuild the node from a + // fresh extent, thus no need to generate delta. + auto left_addr = left_for_merge->impl->laddr(); + return left_for_merge->rebuild_extent(c + ).si_then([c, update_index_after_merge, + left_addr, + merge_stage = merge_stage, + merge_size = merge_size, + left_for_merge = std::move(left_for_merge), + right_for_merge = std::move(right_for_merge)] (auto left_mut) mutable { + if (left_for_merge->impl->node_type() == node_type_t::LEAF) { + auto& left = *static_cast<LeafNode*>(left_for_merge.get()); + left.on_layout_change(); + } + search_position_t left_last_pos = left_for_merge->impl->merge( + left_mut, *right_for_merge->impl, merge_stage, merge_size); + left_for_merge->track_merge(right_for_merge, merge_stage, left_last_pos); + --(c.t.get_onode_tree_stats().extents_num_delta); + return left_for_merge->parent_info().ptr->apply_children_merge( + c, std::move(left_for_merge), left_addr, + std::move(right_for_merge), update_index_after_merge); + }); + } else { + // size will overflow if merge + } + } + + // cannot merge + if (update_parent_index) { + return fix_parent_index(c, std::move(*p_this_ref), false); + } else { + parent_info().ptr->validate_child_tracked(*this); + return eagain_iertr::now(); + } + // XXX: rebalance + }); +} +template eagain_ifuture<> Node::try_merge_adjacent<true>(context_t, bool, Ref<Node>&&); +template eagain_ifuture<> Node::try_merge_adjacent<false>(context_t, bool, Ref<Node>&&); + +eagain_ifuture<> Node::erase_node(context_t c, Ref<Node>&& this_ref) +{ + // To erase a node: + // 1. I'm supposed to have already untracked any children or cursors + // 2. unlink parent/super --ptr-> me + // 3. unlink me --ref-> parent/super + // 4. retire extent + // 5. destruct node + assert(this_ref.get() == this); + assert(!is_tracking()); + assert(!is_root()); + assert(this_ref->use_count() == 1); + return parent_info().ptr->erase_child(c, std::move(this_ref)); +} + +template <bool FORCE_MERGE> +eagain_ifuture<> Node::fix_parent_index( + context_t c, Ref<Node>&& this_ref, bool check_downgrade) +{ + assert(!is_root()); + assert(this == this_ref.get()); + return parent_info().ptr->fix_index<FORCE_MERGE>( + c, std::move(this_ref), check_downgrade); +} +template eagain_ifuture<> Node::fix_parent_index<true>(context_t, Ref<Node>&&, bool); +template eagain_ifuture<> Node::fix_parent_index<false>(context_t, Ref<Node>&&, bool); + +eagain_ifuture<Ref<Node>> Node::load( + context_t c, laddr_t addr, bool expect_is_level_tail) +{ + LOG_PREFIX(OTree::Node::load); + return c.nm.read_extent(c.t, addr + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::input_output_error::handle( + [FNAME, c, addr, expect_is_level_tail] { + ERRORT("EIO -- addr={:x}, is_level_tail={}", + c.t, addr, expect_is_level_tail); + ceph_abort("fatal error"); + }), + crimson::ct_error::invarg::handle( + [FNAME, c, addr, expect_is_level_tail] { + ERRORT("EINVAL -- addr={:x}, is_level_tail={}", + c.t, addr, expect_is_level_tail); + ceph_abort("fatal error"); + }), + crimson::ct_error::enoent::handle( + [FNAME, c, addr, expect_is_level_tail] { + ERRORT("ENOENT -- addr={:x}, is_level_tail={}", + c.t, addr, expect_is_level_tail); + ceph_abort("fatal error"); + }), + crimson::ct_error::erange::handle( + [FNAME, c, addr, expect_is_level_tail] { + ERRORT("ERANGE -- addr={:x}, is_level_tail={}", + c.t, addr, expect_is_level_tail); + ceph_abort("fatal error"); + }) + ).si_then([FNAME, c, addr, expect_is_level_tail](auto extent) + -> eagain_ifuture<Ref<Node>> { + assert(extent); + auto header = extent->get_header(); + auto field_type = header.get_field_type(); + if (!field_type) { + ERRORT("load addr={:x}, is_level_tail={} error, " + "got invalid header -- {}", + c.t, addr, expect_is_level_tail, fmt::ptr(extent)); + ceph_abort("fatal error"); + } + if (header.get_is_level_tail() != expect_is_level_tail) { + ERRORT("load addr={:x}, is_level_tail={} error, " + "is_level_tail mismatch -- {}", + c.t, addr, expect_is_level_tail, fmt::ptr(extent)); + ceph_abort("fatal error"); + } + + auto node_type = header.get_node_type(); + if (node_type == node_type_t::LEAF) { + if (extent->get_length() != c.vb.get_leaf_node_size()) { + ERRORT("load addr={:x}, is_level_tail={} error, " + "leaf length mismatch -- {}", + c.t, addr, expect_is_level_tail, fmt::ptr(extent)); + ceph_abort("fatal error"); + } + auto impl = LeafNodeImpl::load(extent, *field_type); + auto *derived_ptr = impl.get(); + return eagain_iertr::make_ready_future<Ref<Node>>( + new LeafNode(derived_ptr, std::move(impl))); + } else if (node_type == node_type_t::INTERNAL) { + if (extent->get_length() != c.vb.get_internal_node_size()) { + ERRORT("load addr={:x}, is_level_tail={} error, " + "internal length mismatch -- {}", + c.t, addr, expect_is_level_tail, fmt::ptr(extent)); + ceph_abort("fatal error"); + } + auto impl = InternalNodeImpl::load(extent, *field_type); + auto *derived_ptr = impl.get(); + return eagain_iertr::make_ready_future<Ref<Node>>( + new InternalNode(derived_ptr, std::move(impl))); + } else { + ceph_abort("impossible path"); + } + }); +} + +eagain_ifuture<NodeExtentMutable> Node::rebuild_extent(context_t c) +{ + LOG_PREFIX(OTree::Node::rebuild_extent); + DEBUGT("{} ...", c.t, get_name()); + assert(!is_root()); + // assume I'm already ref counted by caller + + // note: laddr can be changed after rebuild, but we don't fix the parent + // mapping as it is part of the merge process. + return impl->rebuild_extent(c); +} + +eagain_ifuture<> Node::retire(context_t c, Ref<Node>&& this_ref) +{ + LOG_PREFIX(OTree::Node::retire); + DEBUGT("{} ...", c.t, get_name()); + assert(this_ref.get() == this); + assert(!is_tracking()); + assert(!is_tracked()); + assert(this_ref->use_count() == 1); + + return impl->retire_extent(c + ).si_then([this_ref = std::move(this_ref)]{ /* deallocate node */}); +} + +void Node::make_tail(context_t c) +{ + LOG_PREFIX(OTree::Node::make_tail); + assert(!impl->is_level_tail()); + assert(!impl->is_keys_empty()); + DEBUGT("{} ...", c.t, get_name()); + impl->prepare_mutate(c); + auto tail_pos = impl->make_tail(); + if (impl->node_type() == node_type_t::INTERNAL) { + auto& node = *static_cast<InternalNode*>(this); + node.track_make_tail(tail_pos); + } +} + +/* + * InternalNode + */ + +InternalNode::InternalNode(InternalNodeImpl* impl, NodeImplURef&& impl_ref) + : Node(std::move(impl_ref)), impl{impl} {} + +eagain_ifuture<Ref<tree_cursor_t>> +InternalNode::get_next_cursor(context_t c, const search_position_t& pos) +{ + impl->validate_non_empty(); + if (pos.is_end()) { + assert(impl->is_level_tail()); + return get_next_cursor_from_parent(c); + } + + search_position_t next_pos = pos; + const laddr_packed_t* p_child_addr = nullptr; + impl->get_next_slot(next_pos, nullptr, &p_child_addr); + if (next_pos.is_end() && !impl->is_level_tail()) { + return get_next_cursor_from_parent(c); + } else { + if (next_pos.is_end()) { + p_child_addr = impl->get_tail_value(); + } + assert(p_child_addr); + return get_or_track_child(c, next_pos, p_child_addr->value + ).si_then([c](auto child) { + return child->lookup_smallest(c); + }); + } +} + +eagain_ifuture<> InternalNode::apply_child_split( + context_t c, Ref<Node>&& left_child, Ref<Node>&& right_child, + bool update_right_index) +{ + LOG_PREFIX(OTree::InternalNode::apply_child_split); + auto& left_pos = left_child->parent_info().position; + +#ifndef NDEBUG + assert(left_child->parent_info().ptr.get() == this); + assert(!left_child->impl->is_level_tail()); + if (left_pos.is_end()) { + assert(impl->is_level_tail()); + assert(right_child->impl->is_level_tail()); + assert(!update_right_index); + } + + // right_child has not assigned parent yet + assert(!right_child->is_tracked()); +#endif + + impl->prepare_mutate(c); + + DEBUGT("apply {}'s child {} to split to {}, update_index={} ...", + c.t, get_name(), left_child->get_name(), + right_child->get_name(), update_right_index); + + // update layout from left_pos => left_child_addr to right_child_addr + auto left_child_addr = left_child->impl->laddr(); + auto right_child_addr = right_child->impl->laddr(); + impl->replace_child_addr(left_pos, right_child_addr, left_child_addr); + + // update track from left_pos => left_child to right_child + replace_track(right_child, left_child, update_right_index); + + auto left_key = *left_child->impl->get_pivot_index(); + Ref<Node> this_ref = this; + return insert_or_split( + c, left_pos, left_key, left_child, + (update_right_index ? right_child : nullptr) + ).si_then([this, c, + this_ref = std::move(this_ref)] (auto split_right) mutable { + if (split_right) { + // even if update_right_index could be true, + // we haven't fixed the right_child index of this node yet, + // so my parent index should be correct now. + return apply_split_to_parent( + c, std::move(this_ref), std::move(split_right), false); + } else { + return eagain_iertr::now(); + } + }).si_then([c, update_right_index, + right_child = std::move(right_child)] () mutable { + if (update_right_index) { + // XXX: might not need to call validate_tracked_children() in fix_index() + return right_child->fix_parent_index(c, std::move(right_child), false); + } else { + // there is no need to call try_merge_adjacent() because + // the filled size of the inserted node or the split right node + // won't be reduced if update_right_index is false. + return eagain_iertr::now(); + } + }); +} + +eagain_ifuture<> InternalNode::erase_child(context_t c, Ref<Node>&& child_ref) +{ + LOG_PREFIX(OTree::InternalNode::erase_child); + // this is a special version of recursive merge + impl->validate_non_empty(); + assert(child_ref->use_count() == 1); + validate_child_tracked(*child_ref); + + // fix the child's previous node as the new tail, + // and trigger prv_child_ref->try_merge_adjacent() at the end + bool fix_tail = (child_ref->parent_info().position.is_end() && + !impl->is_keys_empty()); + return eagain_iertr::now().si_then([c, this, fix_tail] { + if (fix_tail) { + search_position_t new_tail_pos; + const laddr_packed_t* new_tail_p_addr = nullptr; + impl->get_largest_slot(&new_tail_pos, nullptr, &new_tail_p_addr); + return get_or_track_child(c, new_tail_pos, new_tail_p_addr->value); + } else { + return eagain_iertr::make_ready_future<Ref<Node>>(); + } + }).si_then([c, this, child_ref = std::move(child_ref), FNAME] + (auto&& new_tail_child) mutable { + auto child_pos = child_ref->parent_info().position; + if (new_tail_child) { + DEBUGT("erase {}'s child {} at pos({}), " + "and fix new child tail {} at pos({}) ...", + c.t, get_name(), child_ref->get_name(), child_pos, + new_tail_child->get_name(), new_tail_child->parent_info().position); + assert(!new_tail_child->impl->is_level_tail()); + new_tail_child->make_tail(c); + assert(new_tail_child->impl->is_level_tail()); + if (new_tail_child->impl->node_type() == node_type_t::LEAF) { + // no need to proceed merge because the filled size is not changed + new_tail_child.reset(); + } + } else { + DEBUGT("erase {}'s child {} at pos({}) ...", + c.t, get_name(), child_ref->get_name(), child_pos); + } + + Ref<Node> this_ref = child_ref->deref_parent(); + assert(this_ref == this); + return child_ref->retire(c, std::move(child_ref) + ).si_then([c, this, child_pos, FNAME, + this_ref = std::move(this_ref)] () mutable { + if (impl->has_single_value()) { + // fast path without mutating the extent + DEBUGT("{} has one value left, erase ...", c.t, get_name()); +#ifndef NDEBUG + if (impl->is_level_tail()) { + assert(child_pos.is_end()); + } else { + assert(child_pos == search_position_t::begin()); + } +#endif + + if (is_root()) { + // Note: if merge/split works as expected, we should never encounter the + // situation when the internal root has <=1 children: + // + // A newly created internal root (see Node::upgrade_root()) will have 2 + // children after split is finished. + // + // When merge happens, children will try to merge each other, and if the + // root detects there is only one child left, the root will be + // down-graded to the only child. + // + // In order to preserve the invariant, we need to make sure the new + // internal root also has at least 2 children. + ceph_abort("trying to erase the last item from the internal root node"); + } + + // track erase + assert(tracked_child_nodes.empty()); + + // no child should be referencing this node now, this_ref is the last one. + assert(this_ref->use_count() == 1); + return Node::erase_node(c, std::move(this_ref)); + } + + impl->prepare_mutate(c); + auto [erase_stage, next_or_last_pos] = impl->erase(child_pos); + if (child_pos.is_end()) { + // next_or_last_pos as last_pos + track_make_tail(next_or_last_pos); + } else { + // next_or_last_pos as next_pos + track_erase(child_pos, erase_stage); + } + validate_tracked_children(); + + if (is_root()) { + return try_downgrade_root(c, std::move(this_ref)); + } else { + bool update_parent_index; + if (impl->is_level_tail()) { + update_parent_index = false; + } else { + // next_or_last_pos as next_pos + next_or_last_pos.is_end() ? update_parent_index = true + : update_parent_index = false; + } + return try_merge_adjacent(c, update_parent_index, std::move(this_ref)); + } + }).si_then([c, new_tail_child = std::move(new_tail_child)] () mutable { + // finally, check if the new tail child needs to merge + if (new_tail_child && !new_tail_child->is_root()) { + assert(new_tail_child->impl->is_level_tail()); + return new_tail_child->try_merge_adjacent( + c, false, std::move(new_tail_child)); + } else { + return eagain_iertr::now(); + } + }); + }); +} + +template <bool FORCE_MERGE> +eagain_ifuture<> InternalNode::fix_index( + context_t c, Ref<Node>&& child, bool check_downgrade) +{ + LOG_PREFIX(OTree::InternalNode::fix_index); + impl->validate_non_empty(); + + validate_child_inconsistent(*child); + auto& child_pos = child->parent_info().position; + Ref<Node> this_ref = child->deref_parent(); + assert(this_ref == this); + validate_tracked_children(); + + impl->prepare_mutate(c); + + key_view_t new_key = *child->impl->get_pivot_index(); + DEBUGT("fix {}'s index of child {} at pos({}), new_key={} ...", + c.t, get_name(), child->get_name(), child_pos, new_key); + + // erase the incorrect item + auto [erase_stage, next_pos] = impl->erase(child_pos); + track_erase(child_pos, erase_stage); + validate_tracked_children(); + + // find out whether there is a need to fix parent index recursively + bool update_parent_index; + if (impl->is_level_tail()) { + update_parent_index = false; + } else { + next_pos.is_end() ? update_parent_index = true + : update_parent_index = false; + } + + return insert_or_split(c, next_pos, new_key, child + ).si_then([this, c, update_parent_index, check_downgrade, + this_ref = std::move(this_ref)] (auto split_right) mutable { + if (split_right) { + // after split, the parent index to the split_right will be incorrect + // if update_parent_index is true. + return apply_split_to_parent( + c, std::move(this_ref), std::move(split_right), update_parent_index); + } else { + // no split path + if (is_root()) { + if (check_downgrade) { + return try_downgrade_root(c, std::move(this_ref)); + } else { + // no need to call try_downgrade_root() because the number of keys + // has not changed, and I must have at least 2 keys. + assert(!impl->is_keys_empty()); + return eagain_iertr::now(); + } + } else { + // for non-root, maybe need merge adjacent or fix parent, + // because the filled node size may be reduced. + return try_merge_adjacent<FORCE_MERGE>( + c, update_parent_index, std::move(this_ref)); + } + } + }); +} + +template <bool FORCE_MERGE> +eagain_ifuture<> InternalNode::apply_children_merge( + context_t c, Ref<Node>&& left_child, laddr_t origin_left_addr, + Ref<Node>&& right_child, bool update_index) +{ + LOG_PREFIX(OTree::InternalNode::apply_children_merge); + auto left_pos = left_child->parent_info().position; + auto left_addr = left_child->impl->laddr(); + auto& right_pos = right_child->parent_info().position; + auto right_addr = right_child->impl->laddr(); + DEBUGT("apply {}'s child {} (was {:#x}) at pos({}), " + "to merge with {} at pos({}), update_index={} ...", + c.t, get_name(), left_child->get_name(), origin_left_addr, left_pos, + right_child->get_name(), right_pos, update_index); + +#ifndef NDEBUG + assert(left_child->parent_info().ptr == this); + assert(!left_pos.is_end()); + const laddr_packed_t* p_value_left; + impl->get_slot(left_pos, nullptr, &p_value_left); + assert(p_value_left->value == origin_left_addr); + + assert(right_child->use_count() == 1); + assert(right_child->parent_info().ptr == this); + const laddr_packed_t* p_value_right; + if (right_pos.is_end()) { + assert(right_child->impl->is_level_tail()); + assert(left_child->impl->is_level_tail()); + assert(impl->is_level_tail()); + assert(!update_index); + p_value_right = impl->get_tail_value(); + } else { + assert(!right_child->impl->is_level_tail()); + assert(!left_child->impl->is_level_tail()); + impl->get_slot(right_pos, nullptr, &p_value_right); + } + assert(p_value_right->value == right_addr); +#endif + + // XXX: we may jump to try_downgrade_root() without mutating this node. + + // update layout from right_pos => right_addr to left_addr + impl->prepare_mutate(c); + impl->replace_child_addr(right_pos, left_addr, right_addr); + + // update track from right_pos => right_child to left_child + left_child->deref_parent(); + replace_track(left_child, right_child, update_index); + + // erase left_pos from layout + auto [erase_stage, next_pos] = impl->erase(left_pos); + track_erase<false>(left_pos, erase_stage); + assert(next_pos == left_child->parent_info().position); + + // All good to retire the right_child. + // I'm already ref-counted by left_child. + return right_child->retire(c, std::move(right_child) + ).si_then([c, this, update_index, + left_child = std::move(left_child)] () mutable { + if (update_index) { + // I'm all good but: + // - my number of keys is reduced by 1 + // - my size may underflow, but try_merge_adjacent() is already part of fix_index() + return left_child->fix_parent_index<FORCE_MERGE>(c, std::move(left_child), true); + } else { + validate_tracked_children(); + Ref<Node> this_ref = this; + left_child.reset(); + // I'm all good but: + // - my number of keys is reduced by 1 + // - my size may underflow + if (is_root()) { + return try_downgrade_root(c, std::move(this_ref)); + } else { + return try_merge_adjacent<FORCE_MERGE>( + c, false, std::move(this_ref)); + } + } + }); +} +template eagain_ifuture<> InternalNode::apply_children_merge<true>( + context_t, Ref<Node>&&, laddr_t, Ref<Node>&&, bool); +template eagain_ifuture<> InternalNode::apply_children_merge<false>( + context_t, Ref<Node>&&, laddr_t, Ref<Node>&&, bool); + +eagain_ifuture<std::pair<Ref<Node>, Ref<Node>>> InternalNode::get_child_peers( + context_t c, const search_position_t& pos) +{ + // assume I'm already ref counted by caller + search_position_t prev_pos; + const laddr_packed_t* prev_p_child_addr = nullptr; + search_position_t next_pos; + const laddr_packed_t* next_p_child_addr = nullptr; + + if (pos.is_end()) { + assert(impl->is_level_tail()); + if (!impl->is_keys_empty()) { + // got previous child only + impl->get_largest_slot(&prev_pos, nullptr, &prev_p_child_addr); + assert(prev_pos < pos); + assert(prev_p_child_addr != nullptr); + } else { + // no keys, so no peer children + } + } else { // !pos.is_end() + if (pos != search_position_t::begin()) { + // got previous child + prev_pos = pos; + impl->get_prev_slot(prev_pos, nullptr, &prev_p_child_addr); + assert(prev_pos < pos); + assert(prev_p_child_addr != nullptr); + } else { + // is already the first child, so no previous child + } + + next_pos = pos; + impl->get_next_slot(next_pos, nullptr, &next_p_child_addr); + if (next_pos.is_end()) { + if (impl->is_level_tail()) { + // the next child is the tail + next_p_child_addr = impl->get_tail_value(); + assert(pos < next_pos); + assert(next_p_child_addr != nullptr); + } else { + // next child doesn't exist + assert(next_p_child_addr == nullptr); + } + } else { + // got the next child + assert(pos < next_pos); + assert(next_p_child_addr != nullptr); + } + } + + return eagain_iertr::now().si_then([this, c, prev_pos, prev_p_child_addr] { + if (prev_p_child_addr != nullptr) { + return get_or_track_child(c, prev_pos, prev_p_child_addr->value); + } else { + return eagain_iertr::make_ready_future<Ref<Node>>(); + } + }).si_then([this, c, next_pos, next_p_child_addr] (Ref<Node> lnode) { + if (next_p_child_addr != nullptr) { + return get_or_track_child(c, next_pos, next_p_child_addr->value + ).si_then([lnode] (Ref<Node> rnode) { + return seastar::make_ready_future<std::pair<Ref<Node>, Ref<Node>>>( + lnode, rnode); + }); + } else { + return eagain_iertr::make_ready_future<std::pair<Ref<Node>, Ref<Node>>>( + lnode, nullptr); + } + }); +} + +eagain_ifuture<Ref<InternalNode>> InternalNode::allocate_root( + context_t c, laddr_t hint, level_t old_root_level, + laddr_t old_root_addr, Super::URef&& super) +{ + // support tree height up to 256 + ceph_assert(old_root_level < MAX_LEVEL); + return InternalNode::allocate(c, hint, field_type_t::N0, true, old_root_level + 1 + ).si_then([c, old_root_addr, + super = std::move(super)](auto fresh_node) mutable { + auto root = fresh_node.node; + assert(root->impl->is_keys_empty()); + auto p_value = root->impl->get_tail_value(); + fresh_node.mut.copy_in_absolute( + const_cast<laddr_packed_t*>(p_value), old_root_addr); + root->make_root_from(c, std::move(super), old_root_addr); + ++(c.t.get_onode_tree_stats().extents_num_delta); + return root; + }); +} + +eagain_ifuture<Ref<tree_cursor_t>> +InternalNode::lookup_smallest(context_t c) +{ + impl->validate_non_empty(); + auto position = search_position_t::begin(); + const laddr_packed_t* p_child_addr; + impl->get_slot(position, nullptr, &p_child_addr); + return get_or_track_child(c, position, p_child_addr->value + ).si_then([c](auto child) { + return child->lookup_smallest(c); + }); +} + +eagain_ifuture<Ref<tree_cursor_t>> +InternalNode::lookup_largest(context_t c) +{ + // NOTE: unlike LeafNode::lookup_largest(), this only works for the tail + // internal node to return the tail child address. + impl->validate_non_empty(); + assert(impl->is_level_tail()); + auto p_child_addr = impl->get_tail_value(); + return get_or_track_child(c, search_position_t::end(), p_child_addr->value + ).si_then([c](auto child) { + return child->lookup_largest(c); + }); +} + +eagain_ifuture<Node::search_result_t> +InternalNode::lower_bound_tracked( + context_t c, const key_hobj_t& key, MatchHistory& history) +{ + auto result = impl->lower_bound(key, history); + return get_or_track_child(c, result.position, result.p_value->value + ).si_then([c, &key, &history](auto child) { + // XXX(multi-type): pass result.mstat to child + return child->lower_bound_tracked(c, key, history); + }); +} + +eagain_ifuture<> InternalNode::do_get_tree_stats( + context_t c, tree_stats_t& stats) +{ + impl->validate_non_empty(); + auto nstats = impl->get_stats(); + stats.size_persistent_internal += nstats.size_persistent; + stats.size_filled_internal += nstats.size_filled; + stats.size_logical_internal += nstats.size_logical; + stats.size_overhead_internal += nstats.size_overhead; + stats.size_value_internal += nstats.size_value; + stats.num_kvs_internal += nstats.num_kvs; + stats.num_nodes_internal += 1; + + Ref<Node> this_ref = this; + return seastar::do_with( + search_position_t(), (const laddr_packed_t*)(nullptr), + [this, this_ref, c, &stats](auto& pos, auto& p_child_addr) { + pos = search_position_t::begin(); + impl->get_slot(pos, nullptr, &p_child_addr); + return trans_intr::repeat( + [this, this_ref, c, &stats, &pos, &p_child_addr]() + -> eagain_ifuture<seastar::stop_iteration> { + return get_or_track_child(c, pos, p_child_addr->value + ).si_then([c, &stats](auto child) { + return child->do_get_tree_stats(c, stats); + }).si_then([this, this_ref, &pos, &p_child_addr] { + if (pos.is_end()) { + return seastar::stop_iteration::yes; + } else { + impl->get_next_slot(pos, nullptr, &p_child_addr); + if (pos.is_end()) { + if (impl->is_level_tail()) { + p_child_addr = impl->get_tail_value(); + return seastar::stop_iteration::no; + } else { + return seastar::stop_iteration::yes; + } + } else { + return seastar::stop_iteration::no; + } + } + }); + }); + } + ); +} + +void InternalNode::track_merge( + Ref<Node> _right_node, match_stage_t stage, search_position_t& left_last_pos) +{ + assert(level() == _right_node->level()); + assert(impl->node_type() == _right_node->impl->node_type()); + auto& right_node = *static_cast<InternalNode*>(_right_node.get()); + if (right_node.tracked_child_nodes.empty()) { + return; + } + + match_stage_t curr_stage = STAGE_BOTTOM; + + // prepare the initial left_last_pos for offset + while (curr_stage < stage) { + left_last_pos.index_by_stage(curr_stage) = 0; + ++curr_stage; + } + ++left_last_pos.index_by_stage(curr_stage); + + // fix the tracked child nodes of right_node, stage by stage. + auto& right_tracked_children = right_node.tracked_child_nodes; + auto rit = right_tracked_children.begin(); + while (curr_stage <= STAGE_TOP) { + auto right_pos_until = search_position_t::begin(); + right_pos_until.index_by_stage(curr_stage) = INDEX_UPPER_BOUND; + auto rend = right_tracked_children.lower_bound(right_pos_until); + while (rit != rend) { + auto new_pos = rit->second->parent_info().position; + assert(new_pos == rit->first); + assert(rit->second->parent_info().ptr == &right_node); + new_pos += left_last_pos; + auto p_child = rit->second; + rit = right_tracked_children.erase(rit); + p_child->as_child(new_pos, this); + } + left_last_pos.index_by_stage(curr_stage) = 0; + ++curr_stage; + } + + // fix the end tracked child node of right_node, if exists. + if (rit != right_tracked_children.end()) { + assert(rit->first == search_position_t::end()); + assert(rit->second->parent_info().position == search_position_t::end()); + assert(right_node.impl->is_level_tail()); + assert(impl->is_level_tail()); + auto p_child = rit->second; + rit = right_tracked_children.erase(rit); + p_child->as_child(search_position_t::end(), this); + } + assert(right_tracked_children.empty()); + + validate_tracked_children(); +} + +eagain_ifuture<> InternalNode::test_clone_root( + context_t c_other, RootNodeTracker& tracker_other) const +{ + assert(is_root()); + assert(impl->is_level_tail()); + assert(impl->field_type() == field_type_t::N0); + Ref<const Node> this_ref = this; + return InternalNode::allocate(c_other, L_ADDR_MIN, field_type_t::N0, true, impl->level() + ).si_then([this, c_other, &tracker_other](auto fresh_other) { + impl->test_copy_to(fresh_other.mut); + auto cloned_root = fresh_other.node; + return c_other.nm.get_super(c_other.t, tracker_other + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::assert_all{"Invalid error during test clone"} + ).si_then([c_other, cloned_root](auto&& super_other) { + assert(super_other); + cloned_root->make_root_new(c_other, std::move(super_other)); + return cloned_root; + }); + }).si_then([this_ref, this, c_other](auto cloned_root) { + // clone tracked children + // In some unit tests, the children are stubbed out that they + // don't exist in NodeExtentManager, and are only tracked in memory. + return trans_intr::do_for_each( + tracked_child_nodes.begin(), + tracked_child_nodes.end(), + [this_ref, c_other, cloned_root](auto& kv) { + assert(kv.first == kv.second->parent_info().position); + return kv.second->test_clone_non_root(c_other, cloned_root); + } + ); + }); +} + +eagain_ifuture<> InternalNode::try_downgrade_root( + context_t c, Ref<Node>&& this_ref) +{ + LOG_PREFIX(OTree::InternalNode::try_downgrade_root); + assert(this_ref.get() == this); + assert(is_root()); + assert(impl->is_level_tail()); + if (!impl->is_keys_empty()) { + // I have more than 1 values, no need to downgrade + return eagain_iertr::now(); + } + + // proceed downgrade root to the only child + laddr_t child_addr = impl->get_tail_value()->value; + return get_or_track_child(c, search_position_t::end(), child_addr + ).si_then([c, this, FNAME, + this_ref = std::move(this_ref)] (auto child) mutable { + INFOT("downgrade {} to new root {}", + c.t, get_name(), child->get_name()); + // Invariant, see InternalNode::erase_child() + // the new internal root should have at least 2 children. + assert(child->impl->is_level_tail()); + if (child->impl->node_type() == node_type_t::INTERNAL) { + ceph_assert(!child->impl->is_keys_empty()); + } + + assert(tracked_child_nodes.size() == 1); + child->deref_parent(); + auto super_to_move = deref_super(); + child->make_root_from(c, std::move(super_to_move), impl->laddr()); + --(c.t.get_onode_tree_stats().extents_num_delta); + return retire(c, std::move(this_ref)); + }); +} + +eagain_ifuture<Ref<InternalNode>> InternalNode::insert_or_split( + context_t c, + const search_position_t& pos, + const key_view_t& insert_key, + Ref<Node> insert_child, + Ref<Node> outdated_child) +{ + LOG_PREFIX(OTree::InternalNode::insert_or_split); + // XXX: check the insert_child is unlinked from this node +#ifndef NDEBUG + auto _insert_key = *insert_child->impl->get_pivot_index(); + assert(insert_key == _insert_key); +#endif + auto insert_value = insert_child->impl->laddr(); + auto insert_pos = pos; + DEBUGT("insert {} with insert_key={}, insert_child={}, insert_pos({}), " + "outdated_child={} ...", + c.t, get_name(), insert_key, insert_child->get_name(), + insert_pos, (outdated_child ? "True" : "False")); + auto [insert_stage, insert_size] = impl->evaluate_insert( + insert_key, insert_value, insert_pos); + auto free_size = impl->free_size(); + if (free_size >= insert_size) { + // proceed to insert + [[maybe_unused]] auto p_value = impl->insert( + insert_key, insert_value, insert_pos, insert_stage, insert_size); + assert(impl->free_size() == free_size - insert_size); + assert(insert_pos <= pos); + assert(p_value->value == insert_value); + + if (outdated_child) { + track_insert<false>(insert_pos, insert_stage, insert_child); + validate_child_inconsistent(*outdated_child); +#ifndef NDEBUG + do_untrack_child(*outdated_child); + validate_tracked_children(); + do_track_child<false>(*outdated_child); +#endif + } else { + track_insert(insert_pos, insert_stage, insert_child); + validate_tracked_children(); + } + + return eagain_iertr::make_ready_future<Ref<InternalNode>>(nullptr); + } + + // proceed to split with insert + // assume I'm already ref-counted by caller + laddr_t left_hint, right_hint; + { + key_view_t left_key; + impl->get_slot(search_position_t::begin(), &left_key, nullptr); + left_hint = left_key.get_hint(); + key_view_t right_key; + impl->get_largest_slot(nullptr, &right_key, nullptr); + right_hint = right_key.get_hint(); + } + return (is_root() ? upgrade_root(c, left_hint) : eagain_iertr::now() + ).si_then([this, c, right_hint] { + return InternalNode::allocate( + c, right_hint, impl->field_type(), impl->is_level_tail(), impl->level()); + }).si_then([this, insert_key, insert_child, insert_pos, + insert_stage=insert_stage, insert_size=insert_size, + outdated_child, c, FNAME](auto fresh_right) mutable { + // I'm the left_node and need to split into the right_node + auto right_node = fresh_right.node; + DEBUGT("proceed split {} to fresh {} with insert_child={}," + " outdated_child={} ...", + c.t, get_name(), right_node->get_name(), + insert_child->get_name(), + (outdated_child ? outdated_child->get_name() : "N/A")); + auto insert_value = insert_child->impl->laddr(); + auto [split_pos, is_insert_left, p_value] = impl->split_insert( + fresh_right.mut, *right_node->impl, insert_key, insert_value, + insert_pos, insert_stage, insert_size); + assert(p_value->value == insert_value); + track_split(split_pos, right_node); + + if (outdated_child) { + if (is_insert_left) { + track_insert<false>(insert_pos, insert_stage, insert_child); + } else { + right_node->template track_insert<false>(insert_pos, insert_stage, insert_child); + } +#ifndef NDEBUG + auto& _parent = outdated_child->parent_info().ptr; + _parent->validate_child_inconsistent(*outdated_child); + _parent->do_untrack_child(*outdated_child); + validate_tracked_children(); + right_node->validate_tracked_children(); + _parent->do_track_child<false>(*outdated_child); +#endif + } else { + if (is_insert_left) { + track_insert(insert_pos, insert_stage, insert_child); + } else { + right_node->track_insert(insert_pos, insert_stage, insert_child); + } + validate_tracked_children(); + right_node->validate_tracked_children(); + } + ++(c.t.get_onode_tree_stats().extents_num_delta); + return right_node; + }); +} + +eagain_ifuture<Ref<Node>> InternalNode::get_or_track_child( + context_t c, const search_position_t& position, laddr_t child_addr) +{ + LOG_PREFIX(OTree::InternalNode::get_or_track_child); + Ref<Node> this_ref = this; + return [this, position, child_addr, c, FNAME] { + auto found = tracked_child_nodes.find(position); + if (found != tracked_child_nodes.end()) { + TRACET("loaded child tracked {} at pos({}) addr={:x}", + c.t, found->second->get_name(), position, child_addr); + return eagain_iertr::make_ready_future<Ref<Node>>(found->second); + } + // the child is not loaded yet + TRACET("loading child at pos({}) addr={:x} ...", + c.t, position, child_addr); + bool level_tail = position.is_end(); + return Node::load(c, child_addr, level_tail + ).si_then([this, position, c, FNAME] (auto child) { + TRACET("loaded child untracked {}", + c.t, child->get_name()); + if (child->level() + 1 != level()) { + ERRORT("loaded child {} error from parent {} at pos({}), level mismatch", + c.t, child->get_name(), get_name(), position); + ceph_abort("fatal error"); + } + child->as_child(position, this); + return child; + }); + }().si_then([this_ref, this, position, child_addr] (auto child) { + assert(child_addr == child->impl->laddr()); + assert(position == child->parent_info().position); + std::ignore = position; + std::ignore = child_addr; + validate_child_tracked(*child); + return child; + }); +} + +template <bool VALIDATE> +void InternalNode::track_insert( + const search_position_t& insert_pos, match_stage_t insert_stage, + Ref<Node> insert_child, Ref<Node> nxt_child) +{ + // update tracks + auto pos_upper_bound = insert_pos; + pos_upper_bound.index_by_stage(insert_stage) = INDEX_UPPER_BOUND; + auto first = tracked_child_nodes.lower_bound(insert_pos); + auto last = tracked_child_nodes.lower_bound(pos_upper_bound); + std::vector<Node*> nodes; + std::for_each(first, last, [&nodes](auto& kv) { + nodes.push_back(kv.second); + }); + tracked_child_nodes.erase(first, last); + for (auto& node : nodes) { + auto _pos = node->parent_info().position; + assert(!_pos.is_end()); + ++_pos.index_by_stage(insert_stage); + node->as_child<VALIDATE>(_pos, this); + } + // track insert + insert_child->as_child(insert_pos, this); + +#ifndef NDEBUG + // validate left_child is before right_child + if (nxt_child) { + auto iter = tracked_child_nodes.find(insert_pos); + ++iter; + assert(iter->second == nxt_child); + } +#endif +} +template void InternalNode::track_insert<true>(const search_position_t&, match_stage_t, Ref<Node>, Ref<Node>); +template void InternalNode::track_insert<false>(const search_position_t&, match_stage_t, Ref<Node>, Ref<Node>); + +void InternalNode::replace_track( + Ref<Node> new_child, Ref<Node> old_child, bool is_new_child_outdated) +{ + assert(!new_child->is_tracked()); + auto& pos = old_child->parent_info().position; + auto this_ref = old_child->deref_parent(); + assert(this_ref == this); + if (is_new_child_outdated) { + // we need to keep track of the outdated child through + // insert and split. + new_child->as_child<false>(pos, this); + } else { + new_child->as_child(pos, this); + } + +#ifndef NDEBUG + if (is_new_child_outdated) { + validate_child_inconsistent(*new_child); + } else { + validate_child_tracked(*new_child); + } +#endif +} + +void InternalNode::track_split( + const search_position_t& split_pos, Ref<InternalNode> right_node) +{ + auto iter = tracked_child_nodes.lower_bound(split_pos); + while (iter != tracked_child_nodes.end()) { + auto new_pos = iter->first; + auto p_node = iter->second; + iter = tracked_child_nodes.erase(iter); + new_pos -= split_pos; + p_node->as_child<false>(new_pos, right_node); + } +} + +template <bool VALIDATE> +void InternalNode::track_erase( + const search_position_t& erase_pos, match_stage_t erase_stage) +{ + auto first = tracked_child_nodes.lower_bound(erase_pos); + assert(first == tracked_child_nodes.end() || + first->first != erase_pos); + auto pos_upper_bound = erase_pos; + pos_upper_bound.index_by_stage(erase_stage) = INDEX_UPPER_BOUND; + auto last = tracked_child_nodes.lower_bound(pos_upper_bound); + std::vector<Node*> p_nodes; + std::for_each(first, last, [&p_nodes](auto& kv) { + p_nodes.push_back(kv.second); + }); + tracked_child_nodes.erase(first, last); + for (auto& p_node: p_nodes) { + auto new_pos = p_node->parent_info().position; + assert(new_pos.index_by_stage(erase_stage) > 0); + --new_pos.index_by_stage(erase_stage); + p_node->as_child<VALIDATE>(new_pos, this); + } +} +template void InternalNode::track_erase<true>(const search_position_t&, match_stage_t); +template void InternalNode::track_erase<false>(const search_position_t&, match_stage_t); + +void InternalNode::track_make_tail(const search_position_t& last_pos) +{ + // assume I'm ref counted by the caller. + assert(impl->is_level_tail()); + assert(!last_pos.is_end()); + assert(tracked_child_nodes.find(search_position_t::end()) == + tracked_child_nodes.end()); + auto last_it = tracked_child_nodes.find(last_pos); + if (last_it != tracked_child_nodes.end()) { + assert(std::next(last_it) == tracked_child_nodes.end()); + auto p_last_child = last_it->second; + tracked_child_nodes.erase(last_it); + p_last_child->as_child(search_position_t::end(), this); + } else { + assert(tracked_child_nodes.lower_bound(last_pos) == + tracked_child_nodes.end()); + } +} + +void InternalNode::validate_child(const Node& child) const +{ +#ifndef NDEBUG + assert(impl->level() - 1 == child.impl->level()); + assert(this == child.parent_info().ptr); + auto& child_pos = child.parent_info().position; + if (child_pos.is_end()) { + assert(impl->is_level_tail()); + assert(child.impl->is_level_tail()); + assert(impl->get_tail_value()->value == child.impl->laddr()); + } else { + assert(!child.impl->is_level_tail()); + key_view_t index_key; + const laddr_packed_t* p_child_addr; + impl->get_slot(child_pos, &index_key, &p_child_addr); + assert(index_key == *child.impl->get_pivot_index()); + assert(p_child_addr->value == child.impl->laddr()); + } + // XXX(multi-type) + assert(impl->field_type() <= child.impl->field_type()); +#endif +} + +void InternalNode::validate_child_inconsistent(const Node& child) const +{ +#ifndef NDEBUG + assert(impl->level() - 1 == child.impl->level()); + assert(check_is_tracking(child)); + auto& child_pos = child.parent_info().position; + // the tail value has no key to fix + assert(!child_pos.is_end()); + assert(!child.impl->is_level_tail()); + + key_view_t current_key; + const laddr_packed_t* p_value; + impl->get_slot(child_pos, ¤t_key, &p_value); + key_view_t new_key = *child.impl->get_pivot_index(); + assert(current_key != new_key); + assert(p_value->value == child.impl->laddr()); +#endif +} + +eagain_ifuture<InternalNode::fresh_node_t> InternalNode::allocate( + context_t c, laddr_t hint, field_type_t field_type, bool is_level_tail, level_t level) +{ + return InternalNodeImpl::allocate(c, hint, field_type, is_level_tail, level + ).si_then([](auto&& fresh_impl) { + auto *derived_ptr = fresh_impl.impl.get(); + auto node = Ref<InternalNode>(new InternalNode( + derived_ptr, std::move(fresh_impl.impl))); + return fresh_node_t{node, fresh_impl.mut}; + }); +} + +/* + * LeafNode + */ + +LeafNode::LeafNode(LeafNodeImpl* impl, NodeImplURef&& impl_ref) + : Node(std::move(impl_ref)), impl{impl} {} + +bool LeafNode::is_level_tail() const +{ + return impl->is_level_tail(); +} + +node_version_t LeafNode::get_version() const +{ + return {layout_version, impl->get_extent_state()}; +} + +const char* LeafNode::read() const +{ + return impl->read(); +} + +extent_len_t LeafNode::get_node_size() const +{ + return impl->get_node_size(); +} + +std::tuple<key_view_t, const value_header_t*> +LeafNode::get_kv(const search_position_t& pos) const +{ + key_view_t key_view; + const value_header_t* p_value_header; + impl->get_slot(pos, &key_view, &p_value_header); + return {key_view, p_value_header}; +} + +eagain_ifuture<Ref<tree_cursor_t>> +LeafNode::get_next_cursor(context_t c, const search_position_t& pos) +{ + impl->validate_non_empty(); + search_position_t next_pos = pos; + key_view_t index_key; + const value_header_t* p_value_header = nullptr; + impl->get_next_slot(next_pos, &index_key, &p_value_header); + if (next_pos.is_end()) { + if (unlikely(is_level_tail())) { + return eagain_iertr::make_ready_future<Ref<tree_cursor_t>>( + tree_cursor_t::create_end(this)); + } else { + return get_next_cursor_from_parent(c); + } + } else { + return eagain_iertr::make_ready_future<Ref<tree_cursor_t>>( + get_or_track_cursor(next_pos, index_key, p_value_header)); + } +} + +template <bool FORCE_MERGE> +eagain_ifuture<Ref<tree_cursor_t>> +LeafNode::erase(context_t c, const search_position_t& pos, bool get_next) +{ + LOG_PREFIX(OTree::LeafNode::erase); + assert(!pos.is_end()); + assert(!impl->is_keys_empty()); + Ref<Node> this_ref = this; + DEBUGT("erase {}'s pos({}), get_next={} ...", + c.t, get_name(), pos, get_next); + ++(c.t.get_onode_tree_stats().num_erases); + + // get the next cursor + return eagain_iertr::now().si_then([c, &pos, get_next, this] { + if (get_next) { + return get_next_cursor(c, pos); + } else { + return eagain_iertr::make_ready_future<Ref<tree_cursor_t>>(); + } + }).si_then([c, &pos, this_ref = std::move(this_ref), + this, FNAME] (Ref<tree_cursor_t> next_cursor) mutable { + if (next_cursor && next_cursor->is_end()) { + // reset the node reference from the end cursor + next_cursor.reset(); + } + return eagain_iertr::now().si_then( + [c, &pos, this_ref = std::move(this_ref), this, FNAME] () mutable { + assert_moveable(this_ref); +#ifndef NDEBUG + assert(!impl->is_keys_empty()); + if (impl->has_single_value()) { + assert(pos == search_position_t::begin()); + } +#endif + if (!is_root() && impl->has_single_value()) { + // we need to keep the root as an empty leaf node + // fast path without mutating the extent + // track_erase + DEBUGT("{} has one value left, erase ...", c.t, get_name()); + assert(tracked_cursors.size() == 1); + auto iter = tracked_cursors.begin(); + assert(iter->first == pos); + iter->second->invalidate(); + tracked_cursors.clear(); + + // no cursor should be referencing this node now, this_ref is the last one. + assert(this_ref->use_count() == 1); + return Node::erase_node(c, std::move(this_ref)); + } + + on_layout_change(); + impl->prepare_mutate(c); + auto [erase_stage, next_pos] = impl->erase(pos); + track_erase(pos, erase_stage); + validate_tracked_cursors(); + + if (is_root()) { + return eagain_iertr::now(); + } else { + bool update_parent_index; + if (impl->is_level_tail()) { + update_parent_index = false; + } else { + next_pos.is_end() ? update_parent_index = true + : update_parent_index = false; + } + return try_merge_adjacent<FORCE_MERGE>( + c, update_parent_index, std::move(this_ref)); + } + }).si_then([next_cursor] { + return next_cursor; + }); + }); +} +template eagain_ifuture<Ref<tree_cursor_t>> +LeafNode::erase<true>(context_t, const search_position_t&, bool); +template eagain_ifuture<Ref<tree_cursor_t>> +LeafNode::erase<false>(context_t, const search_position_t&, bool); + +eagain_ifuture<> LeafNode::extend_value( + context_t c, const search_position_t& pos, value_size_t extend_size) +{ + ceph_abort("not implemented"); + return eagain_iertr::now(); +} + +eagain_ifuture<> LeafNode::trim_value( + context_t c, const search_position_t& pos, value_size_t trim_size) +{ + ceph_abort("not implemented"); + return eagain_iertr::now(); +} + +std::pair<NodeExtentMutable&, ValueDeltaRecorder*> +LeafNode::prepare_mutate_value_payload(context_t c) +{ + return impl->prepare_mutate_value_payload(c); +} + +eagain_ifuture<Ref<tree_cursor_t>> +LeafNode::lookup_smallest(context_t) +{ + if (unlikely(impl->is_keys_empty())) { + assert(is_root()); + return seastar::make_ready_future<Ref<tree_cursor_t>>( + tree_cursor_t::create_end(this)); + } + auto pos = search_position_t::begin(); + key_view_t index_key; + const value_header_t* p_value_header; + impl->get_slot(pos, &index_key, &p_value_header); + return seastar::make_ready_future<Ref<tree_cursor_t>>( + get_or_track_cursor(pos, index_key, p_value_header)); +} + +eagain_ifuture<Ref<tree_cursor_t>> +LeafNode::lookup_largest(context_t) +{ + if (unlikely(impl->is_keys_empty())) { + assert(is_root()); + return seastar::make_ready_future<Ref<tree_cursor_t>>( + tree_cursor_t::create_end(this)); + } + search_position_t pos; + key_view_t index_key; + const value_header_t* p_value_header = nullptr; + impl->get_largest_slot(&pos, &index_key, &p_value_header); + return seastar::make_ready_future<Ref<tree_cursor_t>>( + get_or_track_cursor(pos, index_key, p_value_header)); +} + +eagain_ifuture<Node::search_result_t> +LeafNode::lower_bound_tracked( + context_t c, const key_hobj_t& key, MatchHistory& history) +{ + key_view_t index_key; + auto result = impl->lower_bound(key, history, &index_key); + Ref<tree_cursor_t> cursor; + if (result.position.is_end()) { + assert(!result.p_value); + cursor = tree_cursor_t::create_end(this); + } else { + cursor = get_or_track_cursor(result.position, index_key, result.p_value); + } + search_result_t ret{cursor, result.mstat}; + ret.validate_input_key(key, c.vb.get_header_magic()); + return seastar::make_ready_future<search_result_t>(ret); +} + +eagain_ifuture<> LeafNode::do_get_tree_stats(context_t, tree_stats_t& stats) +{ + auto nstats = impl->get_stats(); + stats.size_persistent_leaf += nstats.size_persistent; + stats.size_filled_leaf += nstats.size_filled; + stats.size_logical_leaf += nstats.size_logical; + stats.size_overhead_leaf += nstats.size_overhead; + stats.size_value_leaf += nstats.size_value; + stats.num_kvs_leaf += nstats.num_kvs; + stats.num_nodes_leaf += 1; + return eagain_iertr::now(); +} + +void LeafNode::track_merge( + Ref<Node> _right_node, match_stage_t stage, search_position_t& left_last_pos) +{ + assert(level() == _right_node->level()); + // assert(impl->node_type() == _right_node->impl->node_type()); + auto& right_node = *static_cast<LeafNode*>(_right_node.get()); + if (right_node.tracked_cursors.empty()) { + return; + } + + match_stage_t curr_stage = STAGE_BOTTOM; + + // prepare the initial left_last_pos for offset + while (curr_stage < stage) { + left_last_pos.index_by_stage(curr_stage) = 0; + ++curr_stage; + } + ++left_last_pos.index_by_stage(curr_stage); + + // fix the tracked child nodes of right_node, stage by stage. + auto& right_tracked_cursors = right_node.tracked_cursors; + auto rit = right_tracked_cursors.begin(); + while (curr_stage <= STAGE_TOP) { + auto right_pos_until = search_position_t::begin(); + right_pos_until.index_by_stage(curr_stage) = INDEX_UPPER_BOUND; + auto rend = right_tracked_cursors.lower_bound(right_pos_until); + while (rit != rend) { + auto new_pos = rit->second->get_position(); + assert(new_pos == rit->first); + assert(rit->second->get_leaf_node().get() == &right_node); + new_pos += left_last_pos; + auto p_cursor = rit->second; + rit = right_tracked_cursors.erase(rit); + p_cursor->update_track<true>(this, new_pos); + } + left_last_pos.index_by_stage(curr_stage) = 0; + ++curr_stage; + } + assert(right_tracked_cursors.empty()); + + validate_tracked_cursors(); +} + +eagain_ifuture<> LeafNode::test_clone_root( + context_t c_other, RootNodeTracker& tracker_other) const +{ + assert(is_root()); + assert(impl->is_level_tail()); + assert(impl->field_type() == field_type_t::N0); + Ref<const Node> this_ref = this; + return LeafNode::allocate(c_other, L_ADDR_MIN, field_type_t::N0, true + ).si_then([this, c_other, &tracker_other](auto fresh_other) { + impl->test_copy_to(fresh_other.mut); + auto cloned_root = fresh_other.node; + return c_other.nm.get_super(c_other.t, tracker_other + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::assert_all{"Invalid error during test clone"} + ).si_then([c_other, cloned_root](auto&& super_other) { + assert(super_other); + cloned_root->make_root_new(c_other, std::move(super_other)); + }); + }).si_then([this_ref]{}); +} + +eagain_ifuture<Ref<tree_cursor_t>> LeafNode::insert_value( + context_t c, const key_hobj_t& key, value_config_t vconf, + const search_position_t& pos, const MatchHistory& history, + match_stat_t mstat) +{ + LOG_PREFIX(OTree::LeafNode::insert_value); +#ifndef NDEBUG + if (pos.is_end()) { + assert(impl->is_level_tail()); + } +#endif + DEBUGT("insert {} with insert_key={}, insert_value={}, insert_pos({}), " + "history={}, mstat({}) ...", + c.t, get_name(), key, vconf, pos, history, mstat); + ++(c.t.get_onode_tree_stats().num_inserts); + search_position_t insert_pos = pos; + auto [insert_stage, insert_size] = impl->evaluate_insert( + key, vconf, history, mstat, insert_pos); + auto free_size = impl->free_size(); + if (free_size >= insert_size) { + // proceed to insert + on_layout_change(); + impl->prepare_mutate(c); + auto p_value_header = impl->insert(key, vconf, insert_pos, insert_stage, insert_size); + assert(impl->free_size() == free_size - insert_size); + assert(insert_pos <= pos); + assert(p_value_header->payload_size == vconf.payload_size); + auto ret = track_insert(insert_pos, insert_stage, p_value_header); + validate_tracked_cursors(); + return eagain_iertr::make_ready_future<Ref<tree_cursor_t>>(ret); + } + // split and insert + Ref<Node> this_ref = this; + laddr_t left_hint, right_hint; + { + key_view_t left_key; + impl->get_slot(search_position_t::begin(), &left_key, nullptr); + left_hint = left_key.get_hint(); + key_view_t right_key; + impl->get_largest_slot(nullptr, &right_key, nullptr); + right_hint = right_key.get_hint(); + } + return (is_root() ? upgrade_root(c, left_hint) : eagain_iertr::now() + ).si_then([this, c, right_hint] { + return LeafNode::allocate(c, right_hint, impl->field_type(), impl->is_level_tail()); + }).si_then([this_ref = std::move(this_ref), this, c, &key, vconf, FNAME, + insert_pos, insert_stage=insert_stage, insert_size=insert_size](auto fresh_right) mutable { + auto right_node = fresh_right.node; + DEBUGT("proceed split {} to fresh {} ...", + c.t, get_name(), right_node->get_name()); + // no need to bump version for right node, as it is fresh + on_layout_change(); + impl->prepare_mutate(c); + auto [split_pos, is_insert_left, p_value_header] = impl->split_insert( + fresh_right.mut, *right_node->impl, key, vconf, + insert_pos, insert_stage, insert_size); + assert(p_value_header->payload_size == vconf.payload_size); + track_split(split_pos, right_node); + Ref<tree_cursor_t> ret; + if (is_insert_left) { + ret = track_insert(insert_pos, insert_stage, p_value_header); + } else { + ret = right_node->track_insert(insert_pos, insert_stage, p_value_header); + } + validate_tracked_cursors(); + right_node->validate_tracked_cursors(); + + ++(c.t.get_onode_tree_stats().extents_num_delta); + return apply_split_to_parent( + c, std::move(this_ref), std::move(right_node), false + ).si_then([ret] { + return ret; + }); + // TODO (optimize) + // try to acquire space from siblings before split... see btrfs + }); +} + +eagain_ifuture<Ref<LeafNode>> LeafNode::allocate_root( + context_t c, RootNodeTracker& root_tracker) +{ + LOG_PREFIX(OTree::LeafNode::allocate_root); + return LeafNode::allocate(c, L_ADDR_MIN, field_type_t::N0, true + ).si_then([c, &root_tracker, FNAME](auto fresh_node) { + auto root = fresh_node.node; + return c.nm.get_super(c.t, root_tracker + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::input_output_error::handle([FNAME, c] { + ERRORT("EIO during get_super()", c.t); + ceph_abort("fatal error"); + }) + ).si_then([c, root](auto&& super) { + assert(super); + root->make_root_new(c, std::move(super)); + return root; + }); + }); +} + +Ref<tree_cursor_t> LeafNode::get_or_track_cursor( + const search_position_t& position, + const key_view_t& key, const value_header_t* p_value_header) +{ + assert(!position.is_end()); + assert(p_value_header); + Ref<tree_cursor_t> p_cursor; + auto found = tracked_cursors.find(position); + if (found == tracked_cursors.end()) { + p_cursor = tree_cursor_t::create_tracked( + this, position, key, p_value_header); + } else { + p_cursor = found->second; + assert(p_cursor->get_leaf_node() == this); + assert(p_cursor->get_position() == position); + p_cursor->update_cache_same_node(key, p_value_header); + } + return p_cursor; +} + +void LeafNode::validate_cursor(const tree_cursor_t& cursor) const +{ +#ifndef NDEBUG + assert(this == cursor.get_leaf_node().get()); + assert(cursor.is_tracked()); + assert(!impl->is_extent_retired()); + + // We need to make sure user has freed all the cursors before submitting the + // according transaction. Otherwise the below checks will have undefined + // behaviors. + auto [key, p_value_header] = get_kv(cursor.get_position()); + auto magic = p_value_header->magic; + assert(key == cursor.get_key_view(magic)); + assert(p_value_header == cursor.read_value_header(magic)); +#endif +} + +Ref<tree_cursor_t> LeafNode::track_insert( + const search_position_t& insert_pos, match_stage_t insert_stage, + const value_header_t* p_value_header) +{ + // update cursor position + auto pos_upper_bound = insert_pos; + pos_upper_bound.index_by_stage(insert_stage) = INDEX_UPPER_BOUND; + auto first = tracked_cursors.lower_bound(insert_pos); + auto last = tracked_cursors.lower_bound(pos_upper_bound); + std::vector<tree_cursor_t*> p_cursors; + std::for_each(first, last, [&p_cursors](auto& kv) { + p_cursors.push_back(kv.second); + }); + tracked_cursors.erase(first, last); + for (auto& p_cursor : p_cursors) { + search_position_t new_pos = p_cursor->get_position(); + ++new_pos.index_by_stage(insert_stage); + p_cursor->update_track<true>(this, new_pos); + } + + // track insert + // TODO: getting key_view_t from stage::proceed_insert() and + // stage::append_insert() has not supported yet + return tree_cursor_t::create_inserted( + this, insert_pos); +} + +void LeafNode::track_split( + const search_position_t& split_pos, Ref<LeafNode> right_node) +{ + // update cursor ownership and position + auto iter = tracked_cursors.lower_bound(split_pos); + while (iter != tracked_cursors.end()) { + auto new_pos = iter->first; + auto p_cursor = iter->second; + iter = tracked_cursors.erase(iter); + new_pos -= split_pos; + p_cursor->update_track<false>(right_node, new_pos); + } +} + +void LeafNode::track_erase( + const search_position_t& erase_pos, match_stage_t erase_stage) +{ + // erase tracking and invalidate the erased cursor + auto to_erase = tracked_cursors.find(erase_pos); + assert(to_erase != tracked_cursors.end()); + to_erase->second->invalidate(); + auto first = tracked_cursors.erase(to_erase); + + // update cursor position + assert(first == tracked_cursors.lower_bound(erase_pos)); + auto pos_upper_bound = erase_pos; + pos_upper_bound.index_by_stage(erase_stage) = INDEX_UPPER_BOUND; + auto last = tracked_cursors.lower_bound(pos_upper_bound); + std::vector<tree_cursor_t*> p_cursors; + std::for_each(first, last, [&p_cursors](auto& kv) { + p_cursors.push_back(kv.second); + }); + tracked_cursors.erase(first, last); + for (auto& p_cursor : p_cursors) { + search_position_t new_pos = p_cursor->get_position(); + assert(new_pos.index_by_stage(erase_stage) > 0); + --new_pos.index_by_stage(erase_stage); + p_cursor->update_track<true>(this, new_pos); + } +} + +eagain_ifuture<LeafNode::fresh_node_t> LeafNode::allocate( + context_t c, laddr_t hint, field_type_t field_type, bool is_level_tail) +{ + return LeafNodeImpl::allocate(c, hint, field_type, is_level_tail + ).si_then([](auto&& fresh_impl) { + auto *derived_ptr = fresh_impl.impl.get(); + auto node = Ref<LeafNode>(new LeafNode( + derived_ptr, std::move(fresh_impl.impl))); + return fresh_node_t{node, fresh_impl.mut}; + }); +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node.h new file mode 100644 index 000000000..0b764172e --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node.h @@ -0,0 +1,743 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <compare> +#include <map> +#include <memory> +#include <ostream> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include "crimson/common/type_helpers.h" + +#include "node_extent_mutable.h" +#include "stages/key_layout.h" +#include "stages/stage_types.h" +#include "super.h" +#include "value.h" + +/** + * Tree example (2 levels): + * + * Root node keys: [ 3 7 ] + * values: [p1 p2 p3] + * / | \ + * ------- | ------- + * | | | + * V V V + * Leaf node keys: [ 1 2 3] [ 4 5 7] [ 9 11 12] + * values: [v1 v2 v3] [v4 v5 v6] [v7 v8 v9] + * + * Tree structure properties: + * - As illustrated above, the parent key is strictly equal to its left child's + * largest key; + * - If a tree is indexing multiple seastore transactions, each transaction + * will be mapped to a Super which points to a distinct root node. So the + * transactions are isolated at tree level. However, tree nodes from + * different transactions can reference the same seastore CachedExtent before + * modification; + * - The resources of the transactional tree are tracked by tree_cursor_ts held + * by users. As long as any cursor is alive, the according tree hierarchy is + * alive and keeps tracked. See the reversed resource management sections + * below; + */ + +namespace crimson::os::seastore::onode { + +class LeafNode; +class InternalNode; + +using layout_version_t = uint32_t; +struct node_version_t { + layout_version_t layout; + nextent_state_t state; + + bool operator==(const node_version_t& rhs) const { + return (layout == rhs.layout && state == rhs.state); + } + bool operator!=(const node_version_t& rhs) const { + return !(*this == rhs); + } +}; + +/** + * tree_cursor_t + * + * A cursor points to a position (LeafNode and search_position_t) of the tree + * where it can find the according key and value pair. The position is updated + * by LeafNode insert/split/delete/merge internally and is kept valid. It also + * caches the key-value information for a specific node layout version. + * + * Exposes public interfaces for Btree::Cursor. + */ +class tree_cursor_t final + : public boost::intrusive_ref_counter< + tree_cursor_t, boost::thread_unsafe_counter> { + public: + ~tree_cursor_t(); + tree_cursor_t(const tree_cursor_t&) = delete; + tree_cursor_t(tree_cursor_t&&) = delete; + tree_cursor_t& operator=(const tree_cursor_t&) = delete; + tree_cursor_t& operator=(tree_cursor_t&&) = delete; + + // public to Btree + + /** + * is_end + * + * Represents one-past-the-last of all the sorted key-value + * pairs in the tree. An end cursor won't contain valid key-value + * information. + */ + bool is_end() const { return !!ref_leaf_node && position.is_end(); } + + /** + * is_tracked + * + * Represents a key-value pair stored in the tree, which is always tracked + * across insert/split/erase/merge operations. + */ + bool is_tracked() const { return !!ref_leaf_node && !position.is_end(); } + + /** + * is_invalid + * + * Represents an invalid cursor which was once valid and tracked by the tree + * but is now erased and untracked. User may still hold an invalid cursor. + */ + bool is_invalid() const { return !ref_leaf_node; } + + /// Returns the key view in tree if it is not an end cursor. + const key_view_t& get_key_view(value_magic_t magic) const { + assert(is_tracked()); + return cache.get_key_view(magic, position); + } + + /// Returns the next tree_cursor_t in tree, can be end if there's no next. + eagain_ifuture<Ref<tree_cursor_t>> get_next(context_t); + + /// Check that this is next to prv + void assert_next_to(const tree_cursor_t&, value_magic_t) const; + + /// Erases the key-value pair from tree. + template <bool FORCE_MERGE = false> + eagain_ifuture<Ref<tree_cursor_t>> erase(context_t, bool get_next); + + std::strong_ordering compare_to(const tree_cursor_t&, value_magic_t) const; + + // public to Value + + /// Get the latest value_header_t pointer for read. + const value_header_t* read_value_header(value_magic_t magic) const { + assert(is_tracked()); + return cache.get_p_value_header(magic, position); + } + + /// Prepare the node extent to be mutable and recorded. + std::pair<NodeExtentMutable&, ValueDeltaRecorder*> + prepare_mutate_value_payload(context_t c) { + assert(is_tracked()); + if (!is_mutated) { + is_mutated = true; + ++(c.t.get_onode_tree_stats().num_updates); + } + return cache.prepare_mutate_value_payload(c, position); + } + + /// Extends the size of value payload. + eagain_ifuture<> extend_value(context_t, value_size_t); + + /// Trim and shrink the value payload. + eagain_ifuture<> trim_value(context_t, value_size_t); + + static Ref<tree_cursor_t> get_invalid() { + Ref<tree_cursor_t> INVALID = new tree_cursor_t(); + return INVALID; + } + + private: + // create from insert + tree_cursor_t(Ref<LeafNode>, const search_position_t&); + // create from lookup + tree_cursor_t(Ref<LeafNode>, const search_position_t&, + const key_view_t&, const value_header_t*); + // lookup reaches the end, contain leaf node for further insert + tree_cursor_t(Ref<LeafNode>); + // create an invalid tree_cursor_t + tree_cursor_t() : cache{ref_leaf_node} {} + + const search_position_t& get_position() const { return position; } + Ref<LeafNode> get_leaf_node() const { return ref_leaf_node; } + template <bool VALIDATE> + void update_track(Ref<LeafNode>, const search_position_t&); + void update_cache_same_node(const key_view_t&, + const value_header_t*) const; + void invalidate(); + + static Ref<tree_cursor_t> create_inserted( + Ref<LeafNode> node, const search_position_t& pos) { + return new tree_cursor_t(node, pos); + } + + static Ref<tree_cursor_t> create_tracked( + Ref<LeafNode> node, const search_position_t& pos, + const key_view_t& key, const value_header_t* p_header) { + return new tree_cursor_t(node, pos, key, p_header); + } + + static Ref<tree_cursor_t> create_end(Ref<LeafNode> node) { + return new tree_cursor_t(node); + } + + /** + * Reversed resource management (tree_cursor_t) + * + * tree_cursor_t holds a reference to the LeafNode, so the LeafNode will be + * alive as long as any of it's cursors is still referenced by user. + */ + Ref<LeafNode> ref_leaf_node; + search_position_t position; + + // account 1 update even if there are multiple updates to the same value + bool is_mutated = false; + + /** Cache + * + * Cached memory pointers or views which may be outdated due to + * extent copy-on-write or asynchronous leaf node updates. + */ + class Cache { + public: + Cache(Ref<LeafNode>&); + void validate_is_latest(const search_position_t&) const; + void invalidate() { needs_update_all = true; } + void update_all(const node_version_t&, const key_view_t&, const value_header_t*); + const key_view_t& get_key_view( + value_magic_t magic, const search_position_t& pos) { + make_latest(magic, pos); + return *key_view; + } + const value_header_t* get_p_value_header( + value_magic_t magic, const search_position_t& pos) { + make_latest(magic, pos); + return p_value_header; + } + std::pair<NodeExtentMutable&, ValueDeltaRecorder*> + prepare_mutate_value_payload(context_t, const search_position_t&); + + private: + void maybe_duplicate(const node_version_t&); + void make_latest(value_magic_t, const search_position_t&); + + // metadata about how cache is valid + Ref<LeafNode>& ref_leaf_node; + bool needs_update_all = true; + node_version_t version; + + // cached key value info + const char* p_node_base = nullptr; + std::optional<key_view_t> key_view; + const value_header_t* p_value_header = nullptr; + + // cached data-structures to update value payload + std::optional<NodeExtentMutable> value_payload_mut; + ValueDeltaRecorder* p_value_recorder = nullptr; + }; + mutable Cache cache; + + friend class LeafNode; + friend class Node; // get_position(), get_leaf_node() +}; + +/** + * Node + * + * An abstracted class for both InternalNode and LeafNode. + * + * Exposes public interfaces for Btree. + */ +class Node + : public boost::intrusive_ref_counter< + Node, boost::thread_unsafe_counter> { + public: + // public to Btree + struct search_result_t { + bool is_end() const { return p_cursor->is_end(); } + Ref<tree_cursor_t> p_cursor; + match_stat_t mstat; + + MatchKindBS match() const { + assert(mstat >= MSTAT_MIN && mstat <= MSTAT_MAX); + return (mstat == MSTAT_EQ ? MatchKindBS::EQ : MatchKindBS::NE); + } + + void validate_input_key(const key_hobj_t& key, value_magic_t magic) const { +#ifndef NDEBUG + if (match() == MatchKindBS::EQ) { + assert(key == p_cursor->get_key_view(magic)); + } else { + assert(match() == MatchKindBS::NE); + if (p_cursor->is_tracked()) { + assert(key < p_cursor->get_key_view(magic)); + } else if (p_cursor->is_end()) { + // good + } else { + assert(p_cursor->is_invalid()); + ceph_abort("impossible"); + } + } +#endif + } + }; + + virtual ~Node(); + Node(const Node&) = delete; + Node(Node&&) = delete; + Node& operator=(const Node&) = delete; + Node& operator=(Node&&) = delete; + + /** + * level + * + * A positive value denotes the level (or height) of this node in tree. + * 0 means LeafNode, positive means InternalNode. + */ + level_t level() const; + + /** + * lookup_smallest + * + * Returns a cursor pointing to the smallest key in the sub-tree formed by + * this node. + * + * Returns an end cursor if it is an empty root node. + */ + virtual eagain_ifuture<Ref<tree_cursor_t>> lookup_smallest(context_t) = 0; + + /** + * lookup_largest + * + * Returns a cursor pointing to the largest key in the sub-tree formed by + * this node. + * + * Returns an end cursor if it is an empty root node. + */ + virtual eagain_ifuture<Ref<tree_cursor_t>> lookup_largest(context_t) = 0; + + /** + * lower_bound + * + * Returns a cursor pointing to the first element in the range [first, last) + * of the sub-tree which does not compare less than the input key. The + * result also denotes whether the pointed key is equal to the input key. + * + * Returns an end cursor with MatchKindBS::NE if: + * - It is an empty root node; + * - Or the input key is larger than all the keys in the sub-tree; + */ + eagain_ifuture<search_result_t> lower_bound(context_t c, const key_hobj_t& key); + + /** + * insert + * + * Try to insert a key-value pair into the sub-tree formed by this node. + * + * Returns a boolean denoting whether the insertion is successful: + * - If true, the returned cursor points to the inserted element in tree; + * - If false, the returned cursor points to the conflicting element in tree; + */ + eagain_ifuture<std::pair<Ref<tree_cursor_t>, bool>> insert( + context_t, const key_hobj_t&, value_config_t, Ref<Node>&&); + + /** + * erase + * + * Removes a key-value pair from the sub-tree formed by this node. + * + * Returns the number of erased key-value pairs (0 or 1). + */ + eagain_ifuture<std::size_t> erase(context_t, const key_hobj_t&, Ref<Node>&&); + + /// Recursively collects the statistics of the sub-tree formed by this node + eagain_ifuture<tree_stats_t> get_tree_stats(context_t); + + /// Returns an ostream containing a dump of all the elements in the node. + std::ostream& dump(std::ostream&) const; + + /// Returns an ostream containing an one-line summary of this node. + std::ostream& dump_brief(std::ostream&) const; + + /// Print the node name + const std::string& get_name() const; + + /// Initializes the tree by allocating an empty root node. + static eagain_ifuture<> mkfs(context_t, RootNodeTracker&); + + /// Loads the tree root. The tree must be initialized. + static eagain_ifuture<Ref<Node>> load_root(context_t, RootNodeTracker&); + + // Only for unit test purposes. + void test_make_destructable(context_t, NodeExtentMutable&, Super::URef&&); + virtual eagain_ifuture<> test_clone_root(context_t, RootNodeTracker&) const = 0; + + protected: + virtual eagain_ifuture<> test_clone_non_root(context_t, Ref<InternalNode>) const { + ceph_abort("impossible path"); + } + virtual eagain_ifuture<search_result_t> lower_bound_tracked( + context_t, const key_hobj_t&, MatchHistory&) = 0; + virtual eagain_ifuture<> do_get_tree_stats(context_t, tree_stats_t&) = 0; + + virtual bool is_tracking() const = 0; + + virtual void track_merge(Ref<Node>, match_stage_t, search_position_t&) = 0; + + protected: + Node(NodeImplURef&&); + + bool is_tracked() const { + assert(!(super && _parent_info.has_value())); + return (super || _parent_info.has_value()); + } + + bool is_root() const { + assert(is_tracked()); + return !_parent_info.has_value(); + } + + // as root + void make_root(context_t c, Super::URef&& _super); + void make_root_new(context_t c, Super::URef&& _super) { + assert(_super->get_root_laddr() == L_ADDR_NULL); + make_root(c, std::move(_super)); + } + void make_root_from(context_t c, Super::URef&& _super, laddr_t from_addr) { + assert(_super->get_root_laddr() == from_addr); + make_root(c, std::move(_super)); + } + void as_root(Super::URef&& _super); + eagain_ifuture<> upgrade_root(context_t, laddr_t); + + Super::URef deref_super(); + + // as child/non-root + template <bool VALIDATE = true> + void as_child(const search_position_t&, Ref<InternalNode>); + + struct parent_info_t { + search_position_t position; + Ref<InternalNode> ptr; + }; + const parent_info_t& parent_info() const { return *_parent_info; } + + Ref<InternalNode> deref_parent(); + + eagain_ifuture<> apply_split_to_parent(context_t, Ref<Node>&&, Ref<Node>&&, bool); + eagain_ifuture<Ref<tree_cursor_t>> get_next_cursor_from_parent(context_t); + template <bool FORCE_MERGE = false> + eagain_ifuture<> try_merge_adjacent(context_t, bool, Ref<Node>&&); + eagain_ifuture<> erase_node(context_t, Ref<Node>&&); + template <bool FORCE_MERGE = false> + eagain_ifuture<> fix_parent_index(context_t, Ref<Node>&&, bool); + eagain_ifuture<NodeExtentMutable> rebuild_extent(context_t); + eagain_ifuture<> retire(context_t, Ref<Node>&&); + void make_tail(context_t); + + private: + /** + * Reversed resource management (Node) + * + * Root Node holds a reference to its parent Super class, so its parent + * will be alive as long as this root node is alive. + * + * None-root Node holds a reference to its parent Node, so its parent will + * be alive as long as any of it's children is alive. + */ + // as root + Super::URef super; + // as child/non-root + std::optional<parent_info_t> _parent_info; + + private: + static eagain_ifuture<Ref<Node>> load(context_t, laddr_t, bool expect_is_level_tail); + + NodeImplURef impl; + friend class InternalNode; +}; +inline std::ostream& operator<<(std::ostream& os, const Node& node) { + return node.dump_brief(os); +} + +/** + * InternalNode + * + * A concrete implementation of Node class that represents an internal tree + * node. Its level is always positive and its values are logical block + * addresses to its child nodes. An internal node cannot be empty. + */ +class InternalNode final : public Node { + public: + // public to Node + InternalNode(InternalNodeImpl*, NodeImplURef&&); + ~InternalNode() override { assert(tracked_child_nodes.empty()); } + InternalNode(const InternalNode&) = delete; + InternalNode(InternalNode&&) = delete; + InternalNode& operator=(const InternalNode&) = delete; + InternalNode& operator=(InternalNode&&) = delete; + + eagain_ifuture<Ref<tree_cursor_t>> get_next_cursor(context_t, const search_position_t&); + + eagain_ifuture<> apply_child_split(context_t, Ref<Node>&& left, Ref<Node>&& right, bool); + + template <bool VALIDATE> + void do_track_child(Node& child) { + if constexpr (VALIDATE) { + validate_child(child); + } + auto& child_pos = child.parent_info().position; + assert(tracked_child_nodes.find(child_pos) == tracked_child_nodes.end()); + tracked_child_nodes[child_pos] = &child; + } + + void do_untrack_child(const Node& child) { + assert(check_is_tracking(child)); + auto& child_pos = child.parent_info().position; + [[maybe_unused]] auto removed = tracked_child_nodes.erase(child_pos); + assert(removed); + } + + bool check_is_tracking(const Node& child) const { + auto& child_pos = child.parent_info().position; + auto found = tracked_child_nodes.find(child_pos); + if (found != tracked_child_nodes.end() && found->second == &child) { + assert(child.parent_info().ptr == this); + return true; + } else { + return false; + } + } + + eagain_ifuture<std::pair<Ref<Node>, Ref<Node>>> get_child_peers( + context_t, const search_position_t&); + + eagain_ifuture<> erase_child(context_t, Ref<Node>&&); + + template <bool FORCE_MERGE = false> + eagain_ifuture<> fix_index(context_t, Ref<Node>&&, bool); + + template <bool FORCE_MERGE = false> + eagain_ifuture<> apply_children_merge( + context_t, Ref<Node>&& left, laddr_t, Ref<Node>&& right, bool update_index); + + void validate_child_tracked(const Node& child) const { + validate_child(child); + assert(tracked_child_nodes.find(child.parent_info().position) != + tracked_child_nodes.end()); + assert(tracked_child_nodes.find(child.parent_info().position)->second == &child); + } + + void validate_child_inconsistent(const Node& child) const; + + void validate_tracked_children() const { +#ifndef NDEBUG + for (auto& kv : tracked_child_nodes) { + assert(kv.first == kv.second->parent_info().position); + validate_child(*kv.second); + } +#endif + } + + void track_make_tail(const search_position_t&); + + static eagain_ifuture<Ref<InternalNode>> allocate_root( + context_t, laddr_t, level_t, laddr_t, Super::URef&&); + + protected: + eagain_ifuture<Ref<tree_cursor_t>> lookup_smallest(context_t) override; + eagain_ifuture<Ref<tree_cursor_t>> lookup_largest(context_t) override; + eagain_ifuture<search_result_t> lower_bound_tracked( + context_t, const key_hobj_t&, MatchHistory&) override; + eagain_ifuture<> do_get_tree_stats(context_t, tree_stats_t&) override; + bool is_tracking() const override { + return !tracked_child_nodes.empty(); + } + void track_merge(Ref<Node>, match_stage_t, search_position_t&) override; + + eagain_ifuture<> test_clone_root(context_t, RootNodeTracker&) const override; + + private: + eagain_ifuture<> try_downgrade_root(context_t, Ref<Node>&&); + + eagain_ifuture<Ref<InternalNode>> insert_or_split( + context_t, const search_position_t&, const key_view_t&, Ref<Node>, + Ref<Node> outdated_child=nullptr); + + // XXX: extract a common tracker for InternalNode to track Node, + // and LeafNode to track tree_cursor_t. + eagain_ifuture<Ref<Node>> get_or_track_child(context_t, const search_position_t&, laddr_t); + template <bool VALIDATE = true> + void track_insert( + const search_position_t&, match_stage_t, Ref<Node>, Ref<Node> nxt_child = nullptr); + void replace_track(Ref<Node> new_child, Ref<Node> old_child, bool); + void track_split(const search_position_t&, Ref<InternalNode>); + template <bool VALIDATE = true> + void track_erase(const search_position_t&, match_stage_t); + void validate_child(const Node& child) const; + + struct fresh_node_t { + Ref<InternalNode> node; + NodeExtentMutable mut; + std::pair<Ref<Node>, NodeExtentMutable> make_pair() { + return std::make_pair(Ref<Node>(node), mut); + } + }; + static eagain_ifuture<fresh_node_t> allocate(context_t, laddr_t, field_type_t, bool, level_t); + + private: + /** + * Reversed resource management (InternalNode) + * + * InteralNode keeps track of its child nodes which are still alive in + * memory, and their positions will be updated throughout + * insert/split/delete/merge operations of this node. + */ + // XXX: leverage intrusive data structure to control memory overhead + std::map<search_position_t, Node*> tracked_child_nodes; + InternalNodeImpl* impl; +}; + +/** + * LeafNode + * + * A concrete implementation of Node class that represents a leaf tree node. + * Its level is always 0. A leaf node can only be empty if it is root. + */ +class LeafNode final : public Node { + public: + // public to tree_cursor_t + ~LeafNode() override { assert(tracked_cursors.empty()); } + LeafNode(const LeafNode&) = delete; + LeafNode(LeafNode&&) = delete; + LeafNode& operator=(const LeafNode&) = delete; + LeafNode& operator=(LeafNode&&) = delete; + + bool is_level_tail() const; + node_version_t get_version() const; + const char* read() const; + extent_len_t get_node_size() const; + std::tuple<key_view_t, const value_header_t*> get_kv(const search_position_t&) const; + eagain_ifuture<Ref<tree_cursor_t>> get_next_cursor(context_t, const search_position_t&); + + /** + * erase + * + * Removes a key-value pair from the position. + * + * If get_next is true, returns the cursor pointing to the next key-value + * pair that followed the erased element, which can be nullptr if is end. + */ + template <bool FORCE_MERGE> + eagain_ifuture<Ref<tree_cursor_t>> erase( + context_t, const search_position_t&, bool get_next); + + template <bool VALIDATE> + void do_track_cursor(tree_cursor_t& cursor) { + if constexpr (VALIDATE) { + validate_cursor(cursor); + } + auto& cursor_pos = cursor.get_position(); + assert(tracked_cursors.find(cursor_pos) == tracked_cursors.end()); + tracked_cursors.emplace(cursor_pos, &cursor); + } + void do_untrack_cursor(const tree_cursor_t& cursor) { + validate_cursor(cursor); + auto& cursor_pos = cursor.get_position(); + assert(check_is_tracking(cursor)); + [[maybe_unused]] auto removed = tracked_cursors.erase(cursor_pos); + assert(removed); + } + bool check_is_tracking(const tree_cursor_t& cursor) const { + auto& cursor_pos = cursor.get_position(); + auto found = tracked_cursors.find(cursor_pos); + if (found != tracked_cursors.end() && found->second == &cursor) { + assert(cursor.ref_leaf_node == this); + return true; + } else { + return false; + } + } + + eagain_ifuture<> extend_value(context_t, const search_position_t&, value_size_t); + eagain_ifuture<> trim_value(context_t, const search_position_t&, value_size_t); + + std::pair<NodeExtentMutable&, ValueDeltaRecorder*> + prepare_mutate_value_payload(context_t); + + protected: + eagain_ifuture<Ref<tree_cursor_t>> lookup_smallest(context_t) override; + eagain_ifuture<Ref<tree_cursor_t>> lookup_largest(context_t) override; + eagain_ifuture<search_result_t> lower_bound_tracked( + context_t, const key_hobj_t&, MatchHistory&) override; + eagain_ifuture<> do_get_tree_stats(context_t, tree_stats_t&) override; + bool is_tracking() const override { + return !tracked_cursors.empty(); + } + void track_merge(Ref<Node>, match_stage_t, search_position_t&) override; + + eagain_ifuture<> test_clone_root(context_t, RootNodeTracker&) const override; + + private: + LeafNode(LeafNodeImpl*, NodeImplURef&&); + eagain_ifuture<Ref<tree_cursor_t>> insert_value( + context_t, const key_hobj_t&, value_config_t, + const search_position_t&, const MatchHistory&, + match_stat_t mstat); + static eagain_ifuture<Ref<LeafNode>> allocate_root(context_t, RootNodeTracker&); + friend class Node; + + private: + // XXX: extract a common tracker for InternalNode to track Node, + // and LeafNode to track tree_cursor_t. + Ref<tree_cursor_t> get_or_track_cursor( + const search_position_t&, const key_view_t&, const value_header_t*); + Ref<tree_cursor_t> track_insert( + const search_position_t&, match_stage_t, const value_header_t*); + void track_split(const search_position_t&, Ref<LeafNode>); + void track_erase(const search_position_t&, match_stage_t); + void validate_tracked_cursors() const { +#ifndef NDEBUG + for (auto& kv : tracked_cursors) { + assert(kv.first == kv.second->get_position()); + validate_cursor(*kv.second); + } +#endif + } + void validate_cursor(const tree_cursor_t& cursor) const; + // invalidate p_value pointers in tree_cursor_t + void on_layout_change() { ++layout_version; } + + struct fresh_node_t { + Ref<LeafNode> node; + NodeExtentMutable mut; + std::pair<Ref<Node>, NodeExtentMutable> make_pair() { + return std::make_pair(Ref<Node>(node), mut); + } + }; + static eagain_ifuture<fresh_node_t> allocate(context_t, laddr_t, field_type_t, bool); + + private: + /** + * Reversed resource management (LeafNode) + * + * LeafNode keeps track of the referencing cursors which are still alive in + * memory, and their positions will be updated throughout + * insert/split/delete/merge operations of this node. + */ + // XXX: leverage intrusive data structure to control memory overhead + std::map<search_position_t, tree_cursor_t*> tracked_cursors; + LeafNodeImpl* impl; + layout_version_t layout_version = 0; +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_delta_recorder.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_delta_recorder.h new file mode 100644 index 000000000..ea26195de --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_delta_recorder.h @@ -0,0 +1,55 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "include/buffer.h" +#include "node_types.h" +#include "value.h" + +namespace crimson::os::seastore::onode { + +/** + * DeltaRecorder + * + * An abstracted class to encapsulate different implementations to apply delta + * to a specific node layout. + */ +class DeltaRecorder { + public: + virtual ~DeltaRecorder() { + /* May be non-empty if transaction is abandoned without + * being submitted -- conflicts are a particularly common + * example (denoted generally by returning crimson::ct_error::eagain). + */ + } + + bool is_empty() const { + return encoded.length() == 0; + } + + ceph::bufferlist get_delta() { + return std::move(encoded); + } + + ValueDeltaRecorder* get_value_recorder() const { + assert(value_recorder); + return value_recorder.get(); + } + + virtual node_type_t node_type() const = 0; + virtual field_type_t field_type() const = 0; + virtual void apply_delta(ceph::bufferlist::const_iterator&, + NodeExtentMutable&, + const NodeExtent&) = 0; + + protected: + DeltaRecorder() = default; + DeltaRecorder(const ValueBuilder& vb) + : value_recorder{vb.build_value_recorder(encoded)} {} + + ceph::bufferlist encoded; + std::unique_ptr<ValueDeltaRecorder> value_recorder; +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_accessor.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_accessor.h new file mode 100644 index 000000000..1a03036d3 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_accessor.h @@ -0,0 +1,619 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "crimson/os/seastore/logging.h" + +#include "node_extent_manager.h" +#include "node_delta_recorder.h" +#include "node_layout_replayable.h" +#include "value.h" + +#ifndef NDEBUG +#include "node_extent_manager/test_replay.h" +#endif + +namespace crimson::os::seastore::onode { + +/** + * DeltaRecorderT + * + * Responsible to encode and decode delta, and apply delta for a specific node + * layout. + */ +template <typename FieldType, node_type_t NODE_TYPE> +class DeltaRecorderT final: public DeltaRecorder { + public: + using layout_t = NodeLayoutReplayableT<FieldType, NODE_TYPE>; + using node_stage_t = typename layout_t::node_stage_t; + using position_t = typename layout_t::position_t; + using StagedIterator = typename layout_t::StagedIterator; + using value_input_t = typename layout_t::value_input_t; + static constexpr auto FIELD_TYPE = layout_t::FIELD_TYPE; + + ~DeltaRecorderT() override = default; + + template <KeyT KT> + void encode_insert( + const full_key_t<KT>& key, + const value_input_t& value, + const position_t& insert_pos, + const match_stage_t& insert_stage, + const node_offset_t& insert_size) { + ceph::encode(node_delta_op_t::INSERT, encoded); + encode_key(key, encoded); + encode_value(value, encoded); + insert_pos.encode(encoded); + ceph::encode(insert_stage, encoded); + ceph::encode(insert_size, encoded); + } + + void encode_split( + const StagedIterator& split_at, + const char* p_node_start) { + ceph::encode(node_delta_op_t::SPLIT, encoded); + split_at.encode(p_node_start, encoded); + } + + template <KeyT KT> + void encode_split_insert( + const StagedIterator& split_at, + const full_key_t<KT>& key, + const value_input_t& value, + const position_t& insert_pos, + const match_stage_t& insert_stage, + const node_offset_t& insert_size, + const char* p_node_start) { + ceph::encode(node_delta_op_t::SPLIT_INSERT, encoded); + split_at.encode(p_node_start, encoded); + encode_key(key, encoded); + encode_value(value, encoded); + insert_pos.encode(encoded); + ceph::encode(insert_stage, encoded); + ceph::encode(insert_size, encoded); + } + + void encode_update_child_addr( + const laddr_t new_addr, + const laddr_packed_t* p_addr, + const char* p_node_start, + extent_len_t node_size) { + ceph::encode(node_delta_op_t::UPDATE_CHILD_ADDR, encoded); + ceph::encode(new_addr, encoded); + int node_offset = reinterpret_cast<const char*>(p_addr) - p_node_start; + assert(node_offset > 0 && node_offset < (int)node_size); + ceph::encode(static_cast<node_offset_t>(node_offset), encoded); + } + + void encode_erase( + const position_t& erase_pos) { + ceph::encode(node_delta_op_t::ERASE, encoded); + erase_pos.encode(encoded); + } + + void encode_make_tail() { + ceph::encode(node_delta_op_t::MAKE_TAIL, encoded); + } + + static DeltaRecorderURef create_for_encode(const ValueBuilder& v_builder) { + return std::unique_ptr<DeltaRecorder>(new DeltaRecorderT(v_builder)); + } + + static DeltaRecorderURef create_for_replay() { + return std::unique_ptr<DeltaRecorder>(new DeltaRecorderT()); + } + + protected: + DeltaRecorderT() : DeltaRecorder() {} + DeltaRecorderT(const ValueBuilder& vb) : DeltaRecorder(vb) {} + node_type_t node_type() const override { return NODE_TYPE; } + field_type_t field_type() const override { return FIELD_TYPE; } + void apply_delta(ceph::bufferlist::const_iterator& delta, + NodeExtentMutable& mut, + const NodeExtent& node) override { + LOG_PREFIX(OTree::Extent::Replay); + assert(is_empty()); + node_stage_t stage(reinterpret_cast<const FieldType*>(mut.get_read()), + mut.get_length()); + node_delta_op_t op; + try { + ceph::decode(op, delta); + switch (op) { + case node_delta_op_t::INSERT: { + SUBDEBUG(seastore_onode, "decoding INSERT ..."); + auto key = key_hobj_t::decode(delta); + auto value = decode_value(delta); + auto insert_pos = position_t::decode(delta); + match_stage_t insert_stage; + ceph::decode(insert_stage, delta); + node_offset_t insert_size; + ceph::decode(insert_size, delta); + SUBDEBUG(seastore_onode, + "apply {}, {}, insert_pos({}), insert_stage={}, " + "insert_size={}B ...", + key, value, insert_pos, insert_stage, insert_size); + layout_t::template insert<KeyT::HOBJ>( + mut, stage, key, value, insert_pos, insert_stage, insert_size); + break; + } + case node_delta_op_t::SPLIT: { + SUBDEBUG(seastore_onode, "decoding SPLIT ..."); + auto split_at = StagedIterator::decode( + mut.get_read(), mut.get_length(), delta); + SUBDEBUG(seastore_onode, "apply split_at={} ...", split_at); + layout_t::split(mut, stage, split_at); + break; + } + case node_delta_op_t::SPLIT_INSERT: { + SUBDEBUG(seastore_onode, "decoding SPLIT_INSERT ..."); + auto split_at = StagedIterator::decode( + mut.get_read(), mut.get_length(), delta); + auto key = key_hobj_t::decode(delta); + auto value = decode_value(delta); + auto insert_pos = position_t::decode(delta); + match_stage_t insert_stage; + ceph::decode(insert_stage, delta); + node_offset_t insert_size; + ceph::decode(insert_size, delta); + SUBDEBUG(seastore_onode, + "apply split_at={}, {}, {}, insert_pos({}), insert_stage={}, " + "insert_size={}B ...", + split_at, key, value, insert_pos, insert_stage, insert_size); + layout_t::template split_insert<KeyT::HOBJ>( + mut, stage, split_at, key, value, insert_pos, insert_stage, insert_size); + break; + } + case node_delta_op_t::UPDATE_CHILD_ADDR: { + SUBDEBUG(seastore_onode, "decoding UPDATE_CHILD_ADDR ..."); + laddr_t new_addr; + ceph::decode(new_addr, delta); + node_offset_t update_offset; + ceph::decode(update_offset, delta); + auto p_addr = reinterpret_cast<laddr_packed_t*>( + mut.get_write() + update_offset); + SUBDEBUG(seastore_onode, + "apply {:#x} to offset {:#x} ...", + new_addr, update_offset); + layout_t::update_child_addr(mut, new_addr, p_addr); + break; + } + case node_delta_op_t::ERASE: { + SUBDEBUG(seastore_onode, "decoding ERASE ..."); + auto erase_pos = position_t::decode(delta); + SUBDEBUG(seastore_onode, "apply erase_pos({}) ...", erase_pos); + layout_t::erase(mut, stage, erase_pos); + break; + } + case node_delta_op_t::MAKE_TAIL: { + SUBDEBUG(seastore_onode, "decoded MAKE_TAIL, apply ..."); + layout_t::make_tail(mut, stage); + break; + } + case node_delta_op_t::SUBOP_UPDATE_VALUE: { + SUBDEBUG(seastore_onode, "decoding SUBOP_UPDATE_VALUE ..."); + node_offset_t value_header_offset; + ceph::decode(value_header_offset, delta); + auto p_header = mut.get_read() + value_header_offset; + auto p_header_ = reinterpret_cast<const value_header_t*>(p_header); + SUBDEBUG(seastore_onode, "update {} at {:#x} ...", *p_header_, value_header_offset); + auto payload_mut = p_header_->get_payload_mutable(mut); + auto value_addr = node.get_laddr() + payload_mut.get_node_offset(); + get_value_replayer(p_header_->magic)->apply_value_delta( + delta, payload_mut, value_addr); + break; + } + default: + SUBERROR(seastore_onode, + "got unknown op {} when replay {}", + op, node); + ceph_abort("fatal error"); + } + } catch (buffer::error& e) { + SUBERROR(seastore_onode, + "got decode error {} when replay {}", + e.what(), node); + ceph_abort("fatal error"); + } + } + + private: + ValueDeltaRecorder* get_value_replayer(value_magic_t magic) { + // Replay procedure is independent of Btree and happens at lower level in + // seastore. There is no ValueBuilder so the recoder needs to build the + // ValueDeltaRecorder by itself. + if (value_replayer) { + if (value_replayer->get_header_magic() != magic) { + ceph_abort_msgf("OTree::Extent::Replay: value magic mismatch %x != %x", + value_replayer->get_header_magic(), magic); + } + } else { + value_replayer = build_value_recorder_by_type(encoded, magic); + if (!value_replayer) { + ceph_abort_msgf("OTree::Extent::Replay: got unexpected value magic = %x", + magic); + } + } + return value_replayer.get(); + } + + void encode_value(const value_input_t& value, ceph::bufferlist& encoded) const { + if constexpr (std::is_same_v<value_input_t, laddr_t>) { + // NODE_TYPE == node_type_t::INTERNAL + ceph::encode(value, encoded); + } else if constexpr (std::is_same_v<value_input_t, value_config_t>) { + // NODE_TYPE == node_type_t::LEAF + value.encode(encoded); + } else { + ceph_abort("impossible path"); + } + } + + value_input_t decode_value(ceph::bufferlist::const_iterator& delta) const { + if constexpr (std::is_same_v<value_input_t, laddr_t>) { + // NODE_TYPE == node_type_t::INTERNAL + laddr_t value; + ceph::decode(value, delta); + return value; + } else if constexpr (std::is_same_v<value_input_t, value_config_t>) { + // NODE_TYPE == node_type_t::LEAF + return value_config_t::decode(delta); + } else { + ceph_abort("impossible path"); + } + } + + std::unique_ptr<ValueDeltaRecorder> value_replayer; +}; + +/** + * NodeExtentAccessorT + * + * This component is responsible to reference and mutate the underlying + * NodeExtent, record mutation parameters when needed, and apply the recorded + * modifications for a specific node layout. + * + * For possible internal states, see node_types.h. + */ +template <typename FieldType, node_type_t NODE_TYPE> +class NodeExtentAccessorT { + public: + using layout_t = NodeLayoutReplayableT<FieldType, NODE_TYPE>; + using node_stage_t = typename layout_t::node_stage_t; + using position_t = typename layout_t::position_t; + using recorder_t = DeltaRecorderT<FieldType, NODE_TYPE>; + using StagedIterator = typename layout_t::StagedIterator; + using value_input_t = typename layout_t::value_input_t; + using value_t = typename layout_t::value_t; + static constexpr auto FIELD_TYPE = layout_t::FIELD_TYPE; + + NodeExtentAccessorT(NodeExtentRef extent) + : extent{extent}, + node_stage{reinterpret_cast<const FieldType*>(extent->get_read()), + extent->get_length()} { + assert(is_valid_node_size(extent->get_length())); + if (extent->is_initial_pending()) { + state = nextent_state_t::FRESH; + mut.emplace(extent->get_mutable()); + assert(extent->get_recorder() == nullptr); + recorder = nullptr; + } else if (extent->is_mutation_pending()) { + state = nextent_state_t::MUTATION_PENDING; + mut.emplace(extent->get_mutable()); + auto p_recorder = extent->get_recorder(); + assert(p_recorder != nullptr); + assert(p_recorder->node_type() == NODE_TYPE); + assert(p_recorder->field_type() == FIELD_TYPE); + recorder = static_cast<recorder_t*>(p_recorder); + } else if (!extent->is_mutable() && extent->is_valid()) { + state = nextent_state_t::READ_ONLY; + // mut is empty + assert(extent->get_recorder() == nullptr || + extent->get_recorder()->is_empty()); + recorder = nullptr; + } else { + // extent is invalid or retired + ceph_abort("impossible path"); + } +#ifndef NDEBUG + auto ref_recorder = recorder_t::create_for_replay(); + test_recorder = static_cast<recorder_t*>(ref_recorder.get()); + test_extent = TestReplayExtent::create( + get_length(), std::move(ref_recorder)); +#endif + } + ~NodeExtentAccessorT() = default; + NodeExtentAccessorT(const NodeExtentAccessorT&) = delete; + NodeExtentAccessorT(NodeExtentAccessorT&&) = delete; + NodeExtentAccessorT& operator=(const NodeExtentAccessorT&) = delete; + NodeExtentAccessorT& operator=(NodeExtentAccessorT&&) = delete; + + const node_stage_t& read() const { return node_stage; } + laddr_t get_laddr() const { return extent->get_laddr(); } + extent_len_t get_length() const { + auto len = extent->get_length(); + assert(is_valid_node_size(len)); + return len; + } + nextent_state_t get_state() const { + assert(!is_retired()); + // we cannot rely on the underlying extent state because + // FRESH/MUTATION_PENDING can become DIRTY after transaction submission. + return state; + } + + bool is_retired() const { + if (extent) { + return false; + } else { + return true; + } + } + + // must be called before any mutate attempes. + // for the safety of mixed read and mutate, call before read. + void prepare_mutate(context_t c) { + assert(!is_retired()); + if (state == nextent_state_t::READ_ONLY) { + assert(!extent->is_mutable()); + auto ref_recorder = recorder_t::create_for_encode(c.vb); + recorder = static_cast<recorder_t*>(ref_recorder.get()); + extent = extent->mutate(c, std::move(ref_recorder)); + state = nextent_state_t::MUTATION_PENDING; + assert(extent->is_mutation_pending()); + node_stage = node_stage_t(reinterpret_cast<const FieldType*>(extent->get_read()), + get_length()); + assert(recorder == static_cast<recorder_t*>(extent->get_recorder())); + mut.emplace(extent->get_mutable()); + } + assert(extent->is_mutable()); + } + + template <KeyT KT> + const value_t* insert_replayable( + const full_key_t<KT>& key, + const value_input_t& value, + position_t& insert_pos, + match_stage_t& insert_stage, + node_offset_t& insert_size) { + assert(extent->is_mutable()); + assert(state != nextent_state_t::READ_ONLY); + if (state == nextent_state_t::MUTATION_PENDING) { + recorder->template encode_insert<KT>( + key, value, insert_pos, insert_stage, insert_size); + } +#ifndef NDEBUG + test_extent->prepare_replay(extent); + test_recorder->template encode_insert<KT>( + key, value, insert_pos, insert_stage, insert_size); +#endif + auto ret = layout_t::template insert<KT>( + *mut, read(), key, value, + insert_pos, insert_stage, insert_size); +#ifndef NDEBUG + test_extent->replay_and_verify(extent); +#endif + return ret; + } + + void split_replayable(StagedIterator& split_at) { + assert(extent->is_mutable()); + assert(state != nextent_state_t::READ_ONLY); + if (state == nextent_state_t::MUTATION_PENDING) { + recorder->encode_split(split_at, read().p_start()); + } +#ifndef NDEBUG + test_extent->prepare_replay(extent); + test_recorder->encode_split(split_at, read().p_start()); +#endif + layout_t::split(*mut, read(), split_at); +#ifndef NDEBUG + test_extent->replay_and_verify(extent); +#endif + } + + template <KeyT KT> + const value_t* split_insert_replayable( + StagedIterator& split_at, + const full_key_t<KT>& key, + const value_input_t& value, + position_t& insert_pos, + match_stage_t& insert_stage, + node_offset_t& insert_size) { + assert(extent->is_mutable()); + assert(state != nextent_state_t::READ_ONLY); + if (state == nextent_state_t::MUTATION_PENDING) { + recorder->template encode_split_insert<KT>( + split_at, key, value, insert_pos, insert_stage, insert_size, + read().p_start()); + } +#ifndef NDEBUG + test_extent->prepare_replay(extent); + test_recorder->template encode_split_insert<KT>( + split_at, key, value, insert_pos, insert_stage, insert_size, + read().p_start()); +#endif + auto ret = layout_t::template split_insert<KT>( + *mut, read(), split_at, key, value, + insert_pos, insert_stage, insert_size); +#ifndef NDEBUG + test_extent->replay_and_verify(extent); +#endif + return ret; + } + + void update_child_addr_replayable( + const laddr_t new_addr, laddr_packed_t* p_addr) { + assert(extent->is_mutable()); + assert(state != nextent_state_t::READ_ONLY); + if (state == nextent_state_t::MUTATION_PENDING) { + recorder->encode_update_child_addr( + new_addr, p_addr, read().p_start(), get_length()); + } +#ifndef NDEBUG + test_extent->prepare_replay(extent); + test_recorder->encode_update_child_addr( + new_addr, p_addr, read().p_start(), get_length()); +#endif + layout_t::update_child_addr(*mut, new_addr, p_addr); +#ifndef NDEBUG + test_extent->replay_and_verify(extent); +#endif + } + + std::tuple<match_stage_t, position_t> erase_replayable(const position_t& pos) { + assert(extent->is_mutable()); + assert(state != nextent_state_t::READ_ONLY); + if (state == nextent_state_t::MUTATION_PENDING) { + recorder->encode_erase(pos); + } +#ifndef NDEBUG + test_extent->prepare_replay(extent); + test_recorder->encode_erase(pos); +#endif + auto ret = layout_t::erase(*mut, read(), pos); +#ifndef NDEBUG + test_extent->replay_and_verify(extent); +#endif + return ret; + } + + position_t make_tail_replayable() { + assert(extent->is_mutable()); + assert(state != nextent_state_t::READ_ONLY); + if (state == nextent_state_t::MUTATION_PENDING) { + recorder->encode_make_tail(); + } +#ifndef NDEBUG + test_extent->prepare_replay(extent); + test_recorder->encode_make_tail(); +#endif + auto ret = layout_t::make_tail(*mut, read()); +#ifndef NDEBUG + test_extent->replay_and_verify(extent); +#endif + return ret; + } + + std::pair<NodeExtentMutable&, ValueDeltaRecorder*> + prepare_mutate_value_payload(context_t c) { + prepare_mutate(c); + ValueDeltaRecorder* p_value_recorder = nullptr; + if (state == nextent_state_t::MUTATION_PENDING) { + p_value_recorder = recorder->get_value_recorder(); + } + return {*mut, p_value_recorder}; + } + + void test_copy_to(NodeExtentMutable& to) const { + assert(extent->get_length() == to.get_length()); + std::memcpy(to.get_write(), extent->get_read(), get_length()); + } + + eagain_ifuture<NodeExtentMutable> rebuild(context_t c, laddr_t hint) { + LOG_PREFIX(OTree::Extent::rebuild); + assert(!is_retired()); + if (state == nextent_state_t::FRESH) { + assert(extent->is_initial_pending()); + // already fresh and no need to record + return eagain_iertr::make_ready_future<NodeExtentMutable>(*mut); + } + assert(!extent->is_initial_pending()); + auto alloc_size = get_length(); + return c.nm.alloc_extent(c.t, hint, alloc_size + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::input_output_error::handle( + [FNAME, c, alloc_size, l_to_discard = extent->get_laddr()] { + SUBERRORT(seastore_onode, + "EIO during allocate -- node_size={}, to_discard={:x}", + c.t, alloc_size, l_to_discard); + ceph_abort("fatal error"); + }) + ).si_then([this, c, FNAME] (auto fresh_extent) { + SUBDEBUGT(seastore_onode, + "update addr from {:#x} to {:#x} ...", + c.t, extent->get_laddr(), fresh_extent->get_laddr()); + assert(fresh_extent); + assert(fresh_extent->is_initial_pending()); + assert(fresh_extent->get_recorder() == nullptr); + assert(get_length() == fresh_extent->get_length()); + auto fresh_mut = fresh_extent->get_mutable(); + std::memcpy(fresh_mut.get_write(), extent->get_read(), get_length()); + NodeExtentRef to_discard = extent; + + extent = fresh_extent; + node_stage = node_stage_t(reinterpret_cast<const FieldType*>(extent->get_read()), + get_length()); + state = nextent_state_t::FRESH; + mut.emplace(fresh_mut); + recorder = nullptr; + + return c.nm.retire_extent(c.t, to_discard + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::input_output_error::handle( + [FNAME, c, l_to_discard = to_discard->get_laddr(), + l_fresh = fresh_extent->get_laddr()] { + SUBERRORT(seastore_onode, + "EIO during retire -- to_disgard={:x}, fresh={:x}", + c.t, l_to_discard, l_fresh); + ceph_abort("fatal error"); + }), + crimson::ct_error::enoent::handle( + [FNAME, c, l_to_discard = to_discard->get_laddr(), + l_fresh = fresh_extent->get_laddr()] { + SUBERRORT(seastore_onode, + "ENOENT during retire -- to_disgard={:x}, fresh={:x}", + c.t, l_to_discard, l_fresh); + ceph_abort("fatal error"); + }) + ); + }).si_then([this, c] { + boost::ignore_unused(c); // avoid clang warning; + assert(!c.t.is_conflicted()); + return *mut; + }); + } + + eagain_ifuture<> retire(context_t c) { + LOG_PREFIX(OTree::Extent::retire); + assert(!is_retired()); + auto addr = extent->get_laddr(); + return c.nm.retire_extent(c.t, std::move(extent) + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::input_output_error::handle( + [FNAME, c, addr] { + SUBERRORT(seastore_onode, "EIO -- addr={:x}", c.t, addr); + ceph_abort("fatal error"); + }), + crimson::ct_error::enoent::handle( + [FNAME, c, addr] { + SUBERRORT(seastore_onode, "ENOENT -- addr={:x}", c.t, addr); + ceph_abort("fatal error"); + }) +#ifndef NDEBUG + ).si_then([c] { + assert(!c.t.is_conflicted()); + } +#endif + ); + } + + private: + NodeExtentRef extent; + node_stage_t node_stage; + nextent_state_t state; + std::optional<NodeExtentMutable> mut; + // owned by extent + recorder_t* recorder; + +#ifndef NDEBUG + // verify record replay using a different memory block + TestReplayExtent::Ref test_extent; + recorder_t* test_recorder; +#endif +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.cc new file mode 100644 index 000000000..8e6f16a74 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.cc @@ -0,0 +1,32 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "node_extent_manager.h" + +#include "node_extent_manager/dummy.h" +#include "node_extent_manager/seastore.h" + +namespace crimson::os::seastore::onode { + +NodeExtentManagerURef NodeExtentManager::create_dummy(bool is_sync) +{ + if (is_sync) { + return NodeExtentManagerURef(new DummyNodeExtentManager<true>()); + } else { + return NodeExtentManagerURef(new DummyNodeExtentManager<false>()); + } +} + +NodeExtentManagerURef NodeExtentManager::create_seastore( + TransactionManager &tm, laddr_t min_laddr, double p_eagain) +{ + if (p_eagain == 0.0) { + return NodeExtentManagerURef( + new SeastoreNodeExtentManager<false>(tm, min_laddr, p_eagain)); + } else { + return NodeExtentManagerURef( + new SeastoreNodeExtentManager<true>(tm, min_laddr, p_eagain)); + } +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.h new file mode 100644 index 000000000..f8772929c --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.h @@ -0,0 +1,105 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "crimson/common/type_helpers.h" +#include "crimson/os/seastore/cached_extent.h" +#include "crimson/os/seastore/transaction_manager.h" + +#include "fwd.h" +#include "node_extent_mutable.h" +#include "node_types.h" +#include "stages/node_stage_layout.h" +#include "super.h" + +/** + * node_extent_manager.h + * + * Contains general interfaces for different backends (Dummy and Seastore). + */ + +namespace crimson::os::seastore::onode { + +using crimson::os::seastore::LogicalCachedExtent; +class NodeExtent : public LogicalCachedExtent { + public: + virtual ~NodeExtent() = default; + const node_header_t& get_header() const { + return *reinterpret_cast<const node_header_t*>(get_read()); + } + const char* get_read() const { + return get_bptr().c_str(); + } + NodeExtentMutable get_mutable() { + assert(is_mutable()); + return do_get_mutable(); + } + + virtual DeltaRecorder* get_recorder() const = 0; + virtual NodeExtentRef mutate(context_t, DeltaRecorderURef&&) = 0; + + protected: + template <typename... T> + NodeExtent(T&&... t) : LogicalCachedExtent(std::forward<T>(t)...) {} + + NodeExtentMutable do_get_mutable() { + return NodeExtentMutable(get_bptr().c_str(), get_length()); + } + + std::ostream& print_detail_l(std::ostream& out) const final { + return out << ", fltree_header=" << get_header(); + } + + /** + * Abstracted interfaces to implement: + * - CacheExtent::duplicate_for_write() -> CachedExtentRef + * - CacheExtent::get_type() -> extent_types_t + * - CacheExtent::get_delta() -> ceph::bufferlist + * - LogicalCachedExtent::apply_delta(const ceph::bufferlist) -> void + */ +}; + +using crimson::os::seastore::TransactionManager; +class NodeExtentManager { + using base_iertr = TransactionManager::base_iertr; + public: + virtual ~NodeExtentManager() = default; + + virtual bool is_read_isolated() const = 0; + + using read_iertr = base_iertr::extend< + crimson::ct_error::invarg, + crimson::ct_error::enoent, + crimson::ct_error::erange>; + virtual read_iertr::future<NodeExtentRef> read_extent( + Transaction&, laddr_t) = 0; + + using alloc_iertr = base_iertr; + virtual alloc_iertr::future<NodeExtentRef> alloc_extent( + Transaction&, laddr_t hint, extent_len_t) = 0; + + using retire_iertr = base_iertr::extend< + crimson::ct_error::enoent>; + virtual retire_iertr::future<> retire_extent( + Transaction&, NodeExtentRef) = 0; + + using getsuper_iertr = base_iertr; + virtual getsuper_iertr::future<Super::URef> get_super( + Transaction&, RootNodeTracker&) = 0; + + virtual std::ostream& print(std::ostream& os) const = 0; + + static NodeExtentManagerURef create_dummy(bool is_sync); + static NodeExtentManagerURef create_seastore( + TransactionManager &tm, laddr_t min_laddr = L_ADDR_MIN, double p_eagain = 0.0); +}; +inline std::ostream& operator<<(std::ostream& os, const NodeExtentManager& nm) { + return nm.print(os); +} + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::onode::NodeExtent> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/dummy.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/dummy.h new file mode 100644 index 000000000..24df8b548 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/dummy.h @@ -0,0 +1,196 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <chrono> +#include <seastar/core/sleep.hh> + +#include "include/buffer_raw.h" +#include "crimson/os/seastore/logging.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.h" + +/** + * dummy.h + * + * Dummy backend implementations for test purposes. + */ + +namespace crimson::os::seastore::onode { + +class DummySuper final: public Super { + public: + DummySuper(Transaction& t, RootNodeTracker& tracker, laddr_t* p_root_laddr) + : Super(t, tracker), p_root_laddr{p_root_laddr} {} + ~DummySuper() override = default; + protected: + laddr_t get_root_laddr() const override { return *p_root_laddr; } + void write_root_laddr(context_t c, laddr_t addr) override { + LOG_PREFIX(OTree::Dummy); + SUBDEBUGT(seastore_onode, "update root {:#x} ...", c.t, addr); + *p_root_laddr = addr; + } + private: + laddr_t* p_root_laddr; +}; + +class DummyNodeExtent final: public NodeExtent { + public: + DummyNodeExtent(ceph::bufferptr &&ptr) : NodeExtent(std::move(ptr)) { + state = extent_state_t::INITIAL_WRITE_PENDING; + } + DummyNodeExtent(const DummyNodeExtent& other) = delete; + ~DummyNodeExtent() override = default; + + void retire() { + assert(state == extent_state_t::INITIAL_WRITE_PENDING); + state = extent_state_t::INVALID; + bufferptr empty_bptr; + get_bptr().swap(empty_bptr); + } + + protected: + NodeExtentRef mutate(context_t, DeltaRecorderURef&&) override { + ceph_abort("impossible path"); } + DeltaRecorder* get_recorder() const override { + return nullptr; } + CachedExtentRef duplicate_for_write(Transaction&) override { + ceph_abort("impossible path"); } + extent_types_t get_type() const override { + return extent_types_t::TEST_BLOCK; } + ceph::bufferlist get_delta() override { + ceph_abort("impossible path"); } + void apply_delta(const ceph::bufferlist&) override { + ceph_abort("impossible path"); } +}; + +template <bool SYNC> +class DummyNodeExtentManager final: public NodeExtentManager { + static constexpr size_t ALIGNMENT = 4096; + public: + ~DummyNodeExtentManager() override = default; + std::size_t size() const { return allocate_map.size(); } + + protected: + bool is_read_isolated() const override { return false; } + + read_iertr::future<NodeExtentRef> read_extent( + Transaction& t, laddr_t addr) override { + SUBTRACET(seastore_onode, "reading at {:#x} ...", t, addr); + if constexpr (SYNC) { + return read_extent_sync(t, addr); + } else { + using namespace std::chrono_literals; + return seastar::sleep(1us).then([this, &t, addr] { + return read_extent_sync(t, addr); + }); + } + } + + alloc_iertr::future<NodeExtentRef> alloc_extent( + Transaction& t, laddr_t hint, extent_len_t len) override { + SUBTRACET(seastore_onode, "allocating {}B with hint {:#x} ...", t, len, hint); + if constexpr (SYNC) { + return alloc_extent_sync(t, len); + } else { + using namespace std::chrono_literals; + return seastar::sleep(1us).then([this, &t, len] { + return alloc_extent_sync(t, len); + }); + } + } + + retire_iertr::future<> retire_extent( + Transaction& t, NodeExtentRef extent) override { + SUBTRACET(seastore_onode, + "retiring {}B at {:#x} -- {} ...", + t, extent->get_length(), extent->get_laddr(), *extent); + if constexpr (SYNC) { + return retire_extent_sync(t, extent); + } else { + using namespace std::chrono_literals; + return seastar::sleep(1us).then([this, &t, extent] { + return retire_extent_sync(t, extent); + }); + } + } + + getsuper_iertr::future<Super::URef> get_super( + Transaction& t, RootNodeTracker& tracker) override { + SUBTRACET(seastore_onode, "get root ...", t); + if constexpr (SYNC) { + return get_super_sync(t, tracker); + } else { + using namespace std::chrono_literals; + return seastar::sleep(1us).then([this, &t, &tracker] { + return get_super_sync(t, tracker); + }); + } + } + + std::ostream& print(std::ostream& os) const override { + return os << "DummyNodeExtentManager(sync=" << SYNC << ")"; + } + + private: + read_iertr::future<NodeExtentRef> read_extent_sync( + Transaction& t, laddr_t addr) { + auto iter = allocate_map.find(addr); + assert(iter != allocate_map.end()); + auto extent = iter->second; + SUBTRACET(seastore_onode, + "read {}B at {:#x} -- {}", + t, extent->get_length(), extent->get_laddr(), *extent); + assert(extent->get_laddr() == addr); + return read_iertr::make_ready_future<NodeExtentRef>(extent); + } + + alloc_iertr::future<NodeExtentRef> alloc_extent_sync( + Transaction& t, extent_len_t len) { + assert(len % ALIGNMENT == 0); + auto r = ceph::buffer::create_aligned(len, ALIGNMENT); + auto addr = reinterpret_cast<laddr_t>(r->get_data()); + auto bp = ceph::bufferptr(std::move(r)); + auto extent = Ref<DummyNodeExtent>(new DummyNodeExtent(std::move(bp))); + extent->set_laddr(addr); + assert(allocate_map.find(extent->get_laddr()) == allocate_map.end()); + allocate_map.insert({extent->get_laddr(), extent}); + SUBDEBUGT(seastore_onode, + "allocated {}B at {:#x} -- {}", + t, extent->get_length(), extent->get_laddr(), *extent); + assert(extent->get_length() == len); + return alloc_iertr::make_ready_future<NodeExtentRef>(extent); + } + + retire_iertr::future<> retire_extent_sync( + Transaction& t, NodeExtentRef _extent) { + auto& extent = static_cast<DummyNodeExtent&>(*_extent.get()); + auto addr = extent.get_laddr(); + auto len = extent.get_length(); + extent.retire(); + auto iter = allocate_map.find(addr); + assert(iter != allocate_map.end()); + allocate_map.erase(iter); + SUBDEBUGT(seastore_onode, "retired {}B at {:#x}", t, len, addr); + return retire_iertr::now(); + } + + getsuper_iertr::future<Super::URef> get_super_sync( + Transaction& t, RootNodeTracker& tracker) { + SUBTRACET(seastore_onode, "got root {:#x}", t, root_laddr); + return getsuper_iertr::make_ready_future<Super::URef>( + Super::URef(new DummySuper(t, tracker, &root_laddr))); + } + + static LOG_PREFIX(OTree::Dummy); + + std::map<laddr_t, Ref<DummyNodeExtent>> allocate_map; + laddr_t root_laddr = L_ADDR_NULL; +}; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::onode::DummyNodeExtent> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/seastore.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/seastore.cc new file mode 100644 index 000000000..3b52c5dc0 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/seastore.cc @@ -0,0 +1,90 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "seastore.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_accessor.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage_layout.h" + +namespace { +LOG_PREFIX(OTree::Seastore); +} + +SET_SUBSYS(seastore_onode); + +namespace crimson::os::seastore::onode { + +static DeltaRecorderURef create_replay_recorder( + node_type_t node_type, field_type_t field_type) +{ + if (node_type == node_type_t::LEAF) { + if (field_type == field_type_t::N0) { + return DeltaRecorderT<node_fields_0_t, node_type_t::LEAF>::create_for_replay(); + } else if (field_type == field_type_t::N1) { + return DeltaRecorderT<node_fields_1_t, node_type_t::LEAF>::create_for_replay(); + } else if (field_type == field_type_t::N2) { + return DeltaRecorderT<node_fields_2_t, node_type_t::LEAF>::create_for_replay(); + } else if (field_type == field_type_t::N3) { + return DeltaRecorderT<leaf_fields_3_t, node_type_t::LEAF>::create_for_replay(); + } else { + ceph_abort("impossible path"); + } + } else if (node_type == node_type_t::INTERNAL) { + if (field_type == field_type_t::N0) { + return DeltaRecorderT<node_fields_0_t, node_type_t::INTERNAL>::create_for_replay(); + } else if (field_type == field_type_t::N1) { + return DeltaRecorderT<node_fields_1_t, node_type_t::INTERNAL>::create_for_replay(); + } else if (field_type == field_type_t::N2) { + return DeltaRecorderT<node_fields_2_t, node_type_t::INTERNAL>::create_for_replay(); + } else if (field_type == field_type_t::N3) { + return DeltaRecorderT<internal_fields_3_t, node_type_t::INTERNAL>::create_for_replay(); + } else { + ceph_abort("impossible path"); + } + } else { + ceph_abort("impossible path"); + } +} + +NodeExtentRef SeastoreNodeExtent::mutate( + context_t c, DeltaRecorderURef&& _recorder) +{ + DEBUGT("mutate {} ...", c.t, *this); + auto p_handle = static_cast<TransactionManagerHandle*>(&c.nm); + auto extent = p_handle->tm.get_mutable_extent(c.t, this); + auto ret = extent->cast<SeastoreNodeExtent>(); + // A replayed extent may already have an empty recorder, we discard it for + // simplicity. + assert(!ret->recorder || ret->recorder->is_empty()); + ret->recorder = std::move(_recorder); + return ret; +} + +void SeastoreNodeExtent::apply_delta(const ceph::bufferlist& bl) +{ + DEBUG("replay {} ...", *this); + if (!recorder) { + auto header = get_header(); + auto field_type = header.get_field_type(); + if (!field_type.has_value()) { + ERROR("replay got invalid node -- {}", *this); + ceph_abort("fatal error"); + } + auto node_type = header.get_node_type(); + recorder = create_replay_recorder(node_type, *field_type); + } else { +#ifndef NDEBUG + auto header = get_header(); + assert(recorder->node_type() == header.get_node_type()); + assert(recorder->field_type() == *header.get_field_type()); +#endif + } + auto mut = do_get_mutable(); + auto p = bl.cbegin(); + while (p != bl.end()) { + recorder->apply_delta(p, mut, *this); + } + DEBUG("relay done!"); +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/seastore.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/seastore.h new file mode 100644 index 000000000..f7cfa8c21 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/seastore.h @@ -0,0 +1,223 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <random> + +#include "crimson/os/seastore/logging.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/node_delta_recorder.h" + +/** + * seastore.h + * + * Seastore backend implementations. + */ + +namespace crimson::os::seastore::onode { + +class SeastoreSuper final: public Super { + public: + SeastoreSuper(Transaction& t, RootNodeTracker& tracker, + laddr_t root_addr, TransactionManager& tm) + : Super(t, tracker), root_addr{root_addr}, tm{tm} {} + ~SeastoreSuper() override = default; + protected: + laddr_t get_root_laddr() const override { + return root_addr; + } + void write_root_laddr(context_t c, laddr_t addr) override { + LOG_PREFIX(OTree::Seastore); + SUBDEBUGT(seastore_onode, "update root {:#x} ...", c.t, addr); + root_addr = addr; + tm.write_onode_root(c.t, addr); + } + private: + laddr_t root_addr; + TransactionManager &tm; +}; + +class SeastoreNodeExtent final: public NodeExtent { + public: + SeastoreNodeExtent(ceph::bufferptr &&ptr) + : NodeExtent(std::move(ptr)) {} + SeastoreNodeExtent(const SeastoreNodeExtent& other) + : NodeExtent(other) {} + ~SeastoreNodeExtent() override = default; + + constexpr static extent_types_t TYPE = extent_types_t::ONODE_BLOCK_STAGED; + extent_types_t get_type() const override { + return TYPE; + } + + protected: + NodeExtentRef mutate(context_t, DeltaRecorderURef&&) override; + + DeltaRecorder* get_recorder() const override { + return recorder.get(); + } + + CachedExtentRef duplicate_for_write(Transaction&) override { + return CachedExtentRef(new SeastoreNodeExtent(*this)); + } + ceph::bufferlist get_delta() override { + assert(recorder); + return recorder->get_delta(); + } + void apply_delta(const ceph::bufferlist&) override; + + private: + DeltaRecorderURef recorder; +}; + +class TransactionManagerHandle : public NodeExtentManager { + public: + TransactionManagerHandle(TransactionManager &tm) : tm{tm} {} + TransactionManager &tm; +}; + +template <bool INJECT_EAGAIN=false> +class SeastoreNodeExtentManager final: public TransactionManagerHandle { + public: + SeastoreNodeExtentManager( + TransactionManager &tm, laddr_t min, double p_eagain) + : TransactionManagerHandle(tm), addr_min{min}, p_eagain{p_eagain} { + if constexpr (INJECT_EAGAIN) { + assert(p_eagain > 0.0 && p_eagain < 1.0); + } else { + assert(p_eagain == 0.0); + } + } + + ~SeastoreNodeExtentManager() override = default; + + void set_generate_eagain(bool enable) { + generate_eagain = enable; + } + + protected: + bool is_read_isolated() const override { return true; } + + read_iertr::future<NodeExtentRef> read_extent( + Transaction& t, laddr_t addr) override { + SUBTRACET(seastore_onode, "reading at {:#x} ...", t, addr); + if constexpr (INJECT_EAGAIN) { + if (trigger_eagain()) { + SUBDEBUGT(seastore_onode, "reading at {:#x}: trigger eagain", t, addr); + t.test_set_conflict(); + return read_iertr::make_ready_future<NodeExtentRef>(); + } + } + return tm.read_extent<SeastoreNodeExtent>(t, addr + ).si_then([addr, &t](auto&& e) -> read_iertr::future<NodeExtentRef> { + SUBTRACET(seastore_onode, + "read {}B at {:#x} -- {}", + t, e->get_length(), e->get_laddr(), *e); + assert(e->get_laddr() == addr); + std::ignore = addr; + return read_iertr::make_ready_future<NodeExtentRef>(e); + }); + } + + alloc_iertr::future<NodeExtentRef> alloc_extent( + Transaction& t, laddr_t hint, extent_len_t len) override { + SUBTRACET(seastore_onode, "allocating {}B with hint {:#x} ...", t, len, hint); + if constexpr (INJECT_EAGAIN) { + if (trigger_eagain()) { + SUBDEBUGT(seastore_onode, "allocating {}B: trigger eagain", t, len); + t.test_set_conflict(); + return alloc_iertr::make_ready_future<NodeExtentRef>(); + } + } + return tm.alloc_extent<SeastoreNodeExtent>(t, hint, len + ).si_then([len, &t](auto extent) { + SUBDEBUGT(seastore_onode, + "allocated {}B at {:#x} -- {}", + t, extent->get_length(), extent->get_laddr(), *extent); + if (!extent->is_initial_pending()) { + SUBERRORT(seastore_onode, + "allocated {}B but got invalid extent: {}", + t, len, *extent); + ceph_abort("fatal error"); + } + assert(extent->get_length() == len); + std::ignore = len; + return NodeExtentRef(extent); + }); + } + + retire_iertr::future<> retire_extent( + Transaction& t, NodeExtentRef _extent) override { + LogicalCachedExtentRef extent = _extent; + auto addr = extent->get_laddr(); + auto len = extent->get_length(); + SUBDEBUGT(seastore_onode, + "retiring {}B at {:#x} -- {} ...", + t, len, addr, *extent); + if constexpr (INJECT_EAGAIN) { + if (trigger_eagain()) { + SUBDEBUGT(seastore_onode, + "retiring {}B at {:#x} -- {} : trigger eagain", + t, len, addr, *extent); + t.test_set_conflict(); + return retire_iertr::now(); + } + } + return tm.dec_ref(t, extent).si_then([addr, len, &t] (unsigned cnt) { + assert(cnt == 0); + SUBTRACET(seastore_onode, "retired {}B at {:#x} ...", t, len, addr); + }); + } + + getsuper_iertr::future<Super::URef> get_super( + Transaction& t, RootNodeTracker& tracker) override { + SUBTRACET(seastore_onode, "get root ...", t); + if constexpr (INJECT_EAGAIN) { + if (trigger_eagain()) { + SUBDEBUGT(seastore_onode, "get root: trigger eagain", t); + t.test_set_conflict(); + return getsuper_iertr::make_ready_future<Super::URef>(); + } + } + return tm.read_onode_root(t).si_then([this, &t, &tracker](auto root_addr) { + SUBTRACET(seastore_onode, "got root {:#x}", t, root_addr); + return Super::URef(new SeastoreSuper(t, tracker, root_addr, tm)); + }); + } + + std::ostream& print(std::ostream& os) const override { + os << "SeastoreNodeExtentManager"; + if constexpr (INJECT_EAGAIN) { + os << "(p_eagain=" << p_eagain << ")"; + } + return os; + } + + private: + static LOG_PREFIX(OTree::Seastore); + + const laddr_t addr_min; + + // XXX: conditional members by INJECT_EAGAIN + bool trigger_eagain() { + if (generate_eagain) { + double dice = rd(); + assert(rd.min() == 0); + dice /= rd.max(); + return dice <= p_eagain; + } else { + return false; + } + } + bool generate_eagain = true; + std::random_device rd; + double p_eagain; +}; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::onode::SeastoreNodeExtent> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/test_replay.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/test_replay.h new file mode 100644 index 000000000..bce74e381 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager/test_replay.h @@ -0,0 +1,67 @@ +// -*- 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/onode_manager/staged-fltree/node_delta_recorder.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_manager.h" + +/** test_replay.h + * + * A special version of NodeExtent to help verify delta encode, decode and + * replay in recorder_t under debug build. + */ + +namespace crimson::os::seastore::onode { + +class TestReplayExtent final: public NodeExtent { + public: + using Ref = crimson::os::seastore::TCachedExtentRef<TestReplayExtent>; + + void prepare_replay(NodeExtentRef from_extent) { + assert(get_length() == from_extent->get_length()); + auto mut = do_get_mutable(); + std::memcpy(mut.get_write(), from_extent->get_read(), get_length()); + } + + void replay_and_verify(NodeExtentRef replayed_extent) { + assert(get_length() == replayed_extent->get_length()); + auto mut = do_get_mutable(); + auto bl = recorder->get_delta(); + assert(bl.length()); + auto p = bl.cbegin(); + recorder->apply_delta(p, mut, *this); + assert(p == bl.end()); + auto cmp = std::memcmp(get_read(), replayed_extent->get_read(), get_length()); + ceph_assert(cmp == 0 && "replay mismatch!"); + } + + static Ref create(extent_len_t length, DeltaRecorderURef&& recorder) { + auto r = ceph::buffer::create_aligned(length, 4096); + auto bp = ceph::bufferptr(std::move(r)); + return new TestReplayExtent(std::move(bp), std::move(recorder)); + } + + protected: + NodeExtentRef mutate(context_t, DeltaRecorderURef&&) override { + ceph_abort("impossible path"); } + DeltaRecorder* get_recorder() const override { + ceph_abort("impossible path"); } + CachedExtentRef duplicate_for_write(Transaction&) override { + ceph_abort("impossible path"); } + extent_types_t get_type() const override { + return extent_types_t::TEST_BLOCK; } + ceph::bufferlist get_delta() override { + ceph_abort("impossible path"); } + void apply_delta(const ceph::bufferlist&) override { + ceph_abort("impossible path"); } + + private: + TestReplayExtent(ceph::bufferptr&& ptr, DeltaRecorderURef&& recorder) + : NodeExtent(std::move(ptr)), recorder(std::move(recorder)) { + state = extent_state_t::MUTATION_PENDING; + } + DeltaRecorderURef recorder; +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_mutable.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_mutable.h new file mode 100644 index 000000000..6f92ca9ed --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_extent_mutable.h @@ -0,0 +1,113 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include <cstring> + +#include "fwd.h" + +#pragma once + +namespace crimson::os::seastore::onode { + +/** + * NodeExtentMutable + * + * A thin wrapper of NodeExtent to make sure that only the newly allocated + * or the duplicated NodeExtent is mutable, and the memory modifications are + * safe within the extent range. + */ +class NodeExtentMutable { + public: + void copy_in_absolute(void* dst, const void* src, extent_len_t len) { + assert(is_safe(dst, len)); + std::memcpy(dst, src, len); + } + template <typename T> + void copy_in_absolute(void* dst, const T& src) { + copy_in_absolute(dst, &src, sizeof(T)); + } + + const void* copy_in_relative( + extent_len_t dst_offset, const void* src, extent_len_t len) { + auto dst = get_write() + dst_offset; + copy_in_absolute(dst, src, len); + return dst; + } + template <typename T> + const T* copy_in_relative( + extent_len_t dst_offset, const T& src) { + auto dst = copy_in_relative(dst_offset, &src, sizeof(T)); + return static_cast<const T*>(dst); + } + + void shift_absolute(const void* src, extent_len_t len, int offset) { + assert(is_safe(src, len)); + char* to = (char*)src + offset; + assert(is_safe(to, len)); + if (len != 0) { + std::memmove(to, src, len); + } + } + void shift_relative(extent_len_t src_offset, extent_len_t len, int offset) { + shift_absolute(get_write() + src_offset, len, offset); + } + + void set_absolute(void* dst, int value, extent_len_t len) { + assert(is_safe(dst, len)); + std::memset(dst, value, len); + } + void set_relative(extent_len_t dst_offset, int value, extent_len_t len) { + auto dst = get_write() + dst_offset; + set_absolute(dst, value, len); + } + + template <typename T> + void validate_inplace_update(const T& updated) { + assert(is_safe(&updated, sizeof(T))); + } + + const char* get_read() const { return p_start; } + char* get_write() { return p_start; } + extent_len_t get_length() const { +#ifndef NDEBUG + if (node_offset == 0) { + assert(is_valid_node_size(length)); + } +#endif + return length; + } + node_offset_t get_node_offset() const { return node_offset; } + + NodeExtentMutable get_mutable_absolute(const void* dst, node_offset_t len) const { + assert(node_offset == 0); + assert(is_safe(dst, len)); + assert((const char*)dst != get_read()); + auto ret = *this; + node_offset_t offset = (const char*)dst - get_read(); + assert(offset != 0); + ret.p_start += offset; + ret.length = len; + ret.node_offset = offset; + return ret; + } + NodeExtentMutable get_mutable_relative( + node_offset_t offset, node_offset_t len) const { + return get_mutable_absolute(get_read() + offset, len); + } + + private: + NodeExtentMutable(char* p_start, extent_len_t length) + : p_start{p_start}, length{length} {} + bool is_safe(const void* src, extent_len_t len) const { + return ((const char*)src >= p_start) && + ((const char*)src + len <= p_start + length); + } + + char* p_start; + extent_len_t length; + node_offset_t node_offset = 0; + + friend class NodeExtent; +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_impl.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/node_impl.cc new file mode 100644 index 000000000..5db0f83dd --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_impl.cc @@ -0,0 +1,80 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "node_impl.h" +#include "node_layout.h" + +namespace crimson::os::seastore::onode { + +#ifdef UNIT_TESTS_BUILT +last_split_info_t last_split = {}; +#endif + +// XXX: branchless allocation +eagain_ifuture<InternalNodeImpl::fresh_impl_t> +InternalNodeImpl::allocate( + context_t c, laddr_t hint, field_type_t type, bool is_level_tail, level_t level) +{ + if (type == field_type_t::N0) { + return InternalNode0::allocate(c, hint, is_level_tail, level); + } else if (type == field_type_t::N1) { + return InternalNode1::allocate(c, hint, is_level_tail, level); + } else if (type == field_type_t::N2) { + return InternalNode2::allocate(c, hint, is_level_tail, level); + } else if (type == field_type_t::N3) { + return InternalNode3::allocate(c, hint, is_level_tail, level); + } else { + ceph_abort("impossible path"); + } +} + +eagain_ifuture<LeafNodeImpl::fresh_impl_t> +LeafNodeImpl::allocate( + context_t c, laddr_t hint, field_type_t type, bool is_level_tail) +{ + if (type == field_type_t::N0) { + return LeafNode0::allocate(c, hint, is_level_tail, 0); + } else if (type == field_type_t::N1) { + return LeafNode1::allocate(c, hint, is_level_tail, 0); + } else if (type == field_type_t::N2) { + return LeafNode2::allocate(c, hint, is_level_tail, 0); + } else if (type == field_type_t::N3) { + return LeafNode3::allocate(c, hint, is_level_tail, 0); + } else { + ceph_abort("impossible path"); + } +} + +InternalNodeImplURef InternalNodeImpl::load( + NodeExtentRef extent, field_type_t type) +{ + if (type == field_type_t::N0) { + return InternalNode0::load(extent); + } else if (type == field_type_t::N1) { + return InternalNode1::load(extent); + } else if (type == field_type_t::N2) { + return InternalNode2::load(extent); + } else if (type == field_type_t::N3) { + return InternalNode3::load(extent); + } else { + ceph_abort("impossible path"); + } +} + +LeafNodeImplURef LeafNodeImpl::load( + NodeExtentRef extent, field_type_t type) +{ + if (type == field_type_t::N0) { + return LeafNode0::load(extent); + } else if (type == field_type_t::N1) { + return LeafNode1::load(extent); + } else if (type == field_type_t::N2) { + return LeafNode2::load(extent); + } else if (type == field_type_t::N3) { + return LeafNode3::load(extent); + } else { + ceph_abort("impossible path"); + } +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_impl.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_impl.h new file mode 100644 index 000000000..cf452618b --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_impl.h @@ -0,0 +1,270 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <ostream> + +#include "node_extent_mutable.h" +#include "node_types.h" +#include "stages/stage_types.h" + +namespace crimson::os::seastore::onode { + +#ifdef UNIT_TESTS_BUILT +enum class InsertType { BEGIN, LAST, MID }; +struct split_expectation_t { + match_stage_t split_stage; + match_stage_t insert_stage; + bool is_insert_left; + InsertType insert_type; +}; +struct last_split_info_t { + search_position_t split_pos; + match_stage_t insert_stage; + bool is_insert_left; + InsertType insert_type; + bool match(const split_expectation_t& e) const { + match_stage_t split_stage; + if (split_pos.nxt.nxt.index == 0) { + if (split_pos.nxt.index == 0) { + split_stage = 2; + } else { + split_stage = 1; + } + } else { + split_stage = 0; + } + return split_stage == e.split_stage && + insert_stage == e.insert_stage && + is_insert_left == e.is_insert_left && + insert_type == e.insert_type; + } + bool match_split_pos(const search_position_t& pos) const { + return split_pos == pos; + } +}; +extern last_split_info_t last_split; +#endif + +struct key_hobj_t; +struct key_view_t; +class NodeExtentMutable; + +/** + * NodeImpl + * + * Hides type specific node layout implementations for Node. + */ +class NodeImpl { + public: + virtual ~NodeImpl() = default; + + virtual node_type_t node_type() const = 0; + virtual field_type_t field_type() const = 0; + virtual laddr_t laddr() const = 0; + virtual const char* read() const = 0; + virtual extent_len_t get_node_size() const = 0; + virtual nextent_state_t get_extent_state() const = 0; + virtual void prepare_mutate(context_t) = 0; + virtual bool is_level_tail() const = 0; + + /* Invariants for num_keys and num_values: + * - for leaf node and non-tail internal node, num_keys == num_values; + * - for tail internal node, num_keys + 1 == num_values; + * - all node must have at least 1 value, except the root leaf node; + * - the root internal node must have more than 1 values; + */ + virtual void validate_non_empty() const = 0; + virtual bool is_keys_empty() const = 0; + // under the assumption that node is not empty + virtual bool has_single_value() const = 0; + + virtual level_t level() const = 0; + virtual node_offset_t free_size() const = 0; + virtual extent_len_t total_size() const = 0; + virtual bool is_extent_retired() const = 0; + virtual std::optional<key_view_t> get_pivot_index() const = 0; + virtual bool is_size_underflow() const = 0; + + virtual std::tuple<match_stage_t, search_position_t> erase(const search_position_t&) = 0; + virtual std::tuple<match_stage_t, std::size_t> evaluate_merge(NodeImpl&) = 0; + virtual search_position_t merge(NodeExtentMutable&, NodeImpl&, match_stage_t, extent_len_t) = 0; + virtual eagain_ifuture<NodeExtentMutable> rebuild_extent(context_t) = 0; + virtual eagain_ifuture<> retire_extent(context_t) = 0; + virtual search_position_t make_tail() = 0; + + virtual node_stats_t get_stats() const = 0; + virtual std::ostream& dump(std::ostream&) const = 0; + virtual std::ostream& dump_brief(std::ostream&) const = 0; + virtual const std::string& get_name() const = 0; + virtual void validate_layout() const = 0; + + virtual void test_copy_to(NodeExtentMutable&) const = 0; + virtual void test_set_tail(NodeExtentMutable&) = 0; + + protected: + NodeImpl() = default; +}; + +/** + * InternalNodeImpl + * + * Hides type specific node layout implementations for InternalNode. + */ +class InternalNodeImpl : public NodeImpl { + public: + struct internal_marker_t {}; + virtual ~InternalNodeImpl() = default; + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual void get_slot(const search_position_t&, // IN + key_view_t* = nullptr, // OUT + const laddr_packed_t** = nullptr) const { // OUT + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual void get_prev_slot(search_position_t&, // IN&OUT + key_view_t* = nullptr, // OUT + const laddr_packed_t** = nullptr) const { // OUT + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual void get_next_slot(search_position_t&, // IN&OUT + key_view_t* = nullptr, // OUT + const laddr_packed_t** = nullptr) const { // OUT + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual void get_largest_slot(search_position_t* = nullptr, // OUT + key_view_t* = nullptr, // OUT + const laddr_packed_t** = nullptr) const { // OUT + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual lookup_result_t<node_type_t::INTERNAL> lower_bound( + const key_hobj_t&, MatchHistory&, + key_view_t* = nullptr, internal_marker_t = {}) const { + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual const laddr_packed_t* insert( + const key_view_t&, const laddr_t&, search_position_t&, match_stage_t&, node_offset_t&) { + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual std::tuple<search_position_t, bool, const laddr_packed_t*> split_insert( + NodeExtentMutable&, NodeImpl&, const key_view_t&, const laddr_t&, + search_position_t&, match_stage_t&, node_offset_t&) { + ceph_abort("impossible path"); + } + + virtual const laddr_packed_t* get_tail_value() const = 0; + + virtual void replace_child_addr(const search_position_t&, laddr_t dst, laddr_t src) = 0; + + virtual std::tuple<match_stage_t, node_offset_t> evaluate_insert( + const key_view_t&, const laddr_t&, search_position_t&) const = 0; + + struct fresh_impl_t { + InternalNodeImplURef impl; + NodeExtentMutable mut; + std::pair<NodeImplURef, NodeExtentMutable> make_pair() { + return {std::move(impl), mut}; + } + }; + static eagain_ifuture<fresh_impl_t> allocate(context_t, laddr_t, field_type_t, bool, level_t); + + static InternalNodeImplURef load(NodeExtentRef, field_type_t); + + protected: + InternalNodeImpl() = default; +}; + +/** + * LeafNodeImpl + * + * Hides type specific node layout implementations for LeafNode. + */ +class LeafNodeImpl : public NodeImpl { + public: + struct leaf_marker_t {}; + virtual ~LeafNodeImpl() = default; + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual void get_slot(const search_position_t&, // IN + key_view_t* = nullptr, // OUT + const value_header_t** = nullptr) const { // OUT + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual void get_prev_slot(search_position_t&, // IN&OUT + key_view_t* = nullptr, // OUT + const value_header_t** = nullptr) const { // OUT + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual void get_next_slot(search_position_t&, // IN&OUT + key_view_t* = nullptr, // OUT + const value_header_t** = nullptr) const { // OUT + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual void get_largest_slot(search_position_t* = nullptr, // OUT + key_view_t* = nullptr, // OUT + const value_header_t** = nullptr) const { // OUT + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual lookup_result_t<node_type_t::LEAF> lower_bound( + const key_hobj_t&, MatchHistory&, + key_view_t* = nullptr, leaf_marker_t = {}) const { + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual const value_header_t* insert( + const key_hobj_t&, const value_config_t&, search_position_t&, match_stage_t&, node_offset_t&) { + ceph_abort("impossible path"); + } + + #pragma GCC diagnostic ignored "-Woverloaded-virtual" + virtual std::tuple<search_position_t, bool, const value_header_t*> split_insert( + NodeExtentMutable&, NodeImpl&, const key_hobj_t&, const value_config_t&, + search_position_t&, match_stage_t&, node_offset_t&) { + ceph_abort("impossible path"); + } + + virtual std::tuple<match_stage_t, node_offset_t> evaluate_insert( + const key_hobj_t&, const value_config_t&, + const MatchHistory&, match_stat_t, search_position_t&) const = 0; + + virtual std::pair<NodeExtentMutable&, ValueDeltaRecorder*> + prepare_mutate_value_payload(context_t) = 0; + + struct fresh_impl_t { + LeafNodeImplURef impl; + NodeExtentMutable mut; + std::pair<NodeImplURef, NodeExtentMutable> make_pair() { + return {std::move(impl), mut}; + } + }; + static eagain_ifuture<fresh_impl_t> allocate(context_t, laddr_t, field_type_t, bool); + + static LeafNodeImplURef load(NodeExtentRef, field_type_t); + + protected: + LeafNodeImpl() = default; +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_layout.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_layout.h new file mode 100644 index 000000000..783a0c6cc --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_layout.h @@ -0,0 +1,948 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <ostream> +#include <sstream> + +#include "common/likely.h" +#include "crimson/os/seastore/logging.h" + +#include "node_extent_accessor.h" +#include "node_impl.h" +#include "stages/node_stage_layout.h" + +namespace crimson::os::seastore::onode { + +template <node_type_t NODE_TYPE> struct insert_key_type; +template <> struct insert_key_type<node_type_t::INTERNAL> { + static constexpr auto type = KeyT::VIEW; }; +template <> struct insert_key_type<node_type_t::LEAF> { + static constexpr auto type = KeyT::HOBJ; }; + +template <node_type_t NODE_TYPE> struct node_impl_type; +template <> struct node_impl_type<node_type_t::INTERNAL> { + using type = InternalNodeImpl; }; +template <> struct node_impl_type<node_type_t::LEAF> { + using type = LeafNodeImpl; }; + +template <node_type_t NODE_TYPE> struct node_marker_type; +template <> struct node_marker_type<node_type_t::INTERNAL> { + using type = InternalNodeImpl::internal_marker_t; }; +template <> struct node_marker_type<node_type_t::LEAF> { + using type = LeafNodeImpl::leaf_marker_t; }; + +/** + * NodeLayoutT + * + * Contains templated and concrete implementations for both InternalNodeImpl + * and LeafNodeImpl under a specific node layout. + */ +template <typename FieldType, node_type_t NODE_TYPE> +class NodeLayoutT final : public InternalNodeImpl, public LeafNodeImpl { + public: + using URef = std::unique_ptr<NodeLayoutT>; + using extent_t = NodeExtentAccessorT<FieldType, NODE_TYPE>; + using parent_t = typename node_impl_type<NODE_TYPE>::type; + using marker_t = typename node_marker_type<NODE_TYPE>::type; + using node_stage_t = typename extent_t::node_stage_t; + using stage_t = node_to_stage_t<node_stage_t>; + using position_t = typename extent_t::position_t; + using value_input_t = typename extent_t::value_input_t; + using value_t = typename extent_t::value_t; + static constexpr auto FIELD_TYPE = extent_t::FIELD_TYPE; + static constexpr auto KEY_TYPE = insert_key_type<NODE_TYPE>::type; + static constexpr auto STAGE = stage_t::STAGE; + + NodeLayoutT(const NodeLayoutT&) = delete; + NodeLayoutT(NodeLayoutT&&) = delete; + NodeLayoutT& operator=(const NodeLayoutT&) = delete; + NodeLayoutT& operator=(NodeLayoutT&&) = delete; + ~NodeLayoutT() override = default; + + static URef load(NodeExtentRef extent) { + std::unique_ptr<NodeLayoutT> ret(new NodeLayoutT(extent)); + return ret; + } + + static eagain_ifuture<typename parent_t::fresh_impl_t> allocate( + context_t c, laddr_t hint, bool is_level_tail, level_t level) { + LOG_PREFIX(OTree::Layout::allocate); + extent_len_t extent_size; + if constexpr (NODE_TYPE == node_type_t::LEAF) { + extent_size = c.vb.get_leaf_node_size(); + } else { + extent_size = c.vb.get_internal_node_size(); + } + return c.nm.alloc_extent(c.t, hint, extent_size + ).handle_error_interruptible( + eagain_iertr::pass_further{}, + crimson::ct_error::input_output_error::handle( + [FNAME, c, extent_size, is_level_tail, level] { + SUBERRORT(seastore_onode, + "EIO -- extent_size={}, is_level_tail={}, level={}", + c.t, extent_size, is_level_tail, level); + ceph_abort("fatal error"); + }) + ).si_then([is_level_tail, level](auto extent) { + assert(extent); + assert(extent->is_initial_pending()); + auto mut = extent->get_mutable(); + node_stage_t::bootstrap_extent( + mut, FIELD_TYPE, NODE_TYPE, is_level_tail, level); + return typename parent_t::fresh_impl_t{ + std::unique_ptr<parent_t>(new NodeLayoutT(extent)), mut}; + }); + } + + protected: + /* + * NodeImpl + */ + node_type_t node_type() const override { return NODE_TYPE; } + field_type_t field_type() const override { return FIELD_TYPE; } + laddr_t laddr() const override { return extent.get_laddr(); } + const char* read() const override { return extent.read().p_start(); } + extent_len_t get_node_size() const override { return extent.get_length(); } + nextent_state_t get_extent_state() const override { return extent.get_state(); } + void prepare_mutate(context_t c) override { return extent.prepare_mutate(c); } + bool is_level_tail() const override { return extent.read().is_level_tail(); } + + void validate_non_empty() const override { + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + if (is_level_tail()) { + return; + } + } + assert(!is_keys_empty()); + } + + bool is_keys_empty() const override { return extent.read().keys() == 0; } + + bool has_single_value() const override { + validate_non_empty(); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + return ((is_level_tail() && is_keys_empty()) || + (!is_level_tail() && stage_t::is_keys_one(extent.read()))); + } else { + return stage_t::is_keys_one(extent.read()); + } + } + + level_t level() const override { return extent.read().level(); } + node_offset_t free_size() const override { return extent.read().free_size(); } + extent_len_t total_size() const override { return extent.read().total_size(); } + bool is_extent_retired() const override { return extent.is_retired(); } + + std::optional<key_view_t> get_pivot_index() const override { + if (is_level_tail()) { + return std::nullopt; + } + assert(!is_keys_empty()); + key_view_t pivot_index; + stage_t::template get_largest_slot<false, true, false>( + extent.read(), nullptr, &pivot_index, nullptr); + return {pivot_index}; + } + + bool is_size_underflow() const override { + /** + * There might be 2 node-merge strategies: + * + * The first is to rebalance and merge nodes and perfer tree fillness as + * much as possible in order to save space and improve key density for + * lookup, in exchange to the efforts of frequent merge, split and + * rebalance. These operations cannot benefit from seastore deltas because + * they are allocating fresh extents which need to be write into the + * journal as a whole, making write amplification much larger. + * + * The second is to delay rebalance and merge. When submit the transaction, + * simple insert and erase only need to append delta including just enough + * information about the inserted/erase item. The downside is tree fillness + * is not as good as the first strategy. + * + * Currently the decision is the second way by delaying merge until the + * node is 1/4 full, so that: + * - After a split operation (making the node at least 1/2 full): + * - The next merge need to erase items taking at least 1/4 space; + * - The next split need to insert items taking at most 1/2 space; + * - After a merge operation (making the node at least 1/2 full): + * - The next merge need to erase items taking at least 1/4 space; + * - The next split need to insert items taking at most 1/2 space; + * - TODO: before node rebalance is implemented, the node size can be below + * the underflow limit if it cannot be merged with peers; + */ + auto& node_stage = extent.read(); + size_t empty_size = node_stage.size_before(0); + size_t filled_kv_size = filled_size() - empty_size; + size_t full_kv_size = node_stage.total_size() - empty_size; + return filled_kv_size <= full_kv_size / 4; + } + + std::tuple<match_stage_t, search_position_t> + erase(const search_position_t& pos) override { + LOG_PREFIX(OTree::Layout::erase); + SUBDEBUG(seastore_onode, "begin at erase_pos({}) ...", pos); + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::trace))) { + std::ostringstream sos; + dump(sos); + SUBTRACE(seastore_onode, "-- dump\n{}", sos.str()); + } + auto [stage, next_or_last_pos] = extent.erase_replayable(cast_down<STAGE>(pos)); + SUBDEBUG(seastore_onode, "done at erase_stage={}, n/l_pos({})", stage, next_or_last_pos); + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::trace))) { + std::ostringstream sos; + dump(sos); + SUBTRACE(seastore_onode, "-- dump\n{}", sos.str()); + } +#ifndef NDEBUG + if (!is_keys_empty()) { + validate_layout(); + } +#endif + return {stage, normalize(std::move(next_or_last_pos))}; + } + + std::tuple<match_stage_t, std::size_t> evaluate_merge( + NodeImpl& _right_node) override { + auto& left_node_stage = extent.read(); + auto& right_node = dynamic_cast<NodeLayoutT&>(_right_node); + auto& right_node_stage = right_node.extent.read(); + + assert(NODE_TYPE == _right_node.node_type()); + assert(FIELD_TYPE == _right_node.field_type()); + assert(!is_level_tail()); + assert(!is_keys_empty()); + + match_stage_t merge_stage; + node_offset_t size_comp; + if (right_node.is_keys_empty()) { + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + assert(right_node.is_level_tail()); + merge_stage = STAGE; + size_comp = right_node_stage.header_size(); + } else { + ceph_abort("impossible path"); + } + } else { + key_view_t left_pivot_index; + stage_t::template get_largest_slot<false, true, false>( + left_node_stage, nullptr, &left_pivot_index, nullptr); + std::tie(merge_stage, size_comp) = stage_t::evaluate_merge( + left_pivot_index, right_node_stage); + } + auto size_left = filled_size(); + auto size_right = right_node.filled_size(); + assert(size_right > size_comp); + std::size_t merge_size = size_left + size_right - size_comp; + return {merge_stage, merge_size}; + } + + search_position_t merge( + NodeExtentMutable& mut, + NodeImpl& _right_node, + match_stage_t merge_stage, + extent_len_t merge_size) override { + LOG_PREFIX(OTree::Layout::merge); + + auto& left_node_stage = extent.read(); + auto& right_node = dynamic_cast<NodeLayoutT&>(_right_node); + auto& right_node_stage = right_node.extent.read(); + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::debug))) { + { + std::ostringstream sos; + dump(sos); + SUBDEBUG(seastore_onode, "-- left node dump\n{}", sos.str()); + } + { + std::ostringstream sos; + right_node.dump(sos); + SUBDEBUG(seastore_onode, "-- right node dump\n{}", sos.str()); + } + } + + assert(NODE_TYPE == _right_node.node_type()); + assert(FIELD_TYPE == _right_node.field_type()); + assert(!is_level_tail()); + assert(!is_keys_empty()); + + if (right_node.is_level_tail()) { + node_stage_t::update_is_level_tail(mut, left_node_stage, true); + build_name(); + } + position_t left_last_pos; + stage_t::template get_largest_slot<true, false, false>( + left_node_stage, &left_last_pos, nullptr, nullptr); + + if (right_node.is_keys_empty()) { + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + assert(right_node.is_level_tail()); + laddr_t tail_value = right_node_stage.get_end_p_laddr()->value; + auto p_write = left_node_stage.get_end_p_laddr(); + mut.copy_in_absolute((void*)p_write, tail_value); + } else { + ceph_abort("impossible path"); + } + } else { + typename stage_t::template StagedAppender<KeyT::VIEW> left_appender; + left_appender.init_tail(&mut, left_node_stage, merge_stage); + + typename stage_t::StagedIterator right_append_at; + right_append_at.set(right_node_stage); + + auto pos_end = position_t::end(); + stage_t::template append_until<KeyT::VIEW>( + right_append_at, left_appender, pos_end, STAGE); + assert(right_append_at.is_end()); + left_appender.wrap(); + } + + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::debug))) { + std::ostringstream sos; + dump(sos); + SUBDEBUG(seastore_onode, "-- merged node dump\n{}", sos.str()); + } + assert(merge_size == filled_size()); + return normalize(std::move(left_last_pos)); + } + + eagain_ifuture<NodeExtentMutable> + rebuild_extent(context_t c) override { + assert(!is_keys_empty()); + key_view_t first_index; + stage_t::template get_slot<true, false>( + extent.read(), position_t::begin(), &first_index, nullptr); + auto hint = first_index.get_hint(); + return extent.rebuild(c, hint).si_then([this] (auto mut) { + // addr may change + build_name(); + return mut; + }); + } + + eagain_ifuture<> retire_extent(context_t c) override { + return extent.retire(c); + } + + search_position_t make_tail() override { + auto&& ret = extent.make_tail_replayable(); + // is_level_tail is changed + build_name(); + return normalize(std::move(ret)); + } + + node_stats_t get_stats() const override { + node_stats_t stats; + auto& node_stage = extent.read(); + key_view_t index_key; + if (!is_keys_empty()) { + stage_t::get_stats(node_stage, stats, index_key); + } + stats.size_persistent = extent.get_length(); + stats.size_filled = filled_size(); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + if (is_level_tail()) { + stats.size_logical += sizeof(value_t); + stats.size_value += sizeof(value_t); + stats.num_kvs += 1; + } + } + return stats; + } + + std::ostream& dump(std::ostream& os) const override { + auto& node_stage = extent.read(); + auto p_start = node_stage.p_start(); + dump_brief(os); + auto stats = get_stats(); + os << " num_kvs=" << stats.num_kvs + << ", logical=" << stats.size_logical + << "B, overhead=" << stats.size_overhead + << "B, value=" << stats.size_value << "B"; + os << ":\n header: " << node_stage_t::header_size() << "B"; + size_t size = 0u; + if (!is_keys_empty()) { + stage_t::dump(node_stage, os, " ", size, p_start); + } else { + size += node_stage_t::header_size(); + if (NODE_TYPE == node_type_t::LEAF || !node_stage.is_level_tail()) { + os << " empty!"; + } + } + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + if (node_stage.is_level_tail()) { + size += sizeof(laddr_t); + auto value_ptr = node_stage.get_end_p_laddr(); + int offset = reinterpret_cast<const char*>(value_ptr) - p_start; + os << "\n tail value: 0x" + << std::hex << value_ptr->value << std::dec + << " " << size << "B" + << " @" << offset << "B"; + } + } + assert(size == filled_size()); + return os; + } + + std::ostream& dump_brief(std::ostream& os) const override { + os << name + << "(filled=" << filled_size() << "B" + << ", free=" << extent.read().free_size() << "B)"; + return os; + } + + const std::string& get_name() const override { return name; } + + void validate_layout() const override { +#ifndef NDEBUG + stage_t::validate(extent.read()); +#endif + } + + void test_copy_to(NodeExtentMutable& to) const override { + extent.test_copy_to(to); + } + + void test_set_tail(NodeExtentMutable& mut) override { + node_stage_t::update_is_level_tail(mut, extent.read(), true); + build_name(); + } + + /* + * Common + */ + void get_slot(const search_position_t& pos, + key_view_t* p_index_key = nullptr, + const value_t** pp_value = nullptr) const override { + assert(!is_keys_empty()); + assert(!pos.is_end()); + if (p_index_key && pp_value) { + stage_t::template get_slot<true, true>( + extent.read(), cast_down<STAGE>(pos), p_index_key, pp_value); + } else if (!p_index_key && pp_value) { + stage_t::template get_slot<false, true>( + extent.read(), cast_down<STAGE>(pos), nullptr, pp_value); + } else if (p_index_key && !pp_value) { + stage_t::template get_slot<true, false>( + extent.read(), cast_down<STAGE>(pos), p_index_key, nullptr); + } else { + ceph_abort("impossible path"); + } +#ifndef NDEBUG + if (pp_value) { + assert((const char*)(*pp_value) - extent.read().p_start() < + extent.get_length()); + } +#endif + } + + void get_prev_slot(search_position_t& pos, + key_view_t* p_index_key = nullptr, + const value_t** pp_value = nullptr) const override { + assert(!is_keys_empty()); + assert(!pos.is_end()); + auto& _pos = cast_down<STAGE>(pos); +#ifndef NDEBUG + auto nxt_pos = _pos; +#endif + if (!p_index_key && pp_value) { + stage_t::template get_prev_slot<false, true>( + extent.read(), _pos, nullptr, pp_value); + } else { + ceph_abort("not implemented"); + } +#ifndef NDEBUG + auto _nxt_pos = _pos; + stage_t::template get_next_slot<false, false>( + extent.read(), _nxt_pos, nullptr, nullptr); + assert(nxt_pos == _nxt_pos); +#endif + } + + void get_next_slot(search_position_t& pos, + key_view_t* p_index_key = nullptr, + const value_t** pp_value = nullptr) const override { + assert(!is_keys_empty()); + assert(!pos.is_end()); + bool find_next; + if (p_index_key && pp_value) { + find_next = stage_t::template get_next_slot<true, true>( + extent.read(), cast_down<STAGE>(pos), p_index_key, pp_value); + } else if (!p_index_key && pp_value) { + find_next = stage_t::template get_next_slot<false, true>( + extent.read(), cast_down<STAGE>(pos), nullptr, pp_value); + } else { + ceph_abort("not implemented"); + } + if (find_next) { + pos = search_position_t::end(); + } + } + + void get_largest_slot(search_position_t* p_pos = nullptr, + key_view_t* p_index_key = nullptr, + const value_t** pp_value = nullptr) const override { + assert(!is_keys_empty()); + if (p_pos && p_index_key && pp_value) { + stage_t::template get_largest_slot<true, true, true>( + extent.read(), &cast_down_fill_0<STAGE>(*p_pos), p_index_key, pp_value); + } else if (!p_pos && p_index_key && !pp_value) { + stage_t::template get_largest_slot<false, true, false>( + extent.read(), nullptr, p_index_key, nullptr); + } else if (p_pos && !p_index_key && pp_value) { + stage_t::template get_largest_slot<true, false, true>( + extent.read(), &cast_down_fill_0<STAGE>(*p_pos), nullptr, pp_value); + } else if (p_pos && !p_index_key && !pp_value) { + stage_t::template get_largest_slot<true, false, false>( + extent.read(), &cast_down_fill_0<STAGE>(*p_pos), nullptr, nullptr); + } else { + ceph_abort("not implemented"); + } + } + + + lookup_result_t<NODE_TYPE> lower_bound( + const key_hobj_t& key, MatchHistory& history, + key_view_t* index_key=nullptr, marker_t={}) const override { + auto& node_stage = extent.read(); + if constexpr (NODE_TYPE == node_type_t::LEAF) { + if (unlikely(is_keys_empty())) { + history.set<STAGE_LEFT>(MatchKindCMP::LT); + return lookup_result_t<NODE_TYPE>::end(); + } + } + assert(!is_keys_empty()); + + typename stage_t::result_t result_raw; + if (index_key) { + result_raw = stage_t::template lower_bound<true>( + node_stage, key, history, index_key); +#ifndef NDEBUG + if (!result_raw.is_end()) { + key_view_t index; + stage_t::template get_slot<true, false>( + node_stage, result_raw.position, &index, nullptr); + assert(index == *index_key); + } +#endif + } else { + result_raw = stage_t::lower_bound(node_stage, key, history); + } +#ifndef NDEBUG + if (result_raw.is_end()) { + assert(result_raw.mstat == MSTAT_END); + } else { + key_view_t index; + stage_t::template get_slot<true, false>( + node_stage, result_raw.position, &index, nullptr); + assert_mstat(key, index, result_raw.mstat); + } +#endif + + // calculate MSTAT_LT3 + if constexpr (FIELD_TYPE == field_type_t::N0) { + // currently only internal node checks mstat + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + if (result_raw.mstat == MSTAT_LT2) { + auto cmp = + key <=> node_stage[result_raw.position.index].shard_pool; + assert(cmp != std::strong_ordering::greater); + if (cmp != 0) { + result_raw.mstat = MSTAT_LT3; + } + } + } + } + + auto result = normalize(std::move(result_raw)); + if (result.is_end()) { + assert(node_stage.is_level_tail()); + assert(result.p_value == nullptr); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + result.p_value = node_stage.get_end_p_laddr(); + } + } else { + assert(result.p_value != nullptr); + } + return result; + } + + const value_t* insert( + const full_key_t<KEY_TYPE>& key, const value_input_t& value, + search_position_t& insert_pos, match_stage_t& insert_stage, + node_offset_t& insert_size) override { + LOG_PREFIX(OTree::Layout::insert); + SUBDEBUG(seastore_onode, + "begin at insert_pos({}), insert_stage={}, insert_size={}B ...", + insert_pos, insert_stage, insert_size); + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::trace))) { + std::ostringstream sos; + dump(sos); + SUBTRACE(seastore_onode, "-- dump\n{}", sos.str()); + } + auto ret = extent.template insert_replayable<KEY_TYPE>( + key, value, cast_down<STAGE>(insert_pos), insert_stage, insert_size); + SUBDEBUG(seastore_onode, + "done at insert_pos({}), insert_stage={}, insert_size={}B", + insert_pos, insert_stage, insert_size); + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::trace))) { + std::ostringstream sos; + dump(sos); + SUBTRACE(seastore_onode, "-- dump\n{}", sos.str()); + } + validate_layout(); +#ifndef NDEBUG + key_view_t index; + get_slot(insert_pos, &index, nullptr); + assert(index == key); +#endif + return ret; + } + + std::tuple<search_position_t, bool, const value_t*> split_insert( + NodeExtentMutable& right_mut, NodeImpl& _right_impl, + const full_key_t<KEY_TYPE>& key, const value_input_t& value, + search_position_t& _insert_pos, match_stage_t& insert_stage, + node_offset_t& insert_size) override { + LOG_PREFIX(OTree::Layout::split_insert); + assert(_right_impl.node_type() == NODE_TYPE); + assert(_right_impl.field_type() == FIELD_TYPE); + auto& right_impl = dynamic_cast<NodeLayoutT&>(_right_impl); + SUBDEBUG(seastore_onode, + "begin at insert_pos({}), insert_stage={}, insert_size={}B ...", + _insert_pos, insert_stage, insert_size); + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::debug))) { + std::ostringstream sos; + dump(sos); + SUBDEBUG(seastore_onode, "-- dump\n{}", sos.str()); + } +#ifdef UNIT_TESTS_BUILT + auto insert_stage_pre = insert_stage; +#endif + + auto& insert_pos = cast_down<STAGE>(_insert_pos); + auto& node_stage = extent.read(); + typename stage_t::StagedIterator split_at; + bool is_insert_left; + size_t split_size; + size_t target_split_size; + { + size_t empty_size = node_stage.size_before(0); + size_t filled_kv_size = filled_size() - empty_size; + /** NODE_BLOCK_SIZE considerations + * + * Generally, + * target_split_size = (filled_size + insert_size) / 2 + * We can have two locate_split() strategies: + * A. the simpler one is to locate the largest split position where + * the estimated left_node_size <= target_split_size; + * B. the fair one takes a further step to calculate the next slot of + * P KiB, and if left_node_size + P/2 < target_split_size, compensate + * the split position to include the next slot; + * + * Say that the node_block_size = N KiB, the largest allowed + * insert_size = 1/I * N KiB (I > 1). We want to identify the minimal 'I' + * that won't lead to "double split" effect, meaning after a split, + * the right node size is still larger than N KiB and need to split + * again. I think "double split" makes split much more complicated and + * we can no longer identify whether the node is safe under concurrent + * operations. + * + * We need to evaluate the worst case in order to identify 'I'. This means: + * - filled_size ~= N KiB + * - insert_size == N/I KiB + * - target_split_size ~= (I+1)/2I * N KiB + * To simplify the below calculations, node_block_size is normalized to 1. + * + * With strategy A, the worst case is when left_node_size cannot include + * the next slot that will just overflow the target_split_size: + * - left_node_size + 1/I ~= (I+1)/2I + * - left_node_size ~= (I-1)/2I + * - right_node_size ~= 1 + 1/I - left_node_size ~= (I+3)/2I + * The right_node_size cannot larger than the node_block_size in the + * worst case, which means (I+3)/2I < 1, so I > 3, meaning the largest + * possible insert_size must be smaller than 1/3 of the node_block_size. + * + * With strategy B, the worst case is when left_node_size cannot include + * the next slot that will just overflow the threshold + * target_split_size - 1/2I, thus: + * - left_node_size ~= (I+1)/2I - 1/2I ~= 1/2 + * - right_node_size ~= 1 + 1/I - 1/2 ~= (I+2)/2I < node_block_size(1) + * - I > 2 + * This means the largest possible insert_size must be smaller than 1/2 of + * the node_block_size, which is better than strategy A. + * + * In order to avoid "double split", there is another side-effect we need + * to take into consideration: if split happens with snap-gen indexes, the + * according ns-oid string needs to be copied to the right node. That is + * to say: right_node_size + string_size < node_block_size. + * + * Say that the largest allowed string size is 1/S of the largest allowed + * insert_size N/I KiB. If we go with stragety B, and when split happens + * with snap-gen indexes and split just overflow the target_split_size: + * - left_node_size ~= target_split_size - 1/2 * (1/I - 1/IS) + * ~= 1/2 + 1/2IS + * - right_node_size ~= 1 + 1/I - left_node_size + 1/IS + * ~= 1/2 + 1/I + 1/2IS < 1 + * - I > 2 + 1/S (S > 1) + * + * Now back to NODE_BLOCK_SIZE calculation, if we have limits of at most + * X KiB ns-oid string and Y KiB of value to store in this BTree, then: + * - largest_insert_size ~= X+Y KiB + * - 1/S == X/(X+Y) + * - I > (3X+2Y)/(X+Y) + * - node_block_size(N) == I * insert_size > 3X+2Y KiB + * + * In conclusion, + * (TODO) the current node block size (4 KiB) is too small to + * store entire 2 KiB ns-oid string. We need to consider a larger + * node_block_size. + * + * We are setting X = Y = 640 B in order not to break the current + * implementations with 4KiB node. + * + * (TODO) Implement smarter logics to check when "double split" happens. + */ + target_split_size = empty_size + (filled_kv_size + insert_size) / 2; + assert(insert_size < (node_stage.total_size() - empty_size) / 2); + + std::optional<bool> _is_insert_left; + split_at.set(node_stage); + split_size = 0; + bool locate_nxt = stage_t::recursively_locate_split_inserted( + split_size, 0, target_split_size, insert_pos, + insert_stage, insert_size, _is_insert_left, split_at); + is_insert_left = *_is_insert_left; + SUBDEBUG(seastore_onode, + "-- located split_at({}), insert_pos({}), is_insert_left={}, " + "split_size={}B(target={}B, current={}B)", + split_at, insert_pos, is_insert_left, + split_size, target_split_size, filled_size()); + // split_size can be larger than target_split_size in strategy B + // assert(split_size <= target_split_size); + if (locate_nxt) { + assert(insert_stage == STAGE); + assert(split_at.get().is_last()); + split_at.set_end(); + assert(insert_pos.index == split_at.index()); + } + } + + auto append_at = split_at; + // TODO(cross-node string dedup) + typename stage_t::template StagedAppender<KEY_TYPE> right_appender; + right_appender.init_empty(&right_mut, right_mut.get_write()); + const value_t* p_value = nullptr; + if (!is_insert_left) { + // right node: append [start(append_at), insert_pos) + stage_t::template append_until<KEY_TYPE>( + append_at, right_appender, insert_pos, insert_stage); + SUBDEBUG(seastore_onode, + "-- right appended until " + "insert_pos({}), insert_stage={}, insert/append the rest ...", + insert_pos, insert_stage); + // right node: append [insert_pos(key, value)] + bool is_front_insert = (insert_pos == position_t::begin()); + [[maybe_unused]] bool is_end = stage_t::template append_insert<KEY_TYPE>( + key, value, append_at, right_appender, + is_front_insert, insert_stage, p_value); + assert(append_at.is_end() == is_end); + } else { + SUBDEBUG(seastore_onode, "-- right appending ..."); + } + + // right node: append (insert_pos, end) + auto pos_end = position_t::end(); + stage_t::template append_until<KEY_TYPE>( + append_at, right_appender, pos_end, STAGE); + assert(append_at.is_end()); + right_appender.wrap(); + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::debug))) { + std::ostringstream sos; + right_impl.dump(sos); + SUBDEBUG(seastore_onode, "-- right node dump\n{}", sos.str()); + } + right_impl.validate_layout(); + + // mutate left node + if (is_insert_left) { + SUBDEBUG(seastore_onode, + "-- left trim/insert at insert_pos({}), insert_stage={} ...", + insert_pos, insert_stage); + p_value = extent.template split_insert_replayable<KEY_TYPE>( + split_at, key, value, insert_pos, insert_stage, insert_size); +#ifndef NDEBUG + key_view_t index; + get_slot(_insert_pos, &index, nullptr); + assert(index == key); +#endif + } else { + SUBDEBUG(seastore_onode, "-- left trim ..."); +#ifndef NDEBUG + key_view_t index; + right_impl.get_slot(_insert_pos, &index, nullptr); + assert(index == key); +#endif + extent.split_replayable(split_at); + } + if (right_impl.is_level_tail()) { + // is_level_tail of left is changed by split/split_insert + build_name(); + } + if (unlikely(LOGGER(seastore_onode).is_enabled(seastar::log_level::debug))) { + std::ostringstream sos; + dump(sos); + SUBDEBUG(seastore_onode, "-- left node dump\n{}", sos.str()); + } + validate_layout(); + assert(p_value); + + auto split_pos = normalize(split_at.get_pos()); + SUBDEBUG(seastore_onode, + "done at insert_pos({}), insert_stage={}, insert_size={}B, " + "split_at({}), is_insert_left={}, split_size={}B(target={}B)", + _insert_pos, insert_stage, insert_size, split_pos, + is_insert_left, split_size, target_split_size); + assert(split_size == filled_size()); + +#ifdef UNIT_TESTS_BUILT + InsertType insert_type; + search_position_t last_pos; + if (is_insert_left) { + stage_t::template get_largest_slot<true, false, false>( + extent.read(), &cast_down_fill_0<STAGE>(last_pos), nullptr, nullptr); + } else { + node_stage_t right_stage{reinterpret_cast<FieldType*>(right_mut.get_write()), + right_mut.get_length()}; + stage_t::template get_largest_slot<true, false, false>( + right_stage, &cast_down_fill_0<STAGE>(last_pos), nullptr, nullptr); + } + if (_insert_pos == search_position_t::begin()) { + insert_type = InsertType::BEGIN; + } else if (_insert_pos == last_pos) { + insert_type = InsertType::LAST; + } else { + insert_type = InsertType::MID; + } + last_split = {split_pos, insert_stage_pre, is_insert_left, insert_type}; +#endif + return {split_pos, is_insert_left, p_value}; + } + + /* + * InternalNodeImpl + */ + const laddr_packed_t* get_tail_value() const override { + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + assert(is_level_tail()); + return extent.read().get_end_p_laddr(); + } else { + ceph_abort("impossible path"); + } + } + + void replace_child_addr( + const search_position_t& pos, laddr_t dst, laddr_t src) override { + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + LOG_PREFIX(OTree::Layout::replace_child_addr); + SUBDEBUG(seastore_onode, "update from {:#x} to {:#x} at pos({}) ...", src, dst, pos); + const laddr_packed_t* p_value; + if (pos.is_end()) { + assert(is_level_tail()); + p_value = get_tail_value(); + } else { + get_slot(pos, nullptr, &p_value); + } + assert(p_value->value == src); + extent.update_child_addr_replayable(dst, const_cast<laddr_packed_t*>(p_value)); + } else { + ceph_abort("impossible path"); + } + } + + std::tuple<match_stage_t, node_offset_t> evaluate_insert( + const key_view_t& key, const laddr_t& value, + search_position_t& insert_pos) const override { + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + auto& node_stage = extent.read(); + match_stage_t insert_stage; + node_offset_t insert_size; + if (unlikely(is_keys_empty())) { + assert(insert_pos.is_end()); + insert_stage = STAGE; + insert_size = stage_t::insert_size(key, value); + } else { + std::tie(insert_stage, insert_size) = stage_t::evaluate_insert( + node_stage, key, value, cast_down<STAGE>(insert_pos), false); + } + return {insert_stage, insert_size}; + } else { + ceph_abort("impossible path"); + } + } + + /* + * LeafNodeImpl + */ + std::tuple<match_stage_t, node_offset_t> evaluate_insert( + const key_hobj_t& key, const value_config_t& value, + const MatchHistory& history, match_stat_t mstat, + search_position_t& insert_pos) const override { + if constexpr (NODE_TYPE == node_type_t::LEAF) { + if (unlikely(is_keys_empty())) { + assert(insert_pos.is_end()); + assert(is_level_tail()); + return {STAGE, stage_t::insert_size(key, value)}; + } else { + return stage_t::evaluate_insert( + key, value, history, mstat, cast_down<STAGE>(insert_pos)); + } + } else { + ceph_abort("impossible path"); + } + } + + std::pair<NodeExtentMutable&, ValueDeltaRecorder*> + prepare_mutate_value_payload(context_t c) { + return extent.prepare_mutate_value_payload(c); + } + + private: + NodeLayoutT(NodeExtentRef extent) : extent{extent} { + build_name(); + } + + extent_len_t filled_size() const { + auto& node_stage = extent.read(); + auto ret = node_stage.size_before(node_stage.keys()); + assert(ret == node_stage.total_size() - node_stage.free_size()); + return ret; + } + + // rebuild the name whenever addr, type, level, tail is changed + void build_name() { + // XXX: maybe also include the extent state + std::ostringstream sos; + sos << "Node" << NODE_TYPE << FIELD_TYPE + << "@0x" << std::hex << extent.get_laddr() + << "+" << extent.get_length() << std::dec + << "Lv" << (unsigned)level() + << (is_level_tail() ? "$" : ""); + name = sos.str(); + } + + extent_t extent; + std::string name = "Node-N/A"; +}; + +using InternalNode0 = NodeLayoutT<node_fields_0_t, node_type_t::INTERNAL>; +using InternalNode1 = NodeLayoutT<node_fields_1_t, node_type_t::INTERNAL>; +using InternalNode2 = NodeLayoutT<node_fields_2_t, node_type_t::INTERNAL>; +using InternalNode3 = NodeLayoutT<internal_fields_3_t, node_type_t::INTERNAL>; +using LeafNode0 = NodeLayoutT<node_fields_0_t, node_type_t::LEAF>; +using LeafNode1 = NodeLayoutT<node_fields_1_t, node_type_t::LEAF>; +using LeafNode2 = NodeLayoutT<node_fields_2_t, node_type_t::LEAF>; +using LeafNode3 = NodeLayoutT<leaf_fields_3_t, node_type_t::LEAF>; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_layout_replayable.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_layout_replayable.h new file mode 100644 index 000000000..d8a18231e --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_layout_replayable.h @@ -0,0 +1,138 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "node_extent_mutable.h" +#include "stages/node_stage.h" +#include "stages/stage.h" + +namespace crimson::os::seastore::onode { + +/** + * NodeLayoutReplayableT + * + * Contains templated logics to modify the layout of a NodeExtend which are + * also replayable. Used by NodeExtentAccessorT at runtime and by + * DeltaRecorderT during replay. + */ +template <typename FieldType, node_type_t NODE_TYPE> +struct NodeLayoutReplayableT { + using node_stage_t = node_extent_t<FieldType, NODE_TYPE>; + using stage_t = node_to_stage_t<node_stage_t>; + using position_t = typename stage_t::position_t; + using StagedIterator = typename stage_t::StagedIterator; + using value_input_t = value_input_type_t<NODE_TYPE>; + using value_t = value_type_t<NODE_TYPE>; + static constexpr auto FIELD_TYPE = FieldType::FIELD_TYPE; + + template <KeyT KT> + static const value_t* insert( + NodeExtentMutable& mut, + const node_stage_t& node_stage, + const full_key_t<KT>& key, + const value_input_t& value, + position_t& insert_pos, + match_stage_t& insert_stage, + node_offset_t& insert_size) { + auto p_value = stage_t::template proceed_insert<KT, false>( + mut, node_stage, key, value, insert_pos, insert_stage, insert_size); + return p_value; + } + + static void split( + NodeExtentMutable& mut, + const node_stage_t& node_stage, + StagedIterator& split_at) { + node_stage_t::update_is_level_tail(mut, node_stage, false); + stage_t::trim(mut, split_at); + } + + template <KeyT KT> + static const value_t* split_insert( + NodeExtentMutable& mut, + const node_stage_t& node_stage, + StagedIterator& split_at, + const full_key_t<KT>& key, + const value_input_t& value, + position_t& insert_pos, + match_stage_t& insert_stage, + node_offset_t& insert_size) { + node_stage_t::update_is_level_tail(mut, node_stage, false); + stage_t::trim(mut, split_at); + auto p_value = stage_t::template proceed_insert<KT, true>( + mut, node_stage, key, value, insert_pos, insert_stage, insert_size); + return p_value; + } + + static void update_child_addr( + NodeExtentMutable& mut, const laddr_t new_addr, laddr_packed_t* p_addr) { + assert(NODE_TYPE == node_type_t::INTERNAL); + mut.copy_in_absolute(p_addr, new_addr); + } + + static std::tuple<match_stage_t, position_t> erase( + NodeExtentMutable& mut, + const node_stage_t& node_stage, + const position_t& _erase_pos) { + if (_erase_pos.is_end()) { + // must be internal node + assert(node_stage.is_level_tail()); + // return erase_stage, last_pos + return update_last_to_tail(mut, node_stage); + } + + assert(node_stage.keys() != 0); + position_t erase_pos = _erase_pos; + auto erase_stage = stage_t::erase(mut, node_stage, erase_pos); + // return erase_stage, next_pos + return {erase_stage, erase_pos}; + } + + static position_t make_tail( + NodeExtentMutable& mut, + const node_stage_t& node_stage) { + assert(!node_stage.is_level_tail()); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + auto [r_stage, r_last_pos] = update_last_to_tail(mut, node_stage); + std::ignore = r_stage; + return r_last_pos; + } else { + node_stage_t::update_is_level_tail(mut, node_stage, true); + // no need to calculate the last pos + return position_t::end(); + } + } + + private: + static std::tuple<match_stage_t, position_t> update_last_to_tail( + NodeExtentMutable& mut, + const node_stage_t& node_stage) { + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + assert(node_stage.keys() != 0); + position_t last_pos; + laddr_t last_value; + { + const laddr_packed_t* p_last_value; + stage_t::template get_largest_slot<true, false, true>( + node_stage, &last_pos, nullptr, &p_last_value); + last_value = p_last_value->value; + } + + auto erase_pos = last_pos; + auto erase_stage = stage_t::erase(mut, node_stage, erase_pos); + assert(erase_pos.is_end()); + + node_stage_t::update_is_level_tail(mut, node_stage, true); + auto p_last_value = const_cast<laddr_packed_t*>( + node_stage.get_end_p_laddr()); + mut.copy_in_absolute(p_last_value, last_value); + // return erase_stage, last_pos + return {erase_stage, last_pos}; + } else { + ceph_abort("impossible path"); + } + } +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/node_types.h b/src/crimson/os/seastore/onode_manager/staged-fltree/node_types.h new file mode 100644 index 000000000..22c140b59 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/node_types.h @@ -0,0 +1,145 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <cassert> +#include <ostream> + +#include "fwd.h" + +namespace crimson::os::seastore::onode { + +constexpr uint8_t FIELD_TYPE_MAGIC = 0x25; +enum class field_type_t : uint8_t { + N0 = FIELD_TYPE_MAGIC, + N1, + N2, + N3, + _MAX +}; +inline uint8_t to_unsigned(field_type_t type) { + auto value = static_cast<uint8_t>(type); + assert(value >= FIELD_TYPE_MAGIC); + assert(value < static_cast<uint8_t>(field_type_t::_MAX)); + return value - FIELD_TYPE_MAGIC; +} +inline std::ostream& operator<<(std::ostream &os, field_type_t type) { + const char* const names[] = {"0", "1", "2", "3"}; + auto index = to_unsigned(type); + os << names[index]; + return os; +} + +enum class node_type_t : uint8_t { + LEAF = 0, + INTERNAL +}; +inline std::ostream& operator<<(std::ostream &os, const node_type_t& type) { + const char* const names[] = {"L", "I"}; + auto index = static_cast<uint8_t>(type); + assert(index <= 1u); + os << names[index]; + return os; +} + +struct laddr_packed_t { + laddr_t value; +} __attribute__((packed)); +inline std::ostream& operator<<(std::ostream& os, const laddr_packed_t& laddr) { + return os << "laddr_packed(0x" << std::hex << laddr.value << std::dec << ")"; +} + +using match_stat_t = int8_t; +constexpr match_stat_t MSTAT_END = -2; // index is search_position_t::end() +constexpr match_stat_t MSTAT_EQ = -1; // key == index +constexpr match_stat_t MSTAT_LT0 = 0; // key == index [pool/shard crush ns/oid]; key < index [snap/gen] +constexpr match_stat_t MSTAT_LT1 = 1; // key == index [pool/shard crush]; key < index [ns/oid] +constexpr match_stat_t MSTAT_LT2 = 2; // key < index [pool/shard crush ns/oid] || + // key == index [pool/shard]; key < index [crush] +constexpr match_stat_t MSTAT_LT3 = 3; // key < index [pool/shard] +constexpr match_stat_t MSTAT_MIN = MSTAT_END; +constexpr match_stat_t MSTAT_MAX = MSTAT_LT3; + +enum class node_delta_op_t : uint8_t { + INSERT, + SPLIT, + SPLIT_INSERT, + UPDATE_CHILD_ADDR, + ERASE, + MAKE_TAIL, + SUBOP_UPDATE_VALUE = 0xff, +}; + +/** nextent_state_t + * + * The possible states of tree node extent(NodeExtentAccessorT). + * + * State transition implies the following capabilities is changed: + * - mutability is changed; + * - whether to record; + * - memory has been copied; + * + * load()----+ + * | + * alloc() v + * | +--> [READ_ONLY] ---------+ + * | | | | + * | | prepare_mutate() | + * | | | | + * | v v v + * | +--> [MUTATION_PENDING]---+ + * | | | + * | | rebuild() + * | | | + * | v v + * +------->+--> [FRESH] <------------+ + * + * Note that NodeExtentAccessorT might still be MUTATION_PENDING/FRESH while + * the internal extent has become DIRTY after the transaction submission is + * started while nodes destruction and validation has not been completed yet. + */ +enum class nextent_state_t : uint8_t { + READ_ONLY = 0, // requires mutate for recording + // CLEAN/DIRTY + MUTATION_PENDING, // can mutate, needs recording + // MUTATION_PENDING + FRESH, // can mutate, no recording + // INITIAL_WRITE_PENDING +}; + +} + +template <> struct fmt::formatter<crimson::os::seastore::onode::node_delta_op_t> + : fmt::formatter<std::string_view> { + using node_delta_op_t = crimson::os::seastore::onode::node_delta_op_t; + // parse is inherited from formatter<string_view>. + template <typename FormatContext> + auto format(node_delta_op_t op, FormatContext& ctx) { + std::string_view name = "unknown"; + switch (op) { + case node_delta_op_t::INSERT: + name = "insert"; + break; + case node_delta_op_t::SPLIT: + name = "split"; + break; + case node_delta_op_t::SPLIT_INSERT: + name = "split_insert"; + break; + case node_delta_op_t::UPDATE_CHILD_ADDR: + name = "update_child_addr"; + break; + case node_delta_op_t::ERASE: + name = "erase"; + break; + case node_delta_op_t::MAKE_TAIL: + name = "make_tail"; + break; + case node_delta_op_t::SUBOP_UPDATE_VALUE: + name = "subop_update_value"; + break; + } + return formatter<string_view>::format(name, ctx); + } +}; diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/item_iterator_stage.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/item_iterator_stage.cc new file mode 100644 index 000000000..9252fb99a --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/item_iterator_stage.cc @@ -0,0 +1,202 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "item_iterator_stage.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_mutable.h" + +namespace crimson::os::seastore::onode { + +#define ITER_T item_iterator_t<NODE_TYPE> +#define ITER_INST(NT) item_iterator_t<NT> + +template <node_type_t NODE_TYPE> +template <IsFullKey Key> +memory_range_t ITER_T::insert_prefix( + NodeExtentMutable& mut, const ITER_T& iter, const Key& key, + bool is_end, node_offset_t size, const char* p_left_bound) +{ + // 1. insert range + char* p_insert; + if (is_end) { + assert(!iter.has_next()); + p_insert = const_cast<char*>(iter.p_start()); + } else { + p_insert = const_cast<char*>(iter.p_end()); + } + char* p_insert_front = p_insert - size; + + // 2. shift memory + const char* p_shift_start = p_left_bound; + const char* p_shift_end = p_insert; + mut.shift_absolute(p_shift_start, + p_shift_end - p_shift_start, + -(int)size); + + // 3. append header + p_insert -= sizeof(node_offset_t); + node_offset_t back_offset = (p_insert - p_insert_front); + mut.copy_in_absolute(p_insert, back_offset); + ns_oid_view_t::append(mut, key, p_insert); + + return {p_insert_front, p_insert}; +} +#define IP_TEMPLATE(NT, Key) \ + template memory_range_t ITER_INST(NT)::insert_prefix<Key>( \ + NodeExtentMutable&, const ITER_INST(NT)&, const Key&, \ + bool, node_offset_t, const char*) +IP_TEMPLATE(node_type_t::LEAF, key_view_t); +IP_TEMPLATE(node_type_t::INTERNAL, key_view_t); +IP_TEMPLATE(node_type_t::LEAF, key_hobj_t); +IP_TEMPLATE(node_type_t::INTERNAL, key_hobj_t); + +template <node_type_t NODE_TYPE> +void ITER_T::update_size( + NodeExtentMutable& mut, const ITER_T& iter, int change) +{ + node_offset_t offset = iter.get_back_offset(); + int new_size = change + offset; + assert(new_size > 0 && new_size < (int)mut.get_length()); + mut.copy_in_absolute( + (void*)iter.get_item_range().p_end, node_offset_t(new_size)); +} + +template <node_type_t NODE_TYPE> +node_offset_t ITER_T::trim_until(NodeExtentMutable& mut, const ITER_T& iter) +{ + assert(iter.index() != 0); + size_t ret = iter.p_end() - iter.p_items_start; + assert(ret < mut.get_length()); + return ret; +} + +template <node_type_t NODE_TYPE> +node_offset_t ITER_T::trim_at( + NodeExtentMutable& mut, const ITER_T& iter, node_offset_t trimmed) +{ + size_t trim_size = iter.p_start() - iter.p_items_start + trimmed; + assert(trim_size < mut.get_length()); + assert(iter.get_back_offset() > trimmed); + node_offset_t new_offset = iter.get_back_offset() - trimmed; + mut.copy_in_absolute((void*)iter.item_range.p_end, new_offset); + return trim_size; +} + +template <node_type_t NODE_TYPE> +node_offset_t ITER_T::erase( + NodeExtentMutable& mut, const ITER_T& iter, const char* p_left_bound) +{ + node_offset_t erase_size = iter.p_end() - iter.p_start(); + const char* p_shift_start = p_left_bound; + assert(p_left_bound <= iter.p_start()); + extent_len_t shift_len = iter.p_start() - p_left_bound; + int shift_off = erase_size; + mut.shift_absolute(p_shift_start, shift_len, shift_off); + return erase_size; +} + +#define ITER_TEMPLATE(NT) template class ITER_INST(NT) +ITER_TEMPLATE(node_type_t::LEAF); +ITER_TEMPLATE(node_type_t::INTERNAL); + +#define APPEND_T ITER_T::Appender<KT> + +template <node_type_t NODE_TYPE> +template <KeyT KT> +APPEND_T::Appender(NodeExtentMutable* p_mut, + const item_iterator_t& iter, + bool open) : p_mut{p_mut} +{ + assert(!iter.has_next()); + if (open) { + p_append = const_cast<char*>(iter.get_key().p_start()); + p_offset_while_open = const_cast<char*>(iter.item_range.p_end); + } else { + // XXX: this doesn't need to advance the iter to last + p_append = const_cast<char*>(iter.p_items_start); + } +} + +template <node_type_t NODE_TYPE> +template <KeyT KT> +bool APPEND_T::append(const ITER_T& src, index_t& items) +{ + auto p_end = src.p_end(); + bool append_till_end = false; + if (is_valid_index(items)) { + for (auto i = 1u; i <= items; ++i) { + if (!src.has_next()) { + assert(i == items); + append_till_end = true; + break; + } + ++src; + } + } else { + if (items == INDEX_END) { + append_till_end = true; + } else { + assert(items == INDEX_LAST); + } + items = 0; + while (src.has_next()) { + ++src; + ++items; + } + if (append_till_end) { + ++items; + } + } + + const char* p_start; + if (append_till_end) { + p_start = src.p_start(); + } else { + p_start = src.p_end(); + } + assert(p_end >= p_start); + size_t append_size = p_end - p_start; + p_append -= append_size; + p_mut->copy_in_absolute(p_append, p_start, append_size); + return append_till_end; +} + +template <node_type_t NODE_TYPE> +template <KeyT KT> +std::tuple<NodeExtentMutable*, char*> +APPEND_T::open_nxt(const key_get_type& partial_key) +{ + p_append -= sizeof(node_offset_t); + p_offset_while_open = p_append; + ns_oid_view_t::append(*p_mut, partial_key, p_append); + return {p_mut, p_append}; +} + +template <node_type_t NODE_TYPE> +template <KeyT KT> +std::tuple<NodeExtentMutable*, char*> +APPEND_T::open_nxt(const full_key_t<KT>& key) +{ + p_append -= sizeof(node_offset_t); + p_offset_while_open = p_append; + ns_oid_view_t::append(*p_mut, key, p_append); + return {p_mut, p_append}; +} + +template <node_type_t NODE_TYPE> +template <KeyT KT> +void APPEND_T::wrap_nxt(char* _p_append) +{ + assert(_p_append < p_append); + p_mut->copy_in_absolute( + p_offset_while_open, node_offset_t(p_offset_while_open - _p_append)); + p_append = _p_append; +} + +#define APPEND_TEMPLATE(NT, KT) template class ITER_INST(NT)::Appender<KT> +APPEND_TEMPLATE(node_type_t::LEAF, KeyT::VIEW); +APPEND_TEMPLATE(node_type_t::INTERNAL, KeyT::VIEW); +APPEND_TEMPLATE(node_type_t::LEAF, KeyT::HOBJ); +APPEND_TEMPLATE(node_type_t::INTERNAL, KeyT::HOBJ); + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/item_iterator_stage.h b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/item_iterator_stage.h new file mode 100644 index 000000000..9d12474ac --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/item_iterator_stage.h @@ -0,0 +1,193 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_types.h" +#include "key_layout.h" +#include "stage_types.h" + +namespace crimson::os::seastore::onode { + +class NodeExtentMutable; + +/** + * item_iterator_t + * + * The STAGE_STRING implementation for node N0/N1, implements staged contract + * as an iterative container to resolve crush hash conflicts. + * + * The layout of the contaner to index ns, oid strings storing n items: + * + * # <--------- container range ---------> # + * #<~># items [i+1, n) # + * # # items [0, i) #<~># + * # # <------ item i -------------> # # + * # # <--- item_range ---> | # # + * # # | # # + * # # next-stage | ns-oid | back_ # # + * # # contaner | strings | offset # # + * #...# range | | #...# + * ^ ^ | ^ + * | | | | + * | +---------------------------+ | + * + p_items_start p_items_end + + */ +template <node_type_t NODE_TYPE> +class item_iterator_t { + using value_input_t = value_input_type_t<NODE_TYPE>; + using value_t = value_type_t<NODE_TYPE>; + public: + item_iterator_t(const container_range_t& range) + : node_size{range.node_size}, + p_items_start(range.range.p_start), + p_items_end(range.range.p_end) { + assert(is_valid_node_size(node_size)); + assert(p_items_start < p_items_end); + next_item_range(p_items_end); + } + + const char* p_start() const { return item_range.p_start; } + const char* p_end() const { return item_range.p_end + sizeof(node_offset_t); } + const memory_range_t& get_item_range() const { return item_range; } + node_offset_t get_back_offset() const { return back_offset; } + + // container type system + using key_get_type = const ns_oid_view_t&; + static constexpr auto CONTAINER_TYPE = ContainerType::ITERATIVE; + index_t index() const { return _index; } + key_get_type get_key() const { + if (!key.has_value()) { + key = ns_oid_view_t(item_range.p_end); + assert(item_range.p_start < (*key).p_start()); + } + return *key; + } + node_offset_t size() const { + size_t ret = item_range.p_end - item_range.p_start + sizeof(node_offset_t); + assert(ret < node_size); + return ret; + }; + node_offset_t size_to_nxt() const { + size_t ret = get_key().size() + sizeof(node_offset_t); + assert(ret < node_size); + return ret; + } + node_offset_t size_overhead() const { + return sizeof(node_offset_t) + get_key().size_overhead(); + } + container_range_t get_nxt_container() const { + return {{item_range.p_start, get_key().p_start()}, node_size}; + } + bool has_next() const { + assert(p_items_start <= item_range.p_start); + return p_items_start < item_range.p_start; + } + const item_iterator_t<NODE_TYPE>& operator++() const { + assert(has_next()); + next_item_range(item_range.p_start); + key.reset(); + ++_index; + return *this; + } + void encode(const char* p_node_start, ceph::bufferlist& encoded) const { + int start_offset = p_items_start - p_node_start; + int stage_size = p_items_end - p_items_start; + assert(start_offset > 0); + assert(stage_size > 0); + assert(start_offset + stage_size <= (int)node_size); + ceph::encode(static_cast<node_offset_t>(start_offset), encoded); + ceph::encode(static_cast<node_offset_t>(stage_size), encoded); + ceph::encode(_index, encoded); + } + + static item_iterator_t decode(const char* p_node_start, + extent_len_t node_size, + ceph::bufferlist::const_iterator& delta) { + node_offset_t start_offset; + ceph::decode(start_offset, delta); + node_offset_t stage_size; + ceph::decode(stage_size, delta); + assert(start_offset > 0); + assert(stage_size > 0); + assert((unsigned)start_offset + stage_size <= node_size); + index_t index; + ceph::decode(index, delta); + + item_iterator_t ret({{p_node_start + start_offset, + p_node_start + start_offset + stage_size}, + node_size}); + while (index > 0) { + ++ret; + --index; + } + return ret; + } + + static node_offset_t header_size() { return 0u; } + + template <IsFullKey Key> + static node_offset_t estimate_insert( + const Key& key, const value_input_t&) { + return ns_oid_view_t::estimate_size(key) + sizeof(node_offset_t); + } + + template <IsFullKey Key> + static memory_range_t insert_prefix( + NodeExtentMutable& mut, const item_iterator_t<NODE_TYPE>& iter, + const Key& key, bool is_end, + node_offset_t size, const char* p_left_bound); + + static void update_size( + NodeExtentMutable& mut, const item_iterator_t<NODE_TYPE>& iter, int change); + + static node_offset_t trim_until(NodeExtentMutable&, const item_iterator_t<NODE_TYPE>&); + static node_offset_t trim_at( + NodeExtentMutable&, const item_iterator_t<NODE_TYPE>&, node_offset_t trimmed); + + static node_offset_t erase( + NodeExtentMutable&, const item_iterator_t<NODE_TYPE>&, const char*); + + template <KeyT KT> + class Appender; + + private: + void next_item_range(const char* p_end) const { + auto p_item_end = p_end - sizeof(node_offset_t); + assert(p_items_start < p_item_end); + back_offset = reinterpret_cast<const node_offset_packed_t*>(p_item_end)->value; + assert(back_offset); + const char* p_item_start = p_item_end - back_offset; + assert(p_items_start <= p_item_start); + item_range = {p_item_start, p_item_end}; + } + + extent_len_t node_size; + const char* p_items_start; + const char* p_items_end; + mutable memory_range_t item_range; + mutable node_offset_t back_offset; + mutable std::optional<ns_oid_view_t> key; + mutable index_t _index = 0u; +}; + +template <node_type_t NODE_TYPE> +template <KeyT KT> +class item_iterator_t<NODE_TYPE>::Appender { + public: + Appender(NodeExtentMutable* p_mut, char* p_append) + : p_mut{p_mut}, p_append{p_append} {} + Appender(NodeExtentMutable*, const item_iterator_t&, bool open); + bool append(const item_iterator_t<NODE_TYPE>& src, index_t& items); + char* wrap() { return p_append; } + std::tuple<NodeExtentMutable*, char*> open_nxt(const key_get_type&); + std::tuple<NodeExtentMutable*, char*> open_nxt(const full_key_t<KT>&); + void wrap_nxt(char* _p_append); + + private: + NodeExtentMutable* p_mut; + char* p_append; + char* p_offset_while_open; +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/key_layout.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/key_layout.cc new file mode 100644 index 000000000..4bf717dc2 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/key_layout.cc @@ -0,0 +1,34 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "key_layout.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_mutable.h" + +namespace crimson::os::seastore::onode { + +void string_key_view_t::append_str( + NodeExtentMutable& mut, std::string_view str, char*& p_append) +{ + assert(is_valid_size(str.length())); + p_append -= sizeof(string_size_t); + string_size_t len = str.length(); + mut.copy_in_absolute(p_append, len); + p_append -= len; + mut.copy_in_absolute(p_append, str.data(), len); +} + +void string_key_view_t::append_dedup( + NodeExtentMutable& mut, const Type& dedup_type, char*& p_append) +{ + p_append -= sizeof(string_size_t); + if (dedup_type == Type::MIN) { + mut.copy_in_absolute(p_append, MARKER_MIN); + } else if (dedup_type == Type::MAX) { + mut.copy_in_absolute(p_append, MARKER_MAX); + } else { + ceph_abort("impossible path"); + } +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/key_layout.h b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/key_layout.h new file mode 100644 index 000000000..fcd485355 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/key_layout.h @@ -0,0 +1,910 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <cassert> +#include <limits> +#include <optional> +#include <ostream> + +#include "common/hobject.h" +#include "crimson/os/seastore/onode.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/fwd.h" + +namespace crimson::os::seastore::onode { + +using shard_t = int8_t; +using pool_t = int64_t; +// Note: this is the reversed version of the object hash +using crush_hash_t = uint32_t; +using snap_t = uint64_t; +using gen_t = uint64_t; +static_assert(sizeof(shard_t) == sizeof(ghobject_t().shard_id.id)); +static_assert(sizeof(pool_t) == sizeof(ghobject_t().hobj.pool)); +static_assert(sizeof(crush_hash_t) == sizeof(ghobject_t().hobj.get_bitwise_key_u32())); +static_assert(sizeof(snap_t) == sizeof(ghobject_t().hobj.snap.val)); +static_assert(sizeof(gen_t) == sizeof(ghobject_t().generation)); + +constexpr auto MAX_SHARD = std::numeric_limits<shard_t>::max(); +constexpr auto MAX_POOL = std::numeric_limits<pool_t>::max(); +constexpr auto MAX_CRUSH = std::numeric_limits<crush_hash_t>::max(); +constexpr auto MAX_SNAP = std::numeric_limits<snap_t>::max(); +constexpr auto MAX_GEN = std::numeric_limits<gen_t>::max(); + +class NodeExtentMutable; +class key_view_t; +class key_hobj_t; +enum class KeyT { VIEW, HOBJ }; +template <KeyT> struct _full_key_type; +template<> struct _full_key_type<KeyT::VIEW> { using type = key_view_t; }; +template<> struct _full_key_type<KeyT::HOBJ> { using type = key_hobj_t; }; +template <KeyT type> +using full_key_t = typename _full_key_type<type>::type; + +static laddr_t get_lba_hint(shard_t shard, pool_t pool, crush_hash_t crush) { + // FIXME: It is possible that PGs from different pools share the same prefix + // if the mask 0xFF is not long enough, result in unexpected transaction + // conflicts. + return ((uint64_t)(shard & 0XFF)<<56 | + (uint64_t)(pool & 0xFF)<<48 | + (uint64_t)(crush )<<16); +} + +struct node_offset_packed_t { + node_offset_t value; +} __attribute__((packed)); + +/** + * string_key_view_t + * + * The layout to store char array as an oid or an ns string which may be + * compressed. + * + * (TODO) If compressed, the physical block only stores an unsigned int of + * string_size_t, with value MARKER_MIN denoting Type::MIN, and value + * MARKER_MAX denoting Type::MAX. + * + * If not compressed (Type::STR), the physical block stores the char array and + * a valid string_size_t value. + */ +struct string_key_view_t { + enum class Type {MIN, STR, MAX}; + static constexpr auto MARKER_MAX = std::numeric_limits<string_size_t>::max(); + static constexpr auto MARKER_MIN = std::numeric_limits<string_size_t>::max() - 1; + static constexpr auto VALID_UPPER_BOUND = std::numeric_limits<string_size_t>::max() - 2; + static bool is_valid_size(size_t size) { + return size <= VALID_UPPER_BOUND; + } + + string_key_view_t(const char* p_end) { + p_length = p_end - sizeof(string_size_t); + std::memcpy(&length, p_length, sizeof(string_size_t)); + if (is_valid_size(length)) { + auto _p_key = p_length - length; + p_key = static_cast<const char*>(_p_key); + } else { + assert(length == MARKER_MAX || length == MARKER_MIN); + p_key = nullptr; + } + } + Type type() const { + if (length == MARKER_MIN) { + return Type::MIN; + } else if (length == MARKER_MAX) { + return Type::MAX; + } else { + assert(is_valid_size(length)); + return Type::STR; + } + } + const char* p_start() const { + if (p_key) { + return p_key; + } else { + return p_length; + } + } + const char* p_next_end() const { + if (p_key) { + return p_start(); + } else { + return p_length + sizeof(string_size_t); + } + } + node_offset_t size() const { + size_t ret = length + sizeof(string_size_t); + assert(ret < MAX_NODE_SIZE); + return ret; + } + node_offset_t size_logical() const { + assert(type() == Type::STR); + assert(is_valid_size(length)); + return length; + } + node_offset_t size_overhead() const { + assert(type() == Type::STR); + return sizeof(string_size_t); + } + + std::string_view to_string_view() const { + assert(type() == Type::STR); + assert(is_valid_size(length)); + return {p_key, length}; + } + bool operator==(const string_key_view_t& x) const { + if (type() == x.type() && type() != Type::STR) + return true; + if (type() != x.type()) + return false; + if (length != x.length) + return false; + return (memcmp(p_key, x.p_key, length) == 0); + } + bool operator!=(const string_key_view_t& x) const { return !(*this == x); } + + void reset_to(const char* origin_base, + const char* new_base, + extent_len_t node_size) { + reset_ptr(p_key, origin_base, new_base, node_size); + reset_ptr(p_length, origin_base, new_base, node_size); +#ifndef NDEBUG + string_size_t current_length; + std::memcpy(¤t_length, p_length, sizeof(string_size_t)); + assert(length == current_length); +#endif + } + + static void append_str( + NodeExtentMutable&, std::string_view, char*& p_append); + + static void test_append_str(std::string_view str, char*& p_append) { + assert(is_valid_size(str.length())); + p_append -= sizeof(string_size_t); + string_size_t len = str.length(); + std::memcpy(p_append, &len, sizeof(string_size_t)); + p_append -= len; + std::memcpy(p_append, str.data(), len); + } + + static void append_dedup( + NodeExtentMutable&, const Type& dedup_type, char*& p_append); + + static void test_append_dedup(const Type& dedup_type, char*& p_append) { + p_append -= sizeof(string_size_t); + string_size_t len; + if (dedup_type == Type::MIN) { + len = MARKER_MIN; + } else if (dedup_type == Type::MAX) { + len = MARKER_MAX; + } else { + ceph_abort("impossible path"); + } + std::memcpy(p_append, &len, sizeof(string_size_t)); + } + + const char* p_key; + const char* p_length; + // TODO: remove if p_length is aligned + string_size_t length; +}; + +/** + * string_view_masked_t + * + * A common class to hide the underlying string implementation regardless of a + * string_key_view_t (maybe compressed), a string/string_view, or a compressed + * string. And leverage this consistant class to do compare, print, convert and + * append operations. + */ +class string_view_masked_t { + public: + using Type = string_key_view_t::Type; + explicit string_view_masked_t(const string_key_view_t& index) + : type{index.type()} { + if (type == Type::STR) { + view = index.to_string_view(); + } + } + explicit string_view_masked_t(std::string_view str) + : type{Type::STR}, view{str} { + assert(string_key_view_t::is_valid_size(view.size())); + } + + Type get_type() const { return type; } + std::string_view to_string_view() const { + assert(get_type() == Type::STR); + return view; + } + string_size_t size() const { + assert(get_type() == Type::STR); + assert(string_key_view_t::is_valid_size(view.size())); + return view.size(); + } + bool operator==(const string_view_masked_t& x) const { + if (get_type() == x.get_type() && get_type() != Type::STR) + return true; + if (get_type() != x.get_type()) + return false; + if (size() != x.size()) + return false; + return (memcmp(view.data(), x.view.data(), size()) == 0); + } + auto operator<=>(std::string_view rhs) const { + using Type = string_view_masked_t::Type; + assert(string_key_view_t::is_valid_size(rhs.size())); + auto lhs_type = get_type(); + if (lhs_type == Type::MIN) { + return std::strong_ordering::less; + } else if (lhs_type == Type::MAX) { + return std::strong_ordering::greater; + } else { // r_type == Type::STR + assert(string_key_view_t::is_valid_size(size())); + return to_string_view() <=> rhs; + } + } + void encode(ceph::bufferlist& bl) const { + if (get_type() == Type::MIN) { + ceph::encode(string_key_view_t::MARKER_MIN, bl); + } else if (get_type() == Type::MAX) { + ceph::encode(string_key_view_t::MARKER_MAX, bl); + } else { + ceph::encode(size(), bl); + ceph::encode_nohead(view, bl); + } + } + static auto min() { return string_view_masked_t{Type::MIN}; } + static auto max() { return string_view_masked_t{Type::MAX}; } + static string_view_masked_t decode( + std::string& str_storage, ceph::bufferlist::const_iterator& delta) { + string_size_t size; + ceph::decode(size, delta); + if (size == string_key_view_t::MARKER_MIN) { + return min(); + } else if (size == string_key_view_t::MARKER_MAX) { + return max(); + } else { + ceph::decode_nohead(size, str_storage, delta); + return string_view_masked_t(str_storage); + } + } + + private: + explicit string_view_masked_t(Type type) + : type{type} {} + + Type type; + std::string_view view; +}; + +inline auto operator<=>(const string_view_masked_t& l, const string_view_masked_t& r) { + using Type = string_view_masked_t::Type; + auto l_type = l.get_type(); + auto r_type = r.get_type(); + if (l_type == Type::STR && r_type == Type::STR) { + assert(string_key_view_t::is_valid_size(l.size())); + assert(string_key_view_t::is_valid_size(r.size())); + return l.to_string_view() <=> r.to_string_view(); + } else if (l_type == r_type) { + return std::strong_ordering::equal; + } else if (l_type == Type::MIN || r_type == Type::MAX) { + return std::strong_ordering::less; + } else { // l_type == Type::MAX || r_type == Type::MIN + return std::strong_ordering::greater; + } +} + +inline std::ostream& operator<<(std::ostream& os, const string_view_masked_t& masked) { + using Type = string_view_masked_t::Type; + auto type = masked.get_type(); + if (type == Type::MIN) { + return os << "MIN"; + } else if (type == Type::MAX) { + return os << "MAX"; + } else { // type == Type::STR + auto view = masked.to_string_view(); + if (view.length() <= 12) { + os << "\"" << view << "\""; + } else { + os << "\"" << std::string_view(view.data(), 4) << ".." + << std::string_view(view.data() + view.length() - 2, 2) + << "/" << view.length() << "B\""; + } + return os; + } +} + +struct ns_oid_view_t { + using Type = string_key_view_t::Type; + + ns_oid_view_t(const char* p_end) : nspace(p_end), oid(nspace.p_next_end()) {} + Type type() const { return oid.type(); } + const char* p_start() const { return oid.p_start(); } + node_offset_t size() const { + if (type() == Type::STR) { + size_t ret = nspace.size() + oid.size(); + assert(ret < MAX_NODE_SIZE); + return ret; + } else { + return sizeof(string_size_t); + } + } + node_offset_t size_logical() const { + assert(type() == Type::STR); + return nspace.size_logical() + oid.size_logical(); + } + node_offset_t size_overhead() const { + assert(type() == Type::STR); + return nspace.size_overhead() + oid.size_overhead(); + } + bool operator==(const ns_oid_view_t& x) const { + return (string_view_masked_t{nspace} == string_view_masked_t{x.nspace} && + string_view_masked_t{oid} == string_view_masked_t{x.oid}); + } + + void reset_to(const char* origin_base, + const char* new_base, + extent_len_t node_size) { + nspace.reset_to(origin_base, new_base, node_size); + oid.reset_to(origin_base, new_base, node_size); + } + + template <typename Key> + static node_offset_t estimate_size(const Key& key); + + template <typename Key> + static void append(NodeExtentMutable&, + const Key& key, + char*& p_append); + + static void append(NodeExtentMutable& mut, + const ns_oid_view_t& view, + char*& p_append) { + if (view.type() == Type::STR) { + string_key_view_t::append_str(mut, view.nspace.to_string_view(), p_append); + string_key_view_t::append_str(mut, view.oid.to_string_view(), p_append); + } else { + string_key_view_t::append_dedup(mut, view.type(), p_append); + } + } + + template <typename Key> + static void test_append(const Key& key, char*& p_append); + + string_key_view_t nspace; + string_key_view_t oid; +}; +inline std::ostream& operator<<(std::ostream& os, const ns_oid_view_t& ns_oid) { + return os << string_view_masked_t{ns_oid.nspace} << "," + << string_view_masked_t{ns_oid.oid}; +} +inline auto operator<=>(const ns_oid_view_t& l, const ns_oid_view_t& r) { + auto ret = (string_view_masked_t{l.nspace} <=> string_view_masked_t{r.nspace}); + if (ret != 0) + return ret; + return string_view_masked_t{l.oid} <=> string_view_masked_t{r.oid}; +} + +inline const ghobject_t _MIN_OID() { + assert(ghobject_t().is_min()); + // don't extern _MIN_OID + return ghobject_t(); +} + +/* + * Unfortunally the ghobject_t representitive as tree key doesn't have max + * field, so we define our own _MAX_OID and translate it from/to + * ghobject_t::get_max() if necessary. + */ +inline const ghobject_t _MAX_OID() { + auto ret = ghobject_t(shard_id_t(MAX_SHARD), MAX_POOL, MAX_CRUSH, + "MAX", "MAX", MAX_SNAP, MAX_GEN); + assert(ret.hobj.get_hash() == ret.hobj.get_bitwise_key_u32()); + return ret; +} + +// the valid key stored in tree should be in the range of (_MIN_OID, _MAX_OID) +template <typename Key> +bool is_valid_key(const Key& key); + +/** + * key_hobj_t + * + * A specialized implementation of a full_key_t storing a ghobject_t passed + * from user. + */ +class key_hobj_t { + public: + explicit key_hobj_t(const ghobject_t& _ghobj) { + if (_ghobj.is_max()) { + ghobj = _MAX_OID(); + } else { + // including when _ghobj.is_min() + ghobj = _ghobj; + } + // I can be in the range of [_MIN_OID, _MAX_OID] + assert(ghobj >= _MIN_OID()); + assert(ghobj <= _MAX_OID()); + } + /* + * common interfaces as a full_key_t + */ + shard_t shard() const { + return ghobj.shard_id; + } + pool_t pool() const { + return ghobj.hobj.pool; + } + crush_hash_t crush() const { + // Note: this is the reversed version of the object hash + return ghobj.hobj.get_bitwise_key_u32(); + } + laddr_t get_hint() const { + return get_lba_hint(shard(), pool(), crush()); + } + std::string_view nspace() const { + // TODO(cross-node string dedup) + return ghobj.hobj.nspace; + } + string_view_masked_t nspace_masked() const { + // TODO(cross-node string dedup) + return string_view_masked_t{nspace()}; + } + std::string_view oid() const { + // TODO(cross-node string dedup) + return ghobj.hobj.oid.name; + } + string_view_masked_t oid_masked() const { + // TODO(cross-node string dedup) + return string_view_masked_t{oid()}; + } + ns_oid_view_t::Type dedup_type() const { + return _dedup_type; + } + snap_t snap() const { + return ghobj.hobj.snap; + } + gen_t gen() const { + return ghobj.generation; + } + + std::ostream& dump(std::ostream& os) const { + os << "key_hobj(" << (int)shard() << "," + << pool() << ",0x" << std::hex << crush() << std::dec << "; " + << string_view_masked_t{nspace()} << "," + << string_view_masked_t{oid()} << "; " + << snap() << "," << gen() << ")"; + return os; + } + + bool is_valid() const { + return is_valid_key(*this); + } + + static key_hobj_t decode(ceph::bufferlist::const_iterator& delta) { + shard_t shard; + ceph::decode(shard, delta); + pool_t pool; + ceph::decode(pool, delta); + // Note: this is the reversed version of the object hash + crush_hash_t crush; + ceph::decode(crush, delta); + std::string nspace; + [[maybe_unused]] auto nspace_masked = string_view_masked_t::decode(nspace, delta); + // TODO(cross-node string dedup) + assert(nspace_masked.get_type() == string_view_masked_t::Type::STR); + std::string oid; + [[maybe_unused]] auto oid_masked = string_view_masked_t::decode(oid, delta); + // TODO(cross-node string dedup) + assert(oid_masked.get_type() == string_view_masked_t::Type::STR); + snap_t snap; + ceph::decode(snap, delta); + gen_t gen; + ceph::decode(gen, delta); + return key_hobj_t(ghobject_t( + shard_id_t(shard), pool, crush, nspace, oid, snap, gen)); + } + + private: + ns_oid_view_t::Type _dedup_type = ns_oid_view_t::Type::STR; + ghobject_t ghobj; +}; +inline std::ostream& operator<<(std::ostream& os, const key_hobj_t& key) { + return key.dump(os); +} + +struct shard_pool_t; +struct crush_t; +struct shard_pool_crush_t; +struct snap_gen_t; + +/** + * key_view_t + * + * A specialized implementation of a full_key_t pointing to the locations + * storing the full key in a tree node. + */ +class key_view_t { + public: + /** + * common interfaces as a full_key_t + */ + inline shard_t shard() const; + inline pool_t pool() const; + inline crush_hash_t crush() const; + laddr_t get_hint() const { + return get_lba_hint(shard(), pool(), crush()); + } + std::string_view nspace() const { + // TODO(cross-node string dedup) + return ns_oid_view().nspace.to_string_view(); + } + string_view_masked_t nspace_masked() const { + // TODO(cross-node string dedup) + return string_view_masked_t{ns_oid_view().nspace}; + } + std::string_view oid() const { + // TODO(cross-node string dedup) + return ns_oid_view().oid.to_string_view(); + } + string_view_masked_t oid_masked() const { + // TODO(cross-node string dedup) + return string_view_masked_t{ns_oid_view().oid}; + } + ns_oid_view_t::Type dedup_type() const { + return ns_oid_view().type(); + } + inline snap_t snap() const; + inline gen_t gen() const; + + /** + * key_view_t specific interfaces + */ + bool has_shard_pool() const { + return p_shard_pool != nullptr; + } + bool has_crush() const { + return p_crush != nullptr; + } + bool has_ns_oid() const { + return p_ns_oid.has_value(); + } + bool has_snap_gen() const { + return p_snap_gen != nullptr; + } + + const shard_pool_t& shard_pool_packed() const { + assert(has_shard_pool()); + return *p_shard_pool; + } + const crush_t& crush_packed() const { + assert(has_crush()); + return *p_crush; + } + const ns_oid_view_t& ns_oid_view() const { + assert(has_ns_oid()); + return *p_ns_oid; + } + const snap_gen_t& snap_gen_packed() const { + assert(has_snap_gen()); + return *p_snap_gen; + } + + size_t size_logical() const { + return sizeof(shard_t) + sizeof(pool_t) + sizeof(crush_hash_t) + + sizeof(snap_t) + sizeof(gen_t) + ns_oid_view().size_logical(); + } + + ghobject_t to_ghobj() const { + assert(is_valid_key(*this)); + return ghobject_t( + shard_id_t(shard()), pool(), crush(), + std::string(nspace()), std::string(oid()), snap(), gen()); + } + + void replace(const crush_t& key) { p_crush = &key; } + void set(const crush_t& key) { + assert(!has_crush()); + replace(key); + } + inline void replace(const shard_pool_crush_t& key); + inline void set(const shard_pool_crush_t& key); + void replace(const ns_oid_view_t& key) { p_ns_oid = key; } + void set(const ns_oid_view_t& key) { + assert(!has_ns_oid()); + replace(key); + } + void replace(const snap_gen_t& key) { p_snap_gen = &key; } + void set(const snap_gen_t& key) { + assert(!has_snap_gen()); + replace(key); + } + + void reset_to(const char* origin_base, + const char* new_base, + extent_len_t node_size) { + if (p_shard_pool != nullptr) { + reset_ptr(p_shard_pool, origin_base, new_base, node_size); + } + if (p_crush != nullptr) { + reset_ptr(p_crush, origin_base, new_base, node_size); + } + if (p_ns_oid.has_value()) { + p_ns_oid->reset_to(origin_base, new_base, node_size); + } + if (p_snap_gen != nullptr) { + reset_ptr(p_snap_gen, origin_base, new_base, node_size); + } + } + + std::ostream& dump(std::ostream& os) const { + os << "key_view("; + if (has_shard_pool()) { + os << (int)shard() << "," << pool() << ","; + } else { + os << "X,X,"; + } + if (has_crush()) { + os << "0x" << std::hex << crush() << std::dec << "; "; + } else { + os << "X; "; + } + if (has_ns_oid()) { + os << ns_oid_view() << "; "; + } else { + os << "X,X; "; + } + if (has_snap_gen()) { + os << snap() << "," << gen() << ")"; + } else { + os << "X,X)"; + } + return os; + } + + private: + const shard_pool_t* p_shard_pool = nullptr; + const crush_t* p_crush = nullptr; + std::optional<ns_oid_view_t> p_ns_oid; + const snap_gen_t* p_snap_gen = nullptr; +}; + +template<typename T> +concept IsFullKey = std::same_as<T, key_hobj_t> || std::same_as<T, key_view_t>; + +// TODO: consider alignments +struct shard_pool_t { + auto operator<=>(const shard_pool_t&) const = default; + + pool_t pool() const { return _pool; } + + template <IsFullKey Key> + static shard_pool_t from_key(const Key& key) { + if constexpr (std::same_as<Key, key_view_t>) { + return key.shard_pool_packed(); + } else { + return {key.shard(), key.pool()}; + } + } + + shard_t shard; + pool_t _pool; +} __attribute__((packed)); +inline std::ostream& operator<<(std::ostream& os, const shard_pool_t& sp) { + return os << (int)sp.shard << "," << sp.pool(); +} + +// Note: this is the reversed version of the object hash +struct crush_t { + auto operator<=>(const crush_t&) const = default; + + template <IsFullKey Key> + static crush_t from_key(const Key& key) { + if constexpr (std::same_as<Key, key_view_t>) { + return key.crush_packed(); + } else { + return {key.crush()}; + } + } + + crush_hash_t crush; +} __attribute__((packed)); +inline std::ostream& operator<<(std::ostream& os, const crush_t& c) { + return os << "0x" << std::hex << c.crush << std::dec; +} + +struct shard_pool_crush_t { + auto operator<=>(const shard_pool_crush_t&) const = default; + + template <IsFullKey Key> + static shard_pool_crush_t from_key(const Key& key) { + return {shard_pool_t::from_key(key), crush_t::from_key(key)}; + } + + shard_pool_t shard_pool; + crush_t crush; +} __attribute__((packed)); +inline std::ostream& operator<<(std::ostream& os, const shard_pool_crush_t& spc) { + return os << spc.shard_pool << ",0x" << std::hex << spc.crush << std::dec; +} + +struct snap_gen_t { + auto operator<=>(const snap_gen_t&) const = default; + + template <IsFullKey Key> + static snap_gen_t from_key(const Key& key) { + if constexpr (std::same_as<Key, key_view_t>) { + return key.snap_gen_packed(); + } else { + return {key.snap(), key.gen()}; + } + } + + snap_t snap; + gen_t gen; +} __attribute__((packed)); +inline std::ostream& operator<<(std::ostream& os, const snap_gen_t& sg) { + return os << sg.snap << "," << sg.gen; +} + +shard_t key_view_t::shard() const { + return shard_pool_packed().shard; +} + +pool_t key_view_t::pool() const { + return shard_pool_packed().pool(); +} + +crush_hash_t key_view_t::crush() const { + return crush_packed().crush; +} + +snap_t key_view_t::snap() const { + return snap_gen_packed().snap; +} + +gen_t key_view_t::gen() const { + return snap_gen_packed().gen; +} + +void key_view_t::replace(const shard_pool_crush_t& key) { + p_shard_pool = &key.shard_pool; +} + +void key_view_t::set(const shard_pool_crush_t& key) { + set(key.crush); + assert(!has_shard_pool()); + replace(key); +} + +template <IsFullKey Key> +void encode_key(const Key& key, ceph::bufferlist& bl) { + ceph::encode(key.shard(), bl); + ceph::encode(key.pool(), bl); + ceph::encode(key.crush(), bl); + key.nspace_masked().encode(bl); + key.oid_masked().encode(bl); + ceph::encode(key.snap(), bl); + ceph::encode(key.gen(), bl); +} + +template<IsFullKey LHS, IsFullKey RHS> +std::strong_ordering operator<=>(const LHS& lhs, const RHS& rhs) noexcept { + auto ret = lhs.shard() <=> rhs.shard(); + if (ret != 0) + return ret; + ret = lhs.pool() <=> rhs.pool(); + if (ret != 0) + return ret; + ret = lhs.crush() <=> rhs.crush(); + if (ret != 0) + return ret; + ret = lhs.nspace() <=> rhs.nspace(); + if (ret != 0) + return ret; + ret = lhs.oid() <=> rhs.oid(); + if (ret != 0) + return ret; + ret = lhs.snap() <=> rhs.snap(); + if (ret != 0) + return ret; + return lhs.gen() <=> rhs.gen(); +} + +template <typename Key> +bool is_valid_key(const Key& key) { + static_assert(IsFullKey<Key>); + return (key > key_hobj_t(ghobject_t()) && + key < key_hobj_t(ghobject_t::get_max())); +} + +inline std::ostream& operator<<(std::ostream& os, const key_view_t& key) { + return key.dump(os); +} + +template <IsFullKey T> +auto operator<=>(const T& key, const shard_pool_t& target) { + auto ret = key.shard() <=> target.shard; + if (ret != 0) + return ret; + return key.pool() <=> target.pool(); +} + +template <IsFullKey T> +auto operator<=>(const T& key, const crush_t& target) { + return key.crush() <=> target.crush; +} + +template <IsFullKey T> +auto operator<=>(const T& key, const shard_pool_crush_t& target) { + auto ret = key <=> target.shard_pool; + if (ret != 0) + return ret; + return key <=> target.crush; +} + +template <IsFullKey T> +auto operator<=>(const T& key, const ns_oid_view_t& target) { + auto ret = key.nspace() <=> string_view_masked_t{target.nspace}; + if (ret != 0) + return ret; + return key.oid() <=> string_view_masked_t{target.oid}; +} + +template <IsFullKey T> +auto operator<=>(const T& key, const snap_gen_t& target) { + auto ret = key.snap() <=> target.snap; + if (ret != 0) + return ret; + return key.gen() <=> target.gen; +} + +template <IsFullKey LHS, typename RHS> +bool operator==(LHS lhs, RHS rhs) { + return lhs <=> rhs == 0; +} + +template <typename Key> +node_offset_t ns_oid_view_t::estimate_size(const Key& key) { + static_assert(IsFullKey<Key>); + if constexpr (std::same_as<Key, key_view_t>) { + return key.ns_oid_view().size(); + } else { + if (key.dedup_type() != Type::STR) { + // size after deduplication + return sizeof(string_size_t); + } else { + return 2 * sizeof(string_size_t) + key.nspace().size() + key.oid().size(); + } + } +} + +template <typename Key> +void ns_oid_view_t::append( + NodeExtentMutable& mut, const Key& key, char*& p_append) { + static_assert(IsFullKey<Key>); + if (key.dedup_type() == Type::STR) { + string_key_view_t::append_str(mut, key.nspace(), p_append); + string_key_view_t::append_str(mut, key.oid(), p_append); + } else { + string_key_view_t::append_dedup(mut, key.dedup_type(), p_append); + } +} + +template <typename Key> +void ns_oid_view_t::test_append(const Key& key, char*& p_append) { + static_assert(IsFullKey<Key>); + if (key.dedup_type() == Type::STR) { + string_key_view_t::test_append_str(key.nspace(), p_append); + string_key_view_t::test_append_str(key.oid(), p_append); + } else { + string_key_view_t::test_append_dedup(key.dedup_type(), p_append); + } +} + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::onode::key_hobj_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::onode::key_view_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage.cc new file mode 100644 index 000000000..3ed401c37 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage.cc @@ -0,0 +1,420 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "node_stage.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_mutable.h" +#include "node_stage_layout.h" + +namespace crimson::os::seastore::onode { + +#define NODE_T node_extent_t<FieldType, NODE_TYPE> +#define NODE_INST(FT, NT) node_extent_t<FT, NT> + +template <typename FieldType, node_type_t NODE_TYPE> +const char* NODE_T::p_left_bound() const +{ + if constexpr (std::is_same_v<FieldType, internal_fields_3_t>) { + // N3 internal node doesn't have the right part + return nullptr; + } else { + auto ret = p_start() + + fields().get_item_end_offset(keys(), node_size); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + if (is_level_tail()) { + ret -= sizeof(laddr_t); + } + } + return ret; + } +} + +template <typename FieldType, node_type_t NODE_TYPE> +node_offset_t NODE_T::size_to_nxt_at(index_t index) const +{ + assert(index < keys()); + if constexpr (FIELD_TYPE == field_type_t::N0 || + FIELD_TYPE == field_type_t::N1) { + return FieldType::estimate_insert_one(); + } else if constexpr (FIELD_TYPE == field_type_t::N2) { + auto p_end = p_start() + + p_fields->get_item_end_offset(index, node_size); + return FieldType::estimate_insert_one() + ns_oid_view_t(p_end).size(); + } else { + ceph_abort("N3 node is not nested"); + } +} + +template <typename FieldType, node_type_t NODE_TYPE> +container_range_t NODE_T::get_nxt_container(index_t index) const +{ + if constexpr (std::is_same_v<FieldType, internal_fields_3_t>) { + ceph_abort("N3 internal node doesn't have the right part"); + } else { + auto item_start_offset = p_fields->get_item_start_offset( + index, node_size); + auto item_end_offset = p_fields->get_item_end_offset( + index, node_size); + assert(item_start_offset < item_end_offset); + auto item_p_start = p_start() + item_start_offset; + auto item_p_end = p_start() + item_end_offset; + if constexpr (FIELD_TYPE == field_type_t::N2) { + // range for sub_items_t<NODE_TYPE> + item_p_end = ns_oid_view_t(item_p_end).p_start(); + assert(item_p_start < item_p_end); + } else { + // range for item_iterator_t<NODE_TYPE> + } + return {{item_p_start, item_p_end}, node_size}; + } +} + +template <typename FieldType, node_type_t NODE_TYPE> +void NODE_T::bootstrap_extent( + NodeExtentMutable& mut, + field_type_t field_type, node_type_t node_type, + bool is_level_tail, level_t level) +{ + node_header_t::bootstrap_extent( + mut, field_type, node_type, is_level_tail, level); + mut.copy_in_relative( + sizeof(node_header_t), typename FieldType::num_keys_t(0u)); +} + +template <typename FieldType, node_type_t NODE_TYPE> +void NODE_T::update_is_level_tail( + NodeExtentMutable& mut, const node_extent_t& extent, bool value) +{ + assert(mut.get_length() == extent.node_size); + assert(mut.get_read() == extent.p_start()); + node_header_t::update_is_level_tail(mut, extent.p_fields->header, value); +} + +template <typename FieldType, node_type_t NODE_TYPE> +template <IsFullKey Key> +memory_range_t NODE_T::insert_prefix_at( + NodeExtentMutable& mut, const node_extent_t& node, const Key& key, + index_t index, node_offset_t size, const char* p_left_bound) +{ + assert(mut.get_length() == node.node_size); + assert(mut.get_read() == node.p_start()); + if constexpr (FIELD_TYPE == field_type_t::N0 || + FIELD_TYPE == field_type_t::N1) { + assert(index <= node.keys()); + assert(p_left_bound == node.p_left_bound()); + assert(size > FieldType::estimate_insert_one()); + auto size_right = size - FieldType::estimate_insert_one(); + const char* p_insert = node.p_start() + + node.fields().get_item_end_offset(index, mut.get_length()); + const char* p_insert_front = p_insert - size_right; + FieldType::insert_at(mut, key, node.fields(), index, size_right); + mut.shift_absolute(p_left_bound, + p_insert - p_left_bound, + -(int)size_right); + return {p_insert_front, p_insert}; + } else if constexpr (FIELD_TYPE == field_type_t::N2) { + ceph_abort("not implemented"); + } else { + ceph_abort("impossible"); + } +} +#define IPA_TEMPLATE(FT, NT, Key) \ + template memory_range_t NODE_INST(FT, NT)::insert_prefix_at<Key>( \ + NodeExtentMutable&, const node_extent_t&, const Key&, \ + index_t, node_offset_t, const char*) +IPA_TEMPLATE(node_fields_0_t, node_type_t::INTERNAL, key_view_t); +IPA_TEMPLATE(node_fields_1_t, node_type_t::INTERNAL, key_view_t); +IPA_TEMPLATE(node_fields_2_t, node_type_t::INTERNAL, key_view_t); +IPA_TEMPLATE(node_fields_0_t, node_type_t::LEAF, key_view_t); +IPA_TEMPLATE(node_fields_1_t, node_type_t::LEAF, key_view_t); +IPA_TEMPLATE(node_fields_2_t, node_type_t::LEAF, key_view_t); +IPA_TEMPLATE(node_fields_0_t, node_type_t::INTERNAL, key_hobj_t); +IPA_TEMPLATE(node_fields_1_t, node_type_t::INTERNAL, key_hobj_t); +IPA_TEMPLATE(node_fields_2_t, node_type_t::INTERNAL, key_hobj_t); +IPA_TEMPLATE(node_fields_0_t, node_type_t::LEAF, key_hobj_t); +IPA_TEMPLATE(node_fields_1_t, node_type_t::LEAF, key_hobj_t); +IPA_TEMPLATE(node_fields_2_t, node_type_t::LEAF, key_hobj_t); + +template <typename FieldType, node_type_t NODE_TYPE> +void NODE_T::update_size_at( + NodeExtentMutable& mut, const node_extent_t& node, index_t index, int change) +{ + assert(mut.get_length() == node.node_size); + assert(mut.get_read() == node.p_start()); + assert(index < node.keys()); + FieldType::update_size_at(mut, node.fields(), index, change); +} + +template <typename FieldType, node_type_t NODE_TYPE> +node_offset_t NODE_T::trim_until( + NodeExtentMutable& mut, const node_extent_t& node, index_t index) +{ + assert(mut.get_length() == node.node_size); + assert(mut.get_read() == node.p_start()); + assert(!node.is_level_tail()); + auto keys = node.keys(); + assert(index <= keys); + if (index == keys) { + return 0; + } + if constexpr (std::is_same_v<FieldType, internal_fields_3_t>) { + ceph_abort("not implemented"); + } else { + mut.copy_in_absolute( + (void*)&node.p_fields->num_keys, num_keys_t(index)); + } + // no need to calculate trim size for node + return 0; +} + +template <typename FieldType, node_type_t NODE_TYPE> +node_offset_t NODE_T::trim_at( + NodeExtentMutable& mut, const node_extent_t& node, + index_t index, node_offset_t trimmed) +{ + assert(mut.get_length() == node.node_size); + assert(mut.get_read() == node.p_start()); + assert(!node.is_level_tail()); + assert(index < node.keys()); + if constexpr (std::is_same_v<FieldType, internal_fields_3_t>) { + ceph_abort("not implemented"); + } else { + extent_len_t node_size = mut.get_length(); + node_offset_t offset = node.p_fields->get_item_start_offset( + index, node_size); + size_t new_offset = offset + trimmed; + assert(new_offset < node.p_fields->get_item_end_offset(index, node_size)); + mut.copy_in_absolute(const_cast<void*>(node.p_fields->p_offset(index)), + node_offset_t(new_offset)); + mut.copy_in_absolute( + (void*)&node.p_fields->num_keys, num_keys_t(index + 1)); + } + // no need to calculate trim size for node + return 0; +} + +template <typename FieldType, node_type_t NODE_TYPE> +node_offset_t NODE_T::erase_at( + NodeExtentMutable& mut, const node_extent_t& node, + index_t index, const char* p_left_bound) +{ + assert(mut.get_length() == node.node_size); + assert(mut.get_read() == node.p_start()); + if constexpr (FIELD_TYPE == field_type_t::N0 || + FIELD_TYPE == field_type_t::N1) { + assert(node.keys() > 0); + assert(index < node.keys()); + assert(p_left_bound == node.p_left_bound()); + return FieldType::erase_at(mut, node.fields(), index, p_left_bound); + } else { + ceph_abort("not implemented"); + } +} + +#define NODE_TEMPLATE(FT, NT) template class NODE_INST(FT, NT) +NODE_TEMPLATE(node_fields_0_t, node_type_t::INTERNAL); +NODE_TEMPLATE(node_fields_1_t, node_type_t::INTERNAL); +NODE_TEMPLATE(node_fields_2_t, node_type_t::INTERNAL); +NODE_TEMPLATE(internal_fields_3_t, node_type_t::INTERNAL); +NODE_TEMPLATE(node_fields_0_t, node_type_t::LEAF); +NODE_TEMPLATE(node_fields_1_t, node_type_t::LEAF); +NODE_TEMPLATE(node_fields_2_t, node_type_t::LEAF); +NODE_TEMPLATE(leaf_fields_3_t, node_type_t::LEAF); + +#define APPEND_T node_extent_t<FieldType, NODE_TYPE>::Appender<KT> + +template <typename FieldType, node_type_t NODE_TYPE> +template <KeyT KT> +APPEND_T::Appender(NodeExtentMutable* p_mut, const node_extent_t& node, bool open) + : p_mut{p_mut}, p_start{p_mut->get_write()} +{ + assert(p_start == node.p_start()); + assert(node.keys()); + assert(node.node_size == p_mut->get_length()); + extent_len_t node_size = node.node_size; + if (open) { + // seek as open_nxt() + if constexpr (FIELD_TYPE == field_type_t::N0 || + FIELD_TYPE == field_type_t::N1) { + p_append_left = p_start + node.fields().get_key_start_offset( + node.keys() - 1, node_size); + p_append_left += sizeof(typename FieldType::key_t); + p_append_right = p_start + + node.fields().get_item_end_offset(node.keys() - 1, + node_size); + } else if constexpr (FIELD_TYPE == field_type_t::N2) { + ceph_abort("not implemented"); + } else { + ceph_abort("impossible path"); + } + num_keys = node.keys() - 1; + } else { + if constexpr (std::is_same_v<FieldType, internal_fields_3_t>) { + std::ignore = node_size; + ceph_abort("not implemented"); + } else { + p_append_left = p_start + node.fields().get_key_start_offset( + node.keys(), node_size); + p_append_right = p_start + + node.fields().get_item_end_offset(node.keys(), + node_size); + } + num_keys = node.keys(); + } +} + +template <typename FieldType, node_type_t NODE_TYPE> +template <KeyT KT> +void APPEND_T::append(const node_extent_t& src, index_t from, index_t items) +{ + assert(from <= src.keys()); + if (p_src == nullptr) { + p_src = &src; + } else { + assert(p_src == &src); + } + assert(p_src->node_size == p_mut->get_length()); + extent_len_t node_size = src.node_size; + if (items == 0) { + return; + } + assert(from < src.keys()); + assert(from + items <= src.keys()); + num_keys += items; + if constexpr (std::is_same_v<FieldType, internal_fields_3_t>) { + std::ignore = node_size; + ceph_abort("not implemented"); + } else { + // append left part forwards + node_offset_t offset_left_start = src.fields().get_key_start_offset( + from, node_size); + node_offset_t offset_left_end = src.fields().get_key_start_offset( + from + items, node_size); + node_offset_t left_size = offset_left_end - offset_left_start; + if (num_keys == 0) { + // no need to adjust offset + assert(from == 0); + assert(p_start + offset_left_start == p_append_left); + p_mut->copy_in_absolute(p_append_left, + src.p_start() + offset_left_start, left_size); + } else { + node_offset_t step_size = FieldType::estimate_insert_one(); + extent_len_t offset_base = src.fields().get_item_end_offset( + from, node_size); + int offset_change = p_append_right - p_start - offset_base; + auto p_offset_dst = p_append_left; + if constexpr (FIELD_TYPE != field_type_t::N2) { + // copy keys + p_mut->copy_in_absolute(p_append_left, + src.p_start() + offset_left_start, left_size); + // point to offset for update + p_offset_dst += sizeof(typename FieldType::key_t); + } + for (auto i = from; i < from + items; ++i) { + int new_offset = src.fields().get_item_start_offset(i, node_size) + + offset_change; + assert(new_offset > 0); + assert(new_offset < (int)node_size); + p_mut->copy_in_absolute(p_offset_dst, node_offset_t(new_offset)); + p_offset_dst += step_size; + } + assert(p_append_left + left_size + sizeof(typename FieldType::key_t) == + p_offset_dst); + } + p_append_left += left_size; + + // append right part backwards + auto offset_right_start = src.fields().get_item_end_offset( + from + items, node_size); + auto offset_right_end = src.fields().get_item_end_offset( + from, node_size); + int right_size = offset_right_end - offset_right_start; + assert(right_size > 0); + assert(right_size < (int)node_size); + p_append_right -= right_size; + p_mut->copy_in_absolute(p_append_right, + src.p_start() + offset_right_start, node_offset_t(right_size)); + } +} + +template <typename FieldType, node_type_t NODE_TYPE> +template <KeyT KT> +void APPEND_T::append( + const full_key_t<KT>& key, const value_input_t& value, const value_t*& p_value) +{ + if constexpr (FIELD_TYPE == field_type_t::N3) { + ceph_abort("not implemented"); + } else { + ceph_abort("should not happen"); + } +} + +template <typename FieldType, node_type_t NODE_TYPE> +template <KeyT KT> +std::tuple<NodeExtentMutable*, char*> +APPEND_T::open_nxt(const key_get_type& partial_key) +{ + if constexpr (FIELD_TYPE == field_type_t::N0 || + FIELD_TYPE == field_type_t::N1) { + FieldType::append_key(*p_mut, partial_key, p_append_left); + } else if constexpr (FIELD_TYPE == field_type_t::N2) { + FieldType::append_key(*p_mut, partial_key, p_append_right); + } else { + ceph_abort("impossible path"); + } + return {p_mut, p_append_right}; +} + +template <typename FieldType, node_type_t NODE_TYPE> +template <KeyT KT> +std::tuple<NodeExtentMutable*, char*> +APPEND_T::open_nxt(const full_key_t<KT>& key) +{ + if constexpr (FIELD_TYPE == field_type_t::N0 || + FIELD_TYPE == field_type_t::N1) { + FieldType::append_key(*p_mut, key, p_append_left); + } else if constexpr (FIELD_TYPE == field_type_t::N2) { + FieldType::append_key(*p_mut, key, p_append_right); + } else { + ceph_abort("impossible path"); + } + return {p_mut, p_append_right}; +} + +template <typename FieldType, node_type_t NODE_TYPE> +template <KeyT KT> +char* APPEND_T::wrap() +{ + assert(p_append_left <= p_append_right); + assert(p_src); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + if (p_src->is_level_tail()) { + laddr_t tail_value = p_src->get_end_p_laddr()->value; + p_append_right -= sizeof(laddr_t); + assert(p_append_left <= p_append_right); + p_mut->copy_in_absolute(p_append_right, tail_value); + } + } + p_mut->copy_in_absolute(p_start + offsetof(FieldType, num_keys), num_keys); + return p_append_left; +} + +#define APPEND_TEMPLATE(FT, NT, KT) template class node_extent_t<FT, NT>::Appender<KT> +APPEND_TEMPLATE(node_fields_0_t, node_type_t::INTERNAL, KeyT::VIEW); +APPEND_TEMPLATE(node_fields_1_t, node_type_t::INTERNAL, KeyT::VIEW); +APPEND_TEMPLATE(node_fields_2_t, node_type_t::INTERNAL, KeyT::VIEW); +APPEND_TEMPLATE(internal_fields_3_t, node_type_t::INTERNAL, KeyT::VIEW); +APPEND_TEMPLATE(node_fields_0_t, node_type_t::LEAF, KeyT::VIEW); +APPEND_TEMPLATE(node_fields_1_t, node_type_t::LEAF, KeyT::VIEW); +APPEND_TEMPLATE(node_fields_2_t, node_type_t::LEAF, KeyT::VIEW); +APPEND_TEMPLATE(leaf_fields_3_t, node_type_t::LEAF, KeyT::VIEW); +APPEND_TEMPLATE(node_fields_0_t, node_type_t::INTERNAL, KeyT::HOBJ); +APPEND_TEMPLATE(node_fields_1_t, node_type_t::INTERNAL, KeyT::HOBJ); +APPEND_TEMPLATE(node_fields_2_t, node_type_t::INTERNAL, KeyT::HOBJ); +APPEND_TEMPLATE(internal_fields_3_t, node_type_t::INTERNAL, KeyT::HOBJ); +APPEND_TEMPLATE(node_fields_0_t, node_type_t::LEAF, KeyT::HOBJ); +APPEND_TEMPLATE(node_fields_1_t, node_type_t::LEAF, KeyT::HOBJ); +APPEND_TEMPLATE(node_fields_2_t, node_type_t::LEAF, KeyT::HOBJ); +APPEND_TEMPLATE(leaf_fields_3_t, node_type_t::LEAF, KeyT::HOBJ); + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage.h b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage.h new file mode 100644 index 000000000..5615998f8 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage.h @@ -0,0 +1,232 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_types.h" +#include "key_layout.h" +#include "stage_types.h" + +namespace crimson::os::seastore::onode { + +class NodeExtentMutable; + +/** + * node_extent_t + * + * The top indexing stage implementation for node N0/N1/N2/N3, implements + * staged contract as an indexable container, and provides access to node + * header. + * + * The specific field layout are defined by FieldType which are + * node_fields_0_t, node_fields_1_t, node_fields_2_t, internal_fields_3_t and + * leaf_fields_3_t. Diagrams see node_stage_layout.h. + */ +template <typename FieldType, node_type_t _NODE_TYPE> +class node_extent_t { + public: + using value_input_t = value_input_type_t<_NODE_TYPE>; + using value_t = value_type_t<_NODE_TYPE>; + using num_keys_t = typename FieldType::num_keys_t; + static constexpr node_type_t NODE_TYPE = _NODE_TYPE; + static constexpr field_type_t FIELD_TYPE = FieldType::FIELD_TYPE; + + // TODO: remove + node_extent_t() = default; + + node_extent_t(const FieldType* p_fields, extent_len_t node_size) + : p_fields{p_fields}, node_size{node_size} { + assert(is_valid_node_size(node_size)); + validate(*p_fields); + } + + const char* p_start() const { return fields_start(*p_fields); } + + bool is_level_tail() const { return p_fields->is_level_tail(); } + level_t level() const { return p_fields->header.level; } + node_offset_t free_size() const { + return p_fields->template free_size_before<NODE_TYPE>( + keys(), node_size); + } + extent_len_t total_size() const { + return p_fields->total_size(node_size); + } + const char* p_left_bound() const; + template <node_type_t T = NODE_TYPE> + std::enable_if_t<T == node_type_t::INTERNAL, const laddr_packed_t*> + get_end_p_laddr() const { + assert(is_level_tail()); + if constexpr (FIELD_TYPE == field_type_t::N3) { + return p_fields->get_p_child_addr(keys(), node_size); + } else { + auto offset_start = p_fields->get_item_end_offset( + keys(), node_size); + assert(offset_start <= node_size); + offset_start -= sizeof(laddr_packed_t); + auto p_addr = p_start() + offset_start; + return reinterpret_cast<const laddr_packed_t*>(p_addr); + } + } + + // container type system + using key_get_type = typename FieldType::key_get_type; + static constexpr auto CONTAINER_TYPE = ContainerType::INDEXABLE; + index_t keys() const { return p_fields->num_keys; } + key_get_type operator[] (index_t index) const { + return p_fields->get_key(index, node_size); + } + extent_len_t size_before(index_t index) const { + auto free_size = p_fields->template free_size_before<NODE_TYPE>( + index, node_size); + assert(total_size() >= free_size); + return total_size() - free_size; + } + node_offset_t size_to_nxt_at(index_t index) const; + node_offset_t size_overhead_at(index_t index) const { + return FieldType::ITEM_OVERHEAD; } + container_range_t get_nxt_container(index_t index) const; + + template <typename T = FieldType> + std::enable_if_t<T::FIELD_TYPE == field_type_t::N3, const value_t*> + get_p_value(index_t index) const { + assert(index < keys()); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + return p_fields->get_p_child_addr(index, node_size); + } else { + auto range = get_nxt_container(index).range; + auto ret = reinterpret_cast<const value_header_t*>(range.p_start); + assert(range.p_start + ret->allocation_size() == range.p_end); + return ret; + } + } + + void encode(const char* p_node_start, ceph::bufferlist& encoded) const { + assert(p_node_start == p_start()); + // nothing to encode as the container range is the entire extent + } + + static node_extent_t decode(const char* p_node_start, + extent_len_t node_size, + ceph::bufferlist::const_iterator& delta) { + // nothing to decode + return node_extent_t( + reinterpret_cast<const FieldType*>(p_node_start), + node_size); + } + + static void validate(const FieldType& fields) { +#ifndef NDEBUG + assert(fields.header.get_node_type() == NODE_TYPE); + assert(fields.header.get_field_type() == FieldType::FIELD_TYPE); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + assert(fields.header.level > 0u); + } else { + assert(fields.header.level == 0u); + } +#endif + } + + static void bootstrap_extent( + NodeExtentMutable&, field_type_t, node_type_t, bool, level_t); + + static void update_is_level_tail(NodeExtentMutable&, const node_extent_t&, bool); + + static node_offset_t header_size() { return FieldType::HEADER_SIZE; } + + template <IsFullKey Key> + static node_offset_t estimate_insert( + const Key& key, const value_input_t& value) { + auto size = FieldType::estimate_insert_one(); + if constexpr (FIELD_TYPE == field_type_t::N2) { + size += ns_oid_view_t::estimate_size(key); + } else if constexpr (FIELD_TYPE == field_type_t::N3 && + NODE_TYPE == node_type_t::LEAF) { + size += value.allocation_size(); + } + return size; + } + + template <IsFullKey Key> + static const value_t* insert_at( + NodeExtentMutable& mut, const node_extent_t&, + const Key& key, const value_input_t& value, + index_t index, node_offset_t size, const char* p_left_bound) { + if constexpr (FIELD_TYPE == field_type_t::N3) { + ceph_abort("not implemented"); + } else { + ceph_abort("impossible"); + } + } + + template <IsFullKey Key> + static memory_range_t insert_prefix_at( + NodeExtentMutable&, const node_extent_t&, + const Key& key, + index_t index, node_offset_t size, const char* p_left_bound); + + static void update_size_at( + NodeExtentMutable&, const node_extent_t&, index_t index, int change); + + static node_offset_t trim_until( + NodeExtentMutable&, const node_extent_t&, index_t index); + static node_offset_t trim_at(NodeExtentMutable&, const node_extent_t&, + index_t index, node_offset_t trimmed); + + static node_offset_t erase_at(NodeExtentMutable&, const node_extent_t&, + index_t index, const char* p_left_bound); + + template <KeyT KT> + class Appender; + + private: + const FieldType& fields() const { return *p_fields; } + const FieldType* p_fields; + extent_len_t node_size; +}; + +template <typename FieldType, node_type_t NODE_TYPE> +template <KeyT KT> +class node_extent_t<FieldType, NODE_TYPE>::Appender { + public: + Appender(NodeExtentMutable* p_mut, char* p_append) + : p_mut{p_mut}, p_start{p_append} { +#ifndef NDEBUG + auto p_fields = reinterpret_cast<const FieldType*>(p_append); + assert(*(p_fields->header.get_field_type()) == FIELD_TYPE); + assert(p_fields->header.get_node_type() == NODE_TYPE); + assert(p_fields->num_keys == 0); +#endif + p_append_left = p_start + FieldType::HEADER_SIZE; + p_append_right = p_start + p_mut->get_length(); + } + Appender(NodeExtentMutable*, const node_extent_t&, bool open = false); + void append(const node_extent_t& src, index_t from, index_t items); + void append(const full_key_t<KT>&, const value_input_t&, const value_t*&); + char* wrap(); + std::tuple<NodeExtentMutable*, char*> open_nxt(const key_get_type&); + std::tuple<NodeExtentMutable*, char*> open_nxt(const full_key_t<KT>&); + void wrap_nxt(char* p_append) { + if constexpr (FIELD_TYPE != field_type_t::N3) { + assert(p_append < p_append_right); + assert(p_append_left < p_append); + p_append_right = p_append; + auto new_offset = p_append - p_start; + assert(new_offset > 0); + assert(new_offset < p_mut->get_length()); + FieldType::append_offset(*p_mut, new_offset, p_append_left); + ++num_keys; + } else { + ceph_abort("not implemented"); + } + } + + private: + const node_extent_t* p_src = nullptr; + NodeExtentMutable* p_mut; + char* p_start; + char* p_append_left; + char* p_append_right; + num_keys_t num_keys = 0; +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage_layout.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage_layout.cc new file mode 100644 index 000000000..a0752e0fc --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage_layout.cc @@ -0,0 +1,153 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "node_stage_layout.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_mutable.h" + +namespace crimson::os::seastore::onode { + +void node_header_t::bootstrap_extent( + NodeExtentMutable& mut, + field_type_t field_type, node_type_t node_type, + bool is_level_tail, level_t level) +{ + node_header_t header; + header.set_field_type(field_type); + header.set_node_type(node_type); + header.set_is_level_tail(is_level_tail); + header.level = level; + mut.copy_in_relative(0, header); +} + +void node_header_t::update_is_level_tail( + NodeExtentMutable& mut, const node_header_t& header, bool value) +{ + auto& _header = const_cast<node_header_t&>(header); + _header.set_is_level_tail(value); + mut.validate_inplace_update(_header); +} + +#define F013_T _node_fields_013_t<SlotType> +#define F013_INST(ST) _node_fields_013_t<ST> + +template <typename SlotType> +void F013_T::update_size_at( + NodeExtentMutable& mut, const me_t& node, index_t index, int change) +{ + assert(index <= node.num_keys); + [[maybe_unused]] extent_len_t node_size = mut.get_length(); +#ifndef NDEBUG + // check underflow + if (change < 0 && index != node.num_keys) { + assert(node.get_item_start_offset(index, node_size) < + node.get_item_end_offset(index, node_size)); + } +#endif + for (const auto* p_slot = &node.slots[index]; + p_slot < &node.slots[node.num_keys]; + ++p_slot) { + node_offset_t offset = p_slot->right_offset; + int new_offset = offset - change; + assert(new_offset > 0); + assert(new_offset < (int)node_size); + mut.copy_in_absolute( + (void*)&(p_slot->right_offset), + node_offset_t(new_offset)); + } +#ifndef NDEBUG + // check overflow + if (change > 0 && index != node.num_keys) { + assert(node.num_keys > 0); + assert(node.get_key_start_offset(node.num_keys, node_size) <= + node.slots[node.num_keys - 1].right_offset); + } +#endif +} + +template <typename SlotType> +void F013_T::append_key( + NodeExtentMutable& mut, const key_t& key, char*& p_append) +{ + mut.copy_in_absolute(p_append, key); + p_append += sizeof(key_t); +} + +template <typename SlotType> +void F013_T::append_offset( + NodeExtentMutable& mut, node_offset_t offset_to_right, char*& p_append) +{ + mut.copy_in_absolute(p_append, offset_to_right); + p_append += sizeof(node_offset_t); +} + +template <typename SlotType> +template <IsFullKey Key> +void F013_T::insert_at( + NodeExtentMutable& mut, const Key& key, + const me_t& node, index_t index, node_offset_t size_right) +{ + assert(index <= node.num_keys); + extent_len_t node_size = mut.get_length(); + update_size_at(mut, node, index, size_right); + auto p_insert = const_cast<char*>(fields_start(node)) + + node.get_key_start_offset(index, node_size); + auto p_shift_end = fields_start(node) + + node.get_key_start_offset(node.num_keys, node_size); + mut.shift_absolute(p_insert, p_shift_end - p_insert, estimate_insert_one()); + mut.copy_in_absolute((void*)&node.num_keys, num_keys_t(node.num_keys + 1)); + append_key(mut, key_t::from_key(key), p_insert); + int new_offset = node.get_item_end_offset(index, node_size) - size_right; + assert(new_offset > 0); + assert(new_offset < (int)node_size); + append_offset(mut, new_offset, p_insert); +} +#define IA_TEMPLATE(ST, KT) template void F013_INST(ST):: \ + insert_at<KT>(NodeExtentMutable&, const KT&, \ + const F013_INST(ST)&, index_t, node_offset_t) +IA_TEMPLATE(slot_0_t, key_view_t); +IA_TEMPLATE(slot_1_t, key_view_t); +IA_TEMPLATE(slot_3_t, key_view_t); +IA_TEMPLATE(slot_0_t, key_hobj_t); +IA_TEMPLATE(slot_1_t, key_hobj_t); +IA_TEMPLATE(slot_3_t, key_hobj_t); + +template <typename SlotType> +node_offset_t F013_T::erase_at( + NodeExtentMutable& mut, const me_t& node, index_t index, const char* p_left_bound) +{ + extent_len_t node_size = mut.get_length(); + auto offset_item_start = node.get_item_start_offset(index, node_size); + auto offset_item_end = node.get_item_end_offset(index, node_size); + assert(offset_item_start < offset_item_end); + auto erase_size = offset_item_end - offset_item_start; + // fix and shift the left part + update_size_at(mut, node, index + 1, -erase_size); + const char* p_shift_start = fields_start(node) + + node.get_key_start_offset(index + 1, node_size); + extent_len_t shift_len = sizeof(SlotType) * (node.num_keys - index - 1); + int shift_off = -(int)sizeof(SlotType); + mut.shift_absolute(p_shift_start, shift_len, shift_off); + // shift the right part + p_shift_start = p_left_bound; + shift_len = fields_start(node) + offset_item_start - p_left_bound; + shift_off = erase_size; + mut.shift_absolute(p_shift_start, shift_len, shift_off); + // fix num_keys + mut.copy_in_absolute((void*)&node.num_keys, num_keys_t(node.num_keys - 1)); + return erase_size; +} + +#define F013_TEMPLATE(ST) template struct F013_INST(ST) +F013_TEMPLATE(slot_0_t); +F013_TEMPLATE(slot_1_t); +F013_TEMPLATE(slot_3_t); + +void node_fields_2_t::append_offset( + NodeExtentMutable& mut, node_offset_t offset_to_right, char*& p_append) +{ + mut.copy_in_absolute(p_append, offset_to_right); + p_append += sizeof(node_offset_t); +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage_layout.h b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage_layout.h new file mode 100644 index 000000000..1ed4865a6 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/node_stage_layout.h @@ -0,0 +1,406 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include "key_layout.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/node_types.h" + +namespace crimson::os::seastore::onode { + +class NodeExtentMutable; + +struct node_header_t { + static constexpr unsigned FIELD_TYPE_BITS = 6u; + static_assert(static_cast<uint8_t>(field_type_t::_MAX) <= 1u << FIELD_TYPE_BITS); + static constexpr unsigned NODE_TYPE_BITS = 1u; + static constexpr unsigned B_LEVEL_TAIL_BITS = 1u; + using bits_t = uint8_t; + + node_header_t() {} + std::optional<field_type_t> get_field_type() const { + if (field_type >= FIELD_TYPE_MAGIC && + field_type < static_cast<uint8_t>(field_type_t::_MAX)) { + return static_cast<field_type_t>(field_type); + } else { + return std::nullopt; + } + } + node_type_t get_node_type() const { + return static_cast<node_type_t>(node_type); + } + bool get_is_level_tail() const { + return is_level_tail; + } + + static void bootstrap_extent( + NodeExtentMutable&, field_type_t, node_type_t, bool, level_t); + + static void update_is_level_tail(NodeExtentMutable&, const node_header_t&, bool); + + bits_t field_type : FIELD_TYPE_BITS; + bits_t node_type : NODE_TYPE_BITS; + bits_t is_level_tail : B_LEVEL_TAIL_BITS; + static_assert(sizeof(bits_t) * 8 == + FIELD_TYPE_BITS + NODE_TYPE_BITS + B_LEVEL_TAIL_BITS); + level_t level; + + private: + void set_field_type(field_type_t type) { + field_type = static_cast<uint8_t>(type); + } + void set_node_type(node_type_t type) { + node_type = static_cast<uint8_t>(type); + } + void set_is_level_tail(bool value) { + is_level_tail = static_cast<uint8_t>(value); + } +} __attribute__((packed)); +inline std::ostream& operator<<(std::ostream& os, const node_header_t& header) { + auto field_type = header.get_field_type(); + if (field_type.has_value()) { + os << "header" << header.get_node_type() << *field_type + << "(is_level_tail=" << header.get_is_level_tail() + << ", level=" << (unsigned)header.level << ")"; + } else { + os << "header(INVALID)"; + } + return os; +} + +template <typename FixedKeyType, field_type_t _FIELD_TYPE> +struct _slot_t { + using key_t = FixedKeyType; + static constexpr field_type_t FIELD_TYPE = _FIELD_TYPE; + static constexpr node_offset_t OVERHEAD = sizeof(_slot_t) - sizeof(key_t); + + key_t key; + node_offset_t right_offset; +} __attribute__((packed)); +using slot_0_t = _slot_t<shard_pool_crush_t, field_type_t::N0>; +using slot_1_t = _slot_t<crush_t, field_type_t::N1>; +using slot_3_t = _slot_t<snap_gen_t, field_type_t::N3>; + +struct node_range_t { + extent_len_t start; + extent_len_t end; +}; + +template <typename FieldType> +const char* fields_start(const FieldType& node) { + return reinterpret_cast<const char*>(&node); +} + +template <node_type_t NODE_TYPE, typename FieldType> +node_range_t fields_free_range_before( + const FieldType& node, index_t index, extent_len_t node_size) { + assert(index <= node.num_keys); + extent_len_t offset_start = node.get_key_start_offset(index, node_size); + extent_len_t offset_end = node.get_item_end_offset(index, node_size); + if constexpr (NODE_TYPE == node_type_t::INTERNAL) { + if (node.is_level_tail() && index == node.num_keys) { + offset_end -= sizeof(laddr_t); + } + } + assert(offset_start <= offset_end); + assert(offset_end - offset_start < node_size); + return {offset_start, offset_end}; +} + +/** + * _node_fields_013_t (node_fields_0_t, node_fields_1_t, leaf_fields_3_t + * + * The STAGE_LEFT layout implementation for node N0/N1, or the STAGE_RIGHT + * layout implementation for leaf node N3. + * + * The node layout storing n slots: + * + * # <----------------------------- node range --------------------------------------> # + * # #<~># free space # + * # <----- left part -----------------------------> # <~# <----- right slots -------> # + * # # <---- left slots -------------> #~> # # + * # # slots [2, n) |<~># #<~>| right slots [2, n) # + * # # <- slot 0 -> | <- slot 1 -> | # # | <-- s1 --> | <-- s0 --> # + * # # | | # # | | # + * # | num_ # | right | | right | # # | next-stage | next-stage # + * # header | keys # key | offset | key | offset | # # | container | container # + * # | # 0 | 0 | 1 | 1 |...#...#...| or onode 1 | or onode 0 # + * | | ^ ^ + * | | | | + * | +----------------+ | + * +--------------------------------------------+ + */ +template <typename SlotType> +struct _node_fields_013_t { + // should be enough to index all keys under 64 KiB node + using num_keys_t = uint16_t; + using key_t = typename SlotType::key_t; + using key_get_type = const key_t&; + using me_t = _node_fields_013_t<SlotType>; + static constexpr field_type_t FIELD_TYPE = SlotType::FIELD_TYPE; + static constexpr node_offset_t HEADER_SIZE = + sizeof(node_header_t) + sizeof(num_keys_t); + static constexpr node_offset_t ITEM_OVERHEAD = SlotType::OVERHEAD; + + bool is_level_tail() const { return header.get_is_level_tail(); } + extent_len_t total_size(extent_len_t node_size) const { + return node_size; + } + key_get_type get_key( + index_t index, extent_len_t node_size) const { + assert(index < num_keys); + return slots[index].key; + } + node_offset_t get_key_start_offset( + index_t index, extent_len_t node_size) const { + assert(index <= num_keys); + auto offset = HEADER_SIZE + sizeof(SlotType) * index; + assert(offset < node_size); + return offset; + } + node_offset_t get_item_start_offset( + index_t index, extent_len_t node_size) const { + assert(index < num_keys); + auto offset = slots[index].right_offset; + assert(offset < node_size); + return offset; + } + const void* p_offset(index_t index) const { + assert(index < num_keys); + return &slots[index].right_offset; + } + extent_len_t get_item_end_offset( + index_t index, extent_len_t node_size) const { + return index == 0 ? node_size + : get_item_start_offset(index - 1, node_size); + } + template <node_type_t NODE_TYPE> + node_offset_t free_size_before( + index_t index, extent_len_t node_size) const { + auto range = fields_free_range_before<NODE_TYPE>(*this, index, node_size); + return range.end - range.start; + } + + static node_offset_t estimate_insert_one() { return sizeof(SlotType); } + template <IsFullKey Key> + static void insert_at( + NodeExtentMutable&, const Key& key, + const me_t& node, index_t index, node_offset_t size_right); + static node_offset_t erase_at(NodeExtentMutable&, const me_t&, index_t, const char*); + static void update_size_at( + NodeExtentMutable&, const me_t& node, index_t index, int change); + static void append_key( + NodeExtentMutable&, const key_t& key, char*& p_append); + template <IsFullKey Key> + static void append_key( + NodeExtentMutable& mut, const Key& key, char*& p_append) { + append_key(mut, key_t::from_key(key), p_append); + } + static void append_offset( + NodeExtentMutable& mut, node_offset_t offset_to_right, char*& p_append); + + node_header_t header; + num_keys_t num_keys = 0u; + SlotType slots[]; +} __attribute__((packed)); +using node_fields_0_t = _node_fields_013_t<slot_0_t>; +using node_fields_1_t = _node_fields_013_t<slot_1_t>; + +/** + * node_fields_2_t + * + * The STAGE_STRING layout implementation for node N2. + * + * The node layout storing n slots: + * + * # <--------------------------------- node range ----------------------------------------> # + * # #<~># free space # + * # <------- left part ---------------> # <~# <--------- right slots ---------------------> # + * # # <---- offsets ----> #~> #<~>| slots [2, n) # + * # # offsets [2, n) |<~># # | <----- slot 1 ----> | <----- slot 0 ----> # + * # # | # # | | # + * # | num_ # offset | offset | # # | next-stage | ns-oid | next-stage | ns-oid # + * # header | keys # 0 | 1 |...#...#...| container1 | 1 | container0 | 0 # + * | | ^ ^ + * | | | | + * | +----------------+ | + * +-----------------------------------------------+ + */ +struct node_fields_2_t { + // should be enough to index all keys under 64 KiB node + using num_keys_t = uint16_t; + using key_t = ns_oid_view_t; + using key_get_type = key_t; + static constexpr field_type_t FIELD_TYPE = field_type_t::N2; + static constexpr node_offset_t HEADER_SIZE = + sizeof(node_header_t) + sizeof(num_keys_t); + static constexpr node_offset_t ITEM_OVERHEAD = sizeof(node_offset_t); + + bool is_level_tail() const { return header.get_is_level_tail(); } + extent_len_t total_size(extent_len_t node_size) const { + return node_size; + } + key_get_type get_key( + index_t index, extent_len_t node_size) const { + assert(index < num_keys); + auto item_end_offset = get_item_end_offset(index, node_size); + const char* p_start = fields_start(*this); + return key_t(p_start + item_end_offset); + } + node_offset_t get_key_start_offset( + index_t index, extent_len_t node_size) const { + assert(index <= num_keys); + auto offset = HEADER_SIZE + sizeof(node_offset_t) * num_keys; + assert(offset < node_size); + return offset; + } + node_offset_t get_item_start_offset( + index_t index, extent_len_t node_size) const { + assert(index < num_keys); + auto offset = offsets[index]; + assert(offset < node_size); + return offset; + } + const void* p_offset(index_t index) const { + assert(index < num_keys); + return &offsets[index]; + } + extent_len_t get_item_end_offset( + index_t index, extent_len_t node_size) const { + return index == 0 ? node_size + : get_item_start_offset(index - 1, node_size); + } + template <node_type_t NODE_TYPE> + node_offset_t free_size_before( + index_t index, extent_len_t node_size) const { + auto range = fields_free_range_before<NODE_TYPE>(*this, index, node_size); + return range.end - range.start; + } + + static node_offset_t estimate_insert_one() { return sizeof(node_offset_t); } + template <IsFullKey Key> + static void insert_at( + NodeExtentMutable& mut, const Key& key, + const node_fields_2_t& node, index_t index, node_offset_t size_right) { + ceph_abort("not implemented"); + } + static void update_size_at( + NodeExtentMutable& mut, const node_fields_2_t& node, index_t index, int change) { + ceph_abort("not implemented"); + } + static void append_key( + NodeExtentMutable& mut, const key_t& key, char*& p_append) { + ns_oid_view_t::append(mut, key, p_append); + } + template <IsFullKey Key> + static void append_key( + NodeExtentMutable& mut, const Key& key, char*& p_append) { + ns_oid_view_t::append(mut, key, p_append); + } + static void append_offset( + NodeExtentMutable& mut, node_offset_t offset_to_right, char*& p_append); + + node_header_t header; + num_keys_t num_keys = 0u; + node_offset_t offsets[]; +} __attribute__((packed)); + +/** + * internal_fields_3_t + * + * The STAGE_RIGHT layout implementation for N2. + * + * The node layout storing 3 children: + * + * # <---------------- node range ---------------------------> # + * # # <-- keys ---> # <---- laddrs -----------> # + * # free space: # |<~># |<~># + * # # | # | # + * # | num_ # key | key | # laddr | laddr | laddr | # + * # header | keys # 0 | 1 |...# 0 | 1 | 2 |...# + */ +struct internal_fields_3_t { + using key_get_type = const snap_gen_t&; + // should be enough to index all keys under 64 KiB node + using num_keys_t = uint16_t; + static constexpr field_type_t FIELD_TYPE = field_type_t::N3; + static constexpr node_offset_t HEADER_SIZE = + sizeof(node_header_t) + sizeof(num_keys_t); + static constexpr node_offset_t ITEM_SIZE = + sizeof(snap_gen_t) + sizeof(laddr_t); + static constexpr node_offset_t ITEM_OVERHEAD = 0u; + + bool is_level_tail() const { return header.get_is_level_tail(); } + extent_len_t total_size(extent_len_t node_size) const { + if (is_level_tail()) { + return node_size - sizeof(snap_gen_t); + } else { + return node_size; + } + } + key_get_type get_key( + index_t index, extent_len_t node_size) const { + assert(index < num_keys); + return keys[index]; + } + template <node_type_t NODE_TYPE> + std::enable_if_t<NODE_TYPE == node_type_t::INTERNAL, node_offset_t> + free_size_before(index_t index, extent_len_t node_size) const { + assert(index <= num_keys); + assert(num_keys <= get_max_num_keys(node_size)); + extent_len_t free = total_size(node_size) - HEADER_SIZE - + index * ITEM_SIZE; + if (is_level_tail() && index == num_keys) { + free -= sizeof(laddr_t); + } + return free; + } + + const laddr_packed_t* get_p_child_addr( + index_t index, extent_len_t node_size) const { +#ifndef NDEBUG + if (is_level_tail()) { + assert(index <= num_keys); + } else { + assert(index < num_keys); + } +#endif + auto p_addrs = reinterpret_cast<const laddr_packed_t*>( + &keys[get_num_keys_limit(node_size)]); + auto ret = p_addrs + index; + assert((const char*)ret < fields_start(*this) + node_size); + return ret; + } + + static node_offset_t estimate_insert_one() { return ITEM_SIZE; } + + template <IsFullKey Key> + static void insert_at( + NodeExtentMutable& mut, const Key& key, + const internal_fields_3_t& node, + index_t index, node_offset_t size_right) { + ceph_abort("not implemented"); + } + static void update_size_at( + NodeExtentMutable& mut, const internal_fields_3_t& node, + index_t index, int change) { + ceph_abort("not implemented"); + } + + node_header_t header; + num_keys_t num_keys = 0u; + snap_gen_t keys[]; + + private: + num_keys_t get_max_num_keys(extent_len_t node_size) const { + auto num_limit = get_num_keys_limit(node_size); + return (is_level_tail() ? num_limit - 1 : num_limit); + } + static num_keys_t get_num_keys_limit(extent_len_t node_size) { + return (node_size - HEADER_SIZE) / ITEM_SIZE; + } +} __attribute__((packed)); + +using leaf_fields_3_t = _node_fields_013_t<slot_3_t>; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/stage.h b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/stage.h new file mode 100644 index 000000000..7185b15ee --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/stage.h @@ -0,0 +1,2488 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <cassert> +#include <compare> +#include <optional> +#include <ostream> +#include <sstream> +#include <type_traits> + +#include "common/likely.h" + +#include "sub_items_stage.h" +#include "item_iterator_stage.h" + +namespace crimson::os::seastore::onode { + +struct search_result_bs_t { + index_t index; + MatchKindBS match; +}; +template <typename FGetKey> +search_result_bs_t binary_search( + const key_hobj_t& key, + index_t begin, index_t end, FGetKey&& f_get_key) { + assert(begin <= end); + while (begin < end) { + auto total = begin + end; + auto mid = total >> 1; + // do not copy if return value is reference + decltype(f_get_key(mid)) target = f_get_key(mid); + auto match = key <=> target; + if (match == std::strong_ordering::less) { + end = mid; + } else if (match == std::strong_ordering::greater) { + begin = mid + 1; + } else { + return {mid, MatchKindBS::EQ}; + } + } + return {begin , MatchKindBS::NE}; +} + +template <typename PivotType, typename FGet> +search_result_bs_t binary_search_r( + index_t rend, index_t rbegin, FGet&& f_get, const PivotType& key) { + assert(rend <= rbegin); + while (rend < rbegin) { + auto total = rend + rbegin + 1; + auto mid = total >> 1; + // do not copy if return value is reference + decltype(f_get(mid)) target = f_get(mid); + int match = target - key; + if (match < 0) { + rend = mid; + } else if (match > 0) { + rbegin = mid - 1; + } else { + return {mid, MatchKindBS::EQ}; + } + } + return {rbegin, MatchKindBS::NE}; +} + +inline bool matchable(field_type_t type, match_stat_t mstat) { + assert(mstat >= MSTAT_MIN && mstat <= MSTAT_MAX); + /* + * compressed prefix by field type: + * N0: NONE + * N1: pool/shard + * N2: pool/shard crush + * N3: pool/shard crush ns/oid + * + * if key matches the node's compressed prefix, return true + * else, return false + */ +#ifndef NDEBUG + if (mstat == MSTAT_END) { + assert(type == field_type_t::N0); + } +#endif + return mstat + to_unsigned(type) < 4; +} + +inline void assert_mstat( + const key_hobj_t& key, + const key_view_t& index, + match_stat_t mstat) { + assert(mstat >= MSTAT_MIN && mstat <= MSTAT_LT2); + // key < index ... + switch (mstat) { + case MSTAT_EQ: + break; + case MSTAT_LT0: + assert(key < index.snap_gen_packed()); + break; + case MSTAT_LT1: + assert(key < index.ns_oid_view()); + break; + case MSTAT_LT2: + if (index.has_shard_pool()) { + assert((key < shard_pool_crush_t{ + index.shard_pool_packed(), index.crush_packed()})); + } else { + assert(key < index.crush_packed()); + } + break; + default: + ceph_abort("impossible path"); + } + // key == index ... + switch (mstat) { + case MSTAT_EQ: + assert(key == index.snap_gen_packed()); + case MSTAT_LT0: + if (!index.has_ns_oid()) + break; + assert(index.ns_oid_view().type() == ns_oid_view_t::Type::MAX || + key == index.ns_oid_view()); + case MSTAT_LT1: + if (!index.has_crush()) + break; + assert(key == index.crush_packed()); + if (!index.has_shard_pool()) + break; + assert(key == index.shard_pool_packed()); + default: + break; + } +} + +#define NXT_STAGE_T staged<next_param_t> + +enum class TrimType { BEFORE, AFTER, AT }; + +/** + * staged + * + * Implements recursive logic that modifies or reads the node layout + * (N0/N1/N2/N3 * LEAF/INTERNAL) with the multi-stage design. The specific + * stage implementation is flexible. So the implementations for different + * stages can be assembled independently, as long as they follow the + * definitions of container interfaces. + * + * Multi-stage is designed to index different portions of onode keys + * stage-by-stage. There are at most 3 stages for a node: + * - STAGE_LEFT: index shard-pool-crush for N0, or index crush for N1 node; + * - STAGE_STRING: index ns-oid for N0/N1/N2 nodes; + * - STAGE_RIGHT: index snap-gen for N0/N1/N2/N3 nodes; + * + * The intention is to consolidate the high-level indexing implementations at + * the level of stage, so we don't need to write them repeatedly for every + * stage and for every node type. + */ +template <typename Params> +struct staged { + static_assert(Params::STAGE >= STAGE_BOTTOM); + static_assert(Params::STAGE <= STAGE_TOP); + using container_t = typename Params::container_t; + using key_get_type = typename container_t::key_get_type; + using next_param_t = typename Params::next_param_t; + using position_t = staged_position_t<Params::STAGE>; + using result_t = staged_result_t<Params::NODE_TYPE, Params::STAGE>; + using value_input_t = value_input_type_t<Params::NODE_TYPE>; + using value_t = value_type_t<Params::NODE_TYPE>; + static constexpr auto CONTAINER_TYPE = container_t::CONTAINER_TYPE; + static constexpr bool IS_BOTTOM = (Params::STAGE == STAGE_BOTTOM); + static constexpr auto NODE_TYPE = Params::NODE_TYPE; + static constexpr auto STAGE = Params::STAGE; + + template <bool is_exclusive> + static void _left_or_right(index_t& split_index, index_t insert_index, + std::optional<bool>& is_insert_left) { + assert(!is_insert_left.has_value()); + assert(is_valid_index(split_index)); + if constexpr (is_exclusive) { + if (split_index <= insert_index) { + // ...[s_index-1] |!| (i_index) [s_index]... + // offset i_position to right + is_insert_left = false; + } else { + // ...[s_index-1] (i_index)) |?[s_index]| ... + // ...(i_index)...[s_index-1] |?[s_index]| ... + is_insert_left = true; + --split_index; + } + } else { + if (split_index < insert_index) { + // ...[s_index-1] |?[s_index]| ...[(i_index)[s_index_k]... + is_insert_left = false; + } else if (split_index > insert_index) { + // ...[(i_index)s_index-1] |?[s_index]| ... + // ...[(i_index)s_index_k]...[s_index-1] |?[s_index]| ... + is_insert_left = true; + } else { + // ...[s_index-1] |?[(i_index)s_index]| ... + // i_to_left = std::nullopt; + } + } + } + + template <ContainerType CTYPE, typename Enable = void> class _iterator_t; + template <ContainerType CTYPE> + class _iterator_t<CTYPE, std::enable_if_t<CTYPE == ContainerType::INDEXABLE>> { + /* + * indexable container type system: + * CONTAINER_TYPE = ContainerType::INDEXABLE + * keys() const -> index_t + * operator[](index_t) const -> key_get_type + * size_before(index_t) const -> extent_len_t + * size_overhead_at(index_t) const -> node_offset_t + * (IS_BOTTOM) get_p_value(index_t) const -> const value_t* + * (!IS_BOTTOM) size_to_nxt_at(index_t) const -> node_offset_t + * (!IS_BOTTOM) get_nxt_container(index_t) const + * encode(p_node_start, encoded) + * decode(p_node_start, node_size, delta) -> container_t + * static: + * header_size() -> node_offset_t + * estimate_insert(key, value) -> node_offset_t + * (IS_BOTTOM) insert_at(mut, src, key, value, + * index, size, p_left_bound) -> const value_t* + * (!IS_BOTTOM) insert_prefix_at(mut, src, key, + * index, size, p_left_bound) -> memory_range_t + * (!IS_BOTTOM) update_size_at(mut, src, index, size) + * trim_until(mut, container, index) -> trim_size + * (!IS_BOTTOM) trim_at(mut, container, index, trimmed) -> trim_size + * erase_at(mut, container, index, p_left_bound) -> erase_size + * + * Appender::append(const container_t& src, from, items) + */ + public: + using me_t = _iterator_t<CTYPE>; + + _iterator_t(const container_t& container) : container{container} { + assert(container.keys()); + } + + index_t index() const { + return _index; + } + key_get_type get_key() const { + assert(!is_end()); + return container[_index]; + } + node_offset_t size_to_nxt() const { + assert(!is_end()); + return container.size_to_nxt_at(_index); + } + template <typename T = typename NXT_STAGE_T::container_t> + std::enable_if_t<!IS_BOTTOM, T> get_nxt_container() const { + assert(!is_end()); + return container.get_nxt_container(_index); + } + template <typename T = value_t> + std::enable_if_t<IS_BOTTOM, const T*> get_p_value() const { + assert(!is_end()); + return container.get_p_value(_index); + } + bool is_last() const { + return _index + 1 == container.keys(); + } + bool is_end() const { return _index == container.keys(); } + node_offset_t size() const { + assert(!is_end()); + assert(header_size() == container.size_before(0)); + assert(container.size_before(_index + 1) > container.size_before(_index)); + return container.size_before(_index + 1) - + container.size_before(_index); + } + node_offset_t size_overhead() const { + assert(!is_end()); + return container.size_overhead_at(_index); + } + + me_t& operator++() { + assert(!is_end()); + assert(!is_last()); + ++_index; + return *this; + } + void seek_at(index_t index) { + assert(index < container.keys()); + seek_till_end(index); + } + void seek_till_end(index_t index) { + assert(!is_end()); + assert(this->index() == 0); + assert(index <= container.keys()); + _index = index; + } + void seek_last() { + assert(!is_end()); + assert(index() == 0); + _index = container.keys() - 1; + } + void set_end() { + assert(!is_end()); + assert(is_last()); + ++_index; + } + // Note: possible to return an end iterator + MatchKindBS seek(const key_hobj_t& key, bool exclude_last) { + assert(!is_end()); + assert(index() == 0); + index_t end_index = container.keys(); + if (exclude_last) { + assert(end_index); + --end_index; + assert(key < container[end_index]); + } + auto ret = binary_search(key, _index, end_index, + [this] (index_t index) { return container[index]; }); + _index = ret.index; + return ret.match; + } + + template <IsFullKey Key, typename T = value_t> + std::enable_if_t<IS_BOTTOM, const T*> insert( + NodeExtentMutable& mut, + const Key& key, + const value_input_t& value, + node_offset_t insert_size, + const char* p_left_bound) { + return container_t::insert_at( + mut, container, key, value, _index, insert_size, p_left_bound); + } + + template <IsFullKey Key, typename T = memory_range_t> + std::enable_if_t<!IS_BOTTOM, T> insert_prefix( + NodeExtentMutable& mut, const Key& key, + node_offset_t size, const char* p_left_bound) { + return container_t::insert_prefix_at( + mut, container, key, _index, size, p_left_bound); + } + + template <typename T = void> + std::enable_if_t<!IS_BOTTOM, T> + update_size(NodeExtentMutable& mut, int insert_size) { + assert(!is_end()); + container_t::update_size_at(mut, container, _index, insert_size); + } + + // Note: possible to return an end iterator when is_exclusive is true + template <bool is_exclusive> + size_t seek_split_inserted( + size_t start_size, size_t extra_size, size_t target_size, + index_t& insert_index, size_t insert_size, + std::optional<bool>& is_insert_left) { + assert(!is_end()); + assert(index() == 0); + // replace insert_index placeholder + if constexpr (!is_exclusive) { + if (insert_index == INDEX_LAST) { + insert_index = container.keys() - 1; + } + } else { + if (insert_index == INDEX_END) { + insert_index = container.keys(); + } + } + assert(insert_index <= container.keys()); + + auto start_size_1 = start_size + extra_size; + auto f_get_used_size = [this, start_size, start_size_1, + insert_index, insert_size] (index_t index) { + size_t current_size; + if (unlikely(index == 0)) { + current_size = start_size; + } else { + current_size = start_size_1; + if (index > insert_index) { + current_size += insert_size; + if constexpr (is_exclusive) { + --index; + } + } + // already includes header size + current_size += container.size_before(index); + } + return current_size; + }; + index_t s_end; + if constexpr (is_exclusive) { + s_end = container.keys(); + } else { + s_end = container.keys() - 1; + } + _index = binary_search_r(0, s_end, f_get_used_size, target_size).index; + size_t current_size = f_get_used_size(_index); + assert(current_size <= target_size); + + _left_or_right<is_exclusive>(_index, insert_index, is_insert_left); + return current_size; + } + + size_t seek_split(size_t start_size, size_t extra_size, size_t target_size) { + assert(!is_end()); + assert(index() == 0); + auto start_size_1 = start_size + extra_size; + auto f_get_used_size = [this, start_size, start_size_1] (index_t index) { + size_t current_size; + if (unlikely(index == 0)) { + current_size = start_size; + } else { + // already includes header size + current_size = start_size_1 + container.size_before(index); + } + return current_size; + }; + _index = binary_search_r( + 0, container.keys() - 1, f_get_used_size, target_size).index; + size_t current_size = f_get_used_size(_index); + assert(current_size <= target_size); + return current_size; + } + + // Note: possible to return an end iterater if to_index == INDEX_END + template <KeyT KT> + void copy_out_until( + typename container_t::template Appender<KT>& appender, index_t& to_index) { + auto num_keys = container.keys(); + index_t items; + if (to_index == INDEX_END) { + items = num_keys - _index; + appender.append(container, _index, items); + _index = num_keys; + to_index = _index; + } else if (to_index == INDEX_LAST) { + assert(!is_end()); + items = num_keys - 1 - _index; + appender.append(container, _index, items); + _index = num_keys - 1; + to_index = _index; + } else { + assert(_index <= to_index); + assert(to_index <= num_keys); + items = to_index - _index; + appender.append(container, _index, items); + _index = to_index; + } + } + + node_offset_t trim_until(NodeExtentMutable& mut) { + return container_t::trim_until(mut, container, _index); + } + + template <typename T = node_offset_t> + std::enable_if_t<!IS_BOTTOM, T> + trim_at(NodeExtentMutable& mut, node_offset_t trimmed) { + return container_t::trim_at(mut, container, _index, trimmed); + } + + node_offset_t erase(NodeExtentMutable& mut, const char* p_left_bound) { + assert(!is_end()); + return container_t::erase_at(mut, container, _index, p_left_bound); + } + + template <KeyT KT> + typename container_t::template Appender<KT> + get_appender(NodeExtentMutable* p_mut) { + assert(_index + 1 == container.keys()); + return typename container_t::template Appender<KT>(p_mut, container); + } + + template <KeyT KT> + typename container_t::template Appender<KT> + get_appender_opened(NodeExtentMutable* p_mut) { + if constexpr (!IS_BOTTOM) { + assert(_index + 1 == container.keys()); + return typename container_t::template Appender<KT>(p_mut, container, true); + } else { + ceph_abort("impossible path"); + } + } + + void encode(const char* p_node_start, ceph::bufferlist& encoded) const { + container.encode(p_node_start, encoded); + ceph::encode(_index, encoded); + } + + static me_t decode(const char* p_node_start, + extent_len_t node_size, + ceph::bufferlist::const_iterator& delta) { + auto container = container_t::decode( + p_node_start, node_size, delta); + auto ret = me_t(container); + index_t index; + ceph::decode(index, delta); + ret.seek_till_end(index); + return ret; + } + + static node_offset_t header_size() { + return container_t::header_size(); + } + + template <IsFullKey Key> + static node_offset_t estimate_insert( + const Key& key, const value_input_t& value) { + return container_t::estimate_insert(key, value); + } + + private: + container_t container; + index_t _index = 0; + }; + + template <ContainerType CTYPE> + class _iterator_t<CTYPE, std::enable_if_t<CTYPE == ContainerType::ITERATIVE>> { + /* + * iterative container type system (!IS_BOTTOM): + * CONTAINER_TYPE = ContainerType::ITERATIVE + * index() const -> index_t + * get_key() const -> key_get_type + * size() const -> node_offset_t + * size_to_nxt() const -> node_offset_t + * size_overhead() const -> node_offset_t + * get_nxt_container() const + * has_next() const -> bool + * encode(p_node_start, encoded) + * decode(p_node_start, node_length, delta) -> container_t + * operator++() + * static: + * header_size() -> node_offset_t + * estimate_insert(key, value) -> node_offset_t + * insert_prefix(mut, src, key, is_end, size, p_left_bound) -> memory_range_t + * update_size(mut, src, size) + * trim_until(mut, container) -> trim_size + * trim_at(mut, container, trimmed) -> trim_size + * erase(mut, container, p_left_bound) -> erase_size + */ + // currently the iterative iterator is only implemented with STAGE_STRING + // for in-node space efficiency + static_assert(STAGE == STAGE_STRING); + public: + using me_t = _iterator_t<CTYPE>; + + _iterator_t(const container_t& container) : container{container} {} + + index_t index() const { + if (is_end()) { + return container.index() + 1; + } else { + return container.index(); + } + } + key_get_type get_key() const { + assert(!is_end()); + return container.get_key(); + } + node_offset_t size_to_nxt() const { + assert(!is_end()); + return container.size_to_nxt(); + } + const typename NXT_STAGE_T::container_t get_nxt_container() const { + assert(!is_end()); + return container.get_nxt_container(); + } + bool is_last() const { + assert(!is_end()); + return !container.has_next(); + } + bool is_end() const { +#ifndef NDEBUG + if (_is_end) { + assert(!container.has_next()); + } +#endif + return _is_end; + } + node_offset_t size() const { + assert(!is_end()); + return container.size(); + } + node_offset_t size_overhead() const { + assert(!is_end()); + return container.size_overhead(); + } + + me_t& operator++() { + assert(!is_end()); + assert(!is_last()); + ++container; + return *this; + } + void seek_at(index_t index) { + assert(!is_end()); + assert(this->index() == 0); + while (index > 0) { + assert(container.has_next()); + ++container; + --index; + } + } + void seek_till_end(index_t index) { + assert(!is_end()); + assert(this->index() == 0); + while (index > 0) { + if (!container.has_next()) { + assert(index == 1); + set_end(); + break; + } + ++container; + --index; + } + } + void seek_last() { + assert(!is_end()); + assert(index() == 0); + while (container.has_next()) { + ++container; + } + } + void set_end() { + assert(!is_end()); + assert(is_last()); + _is_end = true; + } + // Note: possible to return an end iterator + MatchKindBS seek(const key_hobj_t& key, bool exclude_last) { + assert(!is_end()); + assert(index() == 0); + do { + if (exclude_last && is_last()) { + assert(key < get_key()); + return MatchKindBS::NE; + } + auto match = key <=> get_key(); + if (match == std::strong_ordering::less) { + return MatchKindBS::NE; + } else if (match == std::strong_ordering::equal) { + return MatchKindBS::EQ; + } else { + if (container.has_next()) { + ++container; + } else { + // end + break; + } + } + } while (true); + assert(!exclude_last); + set_end(); + return MatchKindBS::NE; + } + + template <IsFullKey Key> + memory_range_t insert_prefix( + NodeExtentMutable& mut, const Key& key, + node_offset_t size, const char* p_left_bound) { + return container_t::insert_prefix( + mut, container, key, is_end(), size, p_left_bound); + } + + void update_size(NodeExtentMutable& mut, int insert_size) { + assert(!is_end()); + container_t::update_size(mut, container, insert_size); + } + + // Note: possible to return an end iterator when is_exclusive is true + // insert_index can still be INDEX_LAST or INDEX_END + template <bool is_exclusive> + size_t seek_split_inserted( + size_t start_size, size_t extra_size, size_t target_size, + index_t& insert_index, size_t insert_size, + std::optional<bool>& is_insert_left) { + assert(!is_end()); + assert(index() == 0); + size_t current_size = start_size; + index_t split_index = 0; + extra_size += header_size(); + do { + if constexpr (!is_exclusive) { + if (is_last()) { + assert(split_index == index()); + if (insert_index == INDEX_LAST) { + insert_index = index(); + } + assert(insert_index <= index()); + break; + } + } + + size_t nxt_size = current_size; + if (split_index == 0) { + nxt_size += extra_size; + } + if (split_index == insert_index) { + nxt_size += insert_size; + if constexpr (is_exclusive) { + if (nxt_size > target_size) { + break; + } + current_size = nxt_size; + ++split_index; + } + } + nxt_size += size(); + if (nxt_size > target_size) { + break; + } + current_size = nxt_size; + + if constexpr (is_exclusive) { + if (is_last()) { + assert(split_index == index()); + set_end(); + split_index = index(); + if (insert_index == INDEX_END) { + insert_index = index(); + } + assert(insert_index == index()); + break; + } else { + ++(*this); + ++split_index; + } + } else { + ++(*this); + ++split_index; + } + } while (true); + assert(current_size <= target_size); + + _left_or_right<is_exclusive>(split_index, insert_index, is_insert_left); + assert(split_index == index()); + return current_size; + } + + size_t seek_split(size_t start_size, size_t extra_size, size_t target_size) { + assert(!is_end()); + assert(index() == 0); + size_t current_size = start_size; + do { + if (is_last()) { + break; + } + + size_t nxt_size = current_size; + if (index() == 0) { + nxt_size += extra_size; + } + nxt_size += size(); + if (nxt_size > target_size) { + break; + } + current_size = nxt_size; + ++(*this); + } while (true); + assert(current_size <= target_size); + return current_size; + } + + // Note: possible to return an end iterater if to_index == INDEX_END + template <KeyT KT> + void copy_out_until( + typename container_t::template Appender<KT>& appender, index_t& to_index) { + if (is_end()) { + assert(!container.has_next()); + if (to_index == INDEX_END) { + to_index = index(); + } + assert(to_index == index()); + return; + } + index_t items; + if (to_index == INDEX_END || to_index == INDEX_LAST) { + items = to_index; + } else { + assert(is_valid_index(to_index)); + assert(index() <= to_index); + items = to_index - index(); + } + if (appender.append(container, items)) { + set_end(); + } + to_index = index(); + } + + node_offset_t trim_until(NodeExtentMutable& mut) { + if (is_end()) { + return 0; + } + return container_t::trim_until(mut, container); + } + + node_offset_t trim_at(NodeExtentMutable& mut, node_offset_t trimmed) { + assert(!is_end()); + return container_t::trim_at(mut, container, trimmed); + } + + node_offset_t erase(NodeExtentMutable& mut, const char* p_left_bound) { + assert(!is_end()); + return container_t::erase(mut, container, p_left_bound); + } + + template <KeyT KT> + typename container_t::template Appender<KT> + get_appender(NodeExtentMutable* p_mut) { + return typename container_t::template Appender<KT>(p_mut, container, false); + } + + template <KeyT KT> + typename container_t::template Appender<KT> + get_appender_opened(NodeExtentMutable* p_mut) { + if constexpr (!IS_BOTTOM) { + return typename container_t::template Appender<KT>(p_mut, container, true); + } else { + ceph_abort("impossible path"); + } + } + + void encode(const char* p_node_start, ceph::bufferlist& encoded) const { + container.encode(p_node_start, encoded); + uint8_t is_end = _is_end; + ceph::encode(is_end, encoded); + } + + static me_t decode(const char* p_node_start, + extent_len_t node_size, + ceph::bufferlist::const_iterator& delta) { + auto container = container_t::decode( + p_node_start, node_size, delta); + auto ret = me_t(container); + uint8_t is_end; + ceph::decode(is_end, delta); + if (is_end) { + ret.set_end(); + } + return ret; + } + + static node_offset_t header_size() { + return container_t::header_size(); + } + + template <IsFullKey Key> + static node_offset_t estimate_insert(const Key& key, + const value_input_t& value) { + return container_t::estimate_insert(key, value); + } + + private: + container_t container; + bool _is_end = false; + }; + + /* + * iterator_t encapsulates both indexable and iterative implementations + * from a *non-empty* container. + * cstr(const container_t&) + * access: + * index() -> index_t + * get_key() -> key_get_type (const reference or value type) + * is_last() -> bool + * is_end() -> bool + * size() -> node_offset_t + * size_overhead() -> node_offset_t + * (IS_BOTTOM) get_p_value() -> const value_t* + * (!IS_BOTTOM) get_nxt_container() -> container_range_t + * (!IS_BOTTOM) size_to_nxt() -> node_offset_t + * seek: + * operator++() -> iterator_t& + * seek_at(index) + * seek_till_end(index) + * seek_last() + * set_end() + * seek(key, exclude_last) -> MatchKindBS + * insert: + * (IS_BOTTOM) insert(mut, key, value, size, p_left_bound) -> p_value + * (!IS_BOTTOM) insert_prefix(mut, key, size, p_left_bound) -> memory_range_t + * (!IS_BOTTOM) update_size(mut, size) + * split: + * seek_split_inserted<bool is_exclusive>( + * start_size, extra_size, target_size, insert_index, insert_size, + * std::optional<bool>& is_insert_left) + * -> insert to left/right/unknown (!exclusive) + * -> insert to left/right (exclusive, can be end) + * -> split_size + * seek_split(start_size, extra_size, target_size) -> split_size + * copy_out_until(appender, to_index) (can be end) + * trim_until(mut) -> trim_size + * (!IS_BOTTOM) trim_at(mut, trimmed) -> trim_size + * erase: + * erase(mut, p_left_bound) -> erase_size + * merge: + * get_appender(p_mut) -> Appender + * (!IS_BOTTOM)get_appender_opened(p_mut) -> Appender + * denc: + * encode(p_node_start, encoded) + * decode(p_node_start, node_size, delta) -> iterator_t + * static: + * header_size() -> node_offset_t + * estimate_insert(key, value) -> node_offset_t + */ + using iterator_t = _iterator_t<CONTAINER_TYPE>; + /* TODO: detailed comments + * - trim_until(mut) -> trim_size + * * keep 0 to i - 1, and remove the rest, return the size trimmed. + * * if this is the end iterator, do nothing and return 0. + * * if this is the start iterator, normally needs to go to the higher + * stage to trim the entire container. + * - trim_at(mut, trimmed) -> trim_size + * * trim happens inside the current iterator, causing the size reduced by + * <trimmed>, return the total size trimmed. + */ + + /* + * Lookup internals (hide?) + */ + + static bool is_keys_one( + const container_t& container) { // IN + auto iter = iterator_t(container); + iter.seek_last(); + if (iter.index() == 0) { + if constexpr (IS_BOTTOM) { + // ok, there is only 1 key + return true; + } else { + auto nxt_container = iter.get_nxt_container(); + return NXT_STAGE_T::is_keys_one(nxt_container); + } + } else { + // more than 1 keys + return false; + } + } + + template <bool GET_KEY> + static result_t smallest_result( + const iterator_t& iter, key_view_t* p_index_key) { + static_assert(!IS_BOTTOM); + assert(!iter.is_end()); + auto nxt_container = iter.get_nxt_container(); + auto pos_smallest = NXT_STAGE_T::position_t::begin(); + const value_t* p_value; + NXT_STAGE_T::template get_slot<GET_KEY, true>( + nxt_container, pos_smallest, p_index_key, &p_value); + if constexpr (GET_KEY) { + assert(p_index_key); + p_index_key->set(iter.get_key()); + } else { + assert(!p_index_key); + } + return result_t{{iter.index(), pos_smallest}, p_value, STAGE}; + } + + template <bool GET_KEY> + static result_t nxt_lower_bound( + const key_hobj_t& key, iterator_t& iter, + MatchHistory& history, key_view_t* index_key) { + static_assert(!IS_BOTTOM); + assert(!iter.is_end()); + auto nxt_container = iter.get_nxt_container(); + auto nxt_result = NXT_STAGE_T::template lower_bound<GET_KEY>( + nxt_container, key, history, index_key); + if (nxt_result.is_end()) { + if (iter.is_last()) { + return result_t::end(); + } else { + return smallest_result<GET_KEY>(++iter, index_key); + } + } else { + if constexpr (GET_KEY) { + index_key->set(iter.get_key()); + } + return result_t::from_nxt(iter.index(), nxt_result); + } + } + + template <bool GET_POS, bool GET_KEY, bool GET_VAL> + static void get_largest_slot( + const container_t& container, // IN + position_t* p_position, // OUT + key_view_t* p_index_key, // OUT + const value_t** pp_value) { // OUT + auto iter = iterator_t(container); + iter.seek_last(); + if constexpr (GET_KEY) { + assert(p_index_key); + p_index_key->set(iter.get_key()); + } else { + assert(!p_index_key); + } + if constexpr (GET_POS) { + assert(p_position); + p_position->index = iter.index(); + } else { + assert(!p_position); + } + if constexpr (IS_BOTTOM) { + if constexpr (GET_VAL) { + assert(pp_value); + *pp_value = iter.get_p_value(); + } else { + assert(!pp_value); + } + } else { + auto nxt_container = iter.get_nxt_container(); + if constexpr (GET_POS) { + NXT_STAGE_T::template get_largest_slot<true, GET_KEY, GET_VAL>( + nxt_container, &p_position->nxt, p_index_key, pp_value); + } else { + NXT_STAGE_T::template get_largest_slot<false, GET_KEY, GET_VAL>( + nxt_container, nullptr, p_index_key, pp_value); + } + } + } + + template <bool GET_KEY, bool GET_VAL> + static void get_slot( + const container_t& container, // IN + const position_t& pos, // IN + key_view_t* p_index_key, // OUT + const value_t** pp_value) { // OUT + auto iter = iterator_t(container); + iter.seek_at(pos.index); + + if constexpr (GET_KEY) { + assert(p_index_key); + p_index_key->set(iter.get_key()); + } else { + assert(!p_index_key); + } + + if constexpr (!IS_BOTTOM) { + auto nxt_container = iter.get_nxt_container(); + NXT_STAGE_T::template get_slot<GET_KEY, GET_VAL>( + nxt_container, pos.nxt, p_index_key, pp_value); + } else { + if constexpr (GET_VAL) { + assert(pp_value); + *pp_value = iter.get_p_value(); + } else { + assert(!pp_value); + } + } + } + + template <bool GET_KEY = false> + static result_t lower_bound( + const container_t& container, + const key_hobj_t& key, + MatchHistory& history, + key_view_t* index_key = nullptr) { + bool exclude_last = false; + if (history.get<STAGE>().has_value()) { + if (*history.get<STAGE>() == MatchKindCMP::EQ) { + // lookup is short-circuited + if constexpr (!IS_BOTTOM) { + assert(history.get<STAGE - 1>().has_value()); + if (history.is_GT<STAGE - 1>()) { + auto iter = iterator_t(container); + bool test_key_equal; + if constexpr (STAGE == STAGE_STRING) { + // TODO(cross-node string dedup) + // test_key_equal = (iter.get_key().type() == ns_oid_view_t::Type::MIN); + auto cmp = key <=> iter.get_key(); + assert(cmp != std::strong_ordering::greater); + test_key_equal = (cmp == 0); + } else { + auto cmp = key <=> iter.get_key(); + // From history, key[stage] == parent[stage][index - 1] + // which should be the smallest possible value for all + // index[stage][*] + assert(cmp != std::strong_ordering::greater); + test_key_equal = (cmp == 0); + } + if (test_key_equal) { + return nxt_lower_bound<GET_KEY>(key, iter, history, index_key); + } else { + // key[stage] < index[stage][left-most] + return smallest_result<GET_KEY>(iter, index_key); + } + } + } + // IS_BOTTOM || !history.is_GT<STAGE - 1>() + auto iter = iterator_t(container); + iter.seek_last(); + if constexpr (STAGE == STAGE_STRING) { + // TODO(cross-node string dedup) + // assert(iter.get_key().type() == ns_oid_view_t::Type::MAX); + assert(key == iter.get_key()); + } else { + assert(key == iter.get_key()); + } + if constexpr (GET_KEY) { + index_key->set(iter.get_key()); + } + if constexpr (IS_BOTTOM) { + auto value_ptr = iter.get_p_value(); + return result_t{{iter.index()}, value_ptr, MSTAT_EQ}; + } else { + auto nxt_container = iter.get_nxt_container(); + auto nxt_result = NXT_STAGE_T::template lower_bound<GET_KEY>( + nxt_container, key, history, index_key); + // !history.is_GT<STAGE - 1>() means + // key[stage+1 ...] <= index[stage+1 ...][*] + assert(!nxt_result.is_end()); + return result_t::from_nxt(iter.index(), nxt_result); + } + } else if (*history.get<STAGE>() == MatchKindCMP::LT) { + exclude_last = true; + } + } + auto iter = iterator_t(container); + auto bs_match = iter.seek(key, exclude_last); + if (iter.is_end()) { + assert(!exclude_last); + assert(bs_match == MatchKindBS::NE); + history.set<STAGE>(MatchKindCMP::GT); + return result_t::end(); + } + history.set<STAGE>(bs_match == MatchKindBS::EQ ? + MatchKindCMP::EQ : MatchKindCMP::LT); + if constexpr (IS_BOTTOM) { + if constexpr (GET_KEY) { + index_key->set(iter.get_key()); + } + auto value_ptr = iter.get_p_value(); + return result_t{{iter.index()}, value_ptr, + (bs_match == MatchKindBS::EQ ? MSTAT_EQ : MSTAT_LT0)}; + } else { + if (bs_match == MatchKindBS::EQ) { + return nxt_lower_bound<GET_KEY>(key, iter, history, index_key); + } else { + return smallest_result<GET_KEY>(iter, index_key); + } + } + } + + template <IsFullKey Key> + static node_offset_t insert_size(const Key& key, + const value_input_t& value) { + if constexpr (IS_BOTTOM) { + return iterator_t::estimate_insert(key, value); + } else { + return iterator_t::estimate_insert(key, value) + + NXT_STAGE_T::iterator_t::header_size() + + NXT_STAGE_T::insert_size(key, value); + } + } + + template <IsFullKey Key> + static node_offset_t insert_size_at(match_stage_t stage, + const Key& key, + const value_input_t& value) { + if (stage == STAGE) { + return insert_size(key, value); + } else { + assert(stage < STAGE); + return NXT_STAGE_T::template insert_size_at(stage, key, value); + } + } + + template <typename T = std::tuple<match_stage_t, node_offset_t>> + static std::enable_if_t<NODE_TYPE == node_type_t::INTERNAL, T> evaluate_insert( + const container_t& container, const key_view_t& key, + const value_input_t& value, position_t& position, bool evaluate_last) { + auto iter = iterator_t(container); + auto& index = position.index; + if (evaluate_last || index == INDEX_END) { + iter.seek_last(); + index = iter.index(); + // evaluate the previous index + } else { + assert(is_valid_index(index)); + // evaluate the current index + iter.seek_at(index); + auto match = key <=> iter.get_key(); + if (match == 0) { + if constexpr (IS_BOTTOM) { + ceph_abort("insert conflict at current index!"); + } else { + // insert into the current index + auto nxt_container = iter.get_nxt_container(); + return NXT_STAGE_T::evaluate_insert( + nxt_container, key, value, position.nxt, false); + } + } else { + assert(match == std::strong_ordering::less); + if (index == 0) { + // already the first index, so insert at the current index + return {STAGE, insert_size(key, value)}; + } + --index; + iter = iterator_t(container); + iter.seek_at(index); + // proceed to evaluate the previous index + } + } + + // XXX(multi-type): when key is from a different type of node + auto match = key <=> iter.get_key(); + if (match == std::strong_ordering::greater) { + // key doesn't match both indexes, so insert at the current index + ++index; + return {STAGE, insert_size(key, value)}; + } else { + assert(match == std::strong_ordering::equal); + if constexpr (IS_BOTTOM) { + // ceph_abort? + ceph_abort("insert conflict at the previous index!"); + } else { + // insert into the previous index + auto nxt_container = iter.get_nxt_container(); + return NXT_STAGE_T::evaluate_insert( + nxt_container, key, value, position.nxt, true); + } + } + } + + template <typename T = bool> + static std::enable_if_t<NODE_TYPE == node_type_t::LEAF, T> + compensate_insert_position_at(match_stage_t stage, position_t& position) { + auto& index = position.index; + if (stage == STAGE) { + assert(index == 0); + // insert at the end of the current stage + index = INDEX_END; + return true; + } else { + if constexpr (IS_BOTTOM) { + ceph_abort("impossible path"); + } else { + assert(stage < STAGE); + bool compensate = NXT_STAGE_T:: + compensate_insert_position_at(stage, position.nxt); + if (compensate) { + assert(is_valid_index(index)); + if (index == 0) { + // insert into the *last* index of the current stage + index = INDEX_LAST; + return true; + } else { + --index; + return false; + } + } else { + return false; + } + } + } + } + + static void patch_insert_end(position_t& insert_pos, match_stage_t insert_stage) { + assert(insert_stage <= STAGE); + if (insert_stage == STAGE) { + insert_pos.index = INDEX_END; + } else if constexpr (!IS_BOTTOM) { + insert_pos.index = INDEX_LAST; + NXT_STAGE_T::patch_insert_end(insert_pos.nxt, insert_stage); + } + } + + template <typename T = std::tuple<match_stage_t, node_offset_t>> + static std::enable_if_t<NODE_TYPE == node_type_t::LEAF, T> evaluate_insert( + const key_hobj_t& key, const value_config_t& value, + const MatchHistory& history, match_stat_t mstat, position_t& position) { + match_stage_t insert_stage = STAGE_TOP; + while (*history.get_by_stage(insert_stage) == MatchKindCMP::EQ) { + assert(insert_stage != STAGE_BOTTOM && "insert conflict!"); + --insert_stage; + } + + if (history.is_GT()) { + if (position.is_end()) { + // no need to compensate insert position + assert(insert_stage <= STAGE && "impossible insert stage"); + } else if (position == position_t::begin()) { + // I must be short-circuited by staged::smallest_result() + // in staged::lower_bound(), so we need to rely on mstat instead + assert(mstat >= MSTAT_LT0 && mstat <= MSTAT_LT3); + if (mstat == MSTAT_LT0) { + insert_stage = STAGE_RIGHT; + } else if (mstat == MSTAT_LT1) { + insert_stage = STAGE_STRING; + } else { + insert_stage = STAGE_LEFT; + } + // XXX(multi-type): need to upgrade node type before inserting an + // incompatible index at front. + assert(insert_stage <= STAGE && "incompatible insert"); + } else { + assert(insert_stage <= STAGE && "impossible insert stage"); + [[maybe_unused]] bool ret = compensate_insert_position_at(insert_stage, position); + assert(!ret); + } + } + + if (position.is_end()) { + patch_insert_end(position, insert_stage); + } + + node_offset_t insert_size = insert_size_at(insert_stage, key, value); + + return {insert_stage, insert_size}; + } + + template <KeyT KT> + static const value_t* insert_new( + NodeExtentMutable& mut, const memory_range_t& range, + const full_key_t<KT>& key, const value_input_t& value) { + char* p_insert = const_cast<char*>(range.p_end); + const value_t* p_value = nullptr; + StagedAppender<KT> appender; + appender.init_empty(&mut, p_insert); + appender.append(key, value, p_value); + [[maybe_unused]] const char* p_insert_front = appender.wrap(); + assert(p_insert_front == range.p_start); + return p_value; + } + + template <KeyT KT, bool SPLIT> + static const value_t* proceed_insert_recursively( + NodeExtentMutable& mut, const container_t& container, + const full_key_t<KT>& key, const value_input_t& value, + position_t& position, match_stage_t& stage, + node_offset_t& _insert_size, const char* p_left_bound) { + // proceed insert from right to left + assert(stage <= STAGE); + auto iter = iterator_t(container); + auto& index = position.index; + + bool do_insert = false; + if (stage == STAGE) { + if (index == INDEX_END) { + iter.seek_last(); + iter.set_end(); + index = iter.index(); + } else { + assert(is_valid_index(index)); + iter.seek_till_end(index); + } + do_insert = true; + } else { // stage < STAGE + if (index == INDEX_LAST) { + iter.seek_last(); + index = iter.index(); + } else { + assert(is_valid_index(index)); + iter.seek_till_end(index); + } + if constexpr (SPLIT) { + if (iter.is_end()) { + // insert at the higher stage due to split + do_insert = true; + _insert_size = insert_size(key, value); + stage = STAGE; + } + } else { + assert(!iter.is_end()); + } + } + + if (do_insert) { + if constexpr (!IS_BOTTOM) { + position.nxt = position_t::nxt_t::begin(); + } + assert(_insert_size == insert_size(key, value)); + if constexpr (IS_BOTTOM) { + return iter.insert( + mut, key, value, _insert_size, p_left_bound); + } else { + auto range = iter.insert_prefix( + mut, key, _insert_size, p_left_bound); + return NXT_STAGE_T::template insert_new<KT>(mut, range, key, value); + } + } else { + if constexpr (!IS_BOTTOM) { + auto nxt_container = iter.get_nxt_container(); + auto p_value = NXT_STAGE_T::template proceed_insert_recursively<KT, SPLIT>( + mut, nxt_container, key, value, + position.nxt, stage, _insert_size, p_left_bound); + iter.update_size(mut, _insert_size); + return p_value; + } else { + ceph_abort("impossible path"); + } + } + } + + template <KeyT KT, bool SPLIT> + static const value_t* proceed_insert( + NodeExtentMutable& mut, const container_t& container, + const full_key_t<KT>& key, const value_input_t& value, + position_t& position, match_stage_t& stage, node_offset_t& _insert_size) { + auto p_left_bound = container.p_left_bound(); + if (unlikely(!container.keys())) { + if (position.is_end()) { + position = position_t::begin(); + assert(stage == STAGE); + assert(_insert_size == insert_size(key, value)); + } else if (position == position_t::begin()) { + // when insert into a trimmed and empty left node + stage = STAGE; + _insert_size = insert_size(key, value); + } else { + ceph_abort("impossible path"); + } + if constexpr (IS_BOTTOM) { + return container_t::insert_at( + mut, container, key, value, 0, _insert_size, p_left_bound); + } else { + auto range = container_t::template insert_prefix_at( + mut, container, key, 0, _insert_size, p_left_bound); + return NXT_STAGE_T::template insert_new<KT>(mut, range, key, value); + } + } else { + return proceed_insert_recursively<KT, SPLIT>( + mut, container, key, value, + position, stage, _insert_size, p_left_bound); + } + } + + static std::ostream& dump(const container_t& container, + std::ostream& os, + const std::string& prefix, + size_t& size, + const char* p_start) { + auto iter = iterator_t(container); + assert(!iter.is_end()); + std::string prefix_blank(prefix.size(), ' '); + const std::string* p_prefix = &prefix; + size += iterator_t::header_size(); + do { + std::ostringstream sos; + sos << *p_prefix << iter.get_key() << ": "; + std::string i_prefix = sos.str(); + if constexpr (!IS_BOTTOM) { + auto nxt_container = iter.get_nxt_container(); + size += iter.size_to_nxt(); + NXT_STAGE_T::dump(nxt_container, os, i_prefix, size, p_start); + } else { + auto value_ptr = iter.get_p_value(); + int offset = reinterpret_cast<const char*>(value_ptr) - p_start; + size += iter.size(); + os << "\n" << i_prefix; + if constexpr (NODE_TYPE == node_type_t::LEAF) { + os << *value_ptr; + } else { + os << "0x" << std::hex << value_ptr->value << std::dec; + } + os << " " << size << "B" + << " @" << offset << "B"; + } + if (iter.is_last()) { + break; + } else { + ++iter; + p_prefix = &prefix_blank; + } + } while (true); + return os; + } + + static void validate(const container_t& container) { + auto iter = iterator_t(container); + assert(!iter.is_end()); + auto key = iter.get_key(); + do { + if constexpr (!IS_BOTTOM) { + auto nxt_container = iter.get_nxt_container(); + NXT_STAGE_T::validate(nxt_container); + } + if (iter.is_last()) { + break; + } else { + ++iter; + assert(key < iter.get_key()); + key = iter.get_key(); + } + } while (true); + } + + static void get_stats(const container_t& container, node_stats_t& stats, + key_view_t& index_key) { + auto iter = iterator_t(container); + assert(!iter.is_end()); + stats.size_overhead += iterator_t::header_size(); + do { + index_key.replace(iter.get_key()); + stats.size_overhead += iter.size_overhead(); + if constexpr (!IS_BOTTOM) { + auto nxt_container = iter.get_nxt_container(); + NXT_STAGE_T::get_stats(nxt_container, stats, index_key); + } else { + ++stats.num_kvs; + size_t kv_logical_size = index_key.size_logical(); + size_t value_size; + if constexpr (NODE_TYPE == node_type_t::LEAF) { + value_size = iter.get_p_value()->allocation_size(); + } else { + value_size = sizeof(value_t); + } + stats.size_value += value_size; + kv_logical_size += value_size; + stats.size_logical += kv_logical_size; + } + if (iter.is_last()) { + break; + } else { + ++iter; + } + } while (true); + } + + template <bool GET_KEY, bool GET_VAL> + static bool get_next_slot( + const container_t& container, // IN + position_t& pos, // IN&OUT + key_view_t* p_index_key, // OUT + const value_t** pp_value) { // OUT + auto iter = iterator_t(container); + assert(!iter.is_end()); + iter.seek_at(pos.index); + bool find_next; + if constexpr (!IS_BOTTOM) { + auto nxt_container = iter.get_nxt_container(); + find_next = NXT_STAGE_T::template get_next_slot<GET_KEY, GET_VAL>( + nxt_container, pos.nxt, p_index_key, pp_value); + } else { + find_next = true; + } + + if (find_next) { + if (iter.is_last()) { + return true; + } else { + pos.index = iter.index() + 1; + if constexpr (!IS_BOTTOM) { + pos.nxt = NXT_STAGE_T::position_t::begin(); + } + get_slot<GET_KEY, GET_VAL>( + container, pos, p_index_key, pp_value); + return false; + } + } else { // !find_next && !IS_BOTTOM + if constexpr (GET_KEY) { + assert(p_index_key); + p_index_key->set(iter.get_key()); + } else { + assert(!p_index_key); + } + return false; + } + } + + template <bool GET_KEY, bool GET_VAL> + static void get_prev_slot( + const container_t& container, // IN + position_t& pos, // IN&OUT + key_view_t* p_index_key, // OUT + const value_t** pp_value) { // OUT + assert(pos != position_t::begin()); + assert(!pos.is_end()); + auto& index = pos.index; + auto iter = iterator_t(container); + if constexpr (!IS_BOTTOM) { + auto& nxt_pos = pos.nxt; + if (nxt_pos == NXT_STAGE_T::position_t::begin()) { + assert(index); + --index; + iter.seek_at(index); + auto nxt_container = iter.get_nxt_container(); + NXT_STAGE_T::template get_largest_slot<true, GET_KEY, GET_VAL>( + nxt_container, &nxt_pos, p_index_key, pp_value); + } else { + iter.seek_at(index); + auto nxt_container = iter.get_nxt_container(); + NXT_STAGE_T::template get_prev_slot<GET_KEY, GET_VAL>( + nxt_container, nxt_pos, p_index_key, pp_value); + } + } else { + assert(index); + --index; + iter.seek_at(index); + if constexpr (GET_VAL) { + assert(pp_value); + *pp_value = iter.get_p_value(); + } else { + assert(!pp_value); + } + } + if constexpr (GET_KEY) { + p_index_key->set(iter.get_key()); + } else { + assert(!p_index_key); + } + } + + struct _BaseEmpty {}; + class _BaseWithNxtIterator { + protected: + typename NXT_STAGE_T::StagedIterator _nxt; + }; + class StagedIterator + : std::conditional_t<IS_BOTTOM, _BaseEmpty, _BaseWithNxtIterator> { + public: + StagedIterator() = default; + bool valid() const { return iter.has_value(); } + index_t index() const { + return iter->index(); + } + bool is_end() const { return iter->is_end(); } + bool in_progress() const { + assert(valid()); + assert(!is_end()); + if constexpr (!IS_BOTTOM) { + if (this->_nxt.valid()) { + if (this->_nxt.index() == 0) { + return this->_nxt.in_progress(); + } else { + return true; + } + } else { + return false; + } + } else { + return false; + } + } + key_get_type get_key() const { return iter->get_key(); } + + iterator_t& get() { return *iter; } + void set(const container_t& container) { + assert(!valid()); + iter = iterator_t(container); + } + void set_end() { iter->set_end(); } + typename NXT_STAGE_T::StagedIterator& nxt() { + if constexpr (!IS_BOTTOM) { + if (!this->_nxt.valid()) { + auto nxt_container = iter->get_nxt_container(); + this->_nxt.set(nxt_container); + } + return this->_nxt; + } else { + ceph_abort("impossible path"); + } + } + typename NXT_STAGE_T::StagedIterator& get_nxt() { + if constexpr (!IS_BOTTOM) { + return this->_nxt; + } else { + ceph_abort("impossible path"); + } + } + StagedIterator& operator++() { + if (iter->is_last()) { + iter->set_end(); + } else { + ++(*iter); + } + if constexpr (!IS_BOTTOM) { + this->_nxt.reset(); + } + return *this; + } + void reset() { + if (valid()) { + iter.reset(); + if constexpr (!IS_BOTTOM) { + this->_nxt.reset(); + } + } + } + + template<typename OutputIt> + auto do_format_to(OutputIt out, bool is_top) const { + if (valid()) { + if (iter->is_end()) { + return fmt::format_to(out, "END"); + } else { + out = fmt::format_to(out, "{}", index()); + } + } else { + if (is_top) { + return fmt::format_to(out, "invalid StagedIterator!"); + } else { + out = fmt::format_to(out, "0!"); + } + } + if constexpr (!IS_BOTTOM) { + out = fmt::format_to(out, ", "); + return this->_nxt.do_format_to(out, false); + } else { + return out; + } + } + + position_t get_pos() const { + if (valid()) { + if constexpr (IS_BOTTOM) { + return position_t{index()}; + } else { + return position_t{index(), this->_nxt.get_pos()}; + } + } else { + return position_t::begin(); + } + } + void encode(const char* p_node_start, ceph::bufferlist& encoded) const { + uint8_t present = static_cast<bool>(iter); + ceph::encode(present, encoded); + if (iter.has_value()) { + iter->encode(p_node_start, encoded); + if constexpr (!IS_BOTTOM) { + this->_nxt.encode(p_node_start, encoded); + } + } + } + static StagedIterator decode(const char* p_node_start, + extent_len_t node_size, + ceph::bufferlist::const_iterator& delta) { + StagedIterator ret; + uint8_t present; + ceph::decode(present, delta); + if (present) { + ret.iter = iterator_t::decode( + p_node_start, node_size, delta); + if constexpr (!IS_BOTTOM) { + ret._nxt = NXT_STAGE_T::StagedIterator::decode( + p_node_start, node_size, delta); + } + } + return ret; + } + private: + std::optional<iterator_t> iter; + }; + + static bool recursively_locate_split( + size_t& current_size, size_t extra_size, + size_t target_size, StagedIterator& split_at) { + assert(current_size <= target_size); + iterator_t& split_iter = split_at.get(); + current_size = split_iter.seek_split(current_size, extra_size, target_size); + assert(current_size <= target_size); + assert(!split_iter.is_end()); + if (split_iter.index() == 0) { + extra_size += iterator_t::header_size(); + } else { + extra_size = 0; + } + bool locate_nxt; + if constexpr (!IS_BOTTOM) { + locate_nxt = NXT_STAGE_T::recursively_locate_split( + current_size, extra_size + split_iter.size_to_nxt(), + target_size, split_at.nxt()); + } else { // IS_BOTTOM + // located upper_bound, fair split strategy + size_t nxt_size = split_iter.size() + extra_size; + assert(current_size + nxt_size > target_size); + if (current_size + nxt_size/2 < target_size) { + // include next + current_size += nxt_size; + locate_nxt = true; + } else { + // exclude next + locate_nxt = false; + } + } + if (locate_nxt) { + if (split_iter.is_last()) { + return true; + } else { + ++split_at; + return false; + } + } else { + return false; + } + } + + static bool recursively_locate_split_inserted( + size_t& current_size, size_t extra_size, size_t target_size, + position_t& insert_pos, match_stage_t insert_stage, size_t insert_size, + std::optional<bool>& is_insert_left, StagedIterator& split_at) { + assert(current_size <= target_size); + assert(!is_insert_left.has_value()); + iterator_t& split_iter = split_at.get(); + auto& insert_index = insert_pos.index; + if (insert_stage == STAGE) { + current_size = split_iter.template seek_split_inserted<true>( + current_size, extra_size, target_size, + insert_index, insert_size, is_insert_left); + assert(is_insert_left.has_value()); + assert(current_size <= target_size); + if (split_iter.index() == 0) { + if (insert_index == 0) { + if (*is_insert_left == false) { + extra_size += iterator_t::header_size(); + } else { + extra_size = 0; + } + } else { + extra_size += iterator_t::header_size(); + } + } else { + extra_size = 0; + } + if (*is_insert_left == false && split_iter.index() == insert_index) { + // split_iter can be end + // found the lower-bound of target_size + // ...[s_index-1] |!| (i_index) [s_index]... + + // located upper-bound, fair split strategy + // look at the next slot (the insert item) + size_t nxt_size = insert_size + extra_size; + assert(current_size + nxt_size > target_size); + if (current_size + nxt_size/2 < target_size) { + // include next + *is_insert_left = true; + current_size += nxt_size; + if (split_iter.is_end()) { + // ...[s_index-1] (i_index) |!| + return true; + } else { + return false; + } + } else { + // exclude next + return false; + } + } else { + // Already considered insert effect in the current stage. + // Look into the next stage to identify the target_size lower-bound w/o + // insert effect. + assert(!split_iter.is_end()); + bool locate_nxt; + if constexpr (!IS_BOTTOM) { + locate_nxt = NXT_STAGE_T::recursively_locate_split( + current_size, extra_size + split_iter.size_to_nxt(), + target_size, split_at.nxt()); + } else { // IS_BOTTOM + // located upper-bound, fair split strategy + // look at the next slot + size_t nxt_size = split_iter.size() + extra_size; + assert(current_size + nxt_size > target_size); + if (current_size + nxt_size/2 < target_size) { + // include next + current_size += nxt_size; + locate_nxt = true; + } else { + // exclude next + locate_nxt = false; + } + } + if (locate_nxt) { + if (split_iter.is_last()) { + auto end_index = split_iter.index() + 1; + if (insert_index == INDEX_END) { + insert_index = end_index; + } + assert(insert_index <= end_index); + if (insert_index == end_index) { + assert(*is_insert_left == false); + split_iter.set_end(); + // ...[s_index-1] |!| (i_index) + return false; + } else { + assert(*is_insert_left == true); + return true; + } + } else { + ++split_at; + return false; + } + } else { + return false; + } + } + } else { + if constexpr (!IS_BOTTOM) { + assert(insert_stage < STAGE); + current_size = split_iter.template seek_split_inserted<false>( + current_size, extra_size, target_size, + insert_index, insert_size, is_insert_left); + assert(!split_iter.is_end()); + assert(current_size <= target_size); + if (split_iter.index() == 0) { + extra_size += iterator_t::header_size(); + } else { + extra_size = 0; + } + bool locate_nxt; + if (!is_insert_left.has_value()) { + // Considered insert effect in the current stage, and insert happens + // in the lower stage. + // Look into the next stage to identify the target_size lower-bound w/ + // insert effect. + assert(split_iter.index() == insert_index); + locate_nxt = NXT_STAGE_T::recursively_locate_split_inserted( + current_size, extra_size + split_iter.size_to_nxt(), target_size, + insert_pos.nxt, insert_stage, insert_size, + is_insert_left, split_at.nxt()); + assert(is_insert_left.has_value()); +#ifndef NDEBUG + if (locate_nxt) { + assert(*is_insert_left == true); + } +#endif + } else { + // is_insert_left.has_value() == true + // Insert will *not* happen in the lower stage. + // Need to look into the next stage to identify the target_size + // lower-bound w/ insert effect + assert(split_iter.index() != insert_index); + locate_nxt = NXT_STAGE_T::recursively_locate_split( + current_size, extra_size + split_iter.size_to_nxt(), + target_size, split_at.nxt()); +#ifndef NDEBUG + if (split_iter.index() < insert_index) { + assert(*is_insert_left == false); + } else { + assert(*is_insert_left == true); + } +#endif + } + if (locate_nxt) { + if (split_iter.is_last()) { + return true; + } else { + ++split_at; + return false; + } + } else { + return false; + } + } else { + ceph_abort("impossible path"); + return false;; + } + } + } + + /* + * container appender type system + * container_t::Appender(NodeExtentMutable& mut, char* p_append) + * append(const container_t& src, index_t from, index_t items) + * wrap() -> char* + * IF !IS_BOTTOM: + * open_nxt(const key_get_type&) + * open_nxt(const full_key_t&) + * -> std::tuple<NodeExtentMutable&, char*> + * wrap_nxt(char* p_append) + * ELSE + * append(const full_key_t& key, const value_input_t& value) + */ + template <KeyT KT> + struct _BaseWithNxtAppender { + typename NXT_STAGE_T::template StagedAppender<KT> _nxt; + }; + template <KeyT KT> + class StagedAppender + : std::conditional_t<IS_BOTTOM, _BaseEmpty, _BaseWithNxtAppender<KT>> { + public: + StagedAppender() = default; + ~StagedAppender() { + assert(!require_wrap_nxt); + assert(!valid()); + } + bool valid() const { return appender.has_value(); } + index_t index() const { + assert(valid()); + return _index; + } + bool in_progress() const { return require_wrap_nxt; } + // TODO: pass by reference + void init_empty(NodeExtentMutable* p_mut, char* p_start) { + assert(!valid()); + appender = typename container_t::template Appender<KT>(p_mut, p_start); + _index = 0; + } + void init_tail(NodeExtentMutable* p_mut, + const container_t& container, + match_stage_t stage) { + assert(!valid()); + auto iter = iterator_t(container); + iter.seek_last(); + if (stage == STAGE) { + appender = iter.template get_appender<KT>(p_mut); + _index = iter.index() + 1; + if constexpr (!IS_BOTTOM) { + assert(!this->_nxt.valid()); + } + } else { + assert(stage < STAGE); + if constexpr (!IS_BOTTOM) { + appender = iter.template get_appender_opened<KT>(p_mut); + _index = iter.index(); + require_wrap_nxt = true; + auto nxt_container = iter.get_nxt_container(); + this->_nxt.init_tail(p_mut, nxt_container, stage); + } else { + ceph_abort("impossible path"); + } + } + } + // possible to make src_iter end if to_index == INDEX_END + void append_until(StagedIterator& src_iter, index_t& to_index) { + assert(!require_wrap_nxt); + auto s_index = src_iter.index(); + src_iter.get().template copy_out_until<KT>(*appender, to_index); + assert(src_iter.index() == to_index); + assert(to_index >= s_index); + auto increment = (to_index - s_index); + if (increment) { + _index += increment; + if constexpr (!IS_BOTTOM) { + src_iter.get_nxt().reset(); + } + } + } + void append(const full_key_t<KT>& key, + const value_input_t& value, const value_t*& p_value) { + assert(!require_wrap_nxt); + if constexpr (!IS_BOTTOM) { + auto& nxt = open_nxt(key); + nxt.append(key, value, p_value); + wrap_nxt(); + } else { + appender->append(key, value, p_value); + ++_index; + } + } + char* wrap() { + assert(valid()); + assert(_index > 0); + if constexpr (!IS_BOTTOM) { + if (require_wrap_nxt) { + wrap_nxt(); + } + } + auto ret = appender->wrap(); + appender.reset(); + return ret; + } + typename NXT_STAGE_T::template StagedAppender<KT>& + open_nxt(key_get_type paritial_key) { + assert(!require_wrap_nxt); + if constexpr (!IS_BOTTOM) { + require_wrap_nxt = true; + auto [p_mut, p_append] = appender->open_nxt(paritial_key); + this->_nxt.init_empty(p_mut, p_append); + return this->_nxt; + } else { + ceph_abort("impossible path"); + } + } + typename NXT_STAGE_T::template StagedAppender<KT>& + open_nxt(const full_key_t<KT>& key) { + assert(!require_wrap_nxt); + if constexpr (!IS_BOTTOM) { + require_wrap_nxt = true; + auto [p_mut, p_append] = appender->open_nxt(key); + this->_nxt.init_empty(p_mut, p_append); + return this->_nxt; + } else { + ceph_abort("impossible path"); + } + } + typename NXT_STAGE_T::template StagedAppender<KT>& get_nxt() { + if constexpr (!IS_BOTTOM) { + assert(require_wrap_nxt); + return this->_nxt; + } else { + ceph_abort("impossible path"); + } + } + void wrap_nxt() { + if constexpr (!IS_BOTTOM) { + assert(require_wrap_nxt); + require_wrap_nxt = false; + auto p_append = this->_nxt.wrap(); + appender->wrap_nxt(p_append); + ++_index; + } else { + ceph_abort("impossible path"); + } + } + private: + std::optional<typename container_t::template Appender<KT>> appender; + index_t _index; + bool require_wrap_nxt = false; + }; + + template <KeyT KT> + static void _append_range( + StagedIterator& src_iter, StagedAppender<KT>& appender, index_t& to_index) { + if (src_iter.is_end()) { + // append done + assert(to_index == INDEX_END); + to_index = src_iter.index(); + } else if constexpr (!IS_BOTTOM) { + if (appender.in_progress()) { + // appender has appended something at the current item, + // cannot append the current item as-a-whole + index_t to_index_nxt = INDEX_END; + NXT_STAGE_T::template _append_range<KT>( + src_iter.nxt(), appender.get_nxt(), to_index_nxt); + ++src_iter; + appender.wrap_nxt(); + } else if (src_iter.in_progress()) { + // src_iter is not at the beginning of the current item, + // cannot append the current item as-a-whole + index_t to_index_nxt = INDEX_END; + NXT_STAGE_T::template _append_range<KT>( + src_iter.get_nxt(), appender.open_nxt(src_iter.get_key()), to_index_nxt); + ++src_iter; + appender.wrap_nxt(); + } else { + // we can safely append the current item as-a-whole + } + } + appender.append_until(src_iter, to_index); + } + + template <KeyT KT> + static void _append_into(StagedIterator& src_iter, StagedAppender<KT>& appender, + position_t& position, match_stage_t stage) { + assert(position.index == src_iter.index()); + // reaches the last item + if (stage == STAGE) { + // done, end recursion + if constexpr (!IS_BOTTOM) { + position.nxt = position_t::nxt_t::begin(); + } + } else { + assert(stage < STAGE); + // proceed append in the next stage + NXT_STAGE_T::template append_until<KT>( + src_iter.nxt(), appender.open_nxt(src_iter.get_key()), + position.nxt, stage); + } + } + + template <KeyT KT> + static void append_until(StagedIterator& src_iter, StagedAppender<KT>& appender, + position_t& position, match_stage_t stage) { + index_t from_index = src_iter.index(); + index_t& to_index = position.index; + assert(from_index <= to_index); + if constexpr (IS_BOTTOM) { + assert(stage == STAGE); + appender.append_until(src_iter, to_index); + } else { + assert(stage <= STAGE); + if (src_iter.index() == to_index) { + _append_into<KT>(src_iter, appender, position, stage); + } else { + if (to_index == INDEX_END) { + assert(stage == STAGE); + } else if (to_index == INDEX_LAST) { + assert(stage < STAGE); + } + _append_range<KT>(src_iter, appender, to_index); + _append_into<KT>(src_iter, appender, position, stage); + } + } + to_index -= from_index; + } + + template <KeyT KT> + static bool append_insert( + const full_key_t<KT>& key, const value_input_t& value, + StagedIterator& src_iter, StagedAppender<KT>& appender, + bool is_front_insert, match_stage_t& stage, const value_t*& p_value) { + assert(src_iter.valid()); + if (stage == STAGE) { + appender.append(key, value, p_value); + if (src_iter.is_end()) { + return true; + } else { + return false; + } + } else { + assert(stage < STAGE); + if constexpr (!IS_BOTTOM) { + auto nxt_is_end = NXT_STAGE_T::template append_insert<KT>( + key, value, src_iter.get_nxt(), appender.get_nxt(), + is_front_insert, stage, p_value); + if (nxt_is_end) { + appender.wrap_nxt(); + ++src_iter; + if (is_front_insert) { + stage = STAGE; + } + if (src_iter.is_end()) { + return true; + } + } + return false; + } else { + ceph_abort("impossible path"); + } + } + } + + /* TrimType: + * BEFORE: remove the entire container, normally means the according higher + * stage iterator needs to be trimmed as-a-whole. + * AFTER: retain the entire container, normally means the trim should be + * start from the next iterator at the higher stage. + * AT: trim happens in the current container, and the according higher + * stage iterator needs to be adjusted by the trimmed size. + */ + static std::tuple<TrimType, node_offset_t> + recursively_trim(NodeExtentMutable& mut, StagedIterator& trim_at) { + if (!trim_at.valid()) { + return {TrimType::BEFORE, 0u}; + } + if (trim_at.is_end()) { + return {TrimType::AFTER, 0u}; + } + + auto& iter = trim_at.get(); + if constexpr (!IS_BOTTOM) { + auto [type, trimmed] = NXT_STAGE_T::recursively_trim( + mut, trim_at.get_nxt()); + node_offset_t trim_size; + if (type == TrimType::AFTER) { + if (iter.is_last()) { + return {TrimType::AFTER, 0u}; + } + ++trim_at; + trim_size = iter.trim_until(mut); + } else if (type == TrimType::BEFORE) { + if (iter.index() == 0) { + return {TrimType::BEFORE, 0u}; + } + trim_size = iter.trim_until(mut); + } else { + trim_size = iter.trim_at(mut, trimmed); + } + return {TrimType::AT, trim_size}; + } else { + if (iter.index() == 0) { + return {TrimType::BEFORE, 0u}; + } else { + auto trimmed = iter.trim_until(mut); + return {TrimType::AT, trimmed}; + } + } + } + + static void trim(NodeExtentMutable& mut, StagedIterator& trim_at) { + auto [type, trimmed] = recursively_trim(mut, trim_at); + if (type == TrimType::BEFORE) { + assert(trim_at.valid()); + auto& iter = trim_at.get(); + iter.trim_until(mut); + } + } + + static std::optional<std::tuple<match_stage_t, node_offset_t, bool>> + proceed_erase_recursively( + NodeExtentMutable& mut, + const container_t& container, // IN + const char* p_left_bound, // IN + position_t& pos) { // IN&OUT + auto iter = iterator_t(container); + auto& index = pos.index; + assert(is_valid_index(index)); + iter.seek_at(index); + bool is_last = iter.is_last(); + + if constexpr (!IS_BOTTOM) { + auto nxt_container = iter.get_nxt_container(); + auto ret = NXT_STAGE_T::proceed_erase_recursively( + mut, nxt_container, p_left_bound, pos.nxt); + if (ret.has_value()) { + // erased at lower level + auto [r_stage, r_erase_size, r_done] = *ret; + assert(r_erase_size != 0); + iter.update_size(mut, -r_erase_size); + if (r_done) { + // done, the next_pos is calculated + return ret; + } else { + if (is_last) { + // need to find the next pos at upper stage + return ret; + } else { + // done, calculate the next pos + ++index; + pos.nxt = NXT_STAGE_T::position_t::begin(); + return {{r_stage, r_erase_size, true}}; + } + } + } + // not erased at lower level + } + + // not erased yet + if (index == 0 && is_last) { + // need to erase from the upper stage + return std::nullopt; + } else { + auto erase_size = iter.erase(mut, p_left_bound); + assert(erase_size != 0); + if (is_last) { + // need to find the next pos at upper stage + return {{STAGE, erase_size, false}}; + } else { + // done, calculate the next pos (should be correct already) + if constexpr (!IS_BOTTOM) { + assert(pos.nxt == NXT_STAGE_T::position_t::begin()); + } + return {{STAGE, erase_size, true}}; + } + } + } + + static match_stage_t erase( + NodeExtentMutable& mut, + const container_t& node_stage, // IN + position_t& erase_pos) { // IN&OUT + auto p_left_bound = node_stage.p_left_bound(); + auto ret = proceed_erase_recursively( + mut, node_stage, p_left_bound, erase_pos); + if (ret.has_value()) { + auto [r_stage, r_erase_size, r_done] = *ret; + std::ignore = r_erase_size; + if (r_done) { + assert(!erase_pos.is_end()); + return r_stage; + } else { + // erased the last kv + erase_pos = position_t::end(); + return r_stage; + } + } else { + assert(node_stage.keys() == 1); + node_stage.erase_at(mut, node_stage, 0, p_left_bound); + erase_pos = position_t::end(); + return STAGE; + } + } + + static std::tuple<match_stage_t, node_offset_t> evaluate_merge( + const key_view_t& left_pivot_index, + const container_t& right_container) { + auto r_iter = iterator_t(right_container); + r_iter.seek_at(0); + node_offset_t compensate = r_iter.header_size(); + auto cmp = left_pivot_index <=> r_iter.get_key(); + if (cmp == std::strong_ordering::equal) { + if constexpr (!IS_BOTTOM) { + // the index is equal, compensate and look at the lower stage + compensate += r_iter.size_to_nxt(); + auto r_nxt_container = r_iter.get_nxt_container(); + auto [ret_stage, ret_compensate] = NXT_STAGE_T::evaluate_merge( + left_pivot_index, r_nxt_container); + compensate += ret_compensate; + return {ret_stage, compensate}; + } else { + ceph_abort("impossible path: left_pivot_key == right_first_key"); + } + } else if (cmp == std::strong_ordering::less) { + // ok, do merge here + return {STAGE, compensate}; + } else { + ceph_abort("impossible path: left_pivot_key < right_first_key"); + } + } +}; + +/** + * Configurations for struct staged + * + * staged_params_* assembles different container_t implementations (defined by + * stated::_iterator_t) by STAGE, and constructs the final multi-stage + * implementations for different node layouts defined by + * node_extent_t<FieldType, NODE_TYPE>. + * + * The specialized implementations for different layouts are accessible through + * the helper type node_to_stage_t<node_extent_t<FieldType, NODE_TYPE>>. + * + * Specifically, the settings of 8 layouts are: + * + * The layout (N0, LEAF/INTERNAL) has 3 stages: + * - STAGE_LEFT: node_extent_t<node_fields_0_t, LEAF/INTERNAL> + * - STAGE_STRING: item_iterator_t<LEAF/INTERNAL> + * - STAGE_RIGHT: sub_items_t<LEAF/INTERNAL> + * + * The layout (N1, LEAF/INTERNAL) has 3 stages: + * - STAGE_LEFT: node_extent_t<node_fields_1_t, LEAF/INTERNAL> + * - STAGE_STRING: item_iterator_t<LEAF/INTERNAL> + * - STAGE_RIGHT: sub_items_t<LEAF/INTERNAL> + * + * The layout (N2, LEAF/INTERNAL) has 2 stages: + * - STAGE_STRING: node_extent_t<node_fields_2_t, LEAF/INTERNAL> + * - STAGE_RIGHT: sub_items_t<LEAF/INTERNAL> + * + * The layout (N3, LEAF) has 1 stage: + * - STAGE_RIGHT: node_extent_t<leaf_fields_3_t, LEAF> + * + * The layout (N3, INTERNAL) has 1 stage: + * - STAGE_RIGHT: node_extent_t<internal_fields_3_t, INTERNAL> + */ + +template <node_type_t _NODE_TYPE> +struct staged_params_subitems { + using container_t = sub_items_t<_NODE_TYPE>; + static constexpr auto NODE_TYPE = _NODE_TYPE; + static constexpr auto STAGE = STAGE_RIGHT; + + // dummy type in order to make our type system work + // any better solution to get rid of this? + using next_param_t = staged_params_subitems<NODE_TYPE>; +}; + +template <node_type_t _NODE_TYPE> +struct staged_params_item_iterator { + using container_t = item_iterator_t<_NODE_TYPE>; + static constexpr auto NODE_TYPE = _NODE_TYPE; + static constexpr auto STAGE = STAGE_STRING; + + using next_param_t = staged_params_subitems<NODE_TYPE>; +}; + +template <typename NodeType> +struct staged_params_node_01 { + using container_t = NodeType; + static constexpr auto NODE_TYPE = NodeType::NODE_TYPE; + static constexpr auto STAGE = STAGE_LEFT; + + using next_param_t = staged_params_item_iterator<NODE_TYPE>; +}; + +template <typename NodeType> +struct staged_params_node_2 { + using container_t = NodeType; + static constexpr auto NODE_TYPE = NodeType::NODE_TYPE; + static constexpr auto STAGE = STAGE_STRING; + + using next_param_t = staged_params_subitems<NODE_TYPE>; +}; + +template <typename NodeType> +struct staged_params_node_3 { + using container_t = NodeType; + static constexpr auto NODE_TYPE = NodeType::NODE_TYPE; + static constexpr auto STAGE = STAGE_RIGHT; + + // dummy type in order to make our type system work + // any better solution to get rid of this? + using next_param_t = staged_params_node_3<NodeType>; +}; + +template <typename NodeType, typename Enable = void> struct _node_to_stage_t; +template <typename NodeType> +struct _node_to_stage_t<NodeType, + std::enable_if_t<NodeType::FIELD_TYPE == field_type_t::N0 || + NodeType::FIELD_TYPE == field_type_t::N1>> { + using type = staged<staged_params_node_01<NodeType>>; +}; +template <typename NodeType> +struct _node_to_stage_t<NodeType, + std::enable_if_t<NodeType::FIELD_TYPE == field_type_t::N2>> { + using type = staged<staged_params_node_2<NodeType>>; +}; +template <typename NodeType> +struct _node_to_stage_t<NodeType, + std::enable_if_t<NodeType::FIELD_TYPE == field_type_t::N3>> { + using type = staged<staged_params_node_3<NodeType>>; +}; +template <typename NodeType> +using node_to_stage_t = typename _node_to_stage_t<NodeType>::type; + +} + +template<typename T> +concept HasDoFormatTo = requires(T x, std::back_insert_iterator<fmt::memory_buffer> out) { + { x.do_format_to(out, true) } -> std::same_as<decltype(out)>; +}; +template <HasDoFormatTo T> struct fmt::formatter<T> : fmt::formatter<std::string_view> { + template <typename FormatContext> + auto format(const T& staged_iterator, FormatContext& ctx) { + return staged_iterator.do_format_to(ctx.out(), true); + } +}; diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/stage_types.h b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/stage_types.h new file mode 100644 index 000000000..3c1b32a41 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/stage_types.h @@ -0,0 +1,442 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <cassert> +#include <optional> +#include <ostream> + +#include "crimson/os/seastore/onode_manager/staged-fltree/fwd.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/node_types.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/value.h" + +namespace crimson::os::seastore::onode { + +using match_stage_t = int8_t; +constexpr match_stage_t STAGE_LEFT = 2; // shard/pool/crush +constexpr match_stage_t STAGE_STRING = 1; // nspace/oid +constexpr match_stage_t STAGE_RIGHT = 0; // snap/gen +constexpr auto STAGE_TOP = STAGE_LEFT; +constexpr auto STAGE_BOTTOM = STAGE_RIGHT; +constexpr bool is_valid_stage(match_stage_t stage) { + return std::clamp(stage, STAGE_BOTTOM, STAGE_TOP) == stage; +} +// TODO: replace by +// using match_history_t = int8_t; +// left_m, str_m, right_m +// 3: GT, +// 2: EQ, GT, +// 1: EQ, EQ, GT +// 0: EQ, EQ, EQ +// -1: EQ, EQ, LT +// -2: EQ, LT, +// -3: LT, + +struct MatchHistory { + template <match_stage_t STAGE> + const std::optional<MatchKindCMP>& get() const { + static_assert(is_valid_stage(STAGE)); + if constexpr (STAGE == STAGE_RIGHT) { + return right_match; + } else if (STAGE == STAGE_STRING) { + return string_match; + } else { + return left_match; + } + } + + const std::optional<MatchKindCMP>& + get_by_stage(match_stage_t stage) const { + assert(is_valid_stage(stage)); + if (stage == STAGE_RIGHT) { + return right_match; + } else if (stage == STAGE_STRING) { + return string_match; + } else { + return left_match; + } + } + + template <match_stage_t STAGE = STAGE_TOP> + const bool is_GT() const; + + template <match_stage_t STAGE> + void set(MatchKindCMP match) { + static_assert(is_valid_stage(STAGE)); + if constexpr (STAGE < STAGE_TOP) { + assert(*get<STAGE + 1>() == MatchKindCMP::EQ); + } + assert(!get<STAGE>().has_value() || *get<STAGE>() != MatchKindCMP::EQ); + const_cast<std::optional<MatchKindCMP>&>(get<STAGE>()) = match; + } + + std::ostream& dump(std::ostream& os) const { + os << "history("; + dump_each(os, left_match) << ", "; + dump_each(os, string_match) << ", "; + dump_each(os, right_match) << ")"; + return os; + } + + std::ostream& dump_each( + std::ostream& os, const std::optional<MatchKindCMP>& match) const { + if (!match.has_value()) { + return os << "--"; + } else if (*match == MatchKindCMP::LT) { + return os << "LT"; + } else if (*match == MatchKindCMP::EQ) { + return os << "EQ"; + } else if (*match == MatchKindCMP::GT) { + return os << "GT"; + } else { + ceph_abort("impossble path"); + } + } + + std::optional<MatchKindCMP> left_match; + std::optional<MatchKindCMP> string_match; + std::optional<MatchKindCMP> right_match; +}; +inline std::ostream& operator<<(std::ostream& os, const MatchHistory& pos) { + return pos.dump(os); +} + +template <match_stage_t STAGE> +struct _check_GT_t { + static bool eval(const MatchHistory* history) { + return history->get<STAGE>() && + (*history->get<STAGE>() == MatchKindCMP::GT || + (*history->get<STAGE>() == MatchKindCMP::EQ && + _check_GT_t<STAGE - 1>::eval(history))); + } +}; +template <> +struct _check_GT_t<STAGE_RIGHT> { + static bool eval(const MatchHistory* history) { + return history->get<STAGE_RIGHT>() && + *history->get<STAGE_RIGHT>() == MatchKindCMP::GT; + } +}; +template <match_stage_t STAGE> +const bool MatchHistory::is_GT() const { + static_assert(is_valid_stage(STAGE)); + if constexpr (STAGE < STAGE_TOP) { + assert(get<STAGE + 1>() == MatchKindCMP::EQ); + } + return _check_GT_t<STAGE>::eval(this); +} + +template <match_stage_t STAGE> +struct staged_position_t { + static_assert(is_valid_stage(STAGE)); + using me_t = staged_position_t<STAGE>; + using nxt_t = staged_position_t<STAGE - 1>; + bool is_end() const { + if (index == INDEX_END) { + return true; + } else { + assert(is_valid_index(index)); + return false; + } + } + index_t& index_by_stage(match_stage_t stage) { + assert(stage <= STAGE); + if (STAGE == stage) { + return index; + } else { + return nxt.index_by_stage(stage); + } + } + + auto operator<=>(const me_t& o) const = default; + + void assert_next_to(const me_t& prv) const { +#ifndef NDEBUG + if (is_end()) { + assert(!prv.is_end()); + } else if (index == prv.index) { + assert(!nxt.is_end()); + nxt.assert_next_to(prv.nxt); + } else if (index == prv.index + 1) { + assert(!prv.nxt.is_end()); + assert(nxt == nxt_t::begin()); + } else { + assert(false); + } +#endif + } + + me_t& operator-=(const me_t& o) { + assert(is_valid_index(o.index)); + assert(index >= o.index); + if (index != INDEX_END) { + assert(is_valid_index(index)); + index -= o.index; + if (index == 0) { + nxt -= o.nxt; + } + } + return *this; + } + + me_t& operator+=(const me_t& o) { + assert(is_valid_index(index)); + assert(is_valid_index(o.index)); + index += o.index; + nxt += o.nxt; + return *this; + } + + void encode(ceph::bufferlist& encoded) const { + ceph::encode(index, encoded); + nxt.encode(encoded); + } + + static me_t decode(ceph::bufferlist::const_iterator& delta) { + me_t ret; + ceph::decode(ret.index, delta); + ret.nxt = nxt_t::decode(delta); + return ret; + } + + static me_t begin() { return {0u, nxt_t::begin()}; } + static me_t end() { + return {INDEX_END, nxt_t::end()}; + } + + index_t index; + nxt_t nxt; +}; +template <match_stage_t STAGE> +std::ostream& operator<<(std::ostream& os, const staged_position_t<STAGE>& pos) { + if (pos.index == INDEX_END) { + os << "END"; + } else if (pos.index == INDEX_LAST) { + os << "LAST"; + } else { + os << pos.index; + assert(is_valid_index(pos.index)); + } + return os << ", " << pos.nxt; +} + +template <> +struct staged_position_t<STAGE_BOTTOM> { + using me_t = staged_position_t<STAGE_BOTTOM>; + bool is_end() const { + if (index == INDEX_END) { + return true; + } else { + assert(is_valid_index(index)); + return false; + } + } + index_t& index_by_stage(match_stage_t stage) { + assert(stage == STAGE_BOTTOM); + return index; + } + + auto operator<=>(const me_t&) const = default; + + me_t& operator-=(const me_t& o) { + assert(is_valid_index(o.index)); + assert(index >= o.index); + if (index != INDEX_END) { + assert(is_valid_index(index)); + index -= o.index; + } + return *this; + } + + me_t& operator+=(const me_t& o) { + assert(is_valid_index(index)); + assert(is_valid_index(o.index)); + index += o.index; + return *this; + } + + void assert_next_to(const me_t& prv) const { +#ifndef NDEBUG + if (is_end()) { + assert(!prv.is_end()); + } else { + assert(index == prv.index + 1); + } +#endif + } + + void encode(ceph::bufferlist& encoded) const { + ceph::encode(index, encoded); + } + + static me_t decode(ceph::bufferlist::const_iterator& delta) { + me_t ret; + ceph::decode(ret.index, delta); + return ret; + } + + static me_t begin() { return {0u}; } + static me_t end() { return {INDEX_END}; } + + index_t index; +}; +template <> +inline std::ostream& operator<<(std::ostream& os, const staged_position_t<STAGE_BOTTOM>& pos) { + if (pos.index == INDEX_END) { + os << "END"; + } else if (pos.index == INDEX_LAST) { + os << "LAST"; + } else { + os << pos.index; + assert(is_valid_index(pos.index)); + } + return os; +} + +using search_position_t = staged_position_t<STAGE_TOP>; + +template <match_stage_t STAGE> +const staged_position_t<STAGE>& cast_down(const search_position_t& pos) { + if constexpr (STAGE == STAGE_LEFT) { + return pos; + } else if constexpr (STAGE == STAGE_STRING) { +#ifndef NDEBUG + if (pos.is_end()) { + assert(pos.nxt.is_end()); + } else { + assert(pos.index == 0u); + } +#endif + return pos.nxt; + } else if constexpr (STAGE == STAGE_RIGHT) { +#ifndef NDEBUG + if (pos.is_end()) { + assert(pos.nxt.nxt.is_end()); + } else { + assert(pos.index == 0u); + assert(pos.nxt.index == 0u); + } +#endif + return pos.nxt.nxt; + } else { + ceph_abort("impossible path"); + } +} + +template <match_stage_t STAGE> +staged_position_t<STAGE>& cast_down(search_position_t& pos) { + const search_position_t& _pos = pos; + return const_cast<staged_position_t<STAGE>&>(cast_down<STAGE>(_pos)); +} + +template <match_stage_t STAGE> +staged_position_t<STAGE>& cast_down_fill_0(search_position_t& pos) { + if constexpr (STAGE == STAGE_LEFT) { + return pos; + } if constexpr (STAGE == STAGE_STRING) { + pos.index = 0; + return pos.nxt; + } else if constexpr (STAGE == STAGE_RIGHT) { + pos.index = 0; + pos.nxt.index = 0; + return pos.nxt.nxt; + } else { + ceph_abort("impossible path"); + } +} + +inline search_position_t&& normalize(search_position_t&& pos) { return std::move(pos); } + +template <match_stage_t STAGE, typename = std::enable_if_t<STAGE != STAGE_TOP>> +search_position_t normalize(staged_position_t<STAGE>&& pos) { + if (pos.is_end()) { + return search_position_t::end(); + } + if constexpr (STAGE == STAGE_STRING) { + return {0u, std::move(pos)}; + } else if (STAGE == STAGE_RIGHT) { + return {0u, {0u, std::move(pos)}}; + } else { + ceph_abort("impossible path"); + } +} + +struct memory_range_t { + const char* p_start; + const char* p_end; +}; + +struct container_range_t { + memory_range_t range; + extent_len_t node_size; +}; + +enum class ContainerType { ITERATIVE, INDEXABLE }; + +// the input type to construct the value during insert. +template <node_type_t> struct value_input_type; +template<> struct value_input_type<node_type_t::INTERNAL> { using type = laddr_t; }; +template<> struct value_input_type<node_type_t::LEAF> { using type = value_config_t; }; +template <node_type_t NODE_TYPE> +using value_input_type_t = typename value_input_type<NODE_TYPE>::type; + +template <node_type_t> struct value_type; +template<> struct value_type<node_type_t::INTERNAL> { using type = laddr_packed_t; }; +template<> struct value_type<node_type_t::LEAF> { using type = value_header_t; }; +template <node_type_t NODE_TYPE> +using value_type_t = typename value_type<NODE_TYPE>::type; + +template <node_type_t NODE_TYPE, match_stage_t STAGE> +struct staged_result_t { + using me_t = staged_result_t<NODE_TYPE, STAGE>; + bool is_end() const { return position.is_end(); } + + static me_t end() { + return {staged_position_t<STAGE>::end(), nullptr, MSTAT_END}; + } + template <typename T = me_t> + static std::enable_if_t<STAGE != STAGE_BOTTOM, T> from_nxt( + index_t index, const staged_result_t<NODE_TYPE, STAGE - 1>& nxt_stage_result) { + return {{index, nxt_stage_result.position}, + nxt_stage_result.p_value, + nxt_stage_result.mstat}; + } + + staged_position_t<STAGE> position; + const value_type_t<NODE_TYPE>* p_value; + match_stat_t mstat; +}; + +template <node_type_t NODE_TYPE> +using lookup_result_t = staged_result_t<NODE_TYPE, STAGE_TOP>; + +template <node_type_t NODE_TYPE> +lookup_result_t<NODE_TYPE>&& normalize( + lookup_result_t<NODE_TYPE>&& result) { return std::move(result); } + +template <node_type_t NODE_TYPE, match_stage_t STAGE, + typename = std::enable_if_t<STAGE != STAGE_TOP>> +lookup_result_t<NODE_TYPE> normalize( + staged_result_t<NODE_TYPE, STAGE>&& result) { + // FIXME: assert result.mstat correct + return {normalize(std::move(result.position)), result.p_value, result.mstat}; +} + +struct node_stats_t { + size_t size_persistent = 0; + size_t size_filled = 0; + // filled by staged::get_stats() + size_t size_logical = 0; + size_t size_overhead = 0; + size_t size_value = 0; + unsigned num_kvs = 0; +}; + +} + +#if FMT_VERSION >= 90000 +template <crimson::os::seastore::onode::match_stage_t S> +struct fmt::formatter<crimson::os::seastore::onode::staged_position_t<S>> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::onode::MatchHistory> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/sub_items_stage.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/sub_items_stage.cc new file mode 100644 index 000000000..28e6f7102 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/sub_items_stage.cc @@ -0,0 +1,329 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "sub_items_stage.h" + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_extent_mutable.h" + +namespace crimson::os::seastore::onode { + +template <IsFullKey Key> +const laddr_packed_t* internal_sub_items_t::insert_at( + NodeExtentMutable& mut, const internal_sub_items_t& sub_items, + const Key& key, const laddr_t& value, + index_t index, node_offset_t size, const char* p_left_bound) +{ + assert(index <= sub_items.keys()); + assert(size == estimate_insert(key, value)); + const char* p_shift_start = p_left_bound; + const char* p_shift_end = reinterpret_cast<const char*>( + sub_items.p_first_item + 1 - index); + mut.shift_absolute(p_shift_start, p_shift_end - p_shift_start, -(int)size); + + auto p_insert = const_cast<char*>(p_shift_end) - size; + auto item = internal_sub_item_t{ + snap_gen_t::from_key(key), laddr_packed_t{value}}; + mut.copy_in_absolute(p_insert, item); + return &reinterpret_cast<internal_sub_item_t*>(p_insert)->value; +} +#define IA_TEMPLATE(Key) \ + template const laddr_packed_t* internal_sub_items_t::insert_at<Key>( \ + NodeExtentMutable&, const internal_sub_items_t&, const Key&, \ + const laddr_t&, index_t, node_offset_t, const char*) +IA_TEMPLATE(key_view_t); +IA_TEMPLATE(key_hobj_t); + +node_offset_t internal_sub_items_t::trim_until( + NodeExtentMutable& mut, internal_sub_items_t& items, index_t index) +{ + assert(index != 0); + auto keys = items.keys(); + assert(index <= keys); + size_t ret = sizeof(internal_sub_item_t) * (keys - index); + assert(ret < mut.get_length()); + return ret; +} + +node_offset_t internal_sub_items_t::erase_at( + NodeExtentMutable& mut, const internal_sub_items_t& sub_items, + index_t index, const char* p_left_bound) +{ + assert(index < sub_items.keys()); + node_offset_t erase_size = sizeof(internal_sub_item_t); + const char* p_shift_start = p_left_bound; + const char* p_shift_end = reinterpret_cast<const char*>( + sub_items.p_first_item - index); + mut.shift_absolute(p_shift_start, p_shift_end - p_shift_start, erase_size); + return erase_size; +} + +template <KeyT KT> +void internal_sub_items_t::Appender<KT>::append( + const internal_sub_items_t& src, index_t from, index_t items) +{ + assert(from <= src.keys()); + if (items == 0) { + return; + } + assert(from < src.keys()); + assert(from + items <= src.keys()); + node_offset_t size = sizeof(internal_sub_item_t) * items; + p_append -= size; + p_mut->copy_in_absolute(p_append, src.p_first_item + 1 - from - items, size); +} + +template <KeyT KT> +void internal_sub_items_t::Appender<KT>::append( + const full_key_t<KT>& key, const laddr_t& value, + const laddr_packed_t*& p_value) +{ + p_append -= sizeof(internal_sub_item_t); + auto item = internal_sub_item_t{ + snap_gen_t::from_key(key), laddr_packed_t{value}}; + p_mut->copy_in_absolute(p_append, item); + p_value = &reinterpret_cast<internal_sub_item_t*>(p_append)->value; +} + +template <IsFullKey Key> +const value_header_t* leaf_sub_items_t::insert_at( + NodeExtentMutable& mut, const leaf_sub_items_t& sub_items, + const Key& key, const value_config_t& value, + index_t index, node_offset_t size, const char* p_left_bound) +{ + assert(index <= sub_items.keys()); + assert(size == estimate_insert(key, value)); + // a. [... item(index)] << size + const char* p_shift_start = p_left_bound; + const char* p_shift_end = sub_items.get_item_end(index); + mut.shift_absolute(p_shift_start, p_shift_end - p_shift_start, -(int)size); + + // b. insert item + auto p_insert = const_cast<char*>(p_shift_end - size); + auto p_value = reinterpret_cast<value_header_t*>(p_insert); + p_value->initiate(mut, value); + p_insert += value.allocation_size(); + mut.copy_in_absolute(p_insert, snap_gen_t::from_key(key)); + assert(p_insert + sizeof(snap_gen_t) + sizeof(node_offset_t) == p_shift_end); + + // c. compensate affected offsets + auto item_size = value.allocation_size() + sizeof(snap_gen_t); + for (auto i = index; i < sub_items.keys(); ++i) { + const node_offset_packed_t& offset_i = sub_items.get_offset(i); + mut.copy_in_absolute((void*)&offset_i, node_offset_t(offset_i.value + item_size)); + } + + // d. [item(index-1) ... item(0) ... offset(index)] <<< sizeof(node_offset_t) + const char* p_offset = (index == 0 ? + (const char*)&sub_items.get_offset(0) + sizeof(node_offset_t) : + (const char*)&sub_items.get_offset(index - 1)); + p_shift_start = p_shift_end; + p_shift_end = p_offset; + mut.shift_absolute(p_shift_start, p_shift_end - p_shift_start, -(int)sizeof(node_offset_t)); + + // e. insert offset + node_offset_t offset_to_item_start = item_size + sub_items.get_offset_to_end(index); + mut.copy_in_absolute( + const_cast<char*>(p_shift_end) - sizeof(node_offset_t), offset_to_item_start); + + // f. update num_sub_keys + mut.copy_in_absolute((void*)sub_items.p_num_keys, num_keys_t(sub_items.keys() + 1)); + + return p_value; +} +template const value_header_t* leaf_sub_items_t::insert_at<key_hobj_t>( + NodeExtentMutable&, const leaf_sub_items_t&, const key_hobj_t&, + const value_config_t&, index_t, node_offset_t, const char*); + +node_offset_t leaf_sub_items_t::trim_until( + NodeExtentMutable& mut, leaf_sub_items_t& items, index_t index) +{ + assert(index != 0); + auto keys = items.keys(); + assert(index <= keys); + if (index == keys) { + return 0; + } + index_t trim_items = keys - index; + const char* p_items_start = items.p_start(); + const char* p_shift_start = items.get_item_end(index); + const char* p_shift_end = items.get_item_end(0); + size_t size_trim_offsets = sizeof(node_offset_t) * trim_items; + mut.shift_absolute(p_shift_start, p_shift_end - p_shift_start, + size_trim_offsets); + mut.copy_in_absolute((void*)items.p_num_keys, num_keys_t(index)); + size_t ret = size_trim_offsets + (p_shift_start - p_items_start); + assert(ret < mut.get_length()); + return ret; +} + +node_offset_t leaf_sub_items_t::erase_at( + NodeExtentMutable& mut, const leaf_sub_items_t& sub_items, + index_t index, const char* p_left_bound) +{ + assert(sub_items.keys() > 0); + assert(index < sub_items.keys()); + auto p_item_start = sub_items.get_item_start(index); + auto p_item_end = sub_items.get_item_end(index); + assert(p_item_start < p_item_end); + node_offset_t item_erase_size = p_item_end - p_item_start; + node_offset_t erase_size = item_erase_size + sizeof(node_offset_t); + auto p_offset_end = (const char*)&sub_items.get_offset(index); + + // a. compensate affected offset[n] ... offset[index+1] + for (auto i = index + 1; i < sub_items.keys(); ++i) { + const node_offset_packed_t& offset_i = sub_items.get_offset(i); + mut.copy_in_absolute((void*)&offset_i, node_offset_t(offset_i.value - item_erase_size)); + } + + // b. kv[index-1] ... kv[0] ... offset[index+1] >> sizeof(node_offset_t) + mut.shift_absolute(p_item_end, p_offset_end - p_item_end, sizeof(node_offset_t)); + + // c. ... kv[n] ... kv[index+1] >> item_erase_size + mut.shift_absolute(p_left_bound, p_item_start - p_left_bound, erase_size); + + // d. update num_keys + mut.copy_in_absolute((void*)sub_items.p_num_keys, num_keys_t(sub_items.keys() - 1)); + + return erase_size; +} + +template class internal_sub_items_t::Appender<KeyT::VIEW>; +template class internal_sub_items_t::Appender<KeyT::HOBJ>; + +// helper type for the visitor +template<class... Ts> struct overloaded : Ts... { using Ts::operator()...; }; +// explicit deduction guide +template<class... Ts> overloaded(Ts...) -> overloaded<Ts...>; + +template <KeyT KT> +void leaf_sub_items_t::Appender<KT>::append( + const leaf_sub_items_t& src, index_t from, index_t items) +{ + if (p_append) { + // append from empty + assert(cnt <= APPENDER_LIMIT); + assert(from <= src.keys()); + if (items == 0) { + return; + } + if (op_src) { + assert(*op_src == src); + } else { + op_src = src; + } + assert(from < src.keys()); + assert(from + items <= src.keys()); + appends[cnt] = range_items_t{from, items}; + ++cnt; + } else { + // append from existing + assert(op_dst.has_value()); + assert(!p_appended); + assert(from == 0); + assert(items); + assert(items == src.keys()); + + num_keys_t num_keys = op_dst->keys(); + node_offset_t compensate = op_dst->get_offset(num_keys - 1).value; + const char* p_items_start = op_dst->p_start(); + const char* p_items_end = op_dst->p_items_end; + + // update dst num_keys + num_keys += items; + p_mut->copy_in_absolute((char*)op_dst->p_num_keys, num_keys); + + // shift dst items + std::size_t src_offsets_size = sizeof(node_offset_t) * items; + p_mut->shift_absolute(p_items_start, + p_items_end - p_items_start, + -(int)src_offsets_size); + + // fill offsets from src + node_offset_t offset; + char* p_cur_offset = const_cast<char*>(p_items_end); + for (auto i = from; i < from + items; ++i) { + offset = src.get_offset(i).value + compensate; + p_cur_offset -= sizeof(node_offset_t); + p_mut->copy_in_absolute(p_cur_offset, offset); + } + + // fill items from src + auto p_src_items_start = src.get_item_end(from + items); + std::size_t src_items_size = src.get_item_end(from) - p_src_items_start; + p_appended = const_cast<char*>(p_items_start) - src_offsets_size - src_items_size; + p_mut->copy_in_absolute(p_appended, p_src_items_start, src_items_size); + } +} + +template <KeyT KT> +char* leaf_sub_items_t::Appender<KT>::wrap() +{ + if (op_dst.has_value()) { + // append from existing + assert(p_appended); + return p_appended; + } + // append from empty + assert(p_append); + auto p_cur = p_append; + num_keys_t num_keys = 0; + for (auto i = 0u; i < cnt; ++i) { + auto& a = appends[i]; + std::visit(overloaded { + [&] (const range_items_t& arg) { num_keys += arg.items; }, + [&] (const kv_item_t& arg) { ++num_keys; } + }, a); + } + assert(num_keys); + p_cur -= sizeof(num_keys_t); + p_mut->copy_in_absolute(p_cur, num_keys); + + node_offset_t last_offset = 0; + for (auto i = 0u; i < cnt; ++i) { + auto& a = appends[i]; + std::visit(overloaded { + [&] (const range_items_t& arg) { + int compensate = (last_offset - op_src->get_offset_to_end(arg.from)); + node_offset_t offset; + for (auto i = arg.from; i < arg.from + arg.items; ++i) { + offset = op_src->get_offset(i).value + compensate; + p_cur -= sizeof(node_offset_t); + p_mut->copy_in_absolute(p_cur, offset); + } + last_offset = offset; + }, + [&] (const kv_item_t& arg) { + last_offset += sizeof(snap_gen_t) + arg.value_config.allocation_size(); + p_cur -= sizeof(node_offset_t); + p_mut->copy_in_absolute(p_cur, last_offset); + } + }, a); + } + + for (auto i = 0u; i < cnt; ++i) { + auto& a = appends[i]; + std::visit(overloaded { + [&] (const range_items_t& arg) { + auto _p_start = op_src->get_item_end(arg.from + arg.items); + size_t _len = op_src->get_item_end(arg.from) - _p_start; + p_cur -= _len; + p_mut->copy_in_absolute(p_cur, _p_start, _len); + }, + [&] (const kv_item_t& arg) { + assert(pp_value); + p_cur -= sizeof(snap_gen_t); + p_mut->copy_in_absolute(p_cur, snap_gen_t::from_key(*arg.p_key)); + p_cur -= arg.value_config.allocation_size(); + auto p_value = reinterpret_cast<value_header_t*>(p_cur); + p_value->initiate(*p_mut, arg.value_config); + *pp_value = p_value; + } + }, a); + } + return p_cur; +} + +template class leaf_sub_items_t::Appender<KeyT::VIEW>; +template class leaf_sub_items_t::Appender<KeyT::HOBJ>; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/stages/sub_items_stage.h b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/sub_items_stage.h new file mode 100644 index 000000000..e3d1fd7c5 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/stages/sub_items_stage.h @@ -0,0 +1,368 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <variant> + +#include "crimson/os/seastore/onode_manager/staged-fltree/node_types.h" +#include "key_layout.h" +#include "stage_types.h" + +namespace crimson::os::seastore::onode { + +class NodeExtentMutable; + +struct internal_sub_item_t { + const snap_gen_t& get_key() const { return key; } + const laddr_packed_t* get_p_value() const { return &value; } + + snap_gen_t key; + laddr_packed_t value; +} __attribute__((packed)); + +/** + * internal_sub_items_t + * + * The STAGE_RIGHT implementation for internal node N0/N1/N2, implements staged + * contract as an indexable container to index snap-gen to child node + * addresses. + * + * The layout of the contaner storing n sub-items: + * + * # <--------- container range -----------> # + * #<~># sub-items [2, n) # + * # # <- sub-item 1 -> # <- sub-item 0 -> # + * #...# snap-gen | laddr # snap-gen | laddr # + * ^ + * | + * p_first_item + + */ +class internal_sub_items_t { + public: + using num_keys_t = index_t; + + internal_sub_items_t(const container_range_t& _range) + : node_size{_range.node_size} { + assert(is_valid_node_size(node_size)); + auto& range = _range.range; + assert(range.p_start < range.p_end); + assert((range.p_end - range.p_start) % sizeof(internal_sub_item_t) == 0); + num_items = (range.p_end - range.p_start) / sizeof(internal_sub_item_t); + assert(num_items > 0); + auto _p_first_item = range.p_end - sizeof(internal_sub_item_t); + p_first_item = reinterpret_cast<const internal_sub_item_t*>(_p_first_item); + } + + // container type system + using key_get_type = const snap_gen_t&; + static constexpr auto CONTAINER_TYPE = ContainerType::INDEXABLE; + num_keys_t keys() const { return num_items; } + key_get_type operator[](index_t index) const { + assert(index < num_items); + return (p_first_item - index)->get_key(); + } + node_offset_t size_before(index_t index) const { + size_t ret = index * sizeof(internal_sub_item_t); + assert(ret < node_size); + return ret; + } + const laddr_packed_t* get_p_value(index_t index) const { + assert(index < num_items); + return (p_first_item - index)->get_p_value(); + } + node_offset_t size_overhead_at(index_t index) const { return 0u; } + void encode(const char* p_node_start, ceph::bufferlist& encoded) const { + auto p_end = reinterpret_cast<const char*>(p_first_item) + + sizeof(internal_sub_item_t); + auto p_start = p_end - num_items * sizeof(internal_sub_item_t); + int start_offset = p_start - p_node_start; + int stage_size = p_end - p_start; + assert(start_offset > 0); + assert(stage_size > 0); + assert(start_offset + stage_size < (int)node_size); + ceph::encode(static_cast<node_offset_t>(start_offset), encoded); + ceph::encode(static_cast<node_offset_t>(stage_size), encoded); + } + + static internal_sub_items_t decode( + const char* p_node_start, + extent_len_t node_size, + ceph::bufferlist::const_iterator& delta) { + node_offset_t start_offset; + ceph::decode(start_offset, delta); + node_offset_t stage_size; + ceph::decode(stage_size, delta); + assert(start_offset > 0); + assert(stage_size > 0); + assert((unsigned)start_offset + stage_size < node_size); + return internal_sub_items_t({{p_node_start + start_offset, + p_node_start + start_offset + stage_size}, + node_size}); + } + + static node_offset_t header_size() { return 0u; } + + template <IsFullKey Key> + static node_offset_t estimate_insert( + const Key&, const laddr_t&) { + return sizeof(internal_sub_item_t); + } + + template <IsFullKey Key> + static const laddr_packed_t* insert_at( + NodeExtentMutable&, const internal_sub_items_t&, + const Key&, const laddr_t&, + index_t index, node_offset_t size, const char* p_left_bound); + + static node_offset_t trim_until(NodeExtentMutable&, internal_sub_items_t&, index_t); + + static node_offset_t erase_at( + NodeExtentMutable&, const internal_sub_items_t&, index_t, const char*); + + template <KeyT KT> + class Appender; + + private: + extent_len_t node_size; + index_t num_items; + const internal_sub_item_t* p_first_item; +}; + +template <KeyT KT> +class internal_sub_items_t::Appender { + public: + Appender(NodeExtentMutable* p_mut, char* p_append) + : p_mut{p_mut}, p_append{p_append} {} + Appender(NodeExtentMutable* p_mut, const internal_sub_items_t& sub_items) + : p_mut{p_mut}, + p_append{(char*)(sub_items.p_first_item + 1 - sub_items.keys())} { + assert(sub_items.keys()); + } + void append(const internal_sub_items_t& src, index_t from, index_t items); + void append(const full_key_t<KT>&, const laddr_t&, const laddr_packed_t*&); + char* wrap() { return p_append; } + private: + NodeExtentMutable* p_mut; + char* p_append; +}; + +/** + * leaf_sub_items_t + * + * The STAGE_RIGHT implementation for leaf node N0/N1/N2, implements staged + * contract as an indexable container to index snap-gen to value_header_t. + * + * The layout of the contaner storing n sub-items: + * + * # <------------------------ container range -------------------------------> # + * # <---------- sub-items ----------------> # <--- offsets ---------# # + * #<~># sub-items [2, n) #<~>| offsets [2, n) # # + * # # <- sub-item 1 -> # <- sub-item 0 -> # | # # + * #...# snap-gen | value # snap-gen | value #...| offset1 | offset0 # num_keys # + * ^ ^ ^ + * | | | + * p_items_end + p_offsets + | + * p_num_keys + + */ +class leaf_sub_items_t { + public: + // should be enough to index all keys under 64 KiB node + using num_keys_t = uint16_t; + + // TODO: remove if num_keys_t is aligned + struct num_keys_packed_t { + num_keys_t value; + } __attribute__((packed)); + + leaf_sub_items_t(const container_range_t& _range) + : node_size{_range.node_size} { + assert(is_valid_node_size(node_size)); + auto& range = _range.range; + assert(range.p_start < range.p_end); + auto _p_num_keys = range.p_end - sizeof(num_keys_t); + assert(range.p_start < _p_num_keys); + p_num_keys = reinterpret_cast<const num_keys_packed_t*>(_p_num_keys); + assert(keys()); + auto _p_offsets = _p_num_keys - sizeof(node_offset_t); + assert(range.p_start < _p_offsets); + p_offsets = reinterpret_cast<const node_offset_packed_t*>(_p_offsets); + p_items_end = reinterpret_cast<const char*>(&get_offset(keys() - 1)); + assert(range.p_start < p_items_end); + assert(range.p_start == p_start()); + } + + bool operator==(const leaf_sub_items_t& x) { + return (p_num_keys == x.p_num_keys && + p_offsets == x.p_offsets && + p_items_end == x.p_items_end); + } + + const char* p_start() const { return get_item_end(keys()); } + + const node_offset_packed_t& get_offset(index_t index) const { + assert(index < keys()); + return *(p_offsets - index); + } + + const node_offset_t get_offset_to_end(index_t index) const { + assert(index <= keys()); + return index == 0 ? 0 : get_offset(index - 1).value; + } + + const char* get_item_start(index_t index) const { + return p_items_end - get_offset(index).value; + } + + const char* get_item_end(index_t index) const { + return p_items_end - get_offset_to_end(index); + } + + // container type system + using key_get_type = const snap_gen_t&; + static constexpr auto CONTAINER_TYPE = ContainerType::INDEXABLE; + num_keys_t keys() const { return p_num_keys->value; } + key_get_type operator[](index_t index) const { + assert(index < keys()); + auto pointer = get_item_end(index); + assert(get_item_start(index) < pointer); + pointer -= sizeof(snap_gen_t); + assert(get_item_start(index) < pointer); + return *reinterpret_cast<const snap_gen_t*>(pointer); + } + node_offset_t size_before(index_t index) const { + assert(index <= keys()); + size_t ret; + if (index == 0) { + ret = sizeof(num_keys_t); + } else { + --index; + ret = sizeof(num_keys_t) + + (index + 1) * sizeof(node_offset_t) + + get_offset(index).value; + } + assert(ret < node_size); + return ret; + } + node_offset_t size_overhead_at(index_t index) const { return sizeof(node_offset_t); } + const value_header_t* get_p_value(index_t index) const { + assert(index < keys()); + auto pointer = get_item_start(index); + auto value = reinterpret_cast<const value_header_t*>(pointer); + assert(pointer + value->allocation_size() + sizeof(snap_gen_t) == + get_item_end(index)); + return value; + } + void encode(const char* p_node_start, ceph::bufferlist& encoded) const { + auto p_end = reinterpret_cast<const char*>(p_num_keys) + + sizeof(num_keys_t); + int start_offset = p_start() - p_node_start; + int stage_size = p_end - p_start(); + assert(start_offset > 0); + assert(stage_size > 0); + assert(start_offset + stage_size < (int)node_size); + ceph::encode(static_cast<node_offset_t>(start_offset), encoded); + ceph::encode(static_cast<node_offset_t>(stage_size), encoded); + } + + static leaf_sub_items_t decode( + const char* p_node_start, + extent_len_t node_size, + ceph::bufferlist::const_iterator& delta) { + node_offset_t start_offset; + ceph::decode(start_offset, delta); + node_offset_t stage_size; + ceph::decode(stage_size, delta); + assert(start_offset > 0); + assert(stage_size > 0); + assert((unsigned)start_offset + stage_size < node_size); + return leaf_sub_items_t({{p_node_start + start_offset, + p_node_start + start_offset + stage_size}, + node_size}); + } + + static node_offset_t header_size() { return sizeof(num_keys_t); } + + template <IsFullKey Key> + static node_offset_t estimate_insert( + const Key&, const value_config_t& value) { + return value.allocation_size() + sizeof(snap_gen_t) + sizeof(node_offset_t); + } + + template <IsFullKey Key> + static const value_header_t* insert_at( + NodeExtentMutable&, const leaf_sub_items_t&, + const Key&, const value_config_t&, + index_t index, node_offset_t size, const char* p_left_bound); + + static node_offset_t trim_until(NodeExtentMutable&, leaf_sub_items_t&, index_t index); + + static node_offset_t erase_at( + NodeExtentMutable&, const leaf_sub_items_t&, index_t, const char*); + + template <KeyT KT> + class Appender; + + private: + extent_len_t node_size; + const num_keys_packed_t* p_num_keys; + const node_offset_packed_t* p_offsets; + const char* p_items_end; +}; + +constexpr index_t APPENDER_LIMIT = 3u; + +template <KeyT KT> +class leaf_sub_items_t::Appender { + struct range_items_t { + index_t from; + index_t items; + }; + struct kv_item_t { + const full_key_t<KT>* p_key; + value_config_t value_config; + }; + using var_t = std::variant<range_items_t, kv_item_t>; + + public: + Appender(NodeExtentMutable* p_mut, char* p_append) + : p_mut{p_mut}, p_append{p_append} { + } + Appender(NodeExtentMutable* p_mut, const leaf_sub_items_t& sub_items) + : p_mut{p_mut} , op_dst(sub_items) { + assert(sub_items.keys()); + } + + void append(const leaf_sub_items_t& src, index_t from, index_t items); + void append(const full_key_t<KT>& key, + const value_config_t& value, const value_header_t*& p_value) { + // append from empty + assert(p_append); + assert(pp_value == nullptr); + assert(cnt <= APPENDER_LIMIT); + appends[cnt] = kv_item_t{&key, value}; + ++cnt; + pp_value = &p_value; + } + char* wrap(); + + private: + NodeExtentMutable* p_mut; + // append from empty + std::optional<leaf_sub_items_t> op_src; + const value_header_t** pp_value = nullptr; + char* p_append = nullptr; + var_t appends[APPENDER_LIMIT]; + index_t cnt = 0; + // append from existing + std::optional<leaf_sub_items_t> op_dst; + char* p_appended = nullptr; +}; + +template <node_type_t> struct _sub_items_t; +template<> struct _sub_items_t<node_type_t::INTERNAL> { using type = internal_sub_items_t; }; +template<> struct _sub_items_t<node_type_t::LEAF> { using type = leaf_sub_items_t; }; +template <node_type_t NODE_TYPE> +using sub_items_t = typename _sub_items_t<NODE_TYPE>::type; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/super.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/super.cc new file mode 100644 index 000000000..09f20db3a --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/super.cc @@ -0,0 +1,28 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "super.h" +#include "node.h" + +namespace crimson::os::seastore::onode { + +Ref<Node> RootNodeTrackerIsolated::get_root(Transaction& t) const +{ + auto iter = tracked_supers.find(&t); + if (iter == tracked_supers.end()) { + return nullptr; + } else { + return iter->second->get_p_root(); + } +} + +Ref<Node> RootNodeTrackerShared::get_root(Transaction&) const +{ + if (is_clean()) { + return nullptr; + } else { + return tracked_super->get_p_root(); + } +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/super.h b/src/crimson/os/seastore/onode_manager/staged-fltree/super.h new file mode 100644 index 000000000..5eefee9ff --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/super.h @@ -0,0 +1,143 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <memory> + +#include "crimson/common/type_helpers.h" + +#include "fwd.h" + +namespace crimson::os::seastore::onode { + +class Node; +class Super; + +/** + * RootNodeTracker + * + * An abstracted tracker to get the root node by Transaction. + */ +class RootNodeTracker { + public: + virtual ~RootNodeTracker() = default; + virtual bool is_clean() const = 0; + virtual Ref<Node> get_root(Transaction&) const = 0; + static RootNodeTrackerURef create(bool read_isolated); + protected: + RootNodeTracker() = default; + RootNodeTracker(const RootNodeTracker&) = delete; + RootNodeTracker(RootNodeTracker&&) = delete; + RootNodeTracker& operator=(const RootNodeTracker&) = delete; + RootNodeTracker& operator=(RootNodeTracker&&) = delete; + virtual void do_track_super(Transaction&, Super&) = 0; + virtual void do_untrack_super(Transaction&, Super&) = 0; + friend class Super; +}; + +/** + * Super + * + * The parent of root node. It contains the relationship between a Transaction + * and a root node address. + */ +class Super { + public: + using URef = std::unique_ptr<Super>; + Super(const Super&) = delete; + Super(Super&&) = delete; + Super& operator=(const Super&) = delete; + Super& operator=(Super&&) = delete; + virtual ~Super() { + assert(tracked_root_node == nullptr); + tracker.do_untrack_super(t, *this); + } + + virtual laddr_t get_root_laddr() const = 0; + virtual void write_root_laddr(context_t, laddr_t) = 0; + + void do_track_root(Node& root) { + assert(tracked_root_node == nullptr); + tracked_root_node = &root; + } + void do_untrack_root(Node& root) { + assert(tracked_root_node == &root); + tracked_root_node = nullptr; + } + Node* get_p_root() const { + assert(tracked_root_node != nullptr); + return tracked_root_node; + } + + protected: + Super(Transaction& t, RootNodeTracker& tracker) + : t{t}, tracker{tracker} { + tracker.do_track_super(t, *this); + } + + private: + Transaction& t; + RootNodeTracker& tracker; + Node* tracked_root_node = nullptr; +}; + +/** + * RootNodeTrackerIsolated + * + * A concrete RootNodeTracker implementation which provides root node isolation + * between Transactions for Seastore backend. + */ +class RootNodeTrackerIsolated final : public RootNodeTracker { + public: + ~RootNodeTrackerIsolated() override { assert(is_clean()); } + protected: + bool is_clean() const override { + return tracked_supers.empty(); + } + void do_track_super(Transaction& t, Super& super) override { + assert(tracked_supers.find(&t) == tracked_supers.end()); + tracked_supers[&t] = &super; + } + void do_untrack_super(Transaction& t, Super& super) override { + [[maybe_unused]] auto removed = tracked_supers.erase(&t); + assert(removed); + } + ::Ref<Node> get_root(Transaction& t) const override; + std::map<Transaction*, Super*> tracked_supers; +}; + +/** + * RootNodeTrackerShared + * + * A concrete RootNodeTracker implementation which has no isolation between + * Transactions for Dummy backend. + */ +class RootNodeTrackerShared final : public RootNodeTracker { + public: + ~RootNodeTrackerShared() override { assert(is_clean()); } + protected: + bool is_clean() const override { + return tracked_super == nullptr; + } + void do_track_super(Transaction&, Super& super) override { + assert(is_clean()); + tracked_super = &super; + } + void do_untrack_super(Transaction&, Super& super) override { + assert(tracked_super == &super); + tracked_super = nullptr; + } + ::Ref<Node> get_root(Transaction&) const override; + Super* tracked_super = nullptr; +}; + +inline RootNodeTrackerURef RootNodeTracker::create(bool read_isolated) { + if (read_isolated) { + return RootNodeTrackerURef(new RootNodeTrackerIsolated()); + } else { + return RootNodeTrackerURef(new RootNodeTrackerShared()); + } +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/tree.h b/src/crimson/os/seastore/onode_manager/staged-fltree/tree.h new file mode 100644 index 000000000..7385e080c --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/tree.h @@ -0,0 +1,387 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <ostream> + +#include "common/hobject.h" +#include "crimson/common/type_helpers.h" +#include "crimson/os/seastore/logging.h" + +#include "fwd.h" +#include "node.h" +#include "node_extent_manager.h" +#include "stages/key_layout.h" +#include "super.h" +#include "value.h" + +/** + * tree.h + * + * A special-purpose and b-tree-based implementation that: + * - Fulfills requirements of OnodeManager to index ordered onode key-values; + * - Runs above seastore block and transaction layer; + * - Specially optimized for onode key structures and seastore + * delta/transaction semantics; + * + * Note: Cursor/Value are transactional, they cannot be used outside the scope + * of the according transaction, or the behavior is undefined. + */ + +namespace crimson::os::seastore::onode { + +class Node; +class tree_cursor_t; + +template <typename ValueImpl> +class Btree { + public: + Btree(NodeExtentManagerURef&& _nm) + : nm{std::move(_nm)}, + root_tracker{RootNodeTracker::create(nm->is_read_isolated())} {} + ~Btree() { assert(root_tracker->is_clean()); } + + Btree(const Btree&) = delete; + Btree(Btree&&) = delete; + Btree& operator=(const Btree&) = delete; + Btree& operator=(Btree&&) = delete; + + eagain_ifuture<> mkfs(Transaction& t) { + return Node::mkfs(get_context(t), *root_tracker); + } + + class Cursor { + public: + Cursor(const Cursor&) = default; + Cursor(Cursor&&) noexcept = default; + Cursor& operator=(const Cursor&) = default; + Cursor& operator=(Cursor&&) = default; + ~Cursor() = default; + + bool is_end() const { + if (p_cursor->is_tracked()) { + return false; + } else if (p_cursor->is_invalid()) { + return true; + } else { + // we don't actually store end cursor because it will hold a reference + // to an end leaf node and is not kept updated. + assert(p_cursor->is_end()); + ceph_abort("impossible"); + } + } + + /// Invalidate the Cursor before submitting transaction. + void invalidate() { + p_cursor.reset(); + } + + // XXX: return key_view_t to avoid unecessary ghobject_t constructions + ghobject_t get_ghobj() const { + assert(!is_end()); + auto view = p_cursor->get_key_view( + p_tree->value_builder.get_header_magic()); + assert(view.nspace().size() <= + p_tree->value_builder.get_max_ns_size()); + assert(view.oid().size() <= + p_tree->value_builder.get_max_oid_size()); + return view.to_ghobj(); + } + + ValueImpl value() { + assert(!is_end()); + return p_tree->value_builder.build_value( + *p_tree->nm, p_tree->value_builder, p_cursor); + } + + bool operator==(const Cursor& o) const { return operator<=>(o) == 0; } + + eagain_ifuture<Cursor> get_next(Transaction& t) { + assert(!is_end()); + auto this_obj = *this; + return p_cursor->get_next(p_tree->get_context(t) + ).si_then([this_obj] (Ref<tree_cursor_t> next_cursor) { + next_cursor->assert_next_to( + *this_obj.p_cursor, this_obj.p_tree->value_builder.get_header_magic()); + auto ret = Cursor{this_obj.p_tree, next_cursor}; + assert(this_obj < ret); + return ret; + }); + } + + template <bool FORCE_MERGE = false> + eagain_ifuture<Cursor> erase(Transaction& t) { + assert(!is_end()); + auto this_obj = *this; + return p_cursor->erase<FORCE_MERGE>(p_tree->get_context(t), true + ).si_then([this_obj, this] (Ref<tree_cursor_t> next_cursor) { + assert(p_cursor->is_invalid()); + if (next_cursor) { + assert(!next_cursor->is_end()); + return Cursor{p_tree, next_cursor}; + } else { + return Cursor{p_tree}; + } + }); + } + + private: + Cursor(Btree* p_tree, Ref<tree_cursor_t> _p_cursor) : p_tree(p_tree) { + if (_p_cursor->is_invalid()) { + // we don't create Cursor from an invalid tree_cursor_t. + ceph_abort("impossible"); + } else if (_p_cursor->is_end()) { + // we don't actually store end cursor because it will hold a reference + // to an end leaf node and is not kept updated. + } else { + assert(_p_cursor->is_tracked()); + p_cursor = _p_cursor; + } + } + Cursor(Btree* p_tree) : p_tree{p_tree} {} + + std::strong_ordering operator<=>(const Cursor& o) const { + assert(p_tree == o.p_tree); + return p_cursor->compare_to( + *o.p_cursor, p_tree->value_builder.get_header_magic()); + } + + static Cursor make_end(Btree* p_tree) { + return {p_tree}; + } + + Btree* p_tree; + Ref<tree_cursor_t> p_cursor = tree_cursor_t::get_invalid(); + + friend class Btree; + }; + + /* + * lookup + */ + + eagain_ifuture<Cursor> begin(Transaction& t) { + return get_root(t).si_then([this, &t](auto root) { + return root->lookup_smallest(get_context(t)); + }).si_then([this](auto cursor) { + return Cursor{this, cursor}; + }); + } + + eagain_ifuture<Cursor> last(Transaction& t) { + return get_root(t).si_then([this, &t](auto root) { + return root->lookup_largest(get_context(t)); + }).si_then([this](auto cursor) { + return Cursor(this, cursor); + }); + } + + Cursor end() { + return Cursor::make_end(this); + } + + eagain_ifuture<bool> contains(Transaction& t, const ghobject_t& obj) { + return seastar::do_with( + key_hobj_t{obj}, + [this, &t](auto& key) -> eagain_ifuture<bool> { + return get_root(t).si_then([this, &t, &key](auto root) { + // TODO: improve lower_bound() + return root->lower_bound(get_context(t), key); + }).si_then([](auto result) { + return MatchKindBS::EQ == result.match(); + }); + } + ); + } + + eagain_ifuture<Cursor> find(Transaction& t, const ghobject_t& obj) { + return seastar::do_with( + key_hobj_t{obj}, + [this, &t](auto& key) -> eagain_ifuture<Cursor> { + return get_root(t).si_then([this, &t, &key](auto root) { + // TODO: improve lower_bound() + return root->lower_bound(get_context(t), key); + }).si_then([this](auto result) { + if (result.match() == MatchKindBS::EQ) { + return Cursor(this, result.p_cursor); + } else { + return Cursor::make_end(this); + } + }); + } + ); + } + + /** + * lower_bound + * + * Returns a Cursor pointing to the element that is equal to the key, or the + * first element larger than the key, or the end Cursor if that element + * doesn't exist. + */ + eagain_ifuture<Cursor> lower_bound(Transaction& t, const ghobject_t& obj) { + return seastar::do_with( + key_hobj_t{obj}, + [this, &t](auto& key) -> eagain_ifuture<Cursor> { + return get_root(t).si_then([this, &t, &key](auto root) { + return root->lower_bound(get_context(t), key); + }).si_then([this](auto result) { + return Cursor(this, result.p_cursor); + }); + } + ); + } + + eagain_ifuture<Cursor> get_next(Transaction& t, Cursor& cursor) { + return cursor.get_next(t); + } + + /* + * modifiers + */ + + struct tree_value_config_t { + value_size_t payload_size = 256; + }; + using insert_iertr = eagain_iertr::extend< + crimson::ct_error::value_too_large>; + insert_iertr::future<std::pair<Cursor, bool>> + insert(Transaction& t, const ghobject_t& obj, tree_value_config_t _vconf) { + LOG_PREFIX(OTree::insert); + if (_vconf.payload_size > value_builder.get_max_value_payload_size()) { + SUBERRORT(seastore_onode, "value payload size {} too large to insert {}", + t, _vconf.payload_size, key_hobj_t{obj}); + return crimson::ct_error::value_too_large::make(); + } + if (obj.hobj.nspace.size() > value_builder.get_max_ns_size()) { + SUBERRORT(seastore_onode, "namespace size {} too large to insert {}", + t, obj.hobj.nspace.size(), key_hobj_t{obj}); + return crimson::ct_error::value_too_large::make(); + } + if (obj.hobj.oid.name.size() > value_builder.get_max_oid_size()) { + SUBERRORT(seastore_onode, "oid size {} too large to insert {}", + t, obj.hobj.oid.name.size(), key_hobj_t{obj}); + return crimson::ct_error::value_too_large::make(); + } + value_config_t vconf{value_builder.get_header_magic(), _vconf.payload_size}; + return seastar::do_with( + key_hobj_t{obj}, + [this, &t, vconf](auto& key) -> eagain_ifuture<std::pair<Cursor, bool>> { + ceph_assert(key.is_valid()); + return get_root(t).si_then([this, &t, &key, vconf](auto root) { + return root->insert(get_context(t), key, vconf, std::move(root)); + }).si_then([this](auto ret) { + auto& [cursor, success] = ret; + return std::make_pair(Cursor(this, cursor), success); + }); + } + ); + } + + eagain_ifuture<std::size_t> erase(Transaction& t, const ghobject_t& obj) { + return seastar::do_with( + key_hobj_t{obj}, + [this, &t](auto& key) -> eagain_ifuture<std::size_t> { + return get_root(t).si_then([this, &t, &key](auto root) { + return root->erase(get_context(t), key, std::move(root)); + }); + } + ); + } + + eagain_ifuture<Cursor> erase(Transaction& t, Cursor& pos) { + return pos.erase(t); + } + + eagain_ifuture<> erase(Transaction& t, Value& value) { + assert(value.is_tracked()); + auto ref_cursor = value.p_cursor; + return ref_cursor->erase(get_context(t), false + ).si_then([ref_cursor] (auto next_cursor) { + assert(ref_cursor->is_invalid()); + assert(!next_cursor); + }); + } + + /* + * stats + */ + + eagain_ifuture<size_t> height(Transaction& t) { + return get_root(t).si_then([](auto root) { + return size_t(root->level() + 1); + }); + } + + eagain_ifuture<tree_stats_t> get_stats_slow(Transaction& t) { + return get_root(t).si_then([this, &t](auto root) { + unsigned height = root->level() + 1; + return root->get_tree_stats(get_context(t) + ).si_then([height](auto stats) { + stats.height = height; + return seastar::make_ready_future<tree_stats_t>(stats); + }); + }); + } + + std::ostream& dump(Transaction& t, std::ostream& os) { + auto root = root_tracker->get_root(t); + if (root) { + root->dump(os); + } else { + os << "empty tree!"; + } + return os; + } + + std::ostream& print(std::ostream& os) const { + return os << "BTree-" << *nm; + } + + /* + * test_only + */ + + bool test_is_clean() const { + return root_tracker->is_clean(); + } + + eagain_ifuture<> test_clone_from( + Transaction& t, Transaction& t_from, Btree& from) { + // Note: assume the tree to clone is tracked correctly in memory. + // In some unit tests, parts of the tree are stubbed out that they + // should not be loaded from NodeExtentManager. + return from.get_root(t_from + ).si_then([this, &t](auto root_from) { + return root_from->test_clone_root(get_context(t), *root_tracker); + }); + } + + private: + context_t get_context(Transaction& t) { + return {*nm, value_builder, t}; + } + + eagain_ifuture<Ref<Node>> get_root(Transaction& t) { + auto root = root_tracker->get_root(t); + if (root) { + return seastar::make_ready_future<Ref<Node>>(root); + } else { + return Node::load_root(get_context(t), *root_tracker); + } + } + + NodeExtentManagerURef nm; + const ValueBuilderImpl<ValueImpl> value_builder; + RootNodeTrackerURef root_tracker; + + friend class DummyChildPool; +}; + +template <typename ValueImpl> +inline std::ostream& operator<<(std::ostream& os, const Btree<ValueImpl>& tree) { + return tree.print(os); +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/tree_utils.h b/src/crimson/os/seastore/onode_manager/staged-fltree/tree_utils.h new file mode 100644 index 000000000..3ad3564a6 --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/tree_utils.h @@ -0,0 +1,565 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <cassert> +#include <cstring> +#include <random> +#include <string> +#include <sstream> +#include <utility> +#include <vector> + +#include <seastar/core/thread.hh> + +#include "crimson/common/log.h" +#include "stages/key_layout.h" +#include "tree.h" + +/** + * tree_utils.h + * + * Contains shared logic for unit tests and perf tool. + */ + +namespace crimson::os::seastore::onode { + +/** + * templates to work with tree utility classes: + * + * struct ValueItem { + * <public members> + * + * value_size_t get_payload_size() const; + * static ValueItem create(std::size_t expected_size, std::size_t id); + * }; + * std::ostream& operator<<(std::ostream& os, const ValueItem& item); + * + * class ValueImpl final : public Value { + * ... + * + * using item_t = ValueItem; + * void initialize(Transaction& t, const item_t& item); + * void validate(const item_t& item); + * }; + * + */ + +template <typename CursorType> +void initialize_cursor_from_item( + Transaction& t, + const ghobject_t& key, + const typename decltype(std::declval<CursorType>().value())::item_t& item, + CursorType& cursor, + bool insert_success) { + ceph_assert(insert_success); + ceph_assert(!cursor.is_end()); + ceph_assert(cursor.get_ghobj() == key); + auto tree_value = cursor.value(); + tree_value.initialize(t, item); +} + + +template <typename CursorType> +void validate_cursor_from_item( + const ghobject_t& key, + const typename decltype(std::declval<CursorType>().value())::item_t& item, + CursorType& cursor) { + ceph_assert(!cursor.is_end()); + ceph_assert(cursor.get_ghobj() == key); + auto tree_value = cursor.value(); + tree_value.validate(item); +} + +template <typename ValueItem> +class Values { + public: + Values(size_t n) { + for (size_t i = 1; i <= n; ++i) { + auto item = create(i * 8); + values.push_back(item); + } + } + + Values(std::vector<size_t> sizes) { + for (auto& size : sizes) { + auto item = create(size); + values.push_back(item); + } + } + + ~Values() = default; + + ValueItem create(size_t size) { + return ValueItem::create(size, id++); + } + + ValueItem pick() const { + auto index = rd() % values.size(); + return values[index]; + } + + private: + std::size_t id = 0; + mutable std::random_device rd; + std::vector<ValueItem> values; +}; + +template <typename ValueItem> +class KVPool { + public: + struct kv_t { + ghobject_t key; + ValueItem value; + }; + using kv_vector_t = std::vector<kv_t>; + using kvptr_vector_t = std::vector<kv_t*>; + using iterator_t = typename kvptr_vector_t::iterator; + + size_t size() const { + return kvs.size(); + } + + iterator_t begin() { + return serial_p_kvs.begin(); + } + iterator_t end() { + return serial_p_kvs.end(); + } + iterator_t random_begin() { + return random_p_kvs.begin(); + } + iterator_t random_end() { + return random_p_kvs.end(); + } + + void shuffle() { + std::shuffle(random_p_kvs.begin(), random_p_kvs.end(), std::default_random_engine{}); + } + + void erase_from_random(iterator_t begin, iterator_t end) { + random_p_kvs.erase(begin, end); + kv_vector_t new_kvs; + for (auto p_kv : random_p_kvs) { + new_kvs.emplace_back(*p_kv); + } + std::sort(new_kvs.begin(), new_kvs.end(), [](auto& l, auto& r) { + return l.key < r.key; + }); + + kvs.swap(new_kvs); + serial_p_kvs.resize(kvs.size()); + random_p_kvs.resize(kvs.size()); + init(); + } + + static KVPool create_raw_range( + const std::vector<size_t>& ns_sizes, + const std::vector<size_t>& oid_sizes, + const std::vector<size_t>& value_sizes, + const std::pair<index_t, index_t>& range2, + const std::pair<index_t, index_t>& range1, + const std::pair<index_t, index_t>& range0) { + ceph_assert(range2.first < range2.second); + ceph_assert(range2.second - 1 <= MAX_SHARD); + ceph_assert(range2.second - 1 <= MAX_CRUSH); + ceph_assert(range1.first < range1.second); + ceph_assert(range1.second - 1 <= 9); + ceph_assert(range0.first < range0.second); + + kv_vector_t kvs; + std::random_device rd; + Values<ValueItem> values{value_sizes}; + for (index_t i = range2.first; i < range2.second; ++i) { + for (index_t j = range1.first; j < range1.second; ++j) { + size_t ns_size; + size_t oid_size; + if (j == 0) { + // store ns0, oid0 as empty strings for test purposes + ns_size = 0; + oid_size = 0; + } else { + ns_size = ns_sizes[rd() % ns_sizes.size()]; + oid_size = oid_sizes[rd() % oid_sizes.size()]; + assert(ns_size && oid_size); + } + for (index_t k = range0.first; k < range0.second; ++k) { + kvs.emplace_back( + kv_t{make_raw_oid(i, j, k, ns_size, oid_size), values.pick()} + ); + } + } + } + return KVPool(std::move(kvs)); + } + + static KVPool create_range( + const std::pair<index_t, index_t>& range_i, + const std::vector<size_t>& value_sizes, + const uint64_t block_size) { + kv_vector_t kvs; + std::random_device rd; + for (index_t i = range_i.first; i < range_i.second; ++i) { + auto value_size = value_sizes[rd() % value_sizes.size()]; + kvs.emplace_back( + kv_t{make_oid(i), ValueItem::create(value_size, i, block_size)} + ); + } + return KVPool(std::move(kvs)); + } + + private: + KVPool(kv_vector_t&& _kvs) + : kvs(std::move(_kvs)), serial_p_kvs(kvs.size()), random_p_kvs(kvs.size()) { + init(); + } + + void init() { + std::transform(kvs.begin(), kvs.end(), serial_p_kvs.begin(), + [] (kv_t& item) { return &item; }); + std::transform(kvs.begin(), kvs.end(), random_p_kvs.begin(), + [] (kv_t& item) { return &item; }); + shuffle(); + } + + static ghobject_t make_raw_oid( + index_t index2, index_t index1, index_t index0, + size_t ns_size, size_t oid_size) { + assert(index1 < 10); + std::ostringstream os_ns; + std::ostringstream os_oid; + if (index1 == 0) { + assert(!ns_size); + assert(!oid_size); + } else { + os_ns << "ns" << index1; + auto current_size = (size_t)os_ns.tellp(); + assert(ns_size >= current_size); + os_ns << std::string(ns_size - current_size, '_'); + + os_oid << "oid" << index1; + current_size = (size_t)os_oid.tellp(); + assert(oid_size >= current_size); + os_oid << std::string(oid_size - current_size, '_'); + } + + return ghobject_t(shard_id_t(index2), index2, index2, + os_ns.str(), os_oid.str(), index0, index0); + } + + static ghobject_t make_oid(index_t i) { + std::stringstream ss; + ss << "object_" << i; + auto ret = ghobject_t( + hobject_t( + sobject_t(ss.str(), CEPH_NOSNAP))); + ret.set_shard(shard_id_t(0)); + ret.hobj.nspace = "asdf"; + return ret; + } + + kv_vector_t kvs; + kvptr_vector_t serial_p_kvs; + kvptr_vector_t random_p_kvs; +}; + +template <bool TRACK, typename ValueImpl> +class TreeBuilder { + public: + using BtreeImpl = Btree<ValueImpl>; + using BtreeCursor = typename BtreeImpl::Cursor; + using ValueItem = typename ValueImpl::item_t; + using iterator_t = typename KVPool<ValueItem>::iterator_t; + + TreeBuilder(KVPool<ValueItem>& kvs, NodeExtentManagerURef&& nm) + : kvs{kvs} { + tree.emplace(std::move(nm)); + } + + eagain_ifuture<> bootstrap(Transaction& t) { + std::ostringstream oss; +#ifndef NDEBUG + oss << "debug=on, "; +#else + oss << "debug=off, "; +#endif +#ifdef UNIT_TESTS_BUILT + oss << "UNIT_TEST_BUILT=on, "; +#else + oss << "UNIT_TEST_BUILT=off, "; +#endif + if constexpr (TRACK) { + oss << "track=on, "; + } else { + oss << "track=off, "; + } + oss << *tree; + logger().warn("TreeBuilder: {}, bootstrapping ...", oss.str()); + return tree->mkfs(t); + } + + eagain_ifuture<BtreeCursor> insert_one( + Transaction& t, const iterator_t& iter_rd) { + auto p_kv = *iter_rd; + logger().debug("[{}] insert {} -> {}", + iter_rd - kvs.random_begin(), + key_hobj_t{p_kv->key}, + p_kv->value); + return tree->insert( + t, p_kv->key, {p_kv->value.get_payload_size()} + ).si_then([&t, this, p_kv](auto ret) { + boost::ignore_unused(this); // avoid clang warning; + auto success = ret.second; + auto cursor = std::move(ret.first); + initialize_cursor_from_item(t, p_kv->key, p_kv->value, cursor, success); +#ifndef NDEBUG + validate_cursor_from_item(p_kv->key, p_kv->value, cursor); + return tree->find(t, p_kv->key + ).si_then([cursor, p_kv](auto cursor_) mutable { + assert(!cursor_.is_end()); + ceph_assert(cursor_.get_ghobj() == p_kv->key); + ceph_assert(cursor_.value() == cursor.value()); + validate_cursor_from_item(p_kv->key, p_kv->value, cursor_); + return cursor; + }); +#else + return eagain_iertr::make_ready_future<BtreeCursor>(cursor); +#endif + }).handle_error_interruptible( + [] (const crimson::ct_error::value_too_large& e) { + ceph_abort("impossible path"); + }, + crimson::ct_error::pass_further_all{} + ); + } + + eagain_ifuture<> insert(Transaction& t) { + auto ref_kv_iter = seastar::make_lw_shared<iterator_t>(); + *ref_kv_iter = kvs.random_begin(); + auto cursors = seastar::make_lw_shared<std::vector<BtreeCursor>>(); + logger().warn("start inserting {} kvs ...", kvs.size()); + auto start_time = mono_clock::now(); + return trans_intr::repeat([&t, this, cursors, ref_kv_iter, + start_time]() + -> eagain_ifuture<seastar::stop_iteration> { + if (*ref_kv_iter == kvs.random_end()) { + std::chrono::duration<double> duration = mono_clock::now() - start_time; + logger().warn("Insert done! {}s", duration.count()); + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } else { + return insert_one(t, *ref_kv_iter + ).si_then([cursors, ref_kv_iter] (auto cursor) { + if constexpr (TRACK) { + cursors->emplace_back(cursor); + } + ++(*ref_kv_iter); + return seastar::stop_iteration::no; + }); + } + }).si_then([&t, this, cursors, ref_kv_iter] { + if (!cursors->empty()) { + logger().info("Verifing tracked cursors ..."); + *ref_kv_iter = kvs.random_begin(); + return seastar::do_with( + cursors->begin(), + [&t, this, cursors, ref_kv_iter] (auto& c_iter) { + return trans_intr::repeat( + [&t, this, &c_iter, cursors, ref_kv_iter] () + -> eagain_ifuture<seastar::stop_iteration> { + if (*ref_kv_iter == kvs.random_end()) { + logger().info("Verify done!"); + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + assert(c_iter != cursors->end()); + auto p_kv = **ref_kv_iter; + // validate values in tree keep intact + return tree->find(t, p_kv->key).si_then([&c_iter, ref_kv_iter](auto cursor) { + auto p_kv = **ref_kv_iter; + validate_cursor_from_item(p_kv->key, p_kv->value, cursor); + // validate values in cursors keep intact + validate_cursor_from_item(p_kv->key, p_kv->value, *c_iter); + ++(*ref_kv_iter); + ++c_iter; + return seastar::stop_iteration::no; + }); + }); + }); + } else { + return eagain_iertr::now(); + } + }); + } + + eagain_ifuture<> erase_one( + Transaction& t, const iterator_t& iter_rd) { + auto p_kv = *iter_rd; + logger().debug("[{}] erase {} -> {}", + iter_rd - kvs.random_begin(), + key_hobj_t{p_kv->key}, + p_kv->value); + return tree->erase(t, p_kv->key + ).si_then([&t, this, p_kv] (auto size) { + boost::ignore_unused(t); // avoid clang warning; + boost::ignore_unused(this); + boost::ignore_unused(p_kv); + ceph_assert(size == 1); +#ifndef NDEBUG + return tree->contains(t, p_kv->key + ).si_then([] (bool ret) { + ceph_assert(ret == false); + }); +#else + return eagain_iertr::now(); +#endif + }); + } + + eagain_ifuture<> erase(Transaction& t, std::size_t erase_size) { + assert(erase_size <= kvs.size()); + kvs.shuffle(); + auto erase_end = kvs.random_begin() + erase_size; + auto ref_kv_iter = seastar::make_lw_shared<iterator_t>(); + auto cursors = seastar::make_lw_shared<std::map<ghobject_t, BtreeCursor>>(); + return eagain_iertr::now().si_then([&t, this, cursors, ref_kv_iter] { + (void)this; // silence clang warning for !TRACK + (void)t; // silence clang warning for !TRACK + if constexpr (TRACK) { + logger().info("Tracking cursors before erase ..."); + *ref_kv_iter = kvs.begin(); + auto start_time = mono_clock::now(); + return trans_intr::repeat( + [&t, this, cursors, ref_kv_iter, start_time] () + -> eagain_ifuture<seastar::stop_iteration> { + if (*ref_kv_iter == kvs.end()) { + std::chrono::duration<double> duration = mono_clock::now() - start_time; + logger().info("Track done! {}s", duration.count()); + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + auto p_kv = **ref_kv_iter; + return tree->find(t, p_kv->key).si_then([cursors, ref_kv_iter](auto cursor) { + auto p_kv = **ref_kv_iter; + validate_cursor_from_item(p_kv->key, p_kv->value, cursor); + cursors->emplace(p_kv->key, cursor); + ++(*ref_kv_iter); + return seastar::stop_iteration::no; + }); + }); + } else { + return eagain_iertr::now(); + } + }).si_then([&t, this, ref_kv_iter, erase_end] { + *ref_kv_iter = kvs.random_begin(); + logger().warn("start erasing {}/{} kvs ...", + erase_end - kvs.random_begin(), kvs.size()); + auto start_time = mono_clock::now(); + return trans_intr::repeat([&t, this, ref_kv_iter, + start_time, erase_end] () + -> eagain_ifuture<seastar::stop_iteration> { + if (*ref_kv_iter == erase_end) { + std::chrono::duration<double> duration = mono_clock::now() - start_time; + logger().warn("Erase done! {}s", duration.count()); + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } else { + return erase_one(t, *ref_kv_iter + ).si_then([ref_kv_iter] { + ++(*ref_kv_iter); + return seastar::stop_iteration::no; + }); + } + }); + }).si_then([this, cursors, ref_kv_iter, erase_end] { + if constexpr (TRACK) { + logger().info("Verifing tracked cursors ..."); + *ref_kv_iter = kvs.random_begin(); + while (*ref_kv_iter != erase_end) { + auto p_kv = **ref_kv_iter; + auto c_it = cursors->find(p_kv->key); + ceph_assert(c_it != cursors->end()); + ceph_assert(c_it->second.is_end()); + cursors->erase(c_it); + ++(*ref_kv_iter); + } + } + kvs.erase_from_random(kvs.random_begin(), erase_end); + if constexpr (TRACK) { + *ref_kv_iter = kvs.begin(); + for (auto& [k, c] : *cursors) { + assert(*ref_kv_iter != kvs.end()); + auto p_kv = **ref_kv_iter; + validate_cursor_from_item(p_kv->key, p_kv->value, c); + ++(*ref_kv_iter); + } + logger().info("Verify done!"); + } + }); + } + + eagain_ifuture<> get_stats(Transaction& t) { + return tree->get_stats_slow(t + ).si_then([](auto stats) { + logger().warn("{}", stats); + }); + } + + eagain_ifuture<std::size_t> height(Transaction& t) { + return tree->height(t); + } + + void reload(NodeExtentManagerURef&& nm) { + tree.emplace(std::move(nm)); + } + + eagain_ifuture<> validate_one( + Transaction& t, const iterator_t& iter_seq) { + assert(iter_seq != kvs.end()); + auto next_iter = iter_seq + 1; + auto p_kv = *iter_seq; + return tree->find(t, p_kv->key + ).si_then([p_kv, &t] (auto cursor) { + validate_cursor_from_item(p_kv->key, p_kv->value, cursor); + return cursor.get_next(t); + }).si_then([next_iter, this] (auto cursor) { + if (next_iter == kvs.end()) { + ceph_assert(cursor.is_end()); + } else { + auto p_kv = *next_iter; + validate_cursor_from_item(p_kv->key, p_kv->value, cursor); + } + }); + } + + eagain_ifuture<> validate(Transaction& t) { + logger().info("Verifing inserted ..."); + return seastar::do_with( + kvs.begin(), + [this, &t] (auto &iter) { + return trans_intr::repeat( + [this, &t, &iter]() ->eagain_iertr::future<seastar::stop_iteration> { + if (iter == kvs.end()) { + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + } + return validate_one(t, iter).si_then([&iter] { + ++iter; + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::no); + }); + }); + }); + } + + private: + static seastar::logger& logger() { + return crimson::get_logger(ceph_subsys_test); + } + + KVPool<ValueItem>& kvs; + std::optional<BtreeImpl> tree; +}; + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/value.cc b/src/crimson/os/seastore/onode_manager/staged-fltree/value.cc new file mode 100644 index 000000000..694480d4e --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/value.cc @@ -0,0 +1,164 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#include "value.h" + +#include "node.h" +#include "node_delta_recorder.h" +#include "node_layout.h" + +// value implementations +#include "test/crimson/seastore/onode_tree/test_value.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.h" + +namespace crimson::os::seastore::onode { + +ceph::bufferlist& +ValueDeltaRecorder::get_encoded(NodeExtentMutable& payload_mut) +{ + ceph::encode(node_delta_op_t::SUBOP_UPDATE_VALUE, encoded); + node_offset_t offset = payload_mut.get_node_offset(); + assert(offset > sizeof(value_header_t)); + offset -= sizeof(value_header_t); + ceph::encode(offset, encoded); + return encoded; +} + +Value::Value(NodeExtentManager& nm, + const ValueBuilder& vb, + Ref<tree_cursor_t>& p_cursor) + : nm{nm}, vb{vb}, p_cursor{p_cursor} {} + +Value::~Value() {} + +bool Value::is_tracked() const +{ + assert(!p_cursor->is_end()); + return p_cursor->is_tracked(); +} + +void Value::invalidate() +{ + p_cursor.reset(); +} + +eagain_ifuture<> Value::extend(Transaction& t, value_size_t extend_size) +{ + assert(is_tracked()); + [[maybe_unused]] auto target_size = get_payload_size() + extend_size; + return p_cursor->extend_value(get_context(t), extend_size) +#ifndef NDEBUG + .si_then([this, target_size] { + assert(target_size == get_payload_size()); + }) +#endif + ; +} + +eagain_ifuture<> Value::trim(Transaction& t, value_size_t trim_size) +{ + assert(is_tracked()); + assert(get_payload_size() > trim_size); + [[maybe_unused]] auto target_size = get_payload_size() - trim_size; + return p_cursor->trim_value(get_context(t), trim_size) +#ifndef NDEBUG + .si_then([this, target_size] { + assert(target_size == get_payload_size()); + }) +#endif + ; +} + +const value_header_t* Value::read_value_header() const +{ + auto ret = p_cursor->read_value_header(vb.get_header_magic()); + assert(ret->payload_size <= vb.get_max_value_payload_size()); + return ret; +} + +std::pair<NodeExtentMutable&, ValueDeltaRecorder*> +Value::do_prepare_mutate_payload(Transaction& t) +{ + return p_cursor->prepare_mutate_value_payload(get_context(t)); +} + +laddr_t Value::get_hint() const +{ + return p_cursor->get_key_view(vb.get_header_magic()).get_hint(); +} + +std::unique_ptr<ValueDeltaRecorder> +build_value_recorder_by_type(ceph::bufferlist& encoded, + const value_magic_t& magic) +{ + std::unique_ptr<ValueDeltaRecorder> ret; + switch (magic) { + case value_magic_t::ONODE: + ret = std::make_unique<FLTreeOnode::Recorder>(encoded); + break; + case value_magic_t::TEST_UNBOUND: + ret = std::make_unique<UnboundedValue::Recorder>(encoded); + break; + case value_magic_t::TEST_BOUNDED: + ret = std::make_unique<BoundedValue::Recorder>(encoded); + break; + case value_magic_t::TEST_EXTENDED: + ret = std::make_unique<ExtendedValue::Recorder>(encoded); + break; + default: + ret = nullptr; + break; + } + assert(!ret || ret->get_header_magic() == magic); + return ret; +} + +void validate_tree_config(const tree_conf_t& conf) +{ + ceph_assert(conf.max_ns_size < + string_key_view_t::VALID_UPPER_BOUND); + ceph_assert(conf.max_oid_size < + string_key_view_t::VALID_UPPER_BOUND); + ceph_assert(is_valid_node_size(conf.internal_node_size)); + ceph_assert(is_valid_node_size(conf.leaf_node_size)); + + if (conf.do_split_check) { + // In hope to comply with 3 * (oid + ns) + 2 * value < node + // + // see node_layout.h for NODE_BLOCK_SIZE considerations + // + // The below calculations also consider the internal indexing overhead in + // order to be accurate, so the equation has become: + // node-header-size + 2 * max-full-insert-size + + // max-ns/oid-split-overhead <= node-size + + auto obj = ghobject_t{shard_id_t{0}, 0, 0, "", "", 0, 0}; + key_hobj_t key(obj); + auto max_str_size = conf.max_ns_size + conf.max_oid_size; +#define _STAGE_T(NodeType) node_to_stage_t<typename NodeType::node_stage_t> +#define NXT_T(StageType) staged<typename StageType::next_param_t> + + laddr_t i_value{0}; + auto insert_size_2 = + _STAGE_T(InternalNode0)::insert_size(key, i_value); + auto insert_size_0 = + NXT_T(NXT_T(_STAGE_T(InternalNode0)))::insert_size(key, i_value); + unsigned internal_size_bound = sizeof(node_header_t) + + (insert_size_2 + max_str_size) * 2 + + (insert_size_2 - insert_size_0 + max_str_size); + ceph_assert(internal_size_bound <= conf.internal_node_size); + + value_config_t l_value; + l_value.payload_size = conf.max_value_payload_size; + insert_size_2 = + _STAGE_T(LeafNode0)::insert_size(key, l_value); + insert_size_0 = + NXT_T(NXT_T(_STAGE_T(LeafNode0)))::insert_size(key, l_value); + unsigned leaf_size_bound = sizeof(node_header_t) + + (insert_size_2 + max_str_size) * 2 + + (insert_size_2 - insert_size_0 + max_str_size); + ceph_assert(leaf_size_bound <= conf.leaf_node_size); + } +} + +} diff --git a/src/crimson/os/seastore/onode_manager/staged-fltree/value.h b/src/crimson/os/seastore/onode_manager/staged-fltree/value.h new file mode 100644 index 000000000..d9f0c231a --- /dev/null +++ b/src/crimson/os/seastore/onode_manager/staged-fltree/value.h @@ -0,0 +1,337 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <ostream> + +#include "include/buffer.h" +#include "crimson/common/type_helpers.h" + +#include "fwd.h" +#include "node_extent_mutable.h" + +namespace crimson::os::seastore::onode { + +// value size up to 64 KiB +using value_size_t = uint16_t; +enum class value_magic_t : uint8_t { + ONODE = 0x52, + TEST_UNBOUND, + TEST_BOUNDED, + TEST_EXTENDED, +}; +inline std::ostream& operator<<(std::ostream& os, const value_magic_t& magic) { + switch (magic) { + case value_magic_t::ONODE: + return os << "ONODE"; + case value_magic_t::TEST_UNBOUND: + return os << "TEST_UNBOUND"; + case value_magic_t::TEST_BOUNDED: + return os << "TEST_BOUNDED"; + case value_magic_t::TEST_EXTENDED: + return os << "TEST_EXTENDED"; + default: + return os << "UNKNOWN(" << magic << ")"; + } +} + +/** + * value_config_t + * + * Parameters to create a value. + */ +struct value_config_t { + value_magic_t magic; + value_size_t payload_size; + + value_size_t allocation_size() const; + + void encode(ceph::bufferlist& encoded) const { + ceph::encode(magic, encoded); + ceph::encode(payload_size, encoded); + } + + static value_config_t decode(ceph::bufferlist::const_iterator& delta) { + value_magic_t magic; + ceph::decode(magic, delta); + value_size_t payload_size; + ceph::decode(payload_size, delta); + return {magic, payload_size}; + } +}; +inline std::ostream& operator<<(std::ostream& os, const value_config_t& conf) { + return os << "ValueConf(" << conf.magic + << ", " << conf.payload_size << "B)"; +} + +/** + * value_header_t + * + * The header structure in value layout. + * + * Value layout: + * + * # <- alloc size -> # + * # header | payload # + */ +struct value_header_t { + value_magic_t magic; + value_size_t payload_size; + + bool operator==(const value_header_t& rhs) const { + return (magic == rhs.magic && payload_size == rhs.payload_size); + } + bool operator!=(const value_header_t& rhs) const { + return !(*this == rhs); + } + + value_size_t allocation_size() const { + return payload_size + sizeof(value_header_t); + } + + const char* get_payload() const { + return reinterpret_cast<const char*>(this) + sizeof(value_header_t); + } + + NodeExtentMutable get_payload_mutable(NodeExtentMutable& node) const { + return node.get_mutable_absolute(get_payload(), payload_size); + } + + char* get_payload() { + return reinterpret_cast<char*>(this) + sizeof(value_header_t); + } + + void initiate(NodeExtentMutable& mut, const value_config_t& config) { + value_header_t header{config.magic, config.payload_size}; + mut.copy_in_absolute(this, header); + mut.set_absolute(get_payload(), 0, config.payload_size); + } + + static value_size_t estimate_allocation_size(value_size_t payload_size) { + return payload_size + sizeof(value_header_t); + } +} __attribute__((packed)); +inline std::ostream& operator<<(std::ostream& os, const value_header_t& header) { + return os << "Value(" << header.magic + << ", " << header.payload_size << "B)"; +} + +inline value_size_t value_config_t::allocation_size() const { + return value_header_t::estimate_allocation_size(payload_size); +} + +/** + * ValueDeltaRecorder + * + * An abstracted class to handle user-defined value delta encode, decode and + * replay. + */ +class ValueDeltaRecorder { + public: + virtual ~ValueDeltaRecorder() = default; + ValueDeltaRecorder(const ValueDeltaRecorder&) = delete; + ValueDeltaRecorder(ValueDeltaRecorder&&) = delete; + ValueDeltaRecorder& operator=(const ValueDeltaRecorder&) = delete; + ValueDeltaRecorder& operator=(ValueDeltaRecorder&&) = delete; + + /// Returns the value header magic for validation purpose. + virtual value_magic_t get_header_magic() const = 0; + + /// Called by DeltaRecorderT to apply user-defined value delta. + virtual void apply_value_delta(ceph::bufferlist::const_iterator&, + NodeExtentMutable&, + laddr_t) = 0; + + protected: + ValueDeltaRecorder(ceph::bufferlist& encoded) : encoded{encoded} {} + + /// Get the delta buffer to encode user-defined value delta. + ceph::bufferlist& get_encoded(NodeExtentMutable&); + + private: + ceph::bufferlist& encoded; +}; + +/** + * tree_conf_t + * + * Hard limits and compile-time configurations. + */ +struct tree_conf_t { + value_magic_t value_magic; + string_size_t max_ns_size; + string_size_t max_oid_size; + value_size_t max_value_payload_size; + extent_len_t internal_node_size; + extent_len_t leaf_node_size; + bool do_split_check = true; +}; + +class tree_cursor_t; +/** + * Value + * + * Value is a stateless view of the underlying value header and payload content + * stored in a tree leaf node, with the support to implement user-defined value + * deltas and to extend and trim the underlying payload data (not implemented + * yet). + * + * In the current implementation, we don't guarantee any alignment for value + * payload due to unaligned node layout and the according merge and split + * operations. + */ +class Value { + public: + virtual ~Value(); + Value(const Value&) = default; + Value(Value&&) = default; + Value& operator=(const Value&) = delete; + Value& operator=(Value&&) = delete; + + /// Returns whether the Value is still tracked in tree. + bool is_tracked() const; + + /// Invalidate the Value before submitting transaction. + void invalidate(); + + /// Returns the value payload size. + value_size_t get_payload_size() const { + assert(is_tracked()); + return read_value_header()->payload_size; + } + + laddr_t get_hint() const; + + bool operator==(const Value& v) const { return p_cursor == v.p_cursor; } + bool operator!=(const Value& v) const { return !(*this == v); } + + protected: + Value(NodeExtentManager&, const ValueBuilder&, Ref<tree_cursor_t>&); + + /// Extends the payload size. + eagain_ifuture<> extend(Transaction&, value_size_t extend_size); + + /// Trim and shrink the payload. + eagain_ifuture<> trim(Transaction&, value_size_t trim_size); + + /// Get the permission to mutate the payload with the optional value recorder. + template <typename PayloadT, typename ValueDeltaRecorderT> + std::pair<NodeExtentMutable&, ValueDeltaRecorderT*> + prepare_mutate_payload(Transaction& t) { + assert(is_tracked()); + assert(sizeof(PayloadT) <= get_payload_size()); + + auto value_mutable = do_prepare_mutate_payload(t); + assert(value_mutable.first.get_write() == + const_cast<const Value*>(this)->template read_payload<char>()); + assert(value_mutable.first.get_length() == get_payload_size()); + return {value_mutable.first, + static_cast<ValueDeltaRecorderT*>(value_mutable.second)}; + } + + /// Get the latest payload pointer for read. + template <typename PayloadT> + const PayloadT* read_payload() const { + assert(is_tracked()); + // see Value documentation + static_assert(alignof(PayloadT) == 1); + assert(sizeof(PayloadT) <= get_payload_size()); + return reinterpret_cast<const PayloadT*>(read_value_header()->get_payload()); + } + + private: + const value_header_t* read_value_header() const; + context_t get_context(Transaction& t) { + return {nm, vb, t}; + } + + std::pair<NodeExtentMutable&, ValueDeltaRecorder*> + do_prepare_mutate_payload(Transaction&); + + NodeExtentManager& nm; + const ValueBuilder& vb; + Ref<tree_cursor_t> p_cursor; + + template <typename ValueImpl> + friend class Btree; +}; + +/** + * ValueBuilder + * + * For tree nodes to build values without the need to depend on the actual + * implementation. + */ +struct ValueBuilder { + virtual value_magic_t get_header_magic() const = 0; + virtual string_size_t get_max_ns_size() const = 0; + virtual string_size_t get_max_oid_size() const = 0; + virtual value_size_t get_max_value_payload_size() const = 0; + virtual extent_len_t get_internal_node_size() const = 0; + virtual extent_len_t get_leaf_node_size() const = 0; + virtual std::unique_ptr<ValueDeltaRecorder> + build_value_recorder(ceph::bufferlist&) const = 0; +}; + +/** + * ValueBuilderImpl + * + * The concrete ValueBuilder implementation in Btree. + */ +template <typename ValueImpl> +struct ValueBuilderImpl final : public ValueBuilder { + ValueBuilderImpl() { + validate_tree_config(ValueImpl::TREE_CONF); + } + + value_magic_t get_header_magic() const { + return ValueImpl::TREE_CONF.value_magic; + } + string_size_t get_max_ns_size() const override { + return ValueImpl::TREE_CONF.max_ns_size; + } + string_size_t get_max_oid_size() const override { + return ValueImpl::TREE_CONF.max_oid_size; + } + value_size_t get_max_value_payload_size() const override { + return ValueImpl::TREE_CONF.max_value_payload_size; + } + extent_len_t get_internal_node_size() const override { + return ValueImpl::TREE_CONF.internal_node_size; + } + extent_len_t get_leaf_node_size() const override { + return ValueImpl::TREE_CONF.leaf_node_size; + } + + std::unique_ptr<ValueDeltaRecorder> + build_value_recorder(ceph::bufferlist& encoded) const override { + std::unique_ptr<ValueDeltaRecorder> ret = + std::make_unique<typename ValueImpl::Recorder>(encoded); + assert(ret->get_header_magic() == get_header_magic()); + return ret; + } + + ValueImpl build_value(NodeExtentManager& nm, + const ValueBuilder& vb, + Ref<tree_cursor_t>& p_cursor) const { + assert(vb.get_header_magic() == get_header_magic()); + return ValueImpl(nm, vb, p_cursor); + } +}; + +void validate_tree_config(const tree_conf_t& conf); + +/** + * Get the value recorder by type (the magic value) when the ValueBuilder is + * unavailable. + */ +std::unique_ptr<ValueDeltaRecorder> +build_value_recorder_by_type(ceph::bufferlist& encoded, const value_magic_t& magic); + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::onode::value_config_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::onode::value_header_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/ordering_handle.h b/src/crimson/os/seastore/ordering_handle.h new file mode 100644 index 000000000..a7802fda3 --- /dev/null +++ b/src/crimson/os/seastore/ordering_handle.h @@ -0,0 +1,181 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <seastar/core/shared_mutex.hh> + +#include "crimson/common/operation.h" +#include "crimson/osd/osd_operation.h" + +namespace crimson::os::seastore { + +struct WritePipeline { + struct ReserveProjectedUsage : OrderedExclusivePhaseT<ReserveProjectedUsage> { + constexpr static auto type_name = "WritePipeline::reserve_projected_usage"; + } reserve_projected_usage; + struct OolWrites : UnorderedStageT<OolWrites> { + constexpr static auto type_name = "UnorderedStage::ool_writes_stage"; + } ool_writes; + struct Prepare : OrderedExclusivePhaseT<Prepare> { + constexpr static auto type_name = "WritePipeline::prepare_phase"; + } prepare; + struct DeviceSubmission : OrderedConcurrentPhaseT<DeviceSubmission> { + constexpr static auto type_name = "WritePipeline::device_submission_phase"; + } device_submission; + struct Finalize : OrderedExclusivePhaseT<Finalize> { + constexpr static auto type_name = "WritePipeline::finalize_phase"; + } finalize; + + using BlockingEvents = std::tuple< + ReserveProjectedUsage::BlockingEvent, + OolWrites::BlockingEvent, + Prepare::BlockingEvent, + DeviceSubmission::BlockingEvent, + Finalize::BlockingEvent + >; +}; + +/** + * PlaceholderOperation + * + * Once seastore is more complete, I expect to update the externally + * facing interfaces to permit passing the osd level operation through. + * Until then (and for tests likely permanently) we'll use this unregistered + * placeholder for the pipeline phases necessary for journal correctness. + */ +class PlaceholderOperation : public crimson::osd::PhasedOperationT<PlaceholderOperation> { +public: + constexpr static auto type = 0U; + constexpr static auto type_name = + "crimson::os::seastore::PlaceholderOperation"; + + static PlaceholderOperation::IRef create() { + return IRef{new PlaceholderOperation()}; + } + + PipelineHandle handle; + WritePipeline::BlockingEvents tracking_events; + + PipelineHandle& get_handle() { + return handle; + } +private: + void dump_detail(ceph::Formatter *f) const final {} + void print(std::ostream &) const final {} +}; + +struct OperationProxy { + OperationRef op; + OperationProxy(OperationRef op) : op(std::move(op)) {} + + virtual seastar::future<> enter(WritePipeline::ReserveProjectedUsage&) = 0; + virtual seastar::future<> enter(WritePipeline::OolWrites&) = 0; + virtual seastar::future<> enter(WritePipeline::Prepare&) = 0; + virtual seastar::future<> enter(WritePipeline::DeviceSubmission&) = 0; + virtual seastar::future<> enter(WritePipeline::Finalize&) = 0; + + virtual void exit() = 0; + virtual seastar::future<> complete() = 0; + + virtual ~OperationProxy() = default; +}; + +template <typename OpT> +struct OperationProxyT : OperationProxy { + OperationProxyT(typename OpT::IRef op) : OperationProxy(op) {} + + OpT* that() { + return static_cast<OpT*>(op.get()); + } + const OpT* that() const { + return static_cast<const OpT*>(op.get()); + } + + seastar::future<> enter(WritePipeline::ReserveProjectedUsage& s) final { + return that()->enter_stage(s); + } + seastar::future<> enter(WritePipeline::OolWrites& s) final { + return that()->enter_stage(s); + } + seastar::future<> enter(WritePipeline::Prepare& s) final { + return that()->enter_stage(s); + } + seastar::future<> enter(WritePipeline::DeviceSubmission& s) final { + return that()->enter_stage(s); + } + seastar::future<> enter(WritePipeline::Finalize& s) final { + return that()->enter_stage(s); + } + + void exit() final { + return that()->handle.exit(); + } + seastar::future<> complete() final { + return that()->handle.complete(); + } +}; + +struct OrderingHandle { + // we can easily optimize this dynalloc out as all concretes are + // supposed to have exactly the same size. + std::unique_ptr<OperationProxy> op; + seastar::shared_mutex *collection_ordering_lock = nullptr; + + // in the future we might add further constructors / template to type + // erasure while extracting the location of tracking events. + OrderingHandle(std::unique_ptr<OperationProxy> op) : op(std::move(op)) {} + OrderingHandle(OrderingHandle &&other) + : op(std::move(other.op)), + collection_ordering_lock(other.collection_ordering_lock) { + other.collection_ordering_lock = nullptr; + } + + seastar::future<> take_collection_lock(seastar::shared_mutex &mutex) { + ceph_assert(!collection_ordering_lock); + collection_ordering_lock = &mutex; + return collection_ordering_lock->lock(); + } + + void maybe_release_collection_lock() { + if (collection_ordering_lock) { + collection_ordering_lock->unlock(); + collection_ordering_lock = nullptr; + } + } + + template <typename T> + seastar::future<> enter(T &t) { + return op->enter(t); + } + + void exit() { + op->exit(); + } + + seastar::future<> complete() { + return op->complete(); + } + + ~OrderingHandle() { + maybe_release_collection_lock(); + } +}; + +inline OrderingHandle get_dummy_ordering_handle() { + using PlaceholderOpProxy = OperationProxyT<PlaceholderOperation>; + return OrderingHandle{ + std::make_unique<PlaceholderOpProxy>(PlaceholderOperation::create())}; +} + +} // namespace crimson::os::seastore + +namespace crimson { + template <> + struct EventBackendRegistry<os::seastore::PlaceholderOperation> { + static std::tuple<> get_backends() { + return {}; + } + }; +} // namespace crimson + diff --git a/src/crimson/os/seastore/random_block_manager.cc b/src/crimson/os/seastore/random_block_manager.cc new file mode 100644 index 000000000..749edc97f --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager.cc @@ -0,0 +1,21 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/random_block_manager.h" +#include "crimson/os/seastore/random_block_manager/nvme_block_device.h" +#include "crimson/os/seastore/random_block_manager/rbm_device.h" + +namespace crimson::os::seastore { + +seastar::future<random_block_device::RBMDeviceRef> +get_rb_device( + const std::string &device) +{ + return seastar::make_ready_future<random_block_device::RBMDeviceRef>( + std::make_unique< + random_block_device::nvme::NVMeBlockDevice + >(device + "/block")); +} + +} diff --git a/src/crimson/os/seastore/random_block_manager.h b/src/crimson/os/seastore/random_block_manager.h new file mode 100644 index 000000000..d9be1b5e6 --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager.h @@ -0,0 +1,176 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <iosfwd> + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <seastar/core/future.hh> + +#include "include/ceph_assert.h" +#include "crimson/os/seastore/seastore_types.h" +#include "include/buffer_fwd.h" +#include "crimson/osd/exceptions.h" + +#include "crimson/os/seastore/transaction.h" + +#include "crimson/common/layout.h" +#include "include/buffer.h" +#include "crimson/os/seastore/device.h" + +namespace crimson::os::seastore { + +struct rbm_shard_info_t { + std::size_t size = 0; + uint64_t start_offset = 0; + + DENC(rbm_shard_info_t, v, p) { + DENC_START(1, 1, p); + denc(v.size, p); + denc(v.start_offset, p); + DENC_FINISH(p); + } +}; + +struct rbm_metadata_header_t { + size_t size = 0; + size_t block_size = 0; + uint64_t feature = 0; + uint64_t journal_size = 0; + checksum_t crc = 0; + device_config_t config; + unsigned int shard_num = 0; + std::vector<rbm_shard_info_t> shard_infos; + + DENC(rbm_metadata_header_t, v, p) { + DENC_START(1, 1, p); + denc(v.size, p); + denc(v.block_size, p); + denc(v.feature, p); + + denc(v.journal_size, p); + denc(v.crc, p); + denc(v.config, p); + denc(v.shard_num, p); + denc(v.shard_infos, p); + DENC_FINISH(p); + } + + void validate() const { + ceph_assert(shard_num == seastar::smp::count); + ceph_assert(block_size > 0); + for (unsigned int i = 0; i < seastar::smp::count; i ++) { + ceph_assert(shard_infos[i].size > block_size && + shard_infos[i].size % block_size == 0); + ceph_assert_always(shard_infos[i].size <= DEVICE_OFF_MAX); + ceph_assert(journal_size > 0 && + journal_size % block_size == 0); + ceph_assert(shard_infos[i].start_offset < size && + shard_infos[i].start_offset % block_size == 0); + } + ceph_assert(config.spec.magic != 0); + ceph_assert(get_default_backend_of_device(config.spec.dtype) == + backend_type_t::RANDOM_BLOCK); + ceph_assert(config.spec.id <= DEVICE_ID_MAX_VALID); + } +}; + +enum class rbm_extent_state_t { + FREE, // not allocated + RESERVED, // extent is reserved by alloc_new_extent, but is not persistent + ALLOCATED, // extent is persistent +}; + +class Device; +using rbm_abs_addr = uint64_t; +constexpr rbm_abs_addr RBM_START_ADDRESS = 0; +class RandomBlockManager { +public: + + using read_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent, + crimson::ct_error::erange>; + virtual read_ertr::future<> read(paddr_t addr, bufferptr &buffer) = 0; + + using write_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::ebadf, + crimson::ct_error::enospc, + crimson::ct_error::erange + >; + virtual write_ertr::future<> write(paddr_t addr, bufferptr &buf) = 0; + + using open_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent>; + virtual open_ertr::future<> open() = 0; + + using close_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg>; + virtual close_ertr::future<> close() = 0; + + using allocate_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enospc + >; + using allocate_ret = allocate_ertr::future<paddr_t>; + // allocator, return start addr of allocated blocks + virtual paddr_t alloc_extent(size_t size) = 0; + + virtual void mark_space_used(paddr_t paddr, size_t len) = 0; + virtual void mark_space_free(paddr_t paddr, size_t len) = 0; + + virtual void complete_allocation(paddr_t addr, size_t size) = 0; + + virtual size_t get_size() const = 0; + virtual extent_len_t get_block_size() const = 0; + virtual uint64_t get_free_blocks() const = 0; + virtual device_id_t get_device_id() const = 0; + virtual const seastore_meta_t &get_meta() const = 0; + virtual Device* get_device() = 0; + virtual paddr_t get_start() = 0; + virtual rbm_extent_state_t get_extent_state(paddr_t addr, size_t size) = 0; + virtual size_t get_journal_size() const = 0; + virtual ~RandomBlockManager() {} +}; +using RandomBlockManagerRef = std::unique_ptr<RandomBlockManager>; + +inline rbm_abs_addr convert_paddr_to_abs_addr(const paddr_t& paddr) { + const blk_paddr_t& blk_addr = paddr.as_blk_paddr(); + return blk_addr.get_device_off(); +} + +inline paddr_t convert_abs_addr_to_paddr(rbm_abs_addr addr, device_id_t d_id) { + return paddr_t::make_blk_paddr(d_id, addr); +} + +namespace random_block_device { + class RBMDevice; +} + +seastar::future<std::unique_ptr<random_block_device::RBMDevice>> + get_rb_device(const std::string &device); + +std::ostream &operator<<(std::ostream &out, const rbm_metadata_header_t &header); +std::ostream &operator<<(std::ostream &out, const rbm_shard_info_t &shard); +} + +WRITE_CLASS_DENC_BOUNDED( + crimson::os::seastore::rbm_shard_info_t +) +WRITE_CLASS_DENC_BOUNDED( + crimson::os::seastore::rbm_metadata_header_t +) + +#if FMT_VERSION >= 90000 +template<> struct fmt::formatter<crimson::os::seastore::rbm_metadata_header_t> : fmt::ostream_formatter {}; +template<> struct fmt::formatter<crimson::os::seastore::rbm_shard_info_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/random_block_manager/avlallocator.cc b/src/crimson/os/seastore/random_block_manager/avlallocator.cc new file mode 100644 index 000000000..28137a23d --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/avlallocator.cc @@ -0,0 +1,201 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab +// +#include "avlallocator.h" +#include "crimson/os/seastore/logging.h" + +SET_SUBSYS(seastore_device); + +namespace crimson::os::seastore { + +void AvlAllocator::mark_extent_used(rbm_abs_addr addr, size_t size) +{ + LOG_PREFIX(AvlAllocator::mark_extent_used); + DEBUG("addr: {}, size: {}, avail: {}", addr, size, available_size); + _remove_from_tree(addr, size); +} + +void AvlAllocator::init(rbm_abs_addr addr, size_t size, size_t b_size) +{ + LOG_PREFIX(AvlAllocator::init); + DEBUG("addr: {}, size: {}", addr, size); + auto r = new extent_range_t{ addr, addr + size }; + extent_tree.insert(*r); + extent_size_tree.insert(*r); + available_size = size; + block_size = b_size; + total_size = size; + base_addr = addr; +} + +void AvlAllocator::_remove_from_tree(rbm_abs_addr start, rbm_abs_addr size) +{ + LOG_PREFIX(AvlAllocator::_remove_from_tree); + rbm_abs_addr end = start + size; + + ceph_assert(size != 0); + ceph_assert(size <= available_size); + + auto rs = extent_tree.find(extent_range_t{start, end}, extent_tree.key_comp()); + DEBUG("rs start: {}, rs end: {}", rs->start, rs->end); + ceph_assert(rs != extent_tree.end()); + ceph_assert(rs->start <= start); + ceph_assert(rs->end >= end); + + bool left_over = (rs->start != start); + bool right_over = (rs->end != end); + + _extent_size_tree_rm(*rs); + + if (left_over && right_over) { + auto old_right_end = rs->end; + auto insert_pos = rs; + ceph_assert(insert_pos != extent_tree.end()); + ++insert_pos; + rs->end = start; + + auto r = new extent_range_t{end, old_right_end}; + extent_tree.insert_before(insert_pos, *r); + extent_size_tree.insert(*r); + available_size += r->length(); + _extent_size_tree_try_insert(*rs); + } else if (left_over) { + assert(is_aligned(start, block_size)); + rs->end = start; + _extent_size_tree_try_insert(*rs); + } else if (right_over) { + assert(is_aligned(end, block_size)); + rs->start = end; + _extent_size_tree_try_insert(*rs); + } else { + extent_tree.erase_and_dispose(rs, dispose_rs{}); + } +} + +rbm_abs_addr AvlAllocator::find_block(size_t size) +{ + const auto comp = extent_size_tree.key_comp(); + auto iter = extent_size_tree.lower_bound( + extent_range_t{base_addr, base_addr + size}, comp); + for (; iter != extent_size_tree.end(); ++iter) { + assert(is_aligned(iter->start, block_size)); + rbm_abs_addr off = iter->start; + if (off + size <= iter->end) { + return off; + } + } + return total_size; +} + +void AvlAllocator::_add_to_tree(rbm_abs_addr start, rbm_abs_addr size) +{ + LOG_PREFIX(AvlAllocator::_add_to_tree); + ceph_assert(size != 0); + DEBUG("addr: {}, size: {}", start, size); + + rbm_abs_addr end = start + size; + + auto rs_after = extent_tree.upper_bound(extent_range_t{start, end}, + extent_tree.key_comp()); + + auto rs_before = extent_tree.end(); + if (rs_after != extent_tree.begin()) { + rs_before = std::prev(rs_after); + } + + bool merge_before = (rs_before != extent_tree.end() && rs_before->end == start); + bool merge_after = (rs_after != extent_tree.end() && rs_after->start == end); + + if (merge_before && merge_after) { + _extent_size_tree_rm(*rs_before); + _extent_size_tree_rm(*rs_after); + rs_after->start = rs_before->start; + extent_tree.erase_and_dispose(rs_before, dispose_rs{}); + _extent_size_tree_try_insert(*rs_after); + } else if (merge_before) { + _extent_size_tree_rm(*rs_before); + rs_before->end = end; + _extent_size_tree_try_insert(*rs_before); + } else if (merge_after) { + _extent_size_tree_rm(*rs_after); + rs_after->start = start; + _extent_size_tree_try_insert(*rs_after); + } else { + auto r = new extent_range_t{start, end}; + extent_tree.insert(*r); + extent_size_tree.insert(*r); + available_size += r->length(); + } +} + +std::optional<interval_set<rbm_abs_addr>> AvlAllocator::alloc_extent( + size_t size) +{ + LOG_PREFIX(AvlAllocator::alloc_extent); + if (available_size < size) { + return std::nullopt; + } + if (extent_size_tree.empty()) { + return std::nullopt; + } + ceph_assert(size > 0); + ceph_assert(is_aligned(size, block_size)); + + interval_set<rbm_abs_addr> result; + + auto try_to_alloc_block = [this, &result, FNAME] (uint64_t alloc_size) -> uint64_t + { + rbm_abs_addr start = find_block(alloc_size); + if (start != base_addr + total_size) { + _remove_from_tree(start, alloc_size); + DEBUG("allocate addr: {}, allocate size: {}, available size: {}", + start, alloc_size, available_size); + result.insert(start, alloc_size); + return alloc_size; + } + return 0; + }; + + auto alloc = std::min(max_alloc_size, size); + rbm_abs_addr ret = try_to_alloc_block(alloc); + if (ret == 0) { + return std::nullopt; + } + + assert(!result.empty()); + assert(result.num_intervals() == 1); + for (auto p : result) { + INFO("result start: {}, end: {}", p.first, p.first + p.second); + if (detailed) { + assert(!reserved_extent_tracker.contains(p.first, p.second)); + reserved_extent_tracker.insert(p.first, p.second); + } + } + return result; +} + +void AvlAllocator::free_extent(rbm_abs_addr addr, size_t size) +{ + assert(total_size); + assert(total_size > available_size); + _add_to_tree(addr, size); + if (detailed && reserved_extent_tracker.contains(addr, size)) { + reserved_extent_tracker.erase(addr, size); + } +} + +bool AvlAllocator::is_free_extent(rbm_abs_addr start, size_t size) +{ + rbm_abs_addr end = start + size; + ceph_assert(size != 0); + if (start < base_addr || base_addr + total_size < end) { + return false; + } + + auto rs = extent_tree.find(extent_range_t{start, end}, extent_tree.key_comp()); + if (rs != extent_tree.end() && rs->start <= start && rs->end >= end) { + return true; + } + return false; +} +} diff --git a/src/crimson/os/seastore/random_block_manager/avlallocator.h b/src/crimson/os/seastore/random_block_manager/avlallocator.h new file mode 100644 index 000000000..d1a4fabca --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/avlallocator.h @@ -0,0 +1,174 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include "extent_allocator.h" +#include "include/ceph_assert.h" +#include "include/buffer_fwd.h" +#include "crimson/osd/exceptions.h" + +#include "crimson/os/seastore/transaction.h" +#include <string.h> +#include "include/buffer.h" + +#include <boost/intrusive/avl_set.hpp> +#include <optional> +#include <vector> + +namespace crimson::os::seastore { + +struct extent_range_t { + rbm_abs_addr start; + rbm_abs_addr end; + + extent_range_t(rbm_abs_addr start, rbm_abs_addr end) : + start(start), end(end) + {} + + struct before_t { + template<typename KeyLeft, typename KeyRight> + bool operator()(const KeyLeft& lhs, const KeyRight& rhs) const { + return lhs.end <= rhs.start; + } + }; + boost::intrusive::avl_set_member_hook<> offset_hook; + + struct shorter_t { + template<typename KeyType> + bool operator()(const extent_range_t& lhs, const KeyType& rhs) const { + auto lhs_size = lhs.length(); + auto rhs_size = rhs.end - rhs.start; + if (lhs_size < rhs_size) { + return true; + } else if (lhs_size > rhs_size) { + return false; + } else { + return lhs.start < rhs.start; + } + } + }; + + size_t length() const { + return end - start; + } + boost::intrusive::avl_set_member_hook<> size_hook; +}; + +/* + * This is the simplest version of avlallocator from bluestore's avlallocator + */ +class AvlAllocator : public ExtentAllocator { +public: + AvlAllocator(bool detailed) : + detailed(detailed) {} + std::optional<interval_set<rbm_abs_addr>> alloc_extent( + size_t size) final; + + void free_extent(rbm_abs_addr addr, size_t size) final; + void mark_extent_used(rbm_abs_addr addr, size_t size) final; + void init(rbm_abs_addr addr, size_t size, size_t b_size); + + struct dispose_rs { + void operator()(extent_range_t* p) + { + delete p; + } + }; + + ~AvlAllocator() { + close(); + } + + void close() { + if (!detailed) { + assert(reserved_extent_tracker.size() == 0); + } + extent_size_tree.clear(); + extent_tree.clear_and_dispose(dispose_rs{}); + total_size = 0; + block_size = 0; + available_size = 0; + base_addr = 0; + } + + uint64_t get_available_size() const final { + return available_size; + } + + uint64_t get_max_alloc_size() const final { + return max_alloc_size; + } + + bool is_free_extent(rbm_abs_addr start, size_t size); + + void complete_allocation(rbm_abs_addr start, size_t size) final { + if (detailed) { + assert(reserved_extent_tracker.contains(start, size)); + reserved_extent_tracker.erase(start, size); + } + } + + bool is_reserved_extent(rbm_abs_addr start, size_t size) { + if (detailed) { + return reserved_extent_tracker.contains(start, size); + } + return false; + } + + rbm_extent_state_t get_extent_state(rbm_abs_addr addr, size_t size) final { + if (is_reserved_extent(addr, size)) { + return rbm_extent_state_t::RESERVED; + } else if (is_free_extent(addr, size)) { + return rbm_extent_state_t::FREE; + } + return rbm_extent_state_t::ALLOCATED; + } + +private: + void _add_to_tree(rbm_abs_addr start, size_t size); + + void _extent_size_tree_rm(extent_range_t& r) { + ceph_assert(available_size >= r.length()); + available_size -= r.length(); + extent_size_tree.erase(r); + } + + void _extent_size_tree_try_insert(extent_range_t& r) { + extent_size_tree.insert(r); + available_size += r.length(); + } + + void _remove_from_tree(rbm_abs_addr start, rbm_abs_addr size); + rbm_abs_addr find_block(size_t size); + + using extent_tree_t = + boost::intrusive::avl_set< + extent_range_t, + boost::intrusive::compare<extent_range_t::before_t>, + boost::intrusive::member_hook< + extent_range_t, + boost::intrusive::avl_set_member_hook<>, + &extent_range_t::offset_hook>>; + extent_tree_t extent_tree; + + using extent_size_tree_t = + boost::intrusive::avl_set< + extent_range_t, + boost::intrusive::compare<extent_range_t::shorter_t>, + boost::intrusive::member_hook< + extent_range_t, + boost::intrusive::avl_set_member_hook<>, + &extent_range_t::size_hook>>; + extent_size_tree_t extent_size_tree; + + uint64_t block_size = 0; + uint64_t available_size = 0; + uint64_t total_size = 0; + uint64_t base_addr = 0; + uint64_t max_alloc_size = 4 << 20; + bool detailed; + interval_set<rbm_abs_addr> reserved_extent_tracker; +}; + +} diff --git a/src/crimson/os/seastore/random_block_manager/block_rb_manager.cc b/src/crimson/os/seastore/random_block_manager/block_rb_manager.cc new file mode 100644 index 000000000..511b70a2e --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/block_rb_manager.cc @@ -0,0 +1,176 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <sys/mman.h> +#include <string.h> + +#include "crimson/os/seastore/logging.h" + +#include "include/buffer.h" +#include "rbm_device.h" +#include "include/interval_set.h" +#include "include/intarith.h" +#include "block_rb_manager.h" + +SET_SUBSYS(seastore_device); + +namespace crimson::os::seastore { + +device_config_t get_rbm_ephemeral_device_config( + std::size_t index, std::size_t num_devices) +{ + assert(num_devices > index); + magic_t magic = 0xfffa; + auto type = device_type_t::RANDOM_BLOCK_EPHEMERAL; + bool is_major_device; + secondary_device_set_t secondary_devices; + if (index == 0) { + is_major_device = true; + for (std::size_t secondary_index = index + 1; + secondary_index < num_devices; + ++secondary_index) { + device_id_t secondary_id = static_cast<device_id_t>(secondary_index); + secondary_devices.insert({ + secondary_index, device_spec_t{magic, type, secondary_id} + }); + } + } else { // index > 0 + is_major_device = false; + } + + device_id_t id = static_cast<device_id_t>(DEVICE_ID_RANDOM_BLOCK_MIN + index); + seastore_meta_t meta = {}; + return {is_major_device, + device_spec_t{magic, type, id}, + meta, + secondary_devices}; +} + +paddr_t BlockRBManager::alloc_extent(size_t size) +{ + LOG_PREFIX(BlockRBManager::alloc_extent); + assert(allocator); + auto alloc = allocator->alloc_extent(size); + ceph_assert((*alloc).num_intervals() == 1); + auto extent = (*alloc).begin(); + ceph_assert(size == extent.get_len()); + paddr_t paddr = convert_abs_addr_to_paddr( + extent.get_start(), + device->get_device_id()); + DEBUG("allocated addr: {}, size: {}, requested size: {}", + paddr, extent.get_len(), size); + return paddr; +} + +void BlockRBManager::complete_allocation( + paddr_t paddr, size_t size) +{ + assert(allocator); + rbm_abs_addr addr = convert_paddr_to_abs_addr(paddr); + allocator->complete_allocation(addr, size); +} + +BlockRBManager::open_ertr::future<> BlockRBManager::open() +{ + assert(device); + assert(device->get_available_size() > 0); + assert(device->get_block_size() > 0); + auto ool_start = get_start_rbm_addr(); + allocator->init( + ool_start, + device->get_shard_end() - + ool_start, + device->get_block_size()); + return open_ertr::now(); +} + +BlockRBManager::write_ertr::future<> BlockRBManager::write( + paddr_t paddr, + bufferptr &bptr) +{ + LOG_PREFIX(BlockRBManager::write); + ceph_assert(device); + rbm_abs_addr addr = convert_paddr_to_abs_addr(paddr); + rbm_abs_addr start = device->get_shard_start(); + rbm_abs_addr end = device->get_shard_end(); + if (addr < start || addr + bptr.length() > end) { + ERROR("out of range: start {}, end {}, addr {}, length {}", + start, end, addr, bptr.length()); + return crimson::ct_error::erange::make(); + } + bufferptr bp = bufferptr(ceph::buffer::create_page_aligned(bptr.length())); + bp.copy_in(0, bptr.length(), bptr.c_str()); + return device->write( + addr, + std::move(bp)); +} + +BlockRBManager::read_ertr::future<> BlockRBManager::read( + paddr_t paddr, + bufferptr &bptr) +{ + LOG_PREFIX(BlockRBManager::read); + ceph_assert(device); + rbm_abs_addr addr = convert_paddr_to_abs_addr(paddr); + rbm_abs_addr start = device->get_shard_start(); + rbm_abs_addr end = device->get_shard_end(); + if (addr < start || addr + bptr.length() > end) { + ERROR("out of range: start {}, end {}, addr {}, length {}", + start, end, addr, bptr.length()); + return crimson::ct_error::erange::make(); + } + return device->read( + addr, + bptr); +} + +BlockRBManager::close_ertr::future<> BlockRBManager::close() +{ + ceph_assert(device); + allocator->close(); + return device->close(); +} + +BlockRBManager::write_ertr::future<> BlockRBManager::write( + rbm_abs_addr addr, + bufferlist &bl) +{ + LOG_PREFIX(BlockRBManager::write); + ceph_assert(device); + bufferptr bptr; + try { + bptr = bufferptr(ceph::buffer::create_page_aligned(bl.length())); + auto iter = bl.cbegin(); + iter.copy(bl.length(), bptr.c_str()); + } catch (const std::exception &e) { + DEBUG("write: exception creating aligned buffer {}", e); + ceph_assert(0 == "unhandled exception"); + } + return device->write( + addr, + std::move(bptr)); +} + +std::ostream &operator<<(std::ostream &out, const rbm_metadata_header_t &header) +{ + out << " rbm_metadata_header_t(size=" << header.size + << ", block_size=" << header.block_size + << ", feature=" << header.feature + << ", journal_size=" << header.journal_size + << ", crc=" << header.crc + << ", config=" << header.config + << ", shard_num=" << header.shard_num; + for (auto p : header.shard_infos) { + out << p; + } + return out << ")"; +} + +std::ostream &operator<<(std::ostream &out, const rbm_shard_info_t &shard) +{ + out << " rbm_shard_info_t(size=" << shard.size + << ", start_offset=" << shard.start_offset; + return out << ")"; +} + +} diff --git a/src/crimson/os/seastore/random_block_manager/block_rb_manager.h b/src/crimson/os/seastore/random_block_manager/block_rb_manager.h new file mode 100644 index 000000000..b686820d0 --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/block_rb_manager.h @@ -0,0 +1,142 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <iosfwd> + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <seastar/core/future.hh> + +#include "include/ceph_assert.h" +#include "crimson/os/seastore/seastore_types.h" +#include "include/buffer_fwd.h" +#include "crimson/osd/exceptions.h" + +#include "crimson/os/seastore/transaction.h" +#include "rbm_device.h" +#include "crimson/os/seastore/random_block_manager.h" + +#include "crimson/common/layout.h" +#include "include/buffer.h" +#include "include/uuid.h" +#include "avlallocator.h" + + +namespace crimson::os::seastore { + +using RBMDevice = random_block_device::RBMDevice; +using RBMDeviceRef = std::unique_ptr<RBMDevice>; + +device_config_t get_rbm_ephemeral_device_config( + std::size_t index, std::size_t num_devices); + +class BlockRBManager final : public RandomBlockManager { +public: + /* + * Ondisk layout (TODO) + * + * --------------------------------------------------------------------------- + * | rbm_metadata_header_t | metadatas | ... | data blocks | + * --------------------------------------------------------------------------- + */ + + read_ertr::future<> read(paddr_t addr, bufferptr &buffer) final; + write_ertr::future<> write(paddr_t addr, bufferptr &buf) final; + open_ertr::future<> open() final; + close_ertr::future<> close() final; + + /* + * alloc_extent + * + * The role of this function is to find out free blocks the transaction requires. + * To do so, alloc_extent() looks into both in-memory allocator + * and freebitmap blocks. + * + * TODO: multiple allocation + * + */ + paddr_t alloc_extent(size_t size) final; // allocator, return blocks + + void complete_allocation(paddr_t addr, size_t size) final; + + size_t get_start_rbm_addr() const { + return device->get_shard_journal_start() + device->get_journal_size(); + } + size_t get_size() const final { + return device->get_shard_end() - get_start_rbm_addr(); + }; + extent_len_t get_block_size() const final { return device->get_block_size(); } + + BlockRBManager(RBMDevice * device, std::string path, bool detailed) + : device(device), path(path) { + allocator.reset(new AvlAllocator(detailed)); + } + + write_ertr::future<> write(rbm_abs_addr addr, bufferlist &bl); + + device_id_t get_device_id() const final { + assert(device); + return device->get_device_id(); + } + + uint64_t get_free_blocks() const final { + // TODO: return correct free blocks after block allocator is introduced + assert(device); + return get_size() / get_block_size(); + } + const seastore_meta_t &get_meta() const final { + return device->get_meta(); + } + RBMDevice* get_device() { + return device; + } + + void mark_space_used(paddr_t paddr, size_t len) final { + assert(allocator); + rbm_abs_addr addr = convert_paddr_to_abs_addr(paddr); + assert(addr >= get_start_rbm_addr() && + addr + len <= device->get_shard_end()); + allocator->mark_extent_used(addr, len); + } + + void mark_space_free(paddr_t paddr, size_t len) final { + assert(allocator); + rbm_abs_addr addr = convert_paddr_to_abs_addr(paddr); + assert(addr >= get_start_rbm_addr() && + addr + len <= device->get_shard_end()); + allocator->free_extent(addr, len); + } + + paddr_t get_start() final { + return convert_abs_addr_to_paddr( + get_start_rbm_addr(), + device->get_device_id()); + } + + rbm_extent_state_t get_extent_state(paddr_t paddr, size_t size) final { + assert(allocator); + rbm_abs_addr addr = convert_paddr_to_abs_addr(paddr); + assert(addr >= get_start_rbm_addr() && + addr + size <= device->get_shard_end()); + return allocator->get_extent_state(addr, size); + } + + size_t get_journal_size() const final { + return device->get_journal_size(); + } + +private: + /* + * this contains the number of bitmap blocks, free blocks and + * rbm specific information + */ + ExtentAllocatorRef allocator; + RBMDevice * device; + std::string path; + int stream_id; // for multi-stream +}; +using BlockRBManagerRef = std::unique_ptr<BlockRBManager>; + +} diff --git a/src/crimson/os/seastore/random_block_manager/extent_allocator.h b/src/crimson/os/seastore/random_block_manager/extent_allocator.h new file mode 100644 index 000000000..8a3e62c6d --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/extent_allocator.h @@ -0,0 +1,75 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <seastar/core/future.hh> +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/random_block_manager.h" +#include "include/interval_set.h" + +namespace crimson::os::seastore { + +class ExtentAllocator { +public: + /** + * alloc_extent + * + * Allocate continous region as much as given size. + * Note that the inital state of extent is RESERVED after alloc_extent(). + * see rbm_extent_state_t in random_block_manager.h + * + * @param size + * @return nullopt or the address range (rbm_abs_addr, len) + */ + virtual std::optional<interval_set<rbm_abs_addr>> alloc_extent( + size_t size) = 0; + /** + * free_extent + * + * free given region + * + * @param rbm_abs_addr + * @param size + */ + virtual void free_extent(rbm_abs_addr addr, size_t size) = 0; + /** + * mark_extent_used + * + * This marks given region as used without alloc_extent. + * + * @param rbm_abs_addr + * @param size + */ + virtual void mark_extent_used(rbm_abs_addr addr, size_t size) = 0; + /** + * init + * + * Initialize the address space the ExtentAllocator will manage + * + * @param start address (rbm_abs_addr) + * @param total size + * @param block size + */ + virtual void init(rbm_abs_addr addr, size_t size, size_t b_size) = 0; + virtual uint64_t get_available_size() const = 0; + virtual uint64_t get_max_alloc_size() const = 0; + virtual void close() = 0; + /** + * complete_allocation + * + * This changes this extent state from RESERVED to ALLOCATED + * + * @param start address + * @param size + */ + virtual void complete_allocation(rbm_abs_addr start, size_t size) = 0; + virtual rbm_extent_state_t get_extent_state(rbm_abs_addr addr, size_t size) = 0; + virtual ~ExtentAllocator() {} +}; +using ExtentAllocatorRef = std::unique_ptr<ExtentAllocator>; + + +} diff --git a/src/crimson/os/seastore/random_block_manager/nvme_block_device.cc b/src/crimson/os/seastore/random_block_manager/nvme_block_device.cc new file mode 100644 index 000000000..6437f06a4 --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/nvme_block_device.cc @@ -0,0 +1,280 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <sys/mman.h> +#include <string.h> + +#include <fcntl.h> + +#include "crimson/common/log.h" +#include "crimson/common/errorator-loop.h" + +#include "include/buffer.h" +#include "rbm_device.h" +#include "nvme_block_device.h" +#include "block_rb_manager.h" + +namespace { + seastar::logger& logger() { + return crimson::get_logger(ceph_subsys_seastore_tm); + } +} + +namespace crimson::os::seastore::random_block_device::nvme { + +NVMeBlockDevice::mkfs_ret NVMeBlockDevice::mkfs(device_config_t config) { + using crimson::common::get_conf; + return shard_devices.local().do_primary_mkfs(config, + seastar::smp::count, + get_conf<Option::size_t>("seastore_cbjournal_size") + ); +} + +open_ertr::future<> NVMeBlockDevice::open( + const std::string &in_path, + seastar::open_flags mode) { + return seastar::do_with(in_path, [this, mode](auto& in_path) { + return seastar::file_stat(in_path).then([this, mode, in_path](auto stat) { + return seastar::open_file_dma(in_path, mode).then([=, this](auto file) { + device = std::move(file); + logger().debug("open"); + // Get SSD's features from identify_controller and namespace command. + // Do identify_controller first, and then identify_namespace. + return identify_controller(device).safe_then([this, in_path, mode]( + auto id_controller_data) { + support_multistream = id_controller_data.oacs.support_directives; + if (support_multistream) { + stream_id_count = WRITE_LIFE_MAX; + } + awupf = id_controller_data.awupf + 1; + return identify_namespace(device).safe_then([this, in_path, mode] ( + auto id_namespace_data) { + atomic_write_unit = awupf * super.block_size; + data_protection_type = id_namespace_data.dps.protection_type; + data_protection_enabled = (data_protection_type > 0); + if (id_namespace_data.nsfeat.opterf == 1){ + // NPWG and NPWA is 0'based value + write_granularity = super.block_size * (id_namespace_data.npwg + 1); + write_alignment = super.block_size * (id_namespace_data.npwa + 1); + } + return open_for_io(in_path, mode); + }); + }).handle_error(crimson::ct_error::input_output_error::handle([this, in_path, mode]{ + logger().error("open: id ctrlr failed. open without ioctl"); + return open_for_io(in_path, mode); + }), crimson::ct_error::pass_further_all{}); + }); + }); + }); +} + +open_ertr::future<> NVMeBlockDevice::open_for_io( + const std::string& in_path, + seastar::open_flags mode) { + io_device.resize(stream_id_count); + return seastar::do_for_each(io_device, [=, this](auto &target_device) { + return seastar::open_file_dma(in_path, mode).then([this]( + auto file) { + assert(io_device.size() > stream_index_to_open); + io_device[stream_index_to_open] = std::move(file); + return io_device[stream_index_to_open].fcntl( + F_SET_FILE_RW_HINT, + (uintptr_t)&stream_index_to_open).then([this](auto ret) { + stream_index_to_open++; + return seastar::now(); + }); + }); + }); +} + +NVMeBlockDevice::mount_ret NVMeBlockDevice::mount() +{ + logger().debug(" mount "); + return shard_devices.invoke_on_all([](auto &local_device) { + return local_device.do_shard_mount( + ).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in RBMDevice::do_mount" + }); + }); +} + +write_ertr::future<> NVMeBlockDevice::write( + uint64_t offset, + bufferptr &&bptr, + uint16_t stream) { + logger().debug( + "block: write offset {} len {}", + offset, + bptr.length()); + auto length = bptr.length(); + + assert((length % super.block_size) == 0); + uint16_t supported_stream = stream; + if (stream >= stream_id_count) { + supported_stream = WRITE_LIFE_NOT_SET; + } + return seastar::do_with( + std::move(bptr), + [this, offset, length, supported_stream] (auto& bptr) { + return io_device[supported_stream].dma_write( + offset, bptr.c_str(), length).handle_exception( + [](auto e) -> write_ertr::future<size_t> { + logger().error("write: dma_write got error{}", e); + return crimson::ct_error::input_output_error::make(); + }).then([length](auto result) -> write_ertr::future<> { + if (result != length) { + logger().error("write: dma_write got error with not proper length"); + return crimson::ct_error::input_output_error::make(); + } + return write_ertr::now(); + }); + }); +} + +read_ertr::future<> NVMeBlockDevice::read( + uint64_t offset, + bufferptr &bptr) { + logger().debug( + "block: read offset {} len {}", + offset, + bptr.length()); + auto length = bptr.length(); + + assert((length % super.block_size) == 0); + + return device.dma_read(offset, bptr.c_str(), length).handle_exception( + [](auto e) -> read_ertr::future<size_t> { + logger().error("read: dma_read got error{}", e); + return crimson::ct_error::input_output_error::make(); + }).then([length](auto result) -> read_ertr::future<> { + if (result != length) { + logger().error("read: dma_read got error with not proper length"); + return crimson::ct_error::input_output_error::make(); + } + return read_ertr::now(); + }); +} + +write_ertr::future<> NVMeBlockDevice::writev( + uint64_t offset, + ceph::bufferlist bl, + uint16_t stream) { + logger().debug( + "block: write offset {} len {}", + offset, + bl.length()); + + uint16_t supported_stream = stream; + if (stream >= stream_id_count) { + supported_stream = WRITE_LIFE_NOT_SET; + } + bl.rebuild_aligned(super.block_size); + + return seastar::do_with( + bl.prepare_iovs(), + std::move(bl), + [this, supported_stream, offset](auto& iovs, auto& bl) + { + return write_ertr::parallel_for_each( + iovs, + [this, supported_stream, offset](auto& p) mutable + { + auto off = offset + p.offset; + auto len = p.length; + auto& iov = p.iov; + return io_device[supported_stream].dma_write(off, std::move(iov) + ).handle_exception( + [this, off, len](auto e) -> write_ertr::future<size_t> + { + logger().error("{} poffset={}~{} dma_write got error -- {}", + device_id_printer_t{get_device_id()}, off, len, e); + return crimson::ct_error::input_output_error::make(); + }).then([this, off, len](size_t written) -> write_ertr::future<> { + if (written != len) { + logger().error("{} poffset={}~{} dma_write len={} inconsistent", + device_id_printer_t{get_device_id()}, off, len, written); + return crimson::ct_error::input_output_error::make(); + } + return write_ertr::now(); + }); + }); + }); +} + +Device::close_ertr::future<> NVMeBlockDevice::close() { + logger().debug(" close "); + stream_index_to_open = WRITE_LIFE_NOT_SET; + return device.close().then([this]() { + return seastar::do_for_each(io_device, [](auto target_device) { + return target_device.close(); + }); + }); +} + +nvme_command_ertr::future<nvme_identify_controller_data_t> +NVMeBlockDevice::identify_controller(seastar::file f) { + return seastar::do_with( + nvme_admin_command_t(), + nvme_identify_controller_data_t(), + [this, f](auto &admin_command, auto &data) { + admin_command.common.opcode = nvme_admin_command_t::OPCODE_IDENTIFY; + admin_command.common.addr = (uint64_t)&data; + admin_command.common.data_len = sizeof(data); + admin_command.identify.cns = nvme_identify_command_t::CNS_CONTROLLER; + + return pass_admin(admin_command, f).safe_then([&data](auto status) { + return seastar::make_ready_future<nvme_identify_controller_data_t>( + std::move(data)); + }); + }); +} + +discard_ertr::future<> NVMeBlockDevice::discard(uint64_t offset, uint64_t len) { + return device.discard(offset, len); +} + +nvme_command_ertr::future<nvme_identify_namespace_data_t> +NVMeBlockDevice::identify_namespace(seastar::file f) { + return get_nsid(f).safe_then([this, f](auto nsid) { + return seastar::do_with( + nvme_admin_command_t(), + nvme_identify_namespace_data_t(), + [this, nsid, f](auto &admin_command, auto &data) { + admin_command.common.opcode = nvme_admin_command_t::OPCODE_IDENTIFY; + admin_command.common.addr = (uint64_t)&data; + admin_command.common.data_len = sizeof(data); + admin_command.common.nsid = nsid; + admin_command.identify.cns = nvme_identify_command_t::CNS_NAMESPACE; + + return pass_admin(admin_command, f).safe_then([&data](auto status){ + return seastar::make_ready_future<nvme_identify_namespace_data_t>( + std::move(data)); + }); + }); + }); +} + +nvme_command_ertr::future<int> NVMeBlockDevice::get_nsid(seastar::file f) { + return f.ioctl(NVME_IOCTL_ID, nullptr).handle_exception( + [](auto e)->nvme_command_ertr::future<int> { + logger().error("pass_admin: ioctl failed"); + return crimson::ct_error::input_output_error::make(); + }); +} + +nvme_command_ertr::future<int> NVMeBlockDevice::pass_admin( + nvme_admin_command_t& admin_cmd, seastar::file f) { + return f.ioctl(NVME_IOCTL_ADMIN_CMD, &admin_cmd).handle_exception( + [](auto e)->nvme_command_ertr::future<int> { + logger().error("pass_admin: ioctl failed"); + return crimson::ct_error::input_output_error::make(); + }); +} + +nvme_command_ertr::future<int> NVMeBlockDevice::pass_through_io( + nvme_io_command_t& io_cmd) { + return device.ioctl(NVME_IOCTL_IO_CMD, &io_cmd); +} + +} diff --git a/src/crimson/os/seastore/random_block_manager/nvme_block_device.h b/src/crimson/os/seastore/random_block_manager/nvme_block_device.h new file mode 100644 index 000000000..ed8f99be8 --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/nvme_block_device.h @@ -0,0 +1,360 @@ +//-*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <memory> +#include <vector> + +#include <seastar/core/file.hh> +#include <linux/nvme_ioctl.h> + +#include "crimson/osd/exceptions.h" +#include "crimson/common/layout.h" +#include "rbm_device.h" + +namespace ceph { + namespace buffer { + class bufferptr; + } +} + +namespace crimson::os::seastore::random_block_device::nvme { +/* + * NVMe protocol structures (nvme_XX, identify_XX) + * + * All structures relative to NVMe protocol are following NVMe protocol v1.4 + * (latest). NVMe is protocol for fast interfacing between user and SSD device. + * We selectively adopted features among various NVMe features to ease + * implementation. And also, NVMeBlockDevice provides generic command submission + * APIs for IO and Admin commands. Please use pass_through_io() and pass_admin() + * to do it. + * + * For more information about NVMe protocol, refer https://nvmexpress.org/ + */ +struct nvme_identify_command_t { + uint32_t common_dw[10]; + + uint32_t cns : 8; + uint32_t reserved : 8; + uint32_t cnt_id : 16; + + static const uint8_t CNS_NAMESPACE = 0x00; + static const uint8_t CNS_CONTROLLER = 0x01; +}; + +struct nvme_admin_command_t { + union { + nvme_passthru_cmd common; + nvme_identify_command_t identify; + }; + + static const uint8_t OPCODE_IDENTIFY = 0x06; +}; + +// Optional Admin Command Support (OACS) +// Indicates optional commands are supported by SSD or not +struct oacs_t { + uint16_t unused : 5; + uint16_t support_directives : 1; // Support multi-stream + uint16_t unused2 : 10; +}; + +struct nvme_identify_controller_data_t { + union { + struct { + uint8_t unused[256]; // [255:0] + oacs_t oacs; // [257:256] + uint8_t unused2[270]; // [527:258] + uint16_t awupf; // [529:528] + }; + uint8_t raw[4096]; + }; +}; + +// End-to-end Data Protection Capabilities (DPC) +// Indicates type of E2E data protection supported by SSD +struct dpc_t { + uint8_t support_type1 : 1; + uint8_t support_type2 : 1; + uint8_t support_type3 : 1; + uint8_t support_first_meta : 1; + uint8_t support_last_meta : 1; + uint8_t reserved : 3; +}; + +// End-to-end Data Protection Type Settings (DPS) +// Indicates enabled type of E2E data protection +struct dps_t { + uint8_t protection_type : 3; + uint8_t protection_info : 1; + uint8_t reserved : 4; +}; + +// Namespace Features (NSFEAT) +// Indicates features of namespace +struct nsfeat_t { + uint8_t thinp : 1; + uint8_t nsabp : 1; + uint8_t dae : 1; + uint8_t uid_reuse : 1; + uint8_t opterf : 1; // Support NPWG, NPWA + uint8_t reserved : 3; +}; + +// LBA Format (LBAF) +// Indicates LBA format (metadata size, data size, performance) +struct lbaf_t { + uint32_t ms : 16; + uint32_t lbads : 8; + uint32_t rp : 2; + uint32_t reserved : 6; +}; + +struct nvme_identify_namespace_data_t { + union { + struct { + uint8_t unused[24]; // [23:0] + nsfeat_t nsfeat; // [24] + uint8_t unused2[3]; // [27:25] + dpc_t dpc; // [28] + dps_t dps; // [29] + uint8_t unused3[34]; // [63:30] + uint16_t npwg; // [65:64] + uint16_t npwa; // [67:66] + uint8_t unused4[60]; // [127:68] + lbaf_t lbaf0; // [131:128] + }; + uint8_t raw[4096]; + }; +}; + +struct nvme_rw_command_t { + uint32_t common_dw[10]; + + uint64_t s_lba; + + uint32_t nlb : 16; // 0's based value + uint32_t reserved : 4; + uint32_t d_type : 4; + uint32_t reserved2 : 2; + uint32_t prinfo_prchk : 3; + uint32_t prinfo_pract : 1; + uint32_t fua : 1; + uint32_t lr : 1; + + uint32_t reserved3 : 16; + uint32_t dspec : 16; + + static const uint32_t DTYPE_STREAM = 1; +}; + +struct nvme_io_command_t { + union { + nvme_passthru_cmd common; + nvme_rw_command_t rw; + }; + static const uint8_t OPCODE_WRITE = 0x01; + static const uint8_t OPCODE_READ = 0x01; +}; + +/* + * Implementation of NVMeBlockDevice with POSIX APIs + * + * NVMeBlockDevice provides NVMe SSD interfaces through POSIX APIs which is + * generally available at most operating environment. + */ +class NVMeBlockDevice : public RBMDevice { +public: + + /* + * Service NVMe device relative size + * + * size : total size of device in byte. + * + * block_size : IO unit size in byte. Caller should follow every IO command + * aligned with block size. + * + * preffered_write_granularity(PWG), preffered_write_alignment(PWA) : IO unit + * size for write in byte. Caller should request every write IO sized multiple + * times of PWG and aligned starting address by PWA. Available only if NVMe + * Device supports NVMe protocol 1.4 or later versions. + * atomic_write_unit : The maximum size of write whose atomicity is guranteed + * by SSD even on power failure. The write equal to or smaller than + * atomic_write_unit does not require fsync(). + */ + + NVMeBlockDevice(std::string device_path) : device_path(device_path) {} + ~NVMeBlockDevice() = default; + + open_ertr::future<> open( + const std::string &in_path, + seastar::open_flags mode) override; + + write_ertr::future<> write( + uint64_t offset, + bufferptr &&bptr, + uint16_t stream = 0) override; + + using RBMDevice::read; + read_ertr::future<> read( + uint64_t offset, + bufferptr &bptr) final; + + close_ertr::future<> close() override; + + discard_ertr::future<> discard( + uint64_t offset, + uint64_t len) override; + + mount_ret mount() final; + + mkfs_ret mkfs(device_config_t config) final; + + write_ertr::future<> writev( + uint64_t offset, + ceph::bufferlist bl, + uint16_t stream = 0) final; + + stat_device_ret stat_device() final { + return seastar::file_stat(device_path, seastar::follow_symlink::yes + ).handle_exception([](auto e) -> stat_device_ret { + return crimson::ct_error::input_output_error::make(); + }).then([this](auto stat) { + return seastar::open_file_dma( + device_path, + seastar::open_flags::rw | seastar::open_flags::dsync + ).then([this, stat](auto file) mutable { + return file.size().then([this, stat, file](auto size) mutable { + stat.size = size; + return identify_namespace(file + ).safe_then([stat] (auto id_namespace_data) mutable { + // LBA format provides LBA size which is power of 2. LBA is the + // minimum size of read and write. + stat.block_size = (1 << id_namespace_data.lbaf0.lbads); + if (stat.block_size < RBM_SUPERBLOCK_SIZE) { + stat.block_size = RBM_SUPERBLOCK_SIZE; + } + return stat_device_ret( + read_ertr::ready_future_marker{}, + stat + ); + }).handle_error(crimson::ct_error::input_output_error::handle( + [stat]{ + return stat_device_ret( + read_ertr::ready_future_marker{}, + stat + ); + }), crimson::ct_error::pass_further_all{}); + }).safe_then([file](auto st) mutable { + return file.close( + ).then([st] { + return stat_device_ret( + read_ertr::ready_future_marker{}, + st + ); + }); + }); + }); + }); + } + + std::string get_device_path() const final { + return device_path; + } + + seastar::future<> start() final { + return shard_devices.start(device_path); + } + + seastar::future<> stop() final { + return shard_devices.stop(); + } + + Device& get_sharded_device() final { + return shard_devices.local(); + } + + uint64_t get_preffered_write_granularity() const { return write_granularity; } + uint64_t get_preffered_write_alignment() const { return write_alignment; } + uint64_t get_atomic_write_unit() const { return atomic_write_unit; } + /* + * End-to-End Data Protection + * + * NVMe device keeps track of data integrity similar with checksum. Client can + * offload checksuming to NVMe device to reduce its CPU utilization. If data + * protection is enabled, checksum is calculated on every write and used to + * verify data on every read. + */ + bool is_data_protection_enabled() const { return data_protection_enabled; } + + /* + * Data Health + * + * Returns list of LBAs which have almost corrupted data. Data of the LBAs + * will be corrupted very soon. Caller can overwrite, unmap or refresh data to + * protect data + */ + virtual nvme_command_ertr::future<std::list<uint64_t>> get_data_health() { + std::list<uint64_t> fragile_lbas; + return nvme_command_ertr::future<std::list<uint64_t>>( + nvme_command_ertr::ready_future_marker{}, + fragile_lbas + ); + } + + /* + * Recovery Level + * + * Regulate magnitude of SSD-internal data recovery. Caller can get good read + * latency with lower magnitude. + */ + virtual nvme_command_ertr::future<> set_data_recovery_level( + uint32_t level) { return nvme_command_ertr::now(); } + /* + * For passsing through nvme IO or Admin command to SSD + * Caller can construct and execute its own nvme command + */ + nvme_command_ertr::future<int> pass_admin( + nvme_admin_command_t& admin_cmd, seastar::file f); + nvme_command_ertr::future<int> pass_through_io( + nvme_io_command_t& io_cmd); + + bool support_multistream = false; + uint8_t data_protection_type = 0; + + /* + * Predictable Latency + * + * NVMe device can guarantee IO latency within pre-defined time window. This + * functionality will be analyzed soon. + */ + +private: + // identify_controller/namespace are used to get SSD internal information such + // as supported features, NPWG and NPWA + nvme_command_ertr::future<nvme_identify_controller_data_t> + identify_controller(seastar::file f); + nvme_command_ertr::future<nvme_identify_namespace_data_t> + identify_namespace(seastar::file f); + nvme_command_ertr::future<int> get_nsid(seastar::file f); + open_ertr::future<> open_for_io( + const std::string& in_path, + seastar::open_flags mode); + + seastar::file device; + std::vector<seastar::file> io_device; + uint32_t stream_index_to_open = WRITE_LIFE_NOT_SET; + uint32_t stream_id_count = 1; // stream is disabled, defaultly. + uint32_t awupf = 0; + + uint64_t write_granularity = 4096; + uint64_t write_alignment = 4096; + uint32_t atomic_write_unit = 4096; + + bool data_protection_enabled = false; + std::string device_path; + seastar::sharded<NVMeBlockDevice> shard_devices; +}; + +} diff --git a/src/crimson/os/seastore/random_block_manager/rbm_device.cc b/src/crimson/os/seastore/random_block_manager/rbm_device.cc new file mode 100644 index 000000000..cea6c30a7 --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/rbm_device.cc @@ -0,0 +1,271 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <sys/mman.h> +#include <string.h> + +#include <fcntl.h> + +#include "crimson/common/log.h" +#include "crimson/common/errorator-loop.h" + +#include "include/buffer.h" +#include "rbm_device.h" +#include "nvme_block_device.h" +#include "block_rb_manager.h" + +namespace crimson::os::seastore::random_block_device { +#include "crimson/os/seastore/logging.h" +SET_SUBSYS(seastore_device); + +RBMDevice::mkfs_ret RBMDevice::do_primary_mkfs(device_config_t config, + int shard_num, size_t journal_size) { + LOG_PREFIX(RBMDevice::do_primary_mkfs); + return stat_device( + ).handle_error( + mkfs_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error stat_device in RBMDevice::do_primary_mkfs"} + ).safe_then( + [this, FNAME, config=std::move(config), shard_num, journal_size](auto st) { + super.block_size = st.block_size; + super.size = st.size; + super.feature |= RBM_BITMAP_BLOCK_CRC; + super.config = std::move(config); + super.journal_size = journal_size; + ceph_assert_always(super.journal_size > 0); + ceph_assert_always(super.size >= super.journal_size); + ceph_assert_always(shard_num > 0); + + std::vector<rbm_shard_info_t> shard_infos(shard_num); + for (int i = 0; i < shard_num; i++) { + uint64_t aligned_size = + (super.size / shard_num) - + ((super.size / shard_num) % super.block_size); + shard_infos[i].size = aligned_size; + shard_infos[i].start_offset = i * aligned_size; + assert(shard_infos[i].size > super.journal_size); + } + super.shard_infos = shard_infos; + super.shard_num = shard_num; + shard_info = shard_infos[seastar::this_shard_id()]; + DEBUG("super {} ", super); + + // write super block + return open(get_device_path(), + seastar::open_flags::rw | seastar::open_flags::dsync + ).handle_error( + mkfs_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error open in RBMDevice::do_primary_mkfs"} + ).safe_then([this] { + return write_rbm_header( + ).safe_then([this] { + return close(); + }).handle_error( + mkfs_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error write_rbm_header in RBMDevice::do_primary_mkfs" + }); + }); + }); +} + +write_ertr::future<> RBMDevice::write_rbm_header() +{ + bufferlist meta_b_header; + super.crc = 0; + encode(super, meta_b_header); + // If NVMeDevice supports data protection, CRC for checksum is not required + // NVMeDevice is expected to generate and store checksum internally. + // CPU overhead for CRC might be saved. + if (is_data_protection_enabled()) { + super.crc = -1; + } else { + super.crc = meta_b_header.crc32c(-1); + } + + bufferlist bl; + encode(super, bl); + auto iter = bl.begin(); + auto bp = bufferptr(ceph::buffer::create_page_aligned(super.block_size)); + assert(bl.length() < super.block_size); + iter.copy(bl.length(), bp.c_str()); + return write(RBM_START_ADDRESS, std::move(bp)); +} + +read_ertr::future<rbm_metadata_header_t> RBMDevice::read_rbm_header( + rbm_abs_addr addr) +{ + LOG_PREFIX(RBMDevice::read_rbm_header); + assert(super.block_size > 0); + return seastar::do_with( + bufferptr(ceph::buffer::create_page_aligned(super.block_size)), + [this, addr, FNAME](auto &bptr) { + return read( + addr, + bptr + ).safe_then([length=bptr.length(), this, bptr, FNAME]() + -> read_ertr::future<rbm_metadata_header_t> { + bufferlist bl; + bl.append(bptr); + auto p = bl.cbegin(); + rbm_metadata_header_t super_block; + try { + decode(super_block, p); + } + catch (ceph::buffer::error& e) { + DEBUG("read_rbm_header: unable to decode rbm super block {}", + e.what()); + return crimson::ct_error::enoent::make(); + } + checksum_t crc = super_block.crc; + bufferlist meta_b_header; + super_block.crc = 0; + encode(super_block, meta_b_header); + assert(ceph::encoded_sizeof<rbm_metadata_header_t>(super_block) < + super_block.block_size); + + // Do CRC verification only if data protection is not supported. + if (is_data_protection_enabled() == false) { + if (meta_b_header.crc32c(-1) != crc) { + DEBUG("bad crc on super block, expected {} != actual {} ", + meta_b_header.crc32c(-1), crc); + return crimson::ct_error::input_output_error::make(); + } + } else { + ceph_assert_always(crc == (checksum_t)-1); + } + super_block.crc = crc; + super = super_block; + DEBUG("got {} ", super); + return read_ertr::future<rbm_metadata_header_t>( + read_ertr::ready_future_marker{}, + super_block + ); + }); + }); +} + +RBMDevice::mount_ret RBMDevice::do_shard_mount() +{ + return open(get_device_path(), + seastar::open_flags::rw | seastar::open_flags::dsync + ).safe_then([this] { + return stat_device( + ).handle_error( + mount_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error stat_device in RBMDevice::do_shard_mount"} + ).safe_then([this](auto st) { + assert(st.block_size > 0); + super.block_size = st.block_size; + return read_rbm_header(RBM_START_ADDRESS + ).safe_then([this](auto s) { + LOG_PREFIX(RBMDevice::do_shard_mount); + shard_info = s.shard_infos[seastar::this_shard_id()]; + INFO("{} read {}", device_id_printer_t{get_device_id()}, shard_info); + s.validate(); + return seastar::now(); + }); + }); + }).handle_error( + mount_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error mount in RBMDevice::do_shard_mount"} + ); +} + +EphemeralRBMDeviceRef create_test_ephemeral(uint64_t journal_size, uint64_t data_size) { + return EphemeralRBMDeviceRef( + new EphemeralRBMDevice(journal_size + data_size + + random_block_device::RBMDevice::get_shard_reserved_size(), + EphemeralRBMDevice::TEST_BLOCK_SIZE)); +} + +open_ertr::future<> EphemeralRBMDevice::open( + const std::string &in_path, + seastar::open_flags mode) { + LOG_PREFIX(EphemeralRBMDevice::open); + if (buf) { + return open_ertr::now(); + } + + DEBUG( + "Initializing test memory device {}", + size); + + void* addr = ::mmap( + nullptr, + size, + PROT_READ | PROT_WRITE, MAP_SHARED | MAP_ANONYMOUS, + -1, + 0); + + buf = (char*)addr; + + ::memset(buf, 0, size); + return open_ertr::now(); +} + +write_ertr::future<> EphemeralRBMDevice::write( + uint64_t offset, + bufferptr &&bptr, + uint16_t stream) { + LOG_PREFIX(EphemeralRBMDevice::write); + ceph_assert(buf); + DEBUG( + "EphemeralRBMDevice: write offset {} len {}", + offset, + bptr.length()); + + ::memcpy(buf + offset, bptr.c_str(), bptr.length()); + + return write_ertr::now(); +} + +read_ertr::future<> EphemeralRBMDevice::read( + uint64_t offset, + bufferptr &bptr) { + LOG_PREFIX(EphemeralRBMDevice::read); + ceph_assert(buf); + DEBUG( + "EphemeralRBMDevice: read offset {} len {}", + offset, + bptr.length()); + + bptr.copy_in(0, bptr.length(), buf + offset); + return read_ertr::now(); +} + +Device::close_ertr::future<> EphemeralRBMDevice::close() { + LOG_PREFIX(EphemeralRBMDevice::close); + DEBUG(" close "); + return close_ertr::now(); +} + +write_ertr::future<> EphemeralRBMDevice::writev( + uint64_t offset, + ceph::bufferlist bl, + uint16_t stream) { + LOG_PREFIX(EphemeralRBMDevice::writev); + ceph_assert(buf); + DEBUG( + "EphemeralRBMDevice: write offset {} len {}", + offset, + bl.length()); + + bl.begin().copy(bl.length(), buf + offset); + return write_ertr::now(); +} + +EphemeralRBMDevice::mount_ret EphemeralRBMDevice::mount() { + return do_shard_mount(); +} + +EphemeralRBMDevice::mkfs_ret EphemeralRBMDevice::mkfs(device_config_t config) { + return do_primary_mkfs(config, 1, DEFAULT_TEST_CBJOURNAL_SIZE); +} + +} + diff --git a/src/crimson/os/seastore/random_block_manager/rbm_device.h b/src/crimson/os/seastore/random_block_manager/rbm_device.h new file mode 100644 index 000000000..501d9f913 --- /dev/null +++ b/src/crimson/os/seastore/random_block_manager/rbm_device.h @@ -0,0 +1,261 @@ +//-*- 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/seastore_types.h" +#include "crimson/os/seastore/random_block_manager.h" +#include "crimson/os/seastore/device.h" + +namespace ceph { + namespace buffer { + class bufferptr; + } +} + +namespace crimson::os::seastore::random_block_device { + +// from blk/BlockDevice.h +#if defined(__linux__) +#if !defined(F_SET_FILE_RW_HINT) +#define F_LINUX_SPECIFIC_BASE 1024 +#define F_SET_FILE_RW_HINT (F_LINUX_SPECIFIC_BASE + 14) +#endif +// These values match Linux definition +// https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/tree/include/uapi/linux/fcntl.h#n56 +#define WRITE_LIFE_NOT_SET 0 // No hint information set +#define WRITE_LIFE_NONE 1 // No hints about write life time +#define WRITE_LIFE_SHORT 2 // Data written has a short life time +#define WRITE_LIFE_MEDIUM 3 // Data written has a medium life time +#define WRITE_LIFE_LONG 4 // Data written has a long life time +#define WRITE_LIFE_EXTREME 5 // Data written has an extremely long life time +#define WRITE_LIFE_MAX 6 +#else +// On systems don't have WRITE_LIFE_* only use one FD +// And all files are created equal +#define WRITE_LIFE_NOT_SET 0 // No hint information set +#define WRITE_LIFE_NONE 0 // No hints about write life time +#define WRITE_LIFE_SHORT 0 // Data written has a short life time +#define WRITE_LIFE_MEDIUM 0 // Data written has a medium life time +#define WRITE_LIFE_LONG 0 // Data written has a long life time +#define WRITE_LIFE_EXTREME 0 // Data written has an extremely long life time +#define WRITE_LIFE_MAX 1 +#endif + +using read_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent, + crimson::ct_error::erange>; + +using write_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::ebadf, + crimson::ct_error::enospc>; + +using open_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent>; + +using nvme_command_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + +using discard_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + +constexpr uint32_t RBM_SUPERBLOCK_SIZE = 4096; +enum { + // TODO: This allows the device to manage crc on a block by itself + RBM_NVME_END_TO_END_PROTECTION = 1, + RBM_BITMAP_BLOCK_CRC = 2, +}; + +class RBMDevice : public Device { +public: + using Device::read; + read_ertr::future<> read ( + paddr_t addr, + size_t len, + ceph::bufferptr &out) final { + uint64_t rbm_addr = convert_paddr_to_abs_addr(addr); + return read(rbm_addr, out); + } +protected: + rbm_metadata_header_t super; + rbm_shard_info_t shard_info; +public: + RBMDevice() {} + virtual ~RBMDevice() = default; + + template <typename T> + static std::unique_ptr<T> create() { + return std::make_unique<T>(); + } + + device_id_t get_device_id() const { + return super.config.spec.id; + } + + magic_t get_magic() const final { + return super.config.spec.magic; + } + + device_type_t get_device_type() const final { + return device_type_t::RANDOM_BLOCK_SSD; + } + + backend_type_t get_backend_type() const final { + return backend_type_t::RANDOM_BLOCK; + } + + const seastore_meta_t &get_meta() const final { + return super.config.meta; + } + + secondary_device_set_t& get_secondary_devices() final { + return super.config.secondary_devices; + } + std::size_t get_available_size() const { return super.size; } + extent_len_t get_block_size() const { return super.block_size; } + + virtual read_ertr::future<> read( + uint64_t offset, + bufferptr &bptr) = 0; + + /* + * Multi-stream write + * + * Give hint to device about classification of data whose life time is similar + * with each other. Data with same stream value will be managed together in + * SSD for better write performance. + */ + virtual write_ertr::future<> write( + uint64_t offset, + bufferptr &&bptr, + uint16_t stream = 0) = 0; + + virtual discard_ertr::future<> discard( + uint64_t offset, + uint64_t len) { return seastar::now(); } + + virtual open_ertr::future<> open( + const std::string& path, + seastar::open_flags mode) = 0; + + virtual write_ertr::future<> writev( + uint64_t offset, + ceph::bufferlist bl, + uint16_t stream = 0) = 0; + + bool is_data_protection_enabled() const { return false; } + + mkfs_ret do_mkfs(device_config_t); + + // shard 0 mkfs + mkfs_ret do_primary_mkfs(device_config_t, int shard_num, size_t journal_size); + + mount_ret do_mount(); + + mount_ret do_shard_mount(); + + write_ertr::future<> write_rbm_header(); + + read_ertr::future<rbm_metadata_header_t> read_rbm_header(rbm_abs_addr addr); + + using stat_device_ret = + read_ertr::future<seastar::stat_data>; + virtual stat_device_ret stat_device() = 0; + + virtual std::string get_device_path() const = 0; + + uint64_t get_journal_size() const { + return super.journal_size; + } + + static rbm_abs_addr get_shard_reserved_size() { + return RBM_SUPERBLOCK_SIZE; + } + + rbm_abs_addr get_shard_journal_start() { + return shard_info.start_offset + get_shard_reserved_size(); + } + + uint64_t get_shard_start() const { + return shard_info.start_offset; + } + + uint64_t get_shard_end() const { + return shard_info.start_offset + shard_info.size; + } +}; +using RBMDeviceRef = std::unique_ptr<RBMDevice>; + +constexpr uint64_t DEFAULT_TEST_CBJOURNAL_SIZE = 1 << 26; + +class EphemeralRBMDevice : public RBMDevice { +public: + uint64_t size = 0; + uint64_t block_size = 0; + constexpr static uint32_t TEST_BLOCK_SIZE = 4096; + + EphemeralRBMDevice(size_t size, uint64_t block_size) : + size(size), block_size(block_size), buf(nullptr) { + } + ~EphemeralRBMDevice() { + if (buf) { + ::munmap(buf, size); + buf = nullptr; + } + } + + std::size_t get_available_size() const final { return size; } + extent_len_t get_block_size() const final { return block_size; } + + mount_ret mount() final; + mkfs_ret mkfs(device_config_t config) final; + + open_ertr::future<> open( + const std::string &in_path, + seastar::open_flags mode) override; + + write_ertr::future<> write( + uint64_t offset, + bufferptr &&bptr, + uint16_t stream = 0) override; + + using RBMDevice::read; + read_ertr::future<> read( + uint64_t offset, + bufferptr &bptr) override; + + close_ertr::future<> close() override; + + write_ertr::future<> writev( + uint64_t offset, + ceph::bufferlist bl, + uint16_t stream = 0) final; + + stat_device_ret stat_device() final { + seastar::stat_data stat; + stat.block_size = block_size; + stat.size = size; + return stat_device_ret( + read_ertr::ready_future_marker{}, + stat + ); + } + + std::string get_device_path() const final { + return ""; + } + + char *buf; +}; +using EphemeralRBMDeviceRef = std::unique_ptr<EphemeralRBMDevice>; +EphemeralRBMDeviceRef create_test_ephemeral( + uint64_t journal_size = DEFAULT_TEST_CBJOURNAL_SIZE, + uint64_t data_size = DEFAULT_TEST_CBJOURNAL_SIZE); + +} diff --git a/src/crimson/os/seastore/randomblock_manager_group.h b/src/crimson/os/seastore/randomblock_manager_group.h new file mode 100644 index 000000000..77d9cf797 --- /dev/null +++ b/src/crimson/os/seastore/randomblock_manager_group.h @@ -0,0 +1,71 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include <set> + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/random_block_manager.h" +#include "crimson/os/seastore/random_block_manager/block_rb_manager.h" + +namespace crimson::os::seastore { + +class RBMDeviceGroup { +public: + RBMDeviceGroup() { + rb_devices.resize(DEVICE_ID_MAX); + } + + const std::set<device_id_t>& get_device_ids() const { + return device_ids; + } + + std::vector<RandomBlockManager*> get_rb_managers() const { + assert(device_ids.size()); + std::vector<RandomBlockManager*> ret; + for (auto& device_id : device_ids) { + auto rb_device = rb_devices[device_id].get(); + assert(rb_device->get_device_id() == device_id); + ret.emplace_back(rb_device); + } + return ret; + } + + void add_rb_manager(RandomBlockManagerRef rbm) { + auto device_id = rbm->get_device_id(); + ceph_assert(!has_device(device_id)); + rb_devices[device_id] = std::move(rbm); + device_ids.insert(device_id); + } + + void reset() { + rb_devices.clear(); + rb_devices.resize(DEVICE_ID_MAX); + device_ids.clear(); + } + + auto get_block_size() const { + assert(device_ids.size()); + return rb_devices[*device_ids.begin()]->get_block_size(); + } + + const seastore_meta_t &get_meta() const { + assert(device_ids.size()); + return rb_devices[*device_ids.begin()]->get_meta(); + } + +private: + bool has_device(device_id_t id) const { + assert(id <= DEVICE_ID_MAX_VALID); + return device_ids.count(id) >= 1; + } + + std::vector<RandomBlockManagerRef> rb_devices; + std::set<device_id_t> device_ids; +}; + +using RBMDeviceGroupRef = std::unique_ptr<RBMDeviceGroup>; + +} diff --git a/src/crimson/os/seastore/record_scanner.cc b/src/crimson/os/seastore/record_scanner.cc new file mode 100644 index 000000000..9778bbb77 --- /dev/null +++ b/src/crimson/os/seastore/record_scanner.cc @@ -0,0 +1,239 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#include "crimson/os/seastore/record_scanner.h" + +#include "crimson/os/seastore/logging.h" + +SET_SUBSYS(seastore_journal); + +namespace crimson::os::seastore { + +RecordScanner::scan_valid_records_ret +RecordScanner::scan_valid_records( + scan_valid_records_cursor &cursor, + segment_nonce_t nonce, + size_t budget, + found_record_handler_t &handler) +{ + LOG_PREFIX(RecordScanner::scan_valid_records); + initialize_cursor(cursor); + DEBUG("starting at {}, budget={}", cursor, budget); + auto retref = std::make_unique<size_t>(0); + auto &budget_used = *retref; + return crimson::repeat( + [=, &cursor, &budget_used, &handler, this]() mutable + -> scan_valid_records_ertr::future<seastar::stop_iteration> { + return [=, &handler, &cursor, &budget_used, this] { + if (!cursor.last_valid_header_found) { + return read_validate_record_metadata(cursor, nonce + ).safe_then([=, &cursor](auto md) { + if (!md) { + cursor.last_valid_header_found = true; + if (cursor.is_complete()) { + INFO("complete at {}, invalid record group metadata", + cursor); + } else { + DEBUG("found invalid record group metadata at {}, " + "processing {} pending record groups", + cursor.seq, + cursor.pending_record_groups.size()); + } + return scan_valid_records_ertr::now(); + } else { + auto& [header, md_bl] = *md; + DEBUG("found valid {} at {}", header, cursor.seq); + cursor.emplace_record_group(header, std::move(md_bl)); + return scan_valid_records_ertr::now(); + } + }).safe_then([=, &cursor, &budget_used, &handler, this] { + DEBUG("processing committed record groups until {}, {} pending", + cursor.last_committed, + cursor.pending_record_groups.size()); + return crimson::repeat( + [=, &budget_used, &cursor, &handler, this] { + if (cursor.pending_record_groups.empty()) { + /* This is only possible if the segment is empty. + * A record's last_commited must be prior to its own + * location since it itself cannot yet have been committed + * at its own time of submission. Thus, the most recently + * read record must always fall after cursor.last_committed */ + return scan_valid_records_ertr::make_ready_future< + seastar::stop_iteration>(seastar::stop_iteration::yes); + } + auto &next = cursor.pending_record_groups.front(); + journal_seq_t next_seq = {cursor.seq.segment_seq, next.offset}; + if (cursor.last_committed == JOURNAL_SEQ_NULL || + next_seq > cursor.last_committed) { + return scan_valid_records_ertr::make_ready_future< + seastar::stop_iteration>(seastar::stop_iteration::yes); + } + return consume_next_records(cursor, handler, budget_used + ).safe_then([] { + return scan_valid_records_ertr::make_ready_future< + seastar::stop_iteration>(seastar::stop_iteration::no); + }); + }); + }); + } else { + assert(!cursor.pending_record_groups.empty()); + auto &next = cursor.pending_record_groups.front(); + return read_validate_data(next.offset, next.header + ).safe_then([this, FNAME, &budget_used, &cursor, &handler, &next](auto valid) { + if (!valid) { + INFO("complete at {}, invalid record group data at {}, {}", + cursor, next.offset, next.header); + cursor.pending_record_groups.clear(); + return scan_valid_records_ertr::now(); + } + return consume_next_records(cursor, handler, budget_used); + }); + } + }().safe_then([=, &budget_used, &cursor] { + if (cursor.is_complete() || budget_used >= budget) { + DEBUG("finish at {}, budget_used={}, budget={}", + cursor, budget_used, budget); + return seastar::stop_iteration::yes; + } else { + return seastar::stop_iteration::no; + } + }); + }).safe_then([retref=std::move(retref)]() mutable -> scan_valid_records_ret { + return scan_valid_records_ret( + scan_valid_records_ertr::ready_future_marker{}, + std::move(*retref)); + }); +} + +RecordScanner::read_validate_record_metadata_ret +RecordScanner::read_validate_record_metadata( + scan_valid_records_cursor &cursor, + segment_nonce_t nonce) +{ + LOG_PREFIX(RecordScanner::read_validate_record_metadata); + paddr_t start = cursor.seq.offset; + auto block_size = cursor.get_block_size(); + if (get_segment_off(cursor.seq.offset) + block_size > get_segment_end_offset(cursor.seq.offset)) { + DEBUG("failed -- record group header block {}~4096 > segment_size {}", + start, get_segment_end_offset(cursor.seq.offset)); + return read_validate_record_metadata_ret( + read_validate_record_metadata_ertr::ready_future_marker{}, + std::nullopt); + } + TRACE("reading record group header block {}~4096", start); + return read(start, block_size + ).safe_then([=](bufferptr bptr) mutable + -> read_validate_record_metadata_ret { + bufferlist bl; + bl.append(bptr); + auto maybe_header = try_decode_records_header(bl, nonce); + if (!maybe_header.has_value()) { + return read_validate_record_metadata_ret( + read_validate_record_metadata_ertr::ready_future_marker{}, + std::nullopt); + } + + auto& header = *maybe_header; + if (header.mdlength < block_size || + header.mdlength % block_size != 0 || + header.dlength % block_size != 0 || + (header.committed_to != JOURNAL_SEQ_NULL && + get_segment_off(header.committed_to.offset) % + cursor.get_block_size() != 0) || + (get_segment_off(cursor.seq.offset) + header.mdlength + header.dlength > + get_segment_end_offset(cursor.seq.offset))) { + ERROR("failed, invalid record group header {}", header); + return crimson::ct_error::input_output_error::make(); + } + + if (is_record_segment_seq_invalid(cursor, header)) { + return read_validate_record_metadata_ret( + read_validate_record_metadata_ertr::ready_future_marker{}, + std::nullopt); + } + + if (header.mdlength == block_size) { + return read_validate_record_metadata_ret( + read_validate_record_metadata_ertr::ready_future_marker{}, + std::make_pair(std::move(header), std::move(bl)) + ); + } + + paddr_t rest_start = cursor.seq.offset.add_offset(block_size); + auto rest_len = header.mdlength - block_size; + TRACE("reading record group header rest {}~{}", rest_start, rest_len); + return read(rest_start, rest_len + ).safe_then([header=std::move(header), bl=std::move(bl) + ](auto&& bptail) mutable { + bl.push_back(bptail); + return read_validate_record_metadata_ret( + read_validate_record_metadata_ertr::ready_future_marker{}, + std::make_pair(std::move(header), std::move(bl))); + }); + }).safe_then([](auto p) { + if (p && validate_records_metadata(p->second)) { + return read_validate_record_metadata_ret( + read_validate_record_metadata_ertr::ready_future_marker{}, + std::move(*p) + ); + } else { + return read_validate_record_metadata_ret( + read_validate_record_metadata_ertr::ready_future_marker{}, + std::nullopt); + } + }); + +} + +RecordScanner::read_validate_data_ret RecordScanner::read_validate_data( + paddr_t record_base, + const record_group_header_t &header) +{ + LOG_PREFIX(RecordScanner::read_validate_data); + auto data_addr = record_base.add_offset(header.mdlength); + TRACE("reading record group data blocks {}~{}", data_addr, header.dlength); + return read( + data_addr, + header.dlength + ).safe_then([=, &header](auto bptr) { + bufferlist bl; + bl.append(bptr); + return validate_records_data(header, bl); + }); +} + +RecordScanner::consume_record_group_ertr::future<> +RecordScanner::consume_next_records( + scan_valid_records_cursor& cursor, + found_record_handler_t& handler, + std::size_t& budget_used) +{ + LOG_PREFIX(RecordScanner::consume_next_records); + auto& next = cursor.pending_record_groups.front(); + auto total_length = next.header.dlength + next.header.mdlength; + budget_used += total_length; + auto locator = record_locator_t{ + next.offset.add_offset(next.header.mdlength), + write_result_t{ + journal_seq_t{ + cursor.seq.segment_seq, + next.offset + }, + total_length + } + }; + DEBUG("processing {} at {}, budget_used={}", + next.header, locator, budget_used); + return handler( + locator, + next.header, + next.mdbuffer + ).safe_then([FNAME, &cursor] { + cursor.pop_record_group(); + if (cursor.is_complete()) { + INFO("complete at {}, no more record group", cursor); + } + }); +} + +} diff --git a/src/crimson/os/seastore/record_scanner.h b/src/crimson/os/seastore/record_scanner.h new file mode 100644 index 000000000..2cbc7c562 --- /dev/null +++ b/src/crimson/os/seastore/record_scanner.h @@ -0,0 +1,83 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/segment_manager.h" + + +namespace crimson::os::seastore { + +class RecordScanner { +public: + using read_ertr = SegmentManager::read_ertr; + using scan_valid_records_ertr = read_ertr; + using scan_valid_records_ret = scan_valid_records_ertr::future< + size_t>; + using found_record_handler_t = std::function< + scan_valid_records_ertr::future<>( + record_locator_t record_locator, + // callee may assume header and bl will remain valid until + // returned future resolves + const record_group_header_t &header, + const bufferlist &mdbuf)>; + scan_valid_records_ret scan_valid_records( + scan_valid_records_cursor &cursor, ///< [in, out] cursor, updated during call + segment_nonce_t nonce, ///< [in] nonce for segment + size_t budget, ///< [in] max budget to use + found_record_handler_t &handler ///< [in] handler for records + ); ///< @return used budget + + device_off_t get_segment_off(paddr_t addr) const { + if (addr.get_addr_type() == paddr_types_t::SEGMENT) { + auto& seg_addr = addr.as_seg_paddr(); + return seg_addr.get_segment_off(); + } + assert(addr.get_addr_type() == paddr_types_t::RANDOM_BLOCK); + auto& blk_addr = addr.as_blk_paddr(); + return blk_addr.get_device_off(); + } + +protected: + /// read record metadata for record starting at start + using read_validate_record_metadata_ertr = read_ertr; + using read_validate_record_metadata_ret = + read_validate_record_metadata_ertr::future< + std::optional<std::pair<record_group_header_t, bufferlist>> + >; + read_validate_record_metadata_ret read_validate_record_metadata( + scan_valid_records_cursor &cursor, + segment_nonce_t nonce); + + /// read and validate data + using read_validate_data_ertr = read_ertr; + using read_validate_data_ret = read_validate_data_ertr::future<bool>; + read_validate_data_ret read_validate_data( + paddr_t record_base, + const record_group_header_t &header ///< caller must ensure lifetime through + /// future resolution + ); + + virtual bool is_record_segment_seq_invalid(scan_valid_records_cursor &cursor, + record_group_header_t &h) = 0; + + virtual int64_t get_segment_end_offset(paddr_t addr) = 0; + + using read_ret = read_ertr::future<bufferptr>; + virtual read_ret read(paddr_t start, size_t len) = 0; + + using consume_record_group_ertr = scan_valid_records_ertr; + consume_record_group_ertr::future<> consume_next_records( + scan_valid_records_cursor& cursor, + found_record_handler_t& handler, + std::size_t& budget_used); + + virtual void initialize_cursor(scan_valid_records_cursor &cursor) = 0; + + virtual ~RecordScanner() {} + +}; + +} diff --git a/src/crimson/os/seastore/root_block.cc b/src/crimson/os/seastore/root_block.cc new file mode 100644 index 000000000..dc928e81b --- /dev/null +++ b/src/crimson/os/seastore/root_block.cc @@ -0,0 +1,27 @@ +// -*- 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" +#include "crimson/os/seastore/lba_manager/btree/lba_btree_node.h" +#include "crimson/os/seastore/backref/backref_tree_node.h" + +namespace crimson::os::seastore { + +void RootBlock::on_replace_prior(Transaction &t) { + if (!lba_root_node) { + auto &prior = static_cast<RootBlock&>(*get_prior_instance()); + lba_root_node = prior.lba_root_node; + if (lba_root_node) { + ((lba_manager::btree::LBANode*)lba_root_node)->root_block = this; + } + } + if (!backref_root_node) { + auto &prior = static_cast<RootBlock&>(*get_prior_instance()); + backref_root_node = prior.backref_root_node; + if (backref_root_node) { + ((backref::BackrefNode*)backref_root_node)->root_block = this; + } + } +} + +} // namespace crimson::os::seastore diff --git a/src/crimson/os/seastore/root_block.h b/src/crimson/os/seastore/root_block.h new file mode 100644 index 000000000..0e45519ce --- /dev/null +++ b/src/crimson/os/seastore/root_block.h @@ -0,0 +1,109 @@ +// -*- 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 { + +/** + * RootBlock + * + * Holds the physical addresses of all metadata roots. + * In-memory values may be + * - absolute: reference to block which predates the current transaction + * - record_relative: reference to block updated in this transaction + * if !pending() + * + * Journal replay only considers deltas and must always discover the most + * recent value for the RootBlock. Because the contents of root_t above are + * very small, it's simplest to stash the entire root_t value into the delta + * and never actually write the RootBlock to a physical location (safe since + * nothing references the location of the RootBlock). + * + * As a result, Cache treats the root differently in a few ways including: + * - state will only ever be DIRTY or MUTATION_PENDING + * - RootBlock's never show up in the transaction fresh or dirty lists -- + * there's a special Transaction::root member for when the root needs to + * be mutated. + * + * TODO: Journal trimming will need to be aware of the most recent RootBlock + * delta location, or, even easier, just always write one out with the + * mutation which changes the journal trim bound. + */ +struct RootBlock : CachedExtent { + constexpr static extent_len_t SIZE = 4<<10; + using Ref = TCachedExtentRef<RootBlock>; + + root_t root; + + CachedExtent* lba_root_node = nullptr; + CachedExtent* backref_root_node = nullptr; + + RootBlock() : CachedExtent(zero_length_t()) {}; + + RootBlock(const RootBlock &rhs) + : CachedExtent(rhs), + root(rhs.root), + lba_root_node(nullptr), + backref_root_node(nullptr) + {} + + CachedExtentRef duplicate_for_write(Transaction&) final { + return CachedExtentRef(new RootBlock(*this)); + }; + + static constexpr extent_types_t TYPE = extent_types_t::ROOT; + extent_types_t get_type() const final { + return extent_types_t::ROOT; + } + + void on_replace_prior(Transaction &t) final; + + /// dumps root as delta + ceph::bufferlist get_delta() final { + ceph::bufferlist bl; + ceph::buffer::ptr bptr(sizeof(root_t)); + *reinterpret_cast<root_t*>(bptr.c_str()) = root; + bl.append(bptr); + return bl; + } + + /// overwrites root + void apply_delta_and_adjust_crc(paddr_t base, const ceph::bufferlist &_bl) final { + assert(_bl.length() == sizeof(root_t)); + ceph::bufferlist bl = _bl; + bl.rebuild(); + root = *reinterpret_cast<const root_t*>(bl.front().c_str()); + root.adjust_addrs_from_base(base); + } + + /// Patches relative addrs in memory based on record commit addr + void on_delta_write(paddr_t record_block_offset) final { + root.adjust_addrs_from_base(record_block_offset); + } + + complete_load_ertr::future<> complete_load() final { + ceph_abort_msg("Root is only written via deltas"); + } + + void on_initial_write() final { + ceph_abort_msg("Root is only written via deltas"); + } + + root_t &get_root() { return root; } + + std::ostream &print_detail(std::ostream &out) const final { + return out << ", root_block(lba_root_node=" << (void*)lba_root_node + << ", backref_root_node=" << (void*)backref_root_node + << ")"; + } +}; +using RootBlockRef = RootBlock::Ref; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::RootBlock> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/seastore.cc b/src/crimson/os/seastore/seastore.cc new file mode 100644 index 000000000..897a063e0 --- /dev/null +++ b/src/crimson/os/seastore/seastore.cc @@ -0,0 +1,2135 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "seastore.h" + +#include <algorithm> + +#include <boost/algorithm/string/trim.hpp> +#include <fmt/format.h> +#include <fmt/ostream.h> + +#include <seastar/core/file.hh> +#include <seastar/core/fstream.hh> +#include <seastar/core/metrics.hh> +#include <seastar/core/shared_mutex.hh> + +#include "common/safe_io.h" +#include "include/stringify.h" +#include "os/Transaction.h" + +#include "crimson/common/buffer_io.h" + +#include "crimson/os/futurized_collection.h" + +#include "crimson/os/seastore/backref_manager.h" +#include "crimson/os/seastore/async_cleaner.h" +#include "crimson/os/seastore/collection_manager/flat_collection_manager.h" +#include "crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.h" +#include "crimson/os/seastore/omap_manager/btree/btree_omap_manager.h" +#include "crimson/os/seastore/onode_manager.h" +#include "crimson/os/seastore/object_data_handler.h" + + +using std::string; +using crimson::common::local_conf; + +template <> struct fmt::formatter<crimson::os::seastore::op_type_t> + : fmt::formatter<std::string_view> { + using op_type_t = crimson::os::seastore::op_type_t; + // parse is inherited from formatter<string_view>. + template <typename FormatContext> + auto format(op_type_t op, FormatContext& ctx) { + std::string_view name = "unknown"; + switch (op) { + case op_type_t::TRANSACTION: + name = "transaction"; + break; + case op_type_t::READ: + name = "read"; + break; + case op_type_t::WRITE: + name = "write"; + break; + case op_type_t::GET_ATTR: + name = "get_attr"; + break; + case op_type_t::GET_ATTRS: + name = "get_attrs"; + break; + case op_type_t::STAT: + name = "stat"; + break; + case op_type_t::OMAP_GET_VALUES: + name = "omap_get_values"; + break; + case op_type_t::OMAP_LIST: + name = "omap_list"; + break; + case op_type_t::MAX: + name = "unknown"; + break; + } + return formatter<string_view>::format(name, ctx); + } +}; + +SET_SUBSYS(seastore); + +namespace crimson::os::seastore { + +class FileMDStore final : public SeaStore::MDStore { + std::string root; +public: + FileMDStore(const std::string& root) : root(root) {} + + write_meta_ret write_meta( + const std::string& key, const std::string& value) final { + std::string path = fmt::format("{}/{}", root, key); + ceph::bufferlist bl; + bl.append(value + "\n"); + return crimson::write_file(std::move(bl), path); + } + + read_meta_ret read_meta(const std::string& key) final { + std::string path = fmt::format("{}/{}", root, key); + return seastar::file_exists( + path + ).then([path] (bool exist) { + if (exist) { + return crimson::read_file(path) + .then([] (auto tmp_buf) { + std::string v = {tmp_buf.get(), tmp_buf.size()}; + std::size_t pos = v.find("\n"); + std::string str = v.substr(0, pos); + return seastar::make_ready_future<std::optional<std::string>>(str); + }); + } else { + return seastar::make_ready_future<std::optional<std::string>>(std::nullopt); + } + }); + } +}; + +using crimson::common::get_conf; + +SeaStore::Shard::Shard( + std::string root, + Device* dev, + bool is_test) + :root(root), + max_object_size( + get_conf<uint64_t>("seastore_default_max_object_size")), + is_test(is_test), + throttler( + get_conf<uint64_t>("seastore_max_concurrent_transactions")) +{ + device = &(dev->get_sharded_device()); + register_metrics(); +} + +SeaStore::SeaStore( + const std::string& root, + MDStoreRef mdstore) + : root(root), + mdstore(std::move(mdstore)) +{ +} + +SeaStore::~SeaStore() = default; + +void SeaStore::Shard::register_metrics() +{ + namespace sm = seastar::metrics; + using op_type_t = crimson::os::seastore::op_type_t; + std::pair<op_type_t, sm::label_instance> labels_by_op_type[] = { + {op_type_t::TRANSACTION, sm::label_instance("latency", "TRANSACTION")}, + {op_type_t::READ, sm::label_instance("latency", "READ")}, + {op_type_t::WRITE, sm::label_instance("latency", "WRITE")}, + {op_type_t::GET_ATTR, sm::label_instance("latency", "GET_ATTR")}, + {op_type_t::GET_ATTRS, sm::label_instance("latency", "GET_ATTRS")}, + {op_type_t::STAT, sm::label_instance("latency", "STAT")}, + {op_type_t::OMAP_GET_VALUES, sm::label_instance("latency", "OMAP_GET_VALUES")}, + {op_type_t::OMAP_LIST, sm::label_instance("latency", "OMAP_LIST")}, + }; + + for (auto& [op_type, label] : labels_by_op_type) { + auto desc = fmt::format("latency of seastore operation (optype={})", + op_type); + metrics.add_group( + "seastore", + { + sm::make_histogram( + "op_lat", [this, op_type=op_type] { + return get_latency(op_type); + }, + sm::description(desc), + {label} + ), + } + ); + } + + metrics.add_group( + "seastore", + { + sm::make_gauge( + "concurrent_transactions", + [this] { + return throttler.get_current(); + }, + sm::description("transactions that are running inside seastore") + ), + sm::make_gauge( + "pending_transactions", + [this] { + return throttler.get_pending(); + }, + sm::description("transactions waiting to get " + "through seastore's throttler") + ) + } + ); +} + +seastar::future<> SeaStore::start() +{ + ceph_assert(seastar::this_shard_id() == primary_core); +#ifndef NDEBUG + bool is_test = true; +#else + bool is_test = false; +#endif + using crimson::common::get_conf; + std::string type = get_conf<std::string>("seastore_main_device_type"); + device_type_t d_type = string_to_device_type(type); + assert(d_type == device_type_t::SSD || + d_type == device_type_t::RANDOM_BLOCK_SSD); + + ceph_assert(root != ""); + return Device::make_device(root, d_type + ).then([this](DeviceRef device_obj) { + device = std::move(device_obj); + return device->start(); + }).then([this, is_test] { + ceph_assert(device); + return shard_stores.start(root, device.get(), is_test); + }); +} + +seastar::future<> SeaStore::test_start(DeviceRef device_obj) +{ + ceph_assert(device_obj); + ceph_assert(root == ""); + device = std::move(device_obj); + return shard_stores.start_single(root, device.get(), true); +} + +seastar::future<> SeaStore::stop() +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return seastar::do_for_each(secondaries, [](auto& sec_dev) { + return sec_dev->stop(); + }).then([this] { + secondaries.clear(); + if (device) { + return device->stop(); + } else { + return seastar::now(); + } + }).then([this] { + return shard_stores.stop(); + }); +} + +SeaStore::mount_ertr::future<> SeaStore::test_mount() +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.local().mount_managers(); +} + +SeaStore::mount_ertr::future<> SeaStore::mount() +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return device->mount( + ).safe_then([this] { + auto sec_devices = device->get_sharded_device().get_secondary_devices(); + return crimson::do_for_each(sec_devices, [this](auto& device_entry) { + device_id_t id = device_entry.first; + magic_t magic = device_entry.second.magic; + device_type_t dtype = device_entry.second.dtype; + std::string path = + fmt::format("{}/block.{}.{}", root, dtype, std::to_string(id)); + return Device::make_device(path, dtype + ).then([this, path, magic](DeviceRef sec_dev) { + return sec_dev->start( + ).then([this, magic, sec_dev = std::move(sec_dev)]() mutable { + return sec_dev->mount( + ).safe_then([this, sec_dev=std::move(sec_dev), magic]() mutable { + boost::ignore_unused(magic); // avoid clang warning; + assert(sec_dev->get_sharded_device().get_magic() == magic); + secondaries.emplace_back(std::move(sec_dev)); + }); + }).safe_then([this] { + return set_secondaries(); + }); + }); + }).safe_then([this] { + return shard_stores.invoke_on_all([](auto &local_store) { + return local_store.mount_managers(); + }); + }); + }).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::mount" + } + ); +} + +seastar::future<> SeaStore::Shard::mount_managers() +{ + init_managers(); + return transaction_manager->mount( + ).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in mount_managers" + }); +} + +seastar::future<> SeaStore::umount() +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.invoke_on_all([](auto &local_store) { + return local_store.umount(); + }); +} + +seastar::future<> SeaStore::Shard::umount() +{ + return [this] { + if (transaction_manager) { + return transaction_manager->close(); + } else { + return TransactionManager::close_ertr::now(); + } + }().safe_then([this] { + return crimson::do_for_each( + secondaries, + [](auto& sec_dev) -> SegmentManager::close_ertr::future<> + { + return sec_dev->close(); + }); + }).safe_then([this] { + return device->close(); + }).safe_then([this] { + secondaries.clear(); + transaction_manager.reset(); + collection_manager.reset(); + onode_manager.reset(); + }).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::umount" + } + ); +} + +seastar::future<> SeaStore::write_fsid(uuid_d new_osd_fsid) +{ + ceph_assert(seastar::this_shard_id() == primary_core); + LOG_PREFIX(SeaStore::write_fsid); + return read_meta("fsid").then([this, FNAME, new_osd_fsid] (auto tuple) { + auto [ret, fsid] = tuple; + std::string str_fsid = stringify(new_osd_fsid); + if (ret == -1) { + return write_meta("fsid", stringify(new_osd_fsid)); + } else if (ret == 0 && fsid != str_fsid) { + ERROR("on-disk fsid {} != provided {}", + fsid, stringify(new_osd_fsid)); + throw std::runtime_error("store fsid error"); + } else { + return seastar::now(); + } + }); +} + +seastar::future<> +SeaStore::Shard::mkfs_managers() +{ + init_managers(); + return transaction_manager->mkfs( + ).safe_then([this] { + init_managers(); + return transaction_manager->mount(); + }).safe_then([this] { + return repeat_eagain([this] { + return transaction_manager->with_transaction_intr( + Transaction::src_t::MUTATE, + "mkfs_seastore", + [this](auto& t) + { + return onode_manager->mkfs(t + ).si_then([this, &t] { + return collection_manager->mkfs(t); + }).si_then([this, &t](auto coll_root) { + transaction_manager->write_collection_root( + t, coll_root); + return transaction_manager->submit_transaction(t); + }); + }); + }); + }).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in Shard::mkfs_managers" + } + ); +} + +seastar::future<> SeaStore::set_secondaries() +{ + auto sec_dev_ite = secondaries.rbegin(); + Device* sec_dev = sec_dev_ite->get(); + return shard_stores.invoke_on_all([sec_dev](auto &local_store) { + local_store.set_secondaries(sec_dev->get_sharded_device()); + }); +} + +SeaStore::mkfs_ertr::future<> SeaStore::test_mkfs(uuid_d new_osd_fsid) +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return read_meta("mkfs_done").then([this, new_osd_fsid] (auto tuple) { + auto [done, value] = tuple; + if (done == 0) { + return seastar::now(); + } + return shard_stores.local().mkfs_managers( + ).then([this, new_osd_fsid] { + return prepare_meta(new_osd_fsid); + }); + }); +} + +seastar::future<> SeaStore::prepare_meta(uuid_d new_osd_fsid) +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return write_fsid(new_osd_fsid).then([this] { + return read_meta("type").then([this] (auto tuple) { + auto [ret, type] = tuple; + if (ret == 0 && type == "seastore") { + return seastar::now(); + } else if (ret == 0 && type != "seastore") { + LOG_PREFIX(SeaStore::prepare_meta); + ERROR("expected seastore, but type is {}", type); + throw std::runtime_error("store type error"); + } else { + return write_meta("type", "seastore"); + } + }); + }).then([this] { + return write_meta("mkfs_done", "yes"); + }); +} + +SeaStore::mkfs_ertr::future<> SeaStore::mkfs(uuid_d new_osd_fsid) +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return read_meta("mkfs_done").then([this, new_osd_fsid] (auto tuple) { + auto [done, value] = tuple; + if (done == 0) { + return seastar::now(); + } else { + return seastar::do_with( + secondary_device_set_t(), + [this, new_osd_fsid](auto& sds) { + auto fut = seastar::now(); + LOG_PREFIX(SeaStore::mkfs); + DEBUG("root: {}", root); + if (!root.empty()) { + fut = seastar::open_directory(root + ).then([this, &sds, new_osd_fsid](seastar::file rdir) mutable { + std::unique_ptr<seastar::file> root_f = + std::make_unique<seastar::file>(std::move(rdir)); + auto sub = root_f->list_directory( + [this, &sds, new_osd_fsid](auto de) mutable -> seastar::future<> + { + LOG_PREFIX(SeaStore::mkfs); + DEBUG("found file: {}", de.name); + if (de.name.find("block.") == 0 + && de.name.length() > 6 /* 6 for "block." */) { + std::string entry_name = de.name; + auto dtype_end = entry_name.find_first_of('.', 6); + device_type_t dtype = + string_to_device_type( + entry_name.substr(6, dtype_end - 6)); + if (dtype == device_type_t::NONE) { + // invalid device type + return seastar::now(); + } + auto id = std::stoi(entry_name.substr(dtype_end + 1)); + std::string path = fmt::format("{}/{}", root, entry_name); + return Device::make_device(path, dtype + ).then([this, &sds, id, dtype, new_osd_fsid](DeviceRef sec_dev) { + auto p_sec_dev = sec_dev.get(); + secondaries.emplace_back(std::move(sec_dev)); + return p_sec_dev->start( + ).then([&sds, id, dtype, new_osd_fsid, p_sec_dev]() { + magic_t magic = (magic_t)std::rand(); + sds.emplace( + (device_id_t)id, + device_spec_t{magic, dtype, (device_id_t)id}); + return p_sec_dev->mkfs(device_config_t::create_secondary( + new_osd_fsid, id, dtype, magic) + ).handle_error(crimson::ct_error::assert_all{"not possible"}); + }); + }).then([this] { + return set_secondaries(); + }); + } + return seastar::now(); + }); + return sub.done().then([root_f=std::move(root_f)] {}); + }); + } + return fut.then([this, &sds, new_osd_fsid] { + device_id_t id = 0; + device_type_t d_type = device->get_device_type(); + assert(d_type == device_type_t::SSD || + d_type == device_type_t::RANDOM_BLOCK_SSD); + if (d_type == device_type_t::RANDOM_BLOCK_SSD) { + id = static_cast<device_id_t>(DEVICE_ID_RANDOM_BLOCK_MIN); + } + + return device->mkfs( + device_config_t::create_primary(new_osd_fsid, id, d_type, sds) + ); + }).safe_then([this] { + return crimson::do_for_each(secondaries, [](auto& sec_dev) { + return sec_dev->mount(); + }); + }); + }).safe_then([this] { + return device->mount(); + }).safe_then([this] { + return shard_stores.invoke_on_all([] (auto &local_store) { + return local_store.mkfs_managers(); + }); + }).safe_then([this, new_osd_fsid] { + return prepare_meta(new_osd_fsid); + }).safe_then([this] { + return umount(); + }).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::mkfs" + } + ); + } + }); +} + +using coll_core_t = FuturizedStore::coll_core_t; +seastar::future<std::vector<coll_core_t>> +SeaStore::list_collections() +{ + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.map([](auto &local_store) { + return local_store.list_collections(); + }).then([](std::vector<std::vector<coll_core_t>> results) { + std::vector<coll_core_t> collections; + for (auto& colls : results) { + collections.insert(collections.end(), colls.begin(), colls.end()); + } + return seastar::make_ready_future<std::vector<coll_core_t>>( + std::move(collections)); + }); +} + +store_statfs_t SeaStore::Shard::stat() const +{ + return transaction_manager->store_stat(); +} + +seastar::future<store_statfs_t> SeaStore::stat() const +{ + ceph_assert(seastar::this_shard_id() == primary_core); + LOG_PREFIX(SeaStore::stat); + DEBUG(""); + return shard_stores.map_reduce0( + [](const SeaStore::Shard &local_store) { + return local_store.stat(); + }, + store_statfs_t(), + [](auto &&ss, auto &&ret) { + ss.add(ret); + return std::move(ss); + } + ).then([](store_statfs_t ss) { + return seastar::make_ready_future<store_statfs_t>(std::move(ss)); + }); +} + +TransactionManager::read_extent_iertr::future<std::optional<unsigned>> +SeaStore::Shard::get_coll_bits(CollectionRef ch, Transaction &t) const +{ + return transaction_manager->read_collection_root(t) + .si_then([this, ch, &t](auto coll_root) { + return collection_manager->list(coll_root, t); + }).si_then([ch](auto colls) { + auto it = std::find_if(colls.begin(), colls.end(), + [ch](const std::pair<coll_t, coll_info_t>& element) { + return element.first == ch->get_cid(); + }); + if (it != colls.end()) { + return TransactionManager::read_extent_iertr::make_ready_future< + std::optional<unsigned>>(it->second.split_bits); + } else { + return TransactionManager::read_extent_iertr::make_ready_future< + std::optional<unsigned>>(std::nullopt); + } + }); +} + +col_obj_ranges_t +SeaStore::get_objs_range(CollectionRef ch, unsigned bits) +{ + col_obj_ranges_t obj_ranges; + spg_t pgid; + constexpr uint32_t MAX_HASH = std::numeric_limits<uint32_t>::max(); + const std::string_view MAX_NSPACE = "\xff"; + if (ch->get_cid().is_pg(&pgid)) { + obj_ranges.obj_begin.shard_id = pgid.shard; + obj_ranges.temp_begin = obj_ranges.obj_begin; + + obj_ranges.obj_begin.hobj.pool = pgid.pool(); + obj_ranges.temp_begin.hobj.pool = -2ll - pgid.pool(); + + obj_ranges.obj_end = obj_ranges.obj_begin; + obj_ranges.temp_end = obj_ranges.temp_begin; + + uint32_t reverse_hash = hobject_t::_reverse_bits(pgid.ps()); + obj_ranges.obj_begin.hobj.set_bitwise_key_u32(reverse_hash); + obj_ranges.temp_begin.hobj.set_bitwise_key_u32(reverse_hash); + + uint64_t end_hash = reverse_hash + (1ull << (32 - bits)); + if (end_hash > MAX_HASH) { + // make sure end hobj is even greater than the maximum possible hobj + obj_ranges.obj_end.hobj.set_bitwise_key_u32(MAX_HASH); + obj_ranges.temp_end.hobj.set_bitwise_key_u32(MAX_HASH); + obj_ranges.obj_end.hobj.nspace = MAX_NSPACE; + } else { + obj_ranges.obj_end.hobj.set_bitwise_key_u32(end_hash); + obj_ranges.temp_end.hobj.set_bitwise_key_u32(end_hash); + } + } else { + obj_ranges.obj_begin.shard_id = shard_id_t::NO_SHARD; + obj_ranges.obj_begin.hobj.pool = -1ull; + + obj_ranges.obj_end = obj_ranges.obj_begin; + obj_ranges.obj_begin.hobj.set_bitwise_key_u32(0); + obj_ranges.obj_end.hobj.set_bitwise_key_u32(MAX_HASH); + obj_ranges.obj_end.hobj.nspace = MAX_NSPACE; + // no separate temp section + obj_ranges.temp_begin = obj_ranges.obj_end; + obj_ranges.temp_end = obj_ranges.obj_end; + } + + obj_ranges.obj_begin.generation = 0; + obj_ranges.obj_end.generation = 0; + obj_ranges.temp_begin.generation = 0; + obj_ranges.temp_end.generation = 0; + return obj_ranges; +} + +static std::list<std::pair<ghobject_t, ghobject_t>> +get_ranges(CollectionRef ch, + ghobject_t start, + ghobject_t end, + col_obj_ranges_t obj_ranges) +{ + ceph_assert(start <= end); + std::list<std::pair<ghobject_t, ghobject_t>> ranges; + if (start < obj_ranges.temp_end) { + ranges.emplace_back( + std::max(obj_ranges.temp_begin, start), + std::min(obj_ranges.temp_end, end)); + } + if (end > obj_ranges.obj_begin) { + ranges.emplace_back( + std::max(obj_ranges.obj_begin, start), + std::min(obj_ranges.obj_end, end)); + } + return ranges; +} + +seastar::future<std::tuple<std::vector<ghobject_t>, ghobject_t>> +SeaStore::Shard::list_objects(CollectionRef ch, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) const +{ + ceph_assert(start <= end); + using list_iertr = OnodeManager::list_onodes_iertr; + using RetType = typename OnodeManager::list_onodes_bare_ret; + return seastar::do_with( + RetType(std::vector<ghobject_t>(), start), + std::move(limit), + [this, ch, start, end](auto& ret, auto& limit) { + return repeat_eagain([this, ch, start, end, &limit, &ret] { + return transaction_manager->with_transaction_intr( + Transaction::src_t::READ, + "list_objects", + [this, ch, start, end, &limit, &ret](auto &t) + { + return get_coll_bits( + ch, t + ).si_then([this, ch, &t, start, end, &limit, &ret](auto bits) { + if (!bits) { + return list_iertr::make_ready_future< + OnodeManager::list_onodes_bare_ret + >(std::make_tuple( + std::vector<ghobject_t>(), + ghobject_t::get_max())); + } else { + auto filter = SeaStore::get_objs_range(ch, *bits); + using list_iertr = OnodeManager::list_onodes_iertr; + using repeat_ret = list_iertr::future<seastar::stop_iteration>; + return trans_intr::repeat( + [this, &t, &ret, &limit, + filter, ranges = get_ranges(ch, start, end, filter) + ]() mutable -> repeat_ret { + if (limit == 0 || ranges.empty()) { + return list_iertr::make_ready_future< + seastar::stop_iteration + >(seastar::stop_iteration::yes); + } + auto ite = ranges.begin(); + auto pstart = ite->first; + auto pend = ite->second; + ranges.pop_front(); + return onode_manager->list_onodes( + t, pstart, pend, limit + ).si_then([&limit, &ret, pend](auto &&_ret) mutable { + auto &next_objects = std::get<0>(_ret); + auto &ret_objects = std::get<0>(ret); + ret_objects.insert( + ret_objects.end(), + next_objects.begin(), + next_objects.end()); + std::get<1>(ret) = std::get<1>(_ret); + assert(limit >= next_objects.size()); + limit -= next_objects.size(); + assert(limit == 0 || + std::get<1>(_ret) == pend || + std::get<1>(_ret) == ghobject_t::get_max()); + return list_iertr::make_ready_future< + seastar::stop_iteration + >(seastar::stop_iteration::no); + }); + }).si_then([&ret] { + return list_iertr::make_ready_future< + OnodeManager::list_onodes_bare_ret>(std::move(ret)); + }); + } + }); + }).safe_then([&ret](auto&& _ret) { + ret = std::move(_ret); + }); + }).safe_then([&ret] { + return std::move(ret); + }).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::list_objects" + } + ); + }); +} + +seastar::future<CollectionRef> +SeaStore::Shard::create_new_collection(const coll_t& cid) +{ + LOG_PREFIX(SeaStore::create_new_collection); + DEBUG("{}", cid); + return seastar::make_ready_future<CollectionRef>(_get_collection(cid)); +} + +seastar::future<CollectionRef> +SeaStore::Shard::open_collection(const coll_t& cid) +{ + LOG_PREFIX(SeaStore::open_collection); + DEBUG("{}", cid); + return list_collections().then([cid, this] (auto colls_cores) { + if (auto found = std::find(colls_cores.begin(), + colls_cores.end(), + std::make_pair(cid, seastar::this_shard_id())); + found != colls_cores.end()) { + return seastar::make_ready_future<CollectionRef>(_get_collection(cid)); + } else { + return seastar::make_ready_future<CollectionRef>(); + } + }); +} + +seastar::future<std::vector<coll_core_t>> +SeaStore::Shard::list_collections() +{ + return seastar::do_with( + std::vector<coll_core_t>(), + [this](auto &ret) { + return repeat_eagain([this, &ret] { + return transaction_manager->with_transaction_intr( + Transaction::src_t::READ, + "list_collections", + [this, &ret](auto& t) + { + return transaction_manager->read_collection_root(t + ).si_then([this, &t](auto coll_root) { + return collection_manager->list(coll_root, t); + }).si_then([&ret](auto colls) { + ret.resize(colls.size()); + std::transform( + colls.begin(), colls.end(), ret.begin(), + [](auto p) { + return std::make_pair(p.first, seastar::this_shard_id()); + }); + }); + }); + }).safe_then([&ret] { + return seastar::make_ready_future<std::vector<coll_core_t>>(ret); + }); + } + ).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::list_collections" + } + ); +} + +SeaStore::Shard::read_errorator::future<ceph::bufferlist> +SeaStore::Shard::read( + CollectionRef ch, + const ghobject_t& oid, + uint64_t offset, + size_t len, + uint32_t op_flags) +{ + LOG_PREFIX(SeaStore::read); + DEBUG("oid {} offset {} len {}", oid, offset, len); + return repeat_with_onode<ceph::bufferlist>( + ch, + oid, + Transaction::src_t::READ, + "read_obj", + op_type_t::READ, + [=, this](auto &t, auto &onode) -> ObjectDataHandler::read_ret { + size_t size = onode.get_layout().size; + + if (offset >= size) { + return seastar::make_ready_future<ceph::bufferlist>(); + } + + size_t corrected_len = (len == 0) ? + size - offset : + std::min(size - offset, len); + + return ObjectDataHandler(max_object_size).read( + ObjectDataHandler::context_t{ + *transaction_manager, + t, + onode, + }, + offset, + corrected_len); + }); +} + +SeaStore::Shard::read_errorator::future<ceph::bufferlist> +SeaStore::Shard::readv( + CollectionRef ch, + const ghobject_t& _oid, + interval_set<uint64_t>& m, + uint32_t op_flags) +{ + return seastar::do_with( + _oid, + ceph::bufferlist{}, + [=, this, &m](auto &oid, auto &ret) { + return crimson::do_for_each( + m, + [=, this, &oid, &ret](auto &p) { + return read( + ch, oid, p.first, p.second, op_flags + ).safe_then([&ret](auto bl) { + ret.claim_append(bl); + }); + }).safe_then([&ret] { + return read_errorator::make_ready_future<ceph::bufferlist> + (std::move(ret)); + }); + }); + return read_errorator::make_ready_future<ceph::bufferlist>(); +} + +using crimson::os::seastore::omap_manager::BtreeOMapManager; + +SeaStore::Shard::get_attr_errorator::future<ceph::bufferlist> +SeaStore::Shard::get_attr( + CollectionRef ch, + const ghobject_t& oid, + std::string_view name) const +{ + auto c = static_cast<SeastoreCollection*>(ch.get()); + LOG_PREFIX(SeaStore::get_attr); + DEBUG("{} {}", c->get_cid(), oid); + return repeat_with_onode<ceph::bufferlist>( + c, + oid, + Transaction::src_t::READ, + "get_attr", + op_type_t::GET_ATTR, + [=, this](auto &t, auto& onode) -> _omap_get_value_ret { + auto& layout = onode.get_layout(); + if (name == OI_ATTR && layout.oi_size) { + ceph::bufferlist bl; + bl.append(ceph::bufferptr(&layout.oi[0], layout.oi_size)); + return seastar::make_ready_future<ceph::bufferlist>(std::move(bl)); + } + if (name == SS_ATTR && layout.ss_size) { + ceph::bufferlist bl; + bl.append(ceph::bufferptr(&layout.ss[0], layout.ss_size)); + return seastar::make_ready_future<ceph::bufferlist>(std::move(bl)); + } + return _omap_get_value( + t, + layout.xattr_root.get( + onode.get_metadata_hint(device->get_block_size())), + name); + } + ).handle_error(crimson::ct_error::input_output_error::handle([FNAME] { + ERROR("EIO when getting attrs"); + abort(); + }), crimson::ct_error::pass_further_all{}); +} + +SeaStore::Shard::get_attrs_ertr::future<SeaStore::Shard::attrs_t> +SeaStore::Shard::get_attrs( + CollectionRef ch, + const ghobject_t& oid) +{ + LOG_PREFIX(SeaStore::get_attrs); + auto c = static_cast<SeastoreCollection*>(ch.get()); + DEBUG("{} {}", c->get_cid(), oid); + return repeat_with_onode<attrs_t>( + c, + oid, + Transaction::src_t::READ, + "get_addrs", + op_type_t::GET_ATTRS, + [=, this](auto &t, auto& onode) { + auto& layout = onode.get_layout(); + return omap_list(onode, layout.xattr_root, t, std::nullopt, + OMapManager::omap_list_config_t().with_inclusive(false, false) + ).si_then([&layout](auto p) { + auto& attrs = std::get<1>(p); + ceph::bufferlist bl; + if (layout.oi_size) { + bl.append(ceph::bufferptr(&layout.oi[0], layout.oi_size)); + attrs.emplace(OI_ATTR, std::move(bl)); + } + if (layout.ss_size) { + bl.clear(); + bl.append(ceph::bufferptr(&layout.ss[0], layout.ss_size)); + attrs.emplace(SS_ATTR, std::move(bl)); + } + return seastar::make_ready_future<omap_values_t>(std::move(attrs)); + }); + } + ).handle_error(crimson::ct_error::input_output_error::handle([FNAME] { + ERROR("EIO when getting attrs"); + abort(); + }), crimson::ct_error::pass_further_all{}); +} + +seastar::future<struct stat> SeaStore::Shard::stat( + CollectionRef c, + const ghobject_t& oid) +{ + LOG_PREFIX(SeaStore::stat); + return repeat_with_onode<struct stat>( + c, + oid, + Transaction::src_t::READ, + "stat", + op_type_t::STAT, + [=, this](auto &t, auto &onode) { + struct stat st; + auto &olayout = onode.get_layout(); + st.st_size = olayout.size; + st.st_blksize = device->get_block_size(); + st.st_blocks = (st.st_size + st.st_blksize - 1) / st.st_blksize; + st.st_nlink = 1; + DEBUGT("cid {}, oid {}, return size {}", t, c->get_cid(), oid, st.st_size); + return seastar::make_ready_future<struct stat>(st); + } + ).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::stat" + } + ); +} + +SeaStore::Shard::get_attr_errorator::future<ceph::bufferlist> +SeaStore::Shard::omap_get_header( + CollectionRef ch, + const ghobject_t& oid) +{ + return get_attr(ch, oid, OMAP_HEADER_XATTR_KEY); +} + +SeaStore::Shard::read_errorator::future<SeaStore::Shard::omap_values_t> +SeaStore::Shard::omap_get_values( + CollectionRef ch, + const ghobject_t &oid, + const omap_keys_t &keys) +{ + auto c = static_cast<SeastoreCollection*>(ch.get()); + return repeat_with_onode<omap_values_t>( + c, + oid, + Transaction::src_t::READ, + "omap_get_values", + op_type_t::OMAP_GET_VALUES, + [this, keys](auto &t, auto &onode) { + omap_root_t omap_root = onode.get_layout().omap_root.get( + onode.get_metadata_hint(device->get_block_size())); + return _omap_get_values( + t, + std::move(omap_root), + keys); + }); +} + +SeaStore::Shard::_omap_get_value_ret +SeaStore::Shard::_omap_get_value( + Transaction &t, + omap_root_t &&root, + std::string_view key) const +{ + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + std::move(root), + std::string(key), + [&t](auto &manager, auto& root, auto& key) -> _omap_get_value_ret { + if (root.is_null()) { + return crimson::ct_error::enodata::make(); + } + return manager.omap_get_value(root, t, key + ).si_then([](auto opt) -> _omap_get_value_ret { + if (!opt) { + return crimson::ct_error::enodata::make(); + } + return seastar::make_ready_future<ceph::bufferlist>(std::move(*opt)); + }); + } + ); +} + +SeaStore::Shard::_omap_get_values_ret +SeaStore::Shard::_omap_get_values( + Transaction &t, + omap_root_t &&omap_root, + const omap_keys_t &keys) const +{ + if (omap_root.is_null()) { + return seastar::make_ready_future<omap_values_t>(); + } + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + std::move(omap_root), + omap_values_t(), + [&](auto &manager, auto &root, auto &ret) { + return trans_intr::do_for_each( + keys.begin(), + keys.end(), + [&](auto &key) { + return manager.omap_get_value( + root, + t, + key + ).si_then([&ret, &key](auto &&p) { + if (p) { + bufferlist bl; + bl.append(*p); + ret.emplace( + std::move(key), + std::move(bl)); + } + return seastar::now(); + }); + } + ).si_then([&ret] { + return std::move(ret); + }); + } + ); +} + +SeaStore::Shard::omap_list_ret +SeaStore::Shard::omap_list( + Onode &onode, + const omap_root_le_t& omap_root, + Transaction& t, + const std::optional<std::string>& start, + OMapManager::omap_list_config_t config) const +{ + auto root = omap_root.get( + onode.get_metadata_hint(device->get_block_size())); + if (root.is_null()) { + return seastar::make_ready_future<omap_list_bare_ret>( + true, omap_values_t{} + ); + } + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + root, + start, + std::optional<std::string>(std::nullopt), + [&t, config](auto &manager, auto &root, auto &start, auto &end) { + return manager.omap_list(root, t, start, end, config); + }); +} + +SeaStore::Shard::omap_get_values_ret_t +SeaStore::Shard::omap_get_values( + CollectionRef ch, + const ghobject_t &oid, + const std::optional<string> &start) +{ + auto c = static_cast<SeastoreCollection*>(ch.get()); + LOG_PREFIX(SeaStore::omap_get_values); + DEBUG("{} {}", c->get_cid(), oid); + using ret_bare_t = std::tuple<bool, SeaStore::Shard::omap_values_t>; + return repeat_with_onode<ret_bare_t>( + c, + oid, + Transaction::src_t::READ, + "omap_list", + op_type_t::OMAP_LIST, + [this, start](auto &t, auto &onode) { + return omap_list( + onode, + onode.get_layout().omap_root, + t, + start, + OMapManager::omap_list_config_t().with_inclusive(false, false)); + }); +} + +SeaStore::Shard::_fiemap_ret SeaStore::Shard::_fiemap( + Transaction &t, + Onode &onode, + uint64_t off, + uint64_t len) const +{ + return seastar::do_with( + ObjectDataHandler(max_object_size), + [=, this, &t, &onode] (auto &objhandler) { + return objhandler.fiemap( + ObjectDataHandler::context_t{ + *transaction_manager, + t, + onode, + }, + off, + len); + }); +} + +SeaStore::Shard::read_errorator::future<std::map<uint64_t, uint64_t>> +SeaStore::Shard::fiemap( + CollectionRef ch, + const ghobject_t& oid, + uint64_t off, + uint64_t len) +{ + LOG_PREFIX(SeaStore::fiemap); + DEBUG("oid: {}, off: {}, len: {} ", oid, off, len); + return repeat_with_onode<std::map<uint64_t, uint64_t>>( + ch, + oid, + Transaction::src_t::READ, + "fiemap_read", + op_type_t::READ, + [=, this](auto &t, auto &onode) -> _fiemap_ret { + size_t size = onode.get_layout().size; + if (off >= size) { + INFOT("fiemap offset is over onode size!", t); + return seastar::make_ready_future<std::map<uint64_t, uint64_t>>(); + } + size_t adjust_len = (len == 0) ? + size - off: + std::min(size - off, len); + return _fiemap(t, onode, off, adjust_len); + }); +} + +void SeaStore::Shard::on_error(ceph::os::Transaction &t) { + LOG_PREFIX(SeaStore::on_error); + ERROR(" transaction dump:\n"); + JSONFormatter f(true); + f.open_object_section("transaction"); + t.dump(&f); + f.close_section(); + std::stringstream str; + f.flush(str); + ERROR("{}", str.str()); + abort(); +} + +seastar::future<> SeaStore::Shard::do_transaction_no_callbacks( + CollectionRef _ch, + ceph::os::Transaction&& _t) +{ + // repeat_with_internal_context ensures ordering via collection lock + return repeat_with_internal_context( + _ch, + std::move(_t), + Transaction::src_t::MUTATE, + "do_transaction", + op_type_t::TRANSACTION, + [this](auto &ctx) { + return with_trans_intr(*ctx.transaction, [&, this](auto &t) { +#ifndef NDEBUG + LOG_PREFIX(SeaStore::Shard::do_transaction_no_callbacks); + TRACET(" transaction dump:\n", t); + JSONFormatter f(true); + f.open_object_section("transaction"); + ctx.ext_transaction.dump(&f); + f.close_section(); + std::stringstream str; + f.flush(str); + TRACET("{}", t, str.str()); +#endif + return seastar::do_with( + std::vector<OnodeRef>(ctx.iter.objects.size()), + std::vector<OnodeRef>(ctx.iter.objects.size()), + [this, &ctx](auto& onodes, auto& d_onodes) mutable { + return trans_intr::repeat( + [this, &ctx, &onodes, &d_onodes]() mutable + -> tm_iertr::future<seastar::stop_iteration> + { + if (ctx.iter.have_op()) { + return _do_transaction_step( + ctx, ctx.ch, onodes, d_onodes, ctx.iter + ).si_then([] { + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::no); + }); + } else { + return seastar::make_ready_future<seastar::stop_iteration>( + seastar::stop_iteration::yes); + }; + }).si_then([this, &ctx, &d_onodes] { + return onode_manager->write_dirty(*ctx.transaction, d_onodes); + }); + }).si_then([this, &ctx] { + return transaction_manager->submit_transaction(*ctx.transaction); + }); + }); + }); +} + + +seastar::future<> SeaStore::Shard::flush(CollectionRef ch) +{ + return seastar::do_with( + get_dummy_ordering_handle(), + [this, ch](auto &handle) { + return handle.take_collection_lock( + static_cast<SeastoreCollection&>(*ch).ordering_lock + ).then([this, &handle] { + return transaction_manager->flush(handle); + }); + }); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_do_transaction_step( + internal_context_t &ctx, + CollectionRef &col, + std::vector<OnodeRef> &onodes, + std::vector<OnodeRef> &d_onodes, + ceph::os::Transaction::iterator &i) +{ + auto op = i.decode_op(); + + using ceph::os::Transaction; + if (op->op == Transaction::OP_NOP) + return tm_iertr::now(); + + switch (op->op) { + case Transaction::OP_RMCOLL: + { + coll_t cid = i.get_cid(op->cid); + return _remove_collection(ctx, cid); + } + case Transaction::OP_MKCOLL: + { + coll_t cid = i.get_cid(op->cid); + return _create_collection(ctx, cid, op->split_bits); + } + case Transaction::OP_COLL_HINT: + { + ceph::bufferlist hint; + i.decode_bl(hint); + return tm_iertr::now(); + } + } + + using onode_iertr = OnodeManager::get_onode_iertr::extend< + crimson::ct_error::value_too_large>; + auto fut = onode_iertr::make_ready_future<OnodeRef>(OnodeRef()); + bool create = false; + if (op->op == Transaction::OP_TOUCH || + op->op == Transaction::OP_CREATE || + op->op == Transaction::OP_WRITE || + op->op == Transaction::OP_ZERO) { + create = true; + } + if (!onodes[op->oid]) { + if (!create) { + fut = onode_manager->get_onode(*ctx.transaction, i.get_oid(op->oid)); + } else { + fut = onode_manager->get_or_create_onode( + *ctx.transaction, i.get_oid(op->oid)); + } + } + return fut.si_then([&, op](auto get_onode) { + OnodeRef &o = onodes[op->oid]; + if (!o) { + assert(get_onode); + o = get_onode; + d_onodes[op->oid] = get_onode; + } + if (op->op == Transaction::OP_CLONE && !d_onodes[op->dest_oid]) { + //TODO: use when_all_succeed after making onode tree + // support parallel extents loading + return onode_manager->get_or_create_onode( + *ctx.transaction, i.get_oid(op->dest_oid) + ).si_then([&, op](auto dest_onode) { + assert(dest_onode); + auto &d_o = onodes[op->dest_oid]; + assert(!d_o); + assert(!d_onodes[op->dest_oid]); + d_o = dest_onode; + d_onodes[op->dest_oid] = dest_onode; + return seastar::now(); + }); + } else { + return OnodeManager::get_or_create_onode_iertr::now(); + } + }).si_then([&, op, this]() -> tm_ret { + LOG_PREFIX(SeaStore::_do_transaction_step); + try { + switch (op->op) { + case Transaction::OP_REMOVE: + { + TRACET("removing {}", *ctx.transaction, i.get_oid(op->oid)); + return _remove(ctx, onodes[op->oid] + ).si_then([&onodes, &d_onodes, op] { + onodes[op->oid].reset(); + d_onodes[op->oid].reset(); + }); + } + case Transaction::OP_CREATE: + case Transaction::OP_TOUCH: + { + return _touch(ctx, onodes[op->oid]); + } + case Transaction::OP_WRITE: + { + uint64_t off = op->off; + uint64_t len = op->len; + uint32_t fadvise_flags = i.get_fadvise_flags(); + ceph::bufferlist bl; + i.decode_bl(bl); + return _write( + ctx, onodes[op->oid], off, len, std::move(bl), + fadvise_flags); + } + case Transaction::OP_TRUNCATE: + { + uint64_t off = op->off; + return _truncate(ctx, onodes[op->oid], off); + } + case Transaction::OP_SETATTR: + { + std::string name = i.decode_string(); + std::map<std::string, bufferlist> to_set; + ceph::bufferlist& bl = to_set[name]; + i.decode_bl(bl); + return _setattrs(ctx, onodes[op->oid], std::move(to_set)); + } + case Transaction::OP_SETATTRS: + { + std::map<std::string, bufferlist> to_set; + i.decode_attrset(to_set); + return _setattrs(ctx, onodes[op->oid], std::move(to_set)); + } + case Transaction::OP_RMATTR: + { + std::string name = i.decode_string(); + return _rmattr(ctx, onodes[op->oid], name); + } + case Transaction::OP_RMATTRS: + { + return _rmattrs(ctx, onodes[op->oid]); + } + case Transaction::OP_OMAP_SETKEYS: + { + std::map<std::string, ceph::bufferlist> aset; + i.decode_attrset(aset); + return _omap_set_values(ctx, onodes[op->oid], std::move(aset)); + } + case Transaction::OP_OMAP_SETHEADER: + { + ceph::bufferlist bl; + i.decode_bl(bl); + return _omap_set_header(ctx, onodes[op->oid], std::move(bl)); + } + case Transaction::OP_OMAP_RMKEYS: + { + omap_keys_t keys; + i.decode_keyset(keys); + return _omap_rmkeys(ctx, onodes[op->oid], std::move(keys)); + } + case Transaction::OP_OMAP_RMKEYRANGE: + { + string first, last; + first = i.decode_string(); + last = i.decode_string(); + return _omap_rmkeyrange( + ctx, onodes[op->oid], + std::move(first), std::move(last)); + } + case Transaction::OP_OMAP_CLEAR: + { + return _omap_clear(ctx, onodes[op->oid]); + } + case Transaction::OP_ZERO: + { + objaddr_t off = op->off; + extent_len_t len = op->len; + return _zero(ctx, onodes[op->oid], off, len); + } + case Transaction::OP_SETALLOCHINT: + { + // TODO + return tm_iertr::now(); + } + case Transaction::OP_CLONE: + { + TRACET("cloning {} to {}", + *ctx.transaction, + i.get_oid(op->oid), + i.get_oid(op->dest_oid)); + return _clone(ctx, onodes[op->oid], d_onodes[op->dest_oid]); + } + default: + ERROR("bad op {}", static_cast<unsigned>(op->op)); + return crimson::ct_error::input_output_error::make(); + } + } catch (std::exception &e) { + ERROR("got exception {}", e); + return crimson::ct_error::input_output_error::make(); + } + }).handle_error_interruptible( + tm_iertr::pass_further{}, + crimson::ct_error::enoent::handle([op] { + //OMAP_CLEAR, TRUNCATE, REMOVE etc ops will tolerate absent onode. + if (op->op == Transaction::OP_CLONERANGE || + op->op == Transaction::OP_CLONE || + op->op == Transaction::OP_CLONERANGE2 || + op->op == Transaction::OP_COLL_ADD || + op->op == Transaction::OP_SETATTR || + op->op == Transaction::OP_SETATTRS || + op->op == Transaction::OP_RMATTR || + op->op == Transaction::OP_OMAP_SETKEYS || + op->op == Transaction::OP_OMAP_RMKEYS || + op->op == Transaction::OP_OMAP_RMKEYRANGE || + op->op == Transaction::OP_OMAP_SETHEADER) { + ceph_abort_msg("unexpected enoent error"); + } + return seastar::now(); + }), + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::do_transaction_step" + } + ); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_remove( + internal_context_t &ctx, + OnodeRef &onode) +{ + LOG_PREFIX(SeaStore::_remove); + DEBUGT("onode={}", *ctx.transaction, *onode); + auto fut = BtreeOMapManager::omap_clear_iertr::now(); + auto omap_root = onode->get_layout().omap_root.get( + onode->get_metadata_hint(device->get_block_size())); + if (omap_root.get_location() != L_ADDR_NULL) { + fut = seastar::do_with( + BtreeOMapManager(*transaction_manager), + onode->get_layout().omap_root.get( + onode->get_metadata_hint(device->get_block_size())), + [&ctx, onode](auto &omap_manager, auto &omap_root) { + return omap_manager.omap_clear( + omap_root, + *ctx.transaction + ); + }); + } + return fut.si_then([this, &ctx, onode] { + return seastar::do_with( + ObjectDataHandler(max_object_size), + [=, this, &ctx](auto &objhandler) { + return objhandler.clear( + ObjectDataHandler::context_t{ + *transaction_manager, + *ctx.transaction, + *onode, + }); + }); + }).si_then([this, &ctx, onode]() mutable { + return onode_manager->erase_onode(*ctx.transaction, onode); + }).handle_error_interruptible( + crimson::ct_error::input_output_error::pass_further(), + crimson::ct_error::assert_all( + "Invalid error in SeaStore::_remove" + ) + ); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_touch( + internal_context_t &ctx, + OnodeRef &onode) +{ + LOG_PREFIX(SeaStore::_touch); + DEBUGT("onode={}", *ctx.transaction, *onode); + return tm_iertr::now(); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_write( + internal_context_t &ctx, + OnodeRef &onode, + uint64_t offset, size_t len, + ceph::bufferlist &&_bl, + uint32_t fadvise_flags) +{ + LOG_PREFIX(SeaStore::_write); + DEBUGT("onode={} {}~{}", *ctx.transaction, *onode, offset, len); + { + auto &object_size = onode->get_mutable_layout(*ctx.transaction).size; + object_size = std::max<uint64_t>( + offset + len, + object_size); + } + return seastar::do_with( + std::move(_bl), + ObjectDataHandler(max_object_size), + [=, this, &ctx, &onode](auto &bl, auto &objhandler) { + return objhandler.write( + ObjectDataHandler::context_t{ + *transaction_manager, + *ctx.transaction, + *onode, + }, + offset, + bl); + }); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_clone( + internal_context_t &ctx, + OnodeRef &onode, + OnodeRef &d_onode) +{ + LOG_PREFIX(SeaStore::_clone); + DEBUGT("onode={} d_onode={}", *ctx.transaction, *onode, *d_onode); + return seastar::do_with( + ObjectDataHandler(max_object_size), + [this, &ctx, &onode, &d_onode](auto &objHandler) { + //TODO: currently, we only care about object data, leaving cloning + // of xattr/omap for future work + auto &object_size = onode->get_layout().size; + auto &d_object_size = d_onode->get_mutable_layout(*ctx.transaction).size; + d_object_size = object_size; + return objHandler.clone( + ObjectDataHandler::context_t{ + *transaction_manager, + *ctx.transaction, + *onode, + d_onode.get()}); + }); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_zero( + internal_context_t &ctx, + OnodeRef &onode, + objaddr_t offset, + extent_len_t len) +{ + LOG_PREFIX(SeaStore::_zero); + DEBUGT("onode={} {}~{}", *ctx.transaction, *onode, offset, len); + if (offset + len >= max_object_size) { + return crimson::ct_error::input_output_error::make(); + } + auto &object_size = onode->get_mutable_layout(*ctx.transaction).size; + object_size = std::max<uint64_t>(offset + len, object_size); + return seastar::do_with( + ObjectDataHandler(max_object_size), + [=, this, &ctx, &onode](auto &objhandler) { + return objhandler.zero( + ObjectDataHandler::context_t{ + *transaction_manager, + *ctx.transaction, + *onode, + }, + offset, + len); + }); +} + +SeaStore::Shard::omap_set_kvs_ret +SeaStore::Shard::_omap_set_kvs( + OnodeRef &onode, + const omap_root_le_t& omap_root, + Transaction& t, + omap_root_le_t& mutable_omap_root, + std::map<std::string, ceph::bufferlist>&& kvs) +{ + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + omap_root.get(onode->get_metadata_hint(device->get_block_size())), + [&, keys=std::move(kvs)](auto &omap_manager, auto &root) { + tm_iertr::future<> maybe_create_root = + !root.is_null() ? + tm_iertr::now() : + omap_manager.initialize_omap( + t, onode->get_metadata_hint(device->get_block_size()) + ).si_then([&root](auto new_root) { + root = new_root; + }); + return maybe_create_root.si_then( + [&, keys=std::move(keys)]() mutable { + return omap_manager.omap_set_keys(root, t, std::move(keys)); + }).si_then([&] { + return tm_iertr::make_ready_future<omap_root_t>(std::move(root)); + }).si_then([&mutable_omap_root](auto root) { + if (root.must_update()) { + mutable_omap_root.update(root); + } + }); + } + ); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_omap_set_values( + internal_context_t &ctx, + OnodeRef &onode, + std::map<std::string, ceph::bufferlist> &&aset) +{ + LOG_PREFIX(SeaStore::_omap_set_values); + DEBUGT("{} {} keys", *ctx.transaction, *onode, aset.size()); + return _omap_set_kvs( + onode, + onode->get_layout().omap_root, + *ctx.transaction, + onode->get_mutable_layout(*ctx.transaction).omap_root, + std::move(aset)); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_omap_set_header( + internal_context_t &ctx, + OnodeRef &onode, + ceph::bufferlist &&header) +{ + LOG_PREFIX(SeaStore::_omap_set_header); + DEBUGT("{} {} bytes", *ctx.transaction, *onode, header.length()); + std::map<std::string, bufferlist> to_set; + to_set[OMAP_HEADER_XATTR_KEY] = header; + return _setattrs(ctx, onode,std::move(to_set)); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_omap_clear( + internal_context_t &ctx, + OnodeRef &onode) +{ + LOG_PREFIX(SeaStore::_omap_clear); + DEBUGT("{} {} keys", *ctx.transaction, *onode); + return _xattr_rmattr(ctx, onode, std::string(OMAP_HEADER_XATTR_KEY)) + .si_then([this, &ctx, &onode]() -> tm_ret { + if (auto omap_root = onode->get_layout().omap_root.get( + onode->get_metadata_hint(device->get_block_size())); + omap_root.is_null()) { + return seastar::now(); + } else { + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + onode->get_layout().omap_root.get( + onode->get_metadata_hint(device->get_block_size())), + [&ctx, &onode]( + auto &omap_manager, + auto &omap_root) { + return omap_manager.omap_clear( + omap_root, + *ctx.transaction) + .si_then([&] { + if (omap_root.must_update()) { + onode->get_mutable_layout(*ctx.transaction + ).omap_root.update(omap_root); + } + }); + }); + } + }); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_omap_rmkeys( + internal_context_t &ctx, + OnodeRef &onode, + omap_keys_t &&keys) +{ + LOG_PREFIX(SeaStore::_omap_rmkeys); + DEBUGT("{} {} keys", *ctx.transaction, *onode, keys.size()); + auto omap_root = onode->get_layout().omap_root.get( + onode->get_metadata_hint(device->get_block_size())); + if (omap_root.is_null()) { + return seastar::now(); + } else { + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + onode->get_layout().omap_root.get( + onode->get_metadata_hint(device->get_block_size())), + std::move(keys), + [&ctx, &onode]( + auto &omap_manager, + auto &omap_root, + auto &keys) { + return trans_intr::do_for_each( + keys.begin(), + keys.end(), + [&](auto &p) { + return omap_manager.omap_rm_key( + omap_root, + *ctx.transaction, + p); + } + ).si_then([&] { + if (omap_root.must_update()) { + onode->get_mutable_layout(*ctx.transaction + ).omap_root.update(omap_root); + } + }); + } + ); + } +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_omap_rmkeyrange( + internal_context_t &ctx, + OnodeRef &onode, + std::string first, + std::string last) +{ + LOG_PREFIX(SeaStore::_omap_rmkeyrange); + DEBUGT("{} first={} last={}", *ctx.transaction, *onode, first, last); + if (first > last) { + ERRORT("range error, first: {} > last:{}", *ctx.transaction, first, last); + ceph_abort(); + } + auto omap_root = onode->get_layout().omap_root.get( + onode->get_metadata_hint(device->get_block_size())); + if (omap_root.is_null()) { + return seastar::now(); + } else { + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + onode->get_layout().omap_root.get( + onode->get_metadata_hint(device->get_block_size())), + std::move(first), + std::move(last), + [&ctx, &onode]( + auto &omap_manager, + auto &omap_root, + auto &first, + auto &last) { + auto config = OMapManager::omap_list_config_t() + .with_inclusive(true, false) + .without_max(); + return omap_manager.omap_rm_key_range( + omap_root, + *ctx.transaction, + first, + last, + config + ).si_then([&] { + if (omap_root.must_update()) { + onode->get_mutable_layout(*ctx.transaction + ).omap_root.update(omap_root); + } + }); + }); + } +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_truncate( + internal_context_t &ctx, + OnodeRef &onode, + uint64_t size) +{ + LOG_PREFIX(SeaStore::_truncate); + DEBUGT("onode={} size={}", *ctx.transaction, *onode, size); + onode->get_mutable_layout(*ctx.transaction).size = size; + return seastar::do_with( + ObjectDataHandler(max_object_size), + [=, this, &ctx, &onode](auto &objhandler) { + return objhandler.truncate( + ObjectDataHandler::context_t{ + *transaction_manager, + *ctx.transaction, + *onode + }, + size); + }); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_setattrs( + internal_context_t &ctx, + OnodeRef &onode, + std::map<std::string, bufferlist>&& aset) +{ + LOG_PREFIX(SeaStore::_setattrs); + DEBUGT("onode={}", *ctx.transaction, *onode); + + auto fut = tm_iertr::now(); + auto& layout = onode->get_mutable_layout(*ctx.transaction); + if (auto it = aset.find(OI_ATTR); it != aset.end()) { + auto& val = it->second; + if (likely(val.length() <= onode_layout_t::MAX_OI_LENGTH)) { + maybe_inline_memcpy( + &layout.oi[0], + val.c_str(), + val.length(), + onode_layout_t::MAX_OI_LENGTH); + + if (!layout.oi_size) { + // if oi was not in the layout, it probably exists in the omap, + // need to remove it first + fut = _xattr_rmattr(ctx, onode, OI_ATTR); + } + layout.oi_size = val.length(); + aset.erase(it); + } else { + layout.oi_size = 0; + } + } + + if (auto it = aset.find(SS_ATTR); it != aset.end()) { + auto& val = it->second; + if (likely(val.length() <= onode_layout_t::MAX_SS_LENGTH)) { + maybe_inline_memcpy( + &layout.ss[0], + val.c_str(), + val.length(), + onode_layout_t::MAX_SS_LENGTH); + + if (!layout.ss_size) { + fut = _xattr_rmattr(ctx, onode, SS_ATTR); + } + layout.ss_size = val.length(); + + aset.erase(it); + } else { + layout.ss_size = 0; + } + } + + if (aset.empty()) { + return fut; + } + + return fut.si_then( + [this, onode, &ctx, &layout, + aset=std::move(aset)]() mutable { + return _omap_set_kvs( + onode, + onode->get_layout().xattr_root, + *ctx.transaction, + layout.xattr_root, + std::move(aset)); + }); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_rmattr( + internal_context_t &ctx, + OnodeRef &onode, + std::string name) +{ + LOG_PREFIX(SeaStore::_rmattr); + DEBUGT("onode={}", *ctx.transaction, *onode); + auto& layout = onode->get_mutable_layout(*ctx.transaction); + if ((name == OI_ATTR) && (layout.oi_size > 0)) { + memset(&layout.oi[0], 0, layout.oi_size); + layout.oi_size = 0; + return tm_iertr::now(); + } else if ((name == SS_ATTR) && (layout.ss_size > 0)) { + memset(&layout.ss[0], 0, layout.ss_size); + layout.ss_size = 0; + return tm_iertr::now(); + } else { + return _xattr_rmattr( + ctx, + onode, + std::move(name)); + } +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_xattr_rmattr( + internal_context_t &ctx, + OnodeRef &onode, + std::string &&name) +{ + LOG_PREFIX(SeaStore::_xattr_rmattr); + DEBUGT("onode={}", *ctx.transaction, *onode); + auto xattr_root = onode->get_layout().xattr_root.get( + onode->get_metadata_hint(device->get_block_size())); + if (xattr_root.is_null()) { + return seastar::now(); + } else { + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + onode->get_layout().xattr_root.get( + onode->get_metadata_hint(device->get_block_size())), + std::move(name), + [&ctx, &onode](auto &omap_manager, auto &xattr_root, auto &name) { + return omap_manager.omap_rm_key(xattr_root, *ctx.transaction, name) + .si_then([&] { + if (xattr_root.must_update()) { + onode->get_mutable_layout(*ctx.transaction + ).xattr_root.update(xattr_root); + } + }); + }); + } +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_rmattrs( + internal_context_t &ctx, + OnodeRef &onode) +{ + LOG_PREFIX(SeaStore::_rmattrs); + DEBUGT("onode={}", *ctx.transaction, *onode); + auto& layout = onode->get_mutable_layout(*ctx.transaction); + memset(&layout.oi[0], 0, layout.oi_size); + layout.oi_size = 0; + memset(&layout.ss[0], 0, layout.ss_size); + layout.ss_size = 0; + return _xattr_clear(ctx, onode); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_xattr_clear( + internal_context_t &ctx, + OnodeRef &onode) +{ + LOG_PREFIX(SeaStore::_xattr_clear); + DEBUGT("onode={}", *ctx.transaction, *onode); + auto xattr_root = onode->get_layout().xattr_root.get( + onode->get_metadata_hint(device->get_block_size())); + if (xattr_root.is_null()) { + return seastar::now(); + } else { + return seastar::do_with( + BtreeOMapManager(*transaction_manager), + onode->get_layout().xattr_root.get( + onode->get_metadata_hint(device->get_block_size())), + [&ctx, &onode](auto &omap_manager, auto &xattr_root) { + return omap_manager.omap_clear(xattr_root, *ctx.transaction) + .si_then([&] { + if (xattr_root.must_update()) { + onode->get_mutable_layout(*ctx.transaction + ).xattr_root.update(xattr_root); + } + }); + }); + } +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_create_collection( + internal_context_t &ctx, + const coll_t& cid, int bits) +{ + return transaction_manager->read_collection_root( + *ctx.transaction + ).si_then([=, this, &ctx](auto _cmroot) { + return seastar::do_with( + _cmroot, + [=, this, &ctx](auto &cmroot) { + return collection_manager->create( + cmroot, + *ctx.transaction, + cid, + bits + ).si_then([this, &ctx, &cmroot] { + if (cmroot.must_update()) { + transaction_manager->write_collection_root( + *ctx.transaction, + cmroot); + } + }); + } + ); + }).handle_error_interruptible( + tm_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::_create_collection" + } + ); +} + +SeaStore::Shard::tm_ret +SeaStore::Shard::_remove_collection( + internal_context_t &ctx, + const coll_t& cid) +{ + return transaction_manager->read_collection_root( + *ctx.transaction + ).si_then([=, this, &ctx](auto _cmroot) { + return seastar::do_with( + _cmroot, + [=, this, &ctx](auto &cmroot) { + return collection_manager->remove( + cmroot, + *ctx.transaction, + cid + ).si_then([this, &ctx, &cmroot] { + // param here denotes whether it already existed, probably error + if (cmroot.must_update()) { + transaction_manager->write_collection_root( + *ctx.transaction, + cmroot); + } + }); + }); + }).handle_error_interruptible( + tm_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::_create_collection" + } + ); +} + +boost::intrusive_ptr<SeastoreCollection> +SeaStore::Shard::_get_collection(const coll_t& cid) +{ + return new SeastoreCollection{cid}; +} + +seastar::future<> SeaStore::Shard::write_meta( + const std::string& key, + const std::string& value) +{ + LOG_PREFIX(SeaStore::write_meta); + DEBUG("key: {}; value: {}", key, value); + return seastar::do_with( + key, value, + [this, FNAME](auto& key, auto& value) { + return repeat_eagain([this, FNAME, &key, &value] { + return transaction_manager->with_transaction_intr( + Transaction::src_t::MUTATE, + "write_meta", + [this, FNAME, &key, &value](auto& t) + { + DEBUGT("Have transaction, key: {}; value: {}", t, key, value); + return transaction_manager->update_root_meta( + t, key, value + ).si_then([this, &t] { + return transaction_manager->submit_transaction(t); + }); + }); + }); + }).handle_error( + crimson::ct_error::assert_all{"Invalid error in SeaStore::write_meta"} + ); +} + +seastar::future<std::tuple<int, std::string>> +SeaStore::read_meta(const std::string& key) +{ + ceph_assert(seastar::this_shard_id() == primary_core); + LOG_PREFIX(SeaStore::read_meta); + DEBUG("key: {}", key); + return mdstore->read_meta(key).safe_then([](auto v) { + if (v) { + return std::make_tuple(0, std::move(*v)); + } else { + return std::make_tuple(-1, std::string("")); + } + }).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in SeaStore::read_meta" + } + ); +} + +uuid_d SeaStore::Shard::get_fsid() const +{ + return device->get_meta().seastore_id; +} + +void SeaStore::Shard::init_managers() +{ + transaction_manager.reset(); + collection_manager.reset(); + onode_manager.reset(); + + transaction_manager = make_transaction_manager( + device, secondaries, is_test); + collection_manager = std::make_unique<collection_manager::FlatCollectionManager>( + *transaction_manager); + onode_manager = std::make_unique<crimson::os::seastore::onode::FLTreeOnodeManager>( + *transaction_manager); +} + +std::unique_ptr<SeaStore> make_seastore( + const std::string &device) +{ + auto mdstore = std::make_unique<FileMDStore>(device); + return std::make_unique<SeaStore>( + device, + std::move(mdstore)); +} + +std::unique_ptr<SeaStore> make_test_seastore( + SeaStore::MDStoreRef mdstore) +{ + return std::make_unique<SeaStore>( + "", + std::move(mdstore)); +} + +} diff --git a/src/crimson/os/seastore/seastore.h b/src/crimson/os/seastore/seastore.h new file mode 100644 index 000000000..876fadca8 --- /dev/null +++ b/src/crimson/os/seastore/seastore.h @@ -0,0 +1,531 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <string> +#include <unordered_map> +#include <map> +#include <typeinfo> +#include <vector> + +#include <optional> +#include <seastar/core/future.hh> +#include <seastar/core/metrics_types.hh> + +#include "include/uuid.h" + +#include "os/Transaction.h" +#include "crimson/common/throttle.h" +#include "crimson/os/futurized_collection.h" +#include "crimson/os/futurized_store.h" + +#include "crimson/os/seastore/device.h" +#include "crimson/os/seastore/transaction.h" +#include "crimson/os/seastore/onode_manager.h" +#include "crimson/os/seastore/omap_manager.h" +#include "crimson/os/seastore/collection_manager.h" +#include "crimson/os/seastore/object_data_handler.h" + +namespace crimson::os::seastore { + +class Onode; +using OnodeRef = boost::intrusive_ptr<Onode>; +class TransactionManager; + +enum class op_type_t : uint8_t { + TRANSACTION = 0, + READ, + WRITE, + GET_ATTR, + GET_ATTRS, + STAT, + OMAP_GET_VALUES, + OMAP_LIST, + MAX +}; + +class SeastoreCollection final : public FuturizedCollection { +public: + template <typename... T> + SeastoreCollection(T&&... args) : + FuturizedCollection(std::forward<T>(args)...) {} + + seastar::shared_mutex ordering_lock; +}; + +/** + * col_obj_ranges_t + * + * Represents the two ghobject_t ranges spanned by a PG collection. + * Temp objects will be within [temp_begin, temp_end) and normal objects + * will be in [obj_begin, obj_end). + */ +struct col_obj_ranges_t { + ghobject_t temp_begin; + ghobject_t temp_end; + ghobject_t obj_begin; + ghobject_t obj_end; +}; + +class SeaStore final : public FuturizedStore { +public: + class MDStore { + public: + using base_iertr = crimson::errorator< + crimson::ct_error::input_output_error + >; + + using write_meta_ertr = base_iertr; + using write_meta_ret = write_meta_ertr::future<>; + virtual write_meta_ret write_meta( + const std::string &key, + const std::string &val + ) = 0; + + using read_meta_ertr = base_iertr; + using read_meta_ret = write_meta_ertr::future<std::optional<std::string>>; + virtual read_meta_ret read_meta(const std::string &key) = 0; + + virtual ~MDStore() {} + }; + using MDStoreRef = std::unique_ptr<MDStore>; + + class Shard : public FuturizedStore::Shard { + public: + Shard( + std::string root, + Device* device, + bool is_test); + ~Shard() = default; + + seastar::future<struct stat> stat( + CollectionRef c, + const ghobject_t& oid) final; + + read_errorator::future<ceph::bufferlist> read( + CollectionRef c, + const ghobject_t& oid, + uint64_t offset, + size_t len, + uint32_t op_flags = 0) final; + + read_errorator::future<ceph::bufferlist> readv( + CollectionRef c, + const ghobject_t& oid, + interval_set<uint64_t>& m, + uint32_t op_flags = 0) final; + + get_attr_errorator::future<ceph::bufferlist> get_attr( + CollectionRef c, + const ghobject_t& oid, + std::string_view name) const final; + + get_attrs_ertr::future<attrs_t> get_attrs( + CollectionRef c, + const ghobject_t& oid) final; + + read_errorator::future<omap_values_t> omap_get_values( + CollectionRef c, + const ghobject_t& oid, + const omap_keys_t& keys) final; + + /// Retrieves paged set of values > start (if present) + using omap_get_values_ret_bare_t = std::tuple<bool, omap_values_t>; + using omap_get_values_ret_t = read_errorator::future< + omap_get_values_ret_bare_t>; + omap_get_values_ret_t omap_get_values( + CollectionRef c, ///< [in] collection + const ghobject_t &oid, ///< [in] oid + const std::optional<std::string> &start ///< [in] start, empty for begin + ) final; ///< @return <done, values> values.empty() iff done + + get_attr_errorator::future<bufferlist> omap_get_header( + CollectionRef c, + const ghobject_t& oid) final; + + seastar::future<std::tuple<std::vector<ghobject_t>, ghobject_t>> list_objects( + CollectionRef c, + const ghobject_t& start, + const ghobject_t& end, + uint64_t limit) const final; + + seastar::future<CollectionRef> create_new_collection(const coll_t& cid) final; + seastar::future<CollectionRef> open_collection(const coll_t& cid) final; + + seastar::future<> do_transaction_no_callbacks( + CollectionRef ch, + ceph::os::Transaction&& txn) final; + + /* Note, flush() machinery must go through the same pipeline + * stages and locks as do_transaction. */ + seastar::future<> flush(CollectionRef ch) final; + + read_errorator::future<std::map<uint64_t, uint64_t>> fiemap( + CollectionRef ch, + const ghobject_t& oid, + uint64_t off, + uint64_t len) final; + + unsigned get_max_attr_name_length() const final { + return 256; + } + + // only exposed to SeaStore + public: + seastar::future<> umount(); + // init managers and mount transaction_manager + seastar::future<> mount_managers(); + + void set_secondaries(Device& sec_dev) { + secondaries.emplace_back(&sec_dev); + } + + using coll_core_t = FuturizedStore::coll_core_t; + seastar::future<std::vector<coll_core_t>> list_collections(); + + seastar::future<> write_meta(const std::string& key, + const std::string& value); + + store_statfs_t stat() const; + + uuid_d get_fsid() const; + + seastar::future<> mkfs_managers(); + + void init_managers(); + + private: + struct internal_context_t { + CollectionRef ch; + ceph::os::Transaction ext_transaction; + + internal_context_t( + CollectionRef ch, + ceph::os::Transaction &&_ext_transaction, + TransactionRef &&transaction) + : ch(ch), ext_transaction(std::move(_ext_transaction)), + transaction(std::move(transaction)), + iter(ext_transaction.begin()) {} + + TransactionRef transaction; + + ceph::os::Transaction::iterator iter; + std::chrono::steady_clock::time_point begin_timestamp = std::chrono::steady_clock::now(); + + void reset_preserve_handle(TransactionManager &tm) { + tm.reset_transaction_preserve_handle(*transaction); + iter = ext_transaction.begin(); + } + }; + + TransactionManager::read_extent_iertr::future<std::optional<unsigned>> + get_coll_bits(CollectionRef ch, Transaction &t) const; + + static void on_error(ceph::os::Transaction &t); + + template <typename F> + auto repeat_with_internal_context( + CollectionRef ch, + ceph::os::Transaction &&t, + Transaction::src_t src, + const char* tname, + op_type_t op_type, + F &&f) { + return seastar::do_with( + internal_context_t( + ch, std::move(t), + transaction_manager->create_transaction(src, tname)), + std::forward<F>(f), + [this, op_type](auto &ctx, auto &f) { + return ctx.transaction->get_handle().take_collection_lock( + static_cast<SeastoreCollection&>(*(ctx.ch)).ordering_lock + ).then([this] { + return throttler.get(1); + }).then([&, this] { + return repeat_eagain([&, this] { + ctx.reset_preserve_handle(*transaction_manager); + return std::invoke(f, ctx); + }).handle_error( + crimson::ct_error::eagain::pass_further{}, + crimson::ct_error::all_same_way([&ctx](auto e) { + on_error(ctx.ext_transaction); + }) + ); + }).then([this, op_type, &ctx] { + add_latency_sample(op_type, + std::chrono::steady_clock::now() - ctx.begin_timestamp); + }).finally([this] { + throttler.put(); + }); + }); + } + + template <typename Ret, typename F> + auto repeat_with_onode( + CollectionRef ch, + const ghobject_t &oid, + Transaction::src_t src, + const char* tname, + op_type_t op_type, + F &&f) const { + auto begin_time = std::chrono::steady_clock::now(); + return seastar::do_with( + oid, Ret{}, std::forward<F>(f), + [this, src, op_type, begin_time, tname + ](auto &oid, auto &ret, auto &f) + { + return repeat_eagain([&, this, src, tname] { + return transaction_manager->with_transaction_intr( + src, + tname, + [&, this](auto& t) + { + return onode_manager->get_onode(t, oid + ).si_then([&](auto onode) { + return seastar::do_with(std::move(onode), [&](auto& onode) { + return f(t, *onode); + }); + }).si_then([&ret](auto _ret) { + ret = _ret; + }); + }); + }).safe_then([&ret, op_type, begin_time, this] { + const_cast<Shard*>(this)->add_latency_sample(op_type, + std::chrono::steady_clock::now() - begin_time); + return seastar::make_ready_future<Ret>(ret); + }); + }); + } + + using _fiemap_ret = ObjectDataHandler::fiemap_ret; + _fiemap_ret _fiemap( + Transaction &t, + Onode &onode, + uint64_t off, + uint64_t len) const; + + using _omap_get_value_iertr = OMapManager::base_iertr::extend< + crimson::ct_error::enodata + >; + using _omap_get_value_ret = _omap_get_value_iertr::future<ceph::bufferlist>; + _omap_get_value_ret _omap_get_value( + Transaction &t, + omap_root_t &&root, + std::string_view key) const; + + using _omap_get_values_iertr = OMapManager::base_iertr; + using _omap_get_values_ret = _omap_get_values_iertr::future<omap_values_t>; + _omap_get_values_ret _omap_get_values( + Transaction &t, + omap_root_t &&root, + const omap_keys_t &keys) const; + + friend class SeaStoreOmapIterator; + + using omap_list_bare_ret = OMapManager::omap_list_bare_ret; + using omap_list_ret = OMapManager::omap_list_ret; + omap_list_ret omap_list( + Onode &onode, + const omap_root_le_t& omap_root, + Transaction& t, + const std::optional<std::string>& start, + OMapManager::omap_list_config_t config) const; + + using tm_iertr = TransactionManager::base_iertr; + using tm_ret = tm_iertr::future<>; + tm_ret _do_transaction_step( + internal_context_t &ctx, + CollectionRef &col, + std::vector<OnodeRef> &onodes, + std::vector<OnodeRef> &d_onodes, + ceph::os::Transaction::iterator &i); + + tm_ret _remove( + internal_context_t &ctx, + OnodeRef &onode); + tm_ret _touch( + internal_context_t &ctx, + OnodeRef &onode); + tm_ret _write( + internal_context_t &ctx, + OnodeRef &onode, + uint64_t offset, size_t len, + ceph::bufferlist &&bl, + uint32_t fadvise_flags); + tm_ret _clone( + internal_context_t &ctx, + OnodeRef &onode, + OnodeRef &d_onode); + tm_ret _zero( + internal_context_t &ctx, + OnodeRef &onode, + objaddr_t offset, extent_len_t len); + tm_ret _omap_set_values( + internal_context_t &ctx, + OnodeRef &onode, + std::map<std::string, ceph::bufferlist> &&aset); + tm_ret _omap_set_header( + internal_context_t &ctx, + OnodeRef &onode, + ceph::bufferlist &&header); + tm_ret _omap_clear( + internal_context_t &ctx, + OnodeRef &onode); + tm_ret _omap_rmkeys( + internal_context_t &ctx, + OnodeRef &onode, + omap_keys_t &&aset); + tm_ret _omap_rmkeyrange( + internal_context_t &ctx, + OnodeRef &onode, + std::string first, + std::string last); + tm_ret _truncate( + internal_context_t &ctx, + OnodeRef &onode, uint64_t size); + tm_ret _setattrs( + internal_context_t &ctx, + OnodeRef &onode, + std::map<std::string,bufferlist>&& aset); + tm_ret _rmattr( + internal_context_t &ctx, + OnodeRef &onode, + std::string name); + tm_ret _rmattrs( + internal_context_t &ctx, + OnodeRef &onode); + tm_ret _xattr_rmattr( + internal_context_t &ctx, + OnodeRef &onode, + std::string &&name); + tm_ret _xattr_clear( + internal_context_t &ctx, + OnodeRef &onode); + tm_ret _create_collection( + internal_context_t &ctx, + const coll_t& cid, int bits); + tm_ret _remove_collection( + internal_context_t &ctx, + const coll_t& cid); + using omap_set_kvs_ret = tm_iertr::future<>; + omap_set_kvs_ret _omap_set_kvs( + OnodeRef &onode, + const omap_root_le_t& omap_root, + Transaction& t, + omap_root_le_t& mutable_omap_root, + std::map<std::string, ceph::bufferlist>&& kvs); + + boost::intrusive_ptr<SeastoreCollection> _get_collection(const coll_t& cid); + + static constexpr auto LAT_MAX = static_cast<std::size_t>(op_type_t::MAX); + + struct { + std::array<seastar::metrics::histogram, LAT_MAX> op_lat; + } stats; + + seastar::metrics::histogram& get_latency( + op_type_t op_type) { + assert(static_cast<std::size_t>(op_type) < stats.op_lat.size()); + return stats.op_lat[static_cast<std::size_t>(op_type)]; + } + + void add_latency_sample(op_type_t op_type, + std::chrono::steady_clock::duration dur) { + seastar::metrics::histogram& lat = get_latency(op_type); + lat.sample_count++; + lat.sample_sum += std::chrono::duration_cast<std::chrono::milliseconds>(dur).count(); + } + + private: + std::string root; + Device* device; + const uint32_t max_object_size; + bool is_test; + + std::vector<Device*> secondaries; + TransactionManagerRef transaction_manager; + CollectionManagerRef collection_manager; + OnodeManagerRef onode_manager; + + common::Throttle throttler; + + seastar::metrics::metric_group metrics; + void register_metrics(); + }; + +public: + SeaStore( + const std::string& root, + MDStoreRef mdstore); + ~SeaStore(); + + seastar::future<> start() final; + seastar::future<> stop() final; + + mount_ertr::future<> mount() final; + seastar::future<> umount() final; + + mkfs_ertr::future<> mkfs(uuid_d new_osd_fsid) final; + seastar::future<store_statfs_t> stat() const final; + + uuid_d get_fsid() const final { + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.local().get_fsid(); + } + + seastar::future<> write_meta( + const std::string& key, + const std::string& value) final { + ceph_assert(seastar::this_shard_id() == primary_core); + return shard_stores.local().write_meta( + key, value).then([this, key, value] { + return mdstore->write_meta(key, value); + }).handle_error( + crimson::ct_error::assert_all{"Invalid error in SeaStore::write_meta"} + ); + } + + seastar::future<std::tuple<int, std::string>> read_meta(const std::string& key) final; + + seastar::future<std::vector<coll_core_t>> list_collections() final; + + FuturizedStore::Shard& get_sharded_store() final { + return shard_stores.local(); + } + + static col_obj_ranges_t + get_objs_range(CollectionRef ch, unsigned bits); + +// for test +public: + mount_ertr::future<> test_mount(); + mkfs_ertr::future<> test_mkfs(uuid_d new_osd_fsid); + + DeviceRef get_primary_device_ref() { + return std::move(device); + } + + seastar::future<> test_start(DeviceRef dev); + +private: + seastar::future<> write_fsid(uuid_d new_osd_fsid); + + seastar::future<> prepare_meta(uuid_d new_osd_fsid); + + seastar::future<> set_secondaries(); + +private: + std::string root; + MDStoreRef mdstore; + DeviceRef device; + std::vector<DeviceRef> secondaries; + seastar::sharded<SeaStore::Shard> shard_stores; +}; + +std::unique_ptr<SeaStore> make_seastore( + const std::string &device); + +std::unique_ptr<SeaStore> make_test_seastore( + SeaStore::MDStoreRef mdstore); +} diff --git a/src/crimson/os/seastore/seastore_types.cc b/src/crimson/os/seastore/seastore_types.cc new file mode 100644 index 000000000..0acfdb74e --- /dev/null +++ b/src/crimson/os/seastore/seastore_types.cc @@ -0,0 +1,874 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/common/log.h" + +namespace { + +seastar::logger& journal_logger() { + return crimson::get_logger(ceph_subsys_seastore_journal); +} + +} + +namespace crimson::os::seastore { + +bool is_aligned(uint64_t offset, uint64_t alignment) +{ + return (offset % alignment) == 0; +} + +std::ostream& operator<<(std::ostream &out, const omap_root_t &root) +{ + return out << "omap_root{addr=" << root.addr + << ", depth=" << root.depth + << ", hint=" << root.hint + << ", mutated=" << root.mutated + << "}"; +} + +std::ostream& operator<<(std::ostream& out, const seastore_meta_t& meta) +{ + return out << meta.seastore_id; +} + +std::ostream &operator<<(std::ostream &out, const device_id_printer_t &id) +{ + auto _id = id.id; + if (_id == DEVICE_ID_NULL) { + return out << "Dev(NULL)"; + } else if (_id == DEVICE_ID_RECORD_RELATIVE) { + return out << "Dev(RR)"; + } else if (_id == DEVICE_ID_BLOCK_RELATIVE) { + return out << "Dev(BR)"; + } else if (_id == DEVICE_ID_DELAYED) { + return out << "Dev(DELAYED)"; + } else if (_id == DEVICE_ID_FAKE) { + return out << "Dev(FAKE)"; + } else if (_id == DEVICE_ID_ZERO) { + return out << "Dev(ZERO)"; + } else if (_id == DEVICE_ID_ROOT) { + return out << "Dev(ROOT)"; + } else { + return out << "Dev(" << (unsigned)_id << ")"; + } +} + +std::ostream &operator<<(std::ostream &out, const segment_id_t &segment) +{ + if (segment == NULL_SEG_ID) { + return out << "Seg[NULL]"; + } else { + return out << "Seg[" << device_id_printer_t{segment.device_id()} + << "," << segment.device_segment_id() + << "]"; + } +} + +std::ostream& operator<<(std::ostream& out, segment_type_t t) +{ + switch(t) { + case segment_type_t::JOURNAL: + return out << "JOURNAL"; + case segment_type_t::OOL: + return out << "OOL"; + case segment_type_t::NULL_SEG: + return out << "NULL_SEG"; + default: + return out << "INVALID_SEGMENT_TYPE!"; + } +} + +std::ostream& operator<<(std::ostream& out, segment_seq_printer_t seq) +{ + if (seq.seq == NULL_SEG_SEQ) { + return out << "sseq(NULL)"; + } else { + return out << "sseq(" << seq.seq << ")"; + } +} + +std::ostream &operator<<(std::ostream &out, const pladdr_t &pladdr) +{ + if (pladdr.is_laddr()) { + return out << pladdr.get_laddr(); + } else { + return out << pladdr.get_paddr(); + } +} + +std::ostream &operator<<(std::ostream &out, const paddr_t &rhs) +{ + auto id = rhs.get_device_id(); + out << "paddr<"; + if (rhs == P_ADDR_NULL) { + out << "NULL"; + } else if (rhs == P_ADDR_MIN) { + out << "MIN"; + } else if (rhs == P_ADDR_ZERO) { + out << "ZERO"; + } else if (has_device_off(id)) { + auto &s = rhs.as_res_paddr(); + out << device_id_printer_t{id} + << "," + << s.get_device_off(); + } else if (rhs.get_addr_type() == paddr_types_t::SEGMENT) { + auto &s = rhs.as_seg_paddr(); + out << s.get_segment_id() + << "," + << s.get_segment_off(); + } else if (rhs.get_addr_type() == paddr_types_t::RANDOM_BLOCK) { + auto &s = rhs.as_blk_paddr(); + out << device_id_printer_t{s.get_device_id()} + << "," + << s.get_device_off(); + } else { + out << "INVALID!"; + } + return out << ">"; +} + +journal_seq_t journal_seq_t::add_offset( + journal_type_t type, + device_off_t off, + device_off_t roll_start, + device_off_t roll_size) const +{ + assert(offset.is_absolute()); + assert(off <= DEVICE_OFF_MAX && off >= DEVICE_OFF_MIN); + assert(roll_start >= 0); + assert(roll_size > 0); + + segment_seq_t jseq = segment_seq; + device_off_t joff; + if (type == journal_type_t::SEGMENTED) { + joff = offset.as_seg_paddr().get_segment_off(); + } else { + assert(type == journal_type_t::RANDOM_BLOCK); + auto boff = offset.as_blk_paddr().get_device_off(); + joff = boff; + } + auto roll_end = roll_start + roll_size; + assert(joff >= roll_start); + assert(joff <= roll_end); + + if (off >= 0) { + device_off_t new_jseq = jseq + (off / roll_size); + joff += (off % roll_size); + if (joff >= roll_end) { + ++new_jseq; + joff -= roll_size; + } + assert(new_jseq < MAX_SEG_SEQ); + jseq = static_cast<segment_seq_t>(new_jseq); + } else { + device_off_t mod = (-off) / roll_size; + joff -= ((-off) % roll_size); + if (joff < roll_start) { + ++mod; + joff += roll_size; + } + if (jseq >= mod) { + jseq -= mod; + } else { + return JOURNAL_SEQ_MIN; + } + } + assert(joff >= roll_start); + assert(joff < roll_end); + return journal_seq_t{jseq, make_block_relative_paddr(joff)}; +} + +device_off_t journal_seq_t::relative_to( + journal_type_t type, + const journal_seq_t& r, + device_off_t roll_start, + device_off_t roll_size) const +{ + assert(offset.is_absolute()); + assert(r.offset.is_absolute()); + assert(roll_start >= 0); + assert(roll_size > 0); + + device_off_t ret = static_cast<device_off_t>(segment_seq) - r.segment_seq; + ret *= roll_size; + if (type == journal_type_t::SEGMENTED) { + ret += (static_cast<device_off_t>(offset.as_seg_paddr().get_segment_off()) - + static_cast<device_off_t>(r.offset.as_seg_paddr().get_segment_off())); + } else { + assert(type == journal_type_t::RANDOM_BLOCK); + ret += offset.as_blk_paddr().get_device_off() - + r.offset.as_blk_paddr().get_device_off(); + } + assert(ret <= DEVICE_OFF_MAX && ret >= DEVICE_OFF_MIN); + return ret; +} + +std::ostream &operator<<(std::ostream &out, const journal_seq_t &seq) +{ + if (seq == JOURNAL_SEQ_NULL) { + return out << "JOURNAL_SEQ_NULL"; + } else if (seq == JOURNAL_SEQ_MIN) { + return out << "JOURNAL_SEQ_MIN"; + } else { + return out << "jseq(" + << segment_seq_printer_t{seq.segment_seq} + << ", " << seq.offset + << ")"; + } +} + +std::ostream &operator<<(std::ostream &out, extent_types_t t) +{ + switch (t) { + case extent_types_t::ROOT: + return out << "ROOT"; + case extent_types_t::LADDR_INTERNAL: + return out << "LADDR_INTERNAL"; + case extent_types_t::LADDR_LEAF: + return out << "LADDR_LEAF"; + case extent_types_t::DINK_LADDR_LEAF: + return out << "LADDR_LEAF"; + case extent_types_t::ONODE_BLOCK_STAGED: + return out << "ONODE_BLOCK_STAGED"; + case extent_types_t::OMAP_INNER: + return out << "OMAP_INNER"; + case extent_types_t::OMAP_LEAF: + return out << "OMAP_LEAF"; + case extent_types_t::COLL_BLOCK: + return out << "COLL_BLOCK"; + case extent_types_t::OBJECT_DATA_BLOCK: + return out << "OBJECT_DATA_BLOCK"; + case extent_types_t::RETIRED_PLACEHOLDER: + return out << "RETIRED_PLACEHOLDER"; + case extent_types_t::TEST_BLOCK: + return out << "TEST_BLOCK"; + case extent_types_t::TEST_BLOCK_PHYSICAL: + return out << "TEST_BLOCK_PHYSICAL"; + case extent_types_t::BACKREF_INTERNAL: + return out << "BACKREF_INTERNAL"; + case extent_types_t::BACKREF_LEAF: + return out << "BACKREF_LEAF"; + case extent_types_t::NONE: + return out << "NONE"; + default: + return out << "UNKNOWN"; + } +} + +std::ostream &operator<<(std::ostream &out, rewrite_gen_printer_t gen) +{ + if (gen.gen == NULL_GENERATION) { + return out << "GEN_NULL"; + } else if (gen.gen == INIT_GENERATION) { + return out << "GEN_INIT"; + } else if (gen.gen == INLINE_GENERATION) { + return out << "GEN_INL"; + } else if (gen.gen == OOL_GENERATION) { + return out << "GEN_OOL"; + } else if (gen.gen > REWRITE_GENERATIONS) { + return out << "GEN_INVALID(" << (unsigned)gen.gen << ")!"; + } else { + return out << "GEN(" << (unsigned)gen.gen << ")"; + } +} + +std::ostream &operator<<(std::ostream &out, data_category_t c) +{ + switch (c) { + case data_category_t::METADATA: + return out << "MD"; + case data_category_t::DATA: + return out << "DATA"; + default: + return out << "INVALID_CATEGORY!"; + } +} + +std::ostream &operator<<(std::ostream &out, sea_time_point_printer_t tp) +{ + if (tp.tp == NULL_TIME) { + return out << "tp(NULL)"; + } + auto time = seastar::lowres_system_clock::to_time_t(tp.tp); + char buf[32]; + std::strftime(buf, sizeof(buf), "%Y-%m-%d %H:%M:%S", std::localtime(&time)); + return out << "tp(" << buf << ")"; +} + +std::ostream &operator<<(std::ostream &out, mod_time_point_printer_t tp) { + auto time = mod_to_timepoint(tp.tp); + return out << "mod_" << sea_time_point_printer_t{time}; +} + +std::ostream &operator<<(std::ostream &out, const laddr_list_t &rhs) +{ + bool first = false; + for (auto &i: rhs) { + out << (first ? '[' : ',') << '(' << i.first << ',' << i.second << ')'; + first = true; + } + return out << ']'; +} +std::ostream &operator<<(std::ostream &out, const paddr_list_t &rhs) +{ + bool first = false; + for (auto &i: rhs) { + out << (first ? '[' : ',') << '(' << i.first << ',' << i.second << ')'; + first = true; + } + return out << ']'; +} + +std::ostream &operator<<(std::ostream &out, const delta_info_t &delta) +{ + return out << "delta_info_t(" + << "type: " << delta.type + << ", paddr: " << delta.paddr + << ", laddr: " << delta.laddr + << ", prev_crc: " << delta.prev_crc + << ", final_crc: " << delta.final_crc + << ", length: " << delta.length + << ", pversion: " << delta.pversion + << ", ext_seq: " << delta.ext_seq + << ", seg_type: " << delta.seg_type + << ")"; +} + +std::ostream &operator<<(std::ostream &out, const journal_tail_delta_t &delta) +{ + return out << "journal_tail_delta_t(" + << "alloc_tail=" << delta.alloc_tail + << ", dirty_tail=" << delta.dirty_tail + << ")"; +} + +std::ostream &operator<<(std::ostream &out, const extent_info_t &info) +{ + return out << "extent_info_t(" + << "type: " << info.type + << ", addr: " << info.addr + << ", len: " << info.len + << ")"; +} + +std::ostream &operator<<(std::ostream &out, const segment_header_t &header) +{ + return out << "segment_header_t(" + << header.physical_segment_id + << " " << header.type + << " " << segment_seq_printer_t{header.segment_seq} + << " " << header.category + << " " << rewrite_gen_printer_t{header.generation} + << ", dirty_tail=" << header.dirty_tail + << ", alloc_tail=" << header.alloc_tail + << ", segment_nonce=" << header.segment_nonce + << ")"; +} + +std::ostream &operator<<(std::ostream &out, const segment_tail_t &tail) +{ + return out << "segment_tail_t(" + << tail.physical_segment_id + << " " << tail.type + << " " << segment_seq_printer_t{tail.segment_seq} + << ", segment_nonce=" << tail.segment_nonce + << ", modify_time=" << mod_time_point_printer_t{tail.modify_time} + << ", num_extents=" << tail.num_extents + << ")"; +} + +extent_len_t record_size_t::get_raw_mdlength() const +{ + // empty record is allowed to submit + return plain_mdlength + + ceph::encoded_sizeof_bounded<record_header_t>(); +} + +void record_size_t::account_extent(extent_len_t extent_len) +{ + assert(extent_len); + plain_mdlength += ceph::encoded_sizeof_bounded<extent_info_t>(); + dlength += extent_len; +} + +void record_size_t::account(const delta_info_t& delta) +{ + assert(delta.bl.length()); + plain_mdlength += ceph::encoded_sizeof(delta); +} + +std::ostream &operator<<(std::ostream &os, transaction_type_t type) +{ + switch (type) { + case transaction_type_t::MUTATE: + return os << "MUTATE"; + case transaction_type_t::READ: + return os << "READ"; + case transaction_type_t::TRIM_DIRTY: + return os << "TRIM_DIRTY"; + case transaction_type_t::TRIM_ALLOC: + return os << "TRIM_ALLOC"; + case transaction_type_t::CLEANER_MAIN: + return os << "CLEANER_MAIN"; + case transaction_type_t::CLEANER_COLD: + return os << "CLEANER_COLD"; + case transaction_type_t::MAX: + return os << "TRANS_TYPE_NULL"; + default: + return os << "INVALID_TRANS_TYPE(" + << static_cast<std::size_t>(type) + << ")"; + } +} + +std::ostream &operator<<(std::ostream& out, const record_size_t& rsize) +{ + return out << "record_size_t(" + << "raw_md=" << rsize.get_raw_mdlength() + << ", data=" << rsize.dlength + << ")"; +} + +std::ostream &operator<<(std::ostream& out, const record_t& r) +{ + return out << "record_t(" + << "type=" << r.type + << ", num_extents=" << r.extents.size() + << ", num_deltas=" << r.deltas.size() + << ", modify_time=" << sea_time_point_printer_t{r.modify_time} + << ")"; +} + +std::ostream &operator<<(std::ostream& out, const record_header_t& r) +{ + return out << "record_header_t(" + << "type=" << r.type + << ", num_extents=" << r.extents + << ", num_deltas=" << r.deltas + << ", modify_time=" << mod_time_point_printer_t{r.modify_time} + << ")"; +} + +std::ostream& operator<<(std::ostream& out, const record_group_header_t& h) +{ + return out << "record_group_header_t(" + << "num_records=" << h.records + << ", mdlength=" << h.mdlength + << ", dlength=" << h.dlength + << ", nonce=" << h.segment_nonce + << ", committed_to=" << h.committed_to + << ", data_crc=" << h.data_crc + << ")"; +} + +extent_len_t record_group_size_t::get_raw_mdlength() const +{ + return plain_mdlength + + sizeof(checksum_t) + + ceph::encoded_sizeof_bounded<record_group_header_t>(); +} + +void record_group_size_t::account( + const record_size_t& rsize, + extent_len_t _block_size) +{ + // empty record is allowed to submit + assert(_block_size > 0); + assert(rsize.dlength % _block_size == 0); + assert(block_size == 0 || block_size == _block_size); + plain_mdlength += rsize.get_raw_mdlength(); + dlength += rsize.dlength; + block_size = _block_size; +} + +std::ostream& operator<<(std::ostream& out, const record_group_size_t& size) +{ + return out << "record_group_size_t(" + << "raw_md=" << size.get_raw_mdlength() + << ", data=" << size.dlength + << ", block_size=" << size.block_size + << ", fullness=" << size.get_fullness() + << ")"; +} + +std::ostream& operator<<(std::ostream& out, const record_group_t& rg) +{ + return out << "record_group_t(" + << "num_records=" << rg.records.size() + << ", " << rg.size + << ")"; +} + +ceph::bufferlist encode_record( + record_t&& record, + extent_len_t block_size, + const journal_seq_t& committed_to, + segment_nonce_t current_segment_nonce) +{ + record_group_t record_group(std::move(record), block_size); + return encode_records( + record_group, + committed_to, + current_segment_nonce); +} + +ceph::bufferlist encode_records( + record_group_t& record_group, + const journal_seq_t& committed_to, + segment_nonce_t current_segment_nonce) +{ + assert(record_group.size.block_size > 0); + assert(record_group.records.size() > 0); + + bufferlist data_bl; + for (auto& r: record_group.records) { + for (auto& i: r.extents) { + assert(i.bl.length()); + data_bl.append(i.bl); + } + } + + bufferlist bl; + record_group_header_t header{ + static_cast<extent_len_t>(record_group.records.size()), + record_group.size.get_mdlength(), + record_group.size.dlength, + current_segment_nonce, + committed_to, + data_bl.crc32c(-1) + }; + encode(header, bl); + + auto metadata_crc_filler = bl.append_hole(sizeof(checksum_t)); + + for (auto& r: record_group.records) { + record_header_t rheader{ + r.type, + (extent_len_t)r.deltas.size(), + (extent_len_t)r.extents.size(), + timepoint_to_mod(r.modify_time) + }; + encode(rheader, bl); + } + for (auto& r: record_group.records) { + for (const auto& i: r.extents) { + encode(extent_info_t(i), bl); + } + } + for (auto& r: record_group.records) { + for (const auto& i: r.deltas) { + encode(i, bl); + } + } + ceph_assert(bl.length() == record_group.size.get_raw_mdlength()); + + auto aligned_mdlength = record_group.size.get_mdlength(); + if (bl.length() != aligned_mdlength) { + assert(bl.length() < aligned_mdlength); + bl.append_zero(aligned_mdlength - bl.length()); + } + + auto bliter = bl.cbegin(); + auto metadata_crc = bliter.crc32c( + ceph::encoded_sizeof_bounded<record_group_header_t>(), + -1); + bliter += sizeof(checksum_t); /* metadata crc hole */ + metadata_crc = bliter.crc32c( + bliter.get_remaining(), + metadata_crc); + ceph_le32 metadata_crc_le; + metadata_crc_le = metadata_crc; + metadata_crc_filler.copy_in( + sizeof(checksum_t), + reinterpret_cast<const char *>(&metadata_crc_le)); + + bl.claim_append(data_bl); + ceph_assert(bl.length() == record_group.size.get_encoded_length()); + + record_group.clear(); + return bl; +} + +std::optional<record_group_header_t> +try_decode_records_header( + const ceph::bufferlist& header_bl, + segment_nonce_t expected_nonce) +{ + auto bp = header_bl.cbegin(); + record_group_header_t header; + try { + decode(header, bp); + } catch (ceph::buffer::error &e) { + journal_logger().debug( + "try_decode_records_header: failed, " + "cannot decode record_group_header_t, got {}.", + e.what()); + return std::nullopt; + } + if (header.segment_nonce != expected_nonce) { + journal_logger().debug( + "try_decode_records_header: failed, record_group_header nonce mismatch, " + "read {}, expected {}!", + header.segment_nonce, + expected_nonce); + return std::nullopt; + } + return header; +} + +bool validate_records_metadata( + const ceph::bufferlist& md_bl) +{ + auto bliter = md_bl.cbegin(); + auto test_crc = bliter.crc32c( + ceph::encoded_sizeof_bounded<record_group_header_t>(), + -1); + ceph_le32 recorded_crc_le; + decode(recorded_crc_le, bliter); + uint32_t recorded_crc = recorded_crc_le; + test_crc = bliter.crc32c( + bliter.get_remaining(), + test_crc); + bool success = (test_crc == recorded_crc); + if (!success) { + journal_logger().debug( + "validate_records_metadata: failed, metadata crc mismatch."); + } + return success; +} + +bool validate_records_data( + const record_group_header_t& header, + const ceph::bufferlist& data_bl) +{ + bool success = (data_bl.crc32c(-1) == header.data_crc); + if (!success) { + journal_logger().debug( + "validate_records_data: failed, data crc mismatch!"); + } + return success; +} + +std::optional<std::vector<record_header_t>> +try_decode_record_headers( + const record_group_header_t& header, + const ceph::bufferlist& md_bl) +{ + auto bliter = md_bl.cbegin(); + bliter += ceph::encoded_sizeof_bounded<record_group_header_t>(); + bliter += sizeof(checksum_t); /* metadata crc hole */ + std::vector<record_header_t> record_headers(header.records); + for (auto &&i: record_headers) { + try { + decode(i, bliter); + } catch (ceph::buffer::error &e) { + journal_logger().debug( + "try_decode_record_headers: failed, " + "cannot decode record_header_t, got {}.", + e.what()); + return std::nullopt; + } + } + return record_headers; +} + +std::optional<std::vector<record_extent_infos_t> > +try_decode_extent_infos( + const record_group_header_t& header, + const ceph::bufferlist& md_bl) +{ + auto maybe_headers = try_decode_record_headers(header, md_bl); + if (!maybe_headers) { + return std::nullopt; + } + + auto bliter = md_bl.cbegin(); + bliter += ceph::encoded_sizeof_bounded<record_group_header_t>(); + bliter += sizeof(checksum_t); /* metadata crc hole */ + bliter += (ceph::encoded_sizeof_bounded<record_header_t>() * + maybe_headers->size()); + + std::vector<record_extent_infos_t> record_extent_infos( + maybe_headers->size()); + auto result_iter = record_extent_infos.begin(); + for (auto& h: *maybe_headers) { + result_iter->header = h; + result_iter->extent_infos.resize(h.extents); + for (auto& i: result_iter->extent_infos) { + try { + decode(i, bliter); + } catch (ceph::buffer::error &e) { + journal_logger().debug( + "try_decode_extent_infos: failed, " + "cannot decode extent_info_t, got {}.", + e.what()); + return std::nullopt; + } + } + ++result_iter; + } + return record_extent_infos; +} + +std::optional<std::vector<record_deltas_t> > +try_decode_deltas( + const record_group_header_t& header, + const ceph::bufferlist& md_bl, + paddr_t record_block_base) +{ + auto maybe_record_extent_infos = try_decode_extent_infos(header, md_bl); + if (!maybe_record_extent_infos) { + return std::nullopt; + } + + auto bliter = md_bl.cbegin(); + bliter += ceph::encoded_sizeof_bounded<record_group_header_t>(); + bliter += sizeof(checksum_t); /* metadata crc hole */ + bliter += (ceph::encoded_sizeof_bounded<record_header_t>() * + maybe_record_extent_infos->size()); + for (auto& r: *maybe_record_extent_infos) { + bliter += (ceph::encoded_sizeof_bounded<extent_info_t>() * + r.extent_infos.size()); + } + + std::vector<record_deltas_t> record_deltas( + maybe_record_extent_infos->size()); + auto result_iter = record_deltas.begin(); + for (auto& r: *maybe_record_extent_infos) { + result_iter->record_block_base = record_block_base; + result_iter->deltas.resize(r.header.deltas); + for (auto& i: result_iter->deltas) { + try { + decode(i.second, bliter); + i.first = mod_to_timepoint(r.header.modify_time); + } catch (ceph::buffer::error &e) { + journal_logger().debug( + "try_decode_deltas: failed, " + "cannot decode delta_info_t, got {}.", + e.what()); + return std::nullopt; + } + } + for (auto& i: r.extent_infos) { + record_block_base = record_block_base.add_offset(i.len); + } + ++result_iter; + } + return record_deltas; +} + +std::ostream& operator<<(std::ostream& out, placement_hint_t h) +{ + switch (h) { + case placement_hint_t::HOT: + return out << "Hint(HOT)"; + case placement_hint_t::COLD: + return out << "Hint(COLD)"; + case placement_hint_t::REWRITE: + return out << "Hint(REWRITE)"; + case PLACEMENT_HINT_NULL: + return out << "Hint(NULL)"; + default: + return out << "INVALID_PLACEMENT_HINT_TYPE!"; + } +} + +bool can_delay_allocation(device_type_t type) { + // Some types of device may not support delayed allocation, for example PMEM. + // All types of device currently support delayed allocation. + return true; +} + +device_type_t string_to_device_type(std::string type) { + if (type == "HDD") { + return device_type_t::HDD; + } + if (type == "SSD") { + return device_type_t::SSD; + } + if (type == "ZBD") { + return device_type_t::ZBD; + } + if (type == "RANDOM_BLOCK_SSD") { + return device_type_t::RANDOM_BLOCK_SSD; + } + return device_type_t::NONE; +} + +std::ostream& operator<<(std::ostream& out, device_type_t t) +{ + switch (t) { + case device_type_t::NONE: + return out << "NONE"; + case device_type_t::HDD: + return out << "HDD"; + case device_type_t::SSD: + return out << "SSD"; + case device_type_t::ZBD: + return out << "ZBD"; + case device_type_t::EPHEMERAL_COLD: + return out << "EPHEMERAL_COLD"; + case device_type_t::EPHEMERAL_MAIN: + return out << "EPHEMERAL_MAIN"; + case device_type_t::RANDOM_BLOCK_SSD: + return out << "RANDOM_BLOCK_SSD"; + case device_type_t::RANDOM_BLOCK_EPHEMERAL: + return out << "RANDOM_BLOCK_EPHEMERAL"; + default: + return out << "INVALID_DEVICE_TYPE!"; + } +} + +std::ostream& operator<<(std::ostream& out, backend_type_t btype) { + if (btype == backend_type_t::SEGMENTED) { + return out << "SEGMENTED"; + } else { + return out << "RANDOM_BLOCK"; + } +} + +std::ostream& operator<<(std::ostream& out, const write_result_t& w) +{ + return out << "write_result_t(" + << "start=" << w.start_seq + << ", length=" << w.length + << ")"; +} + +std::ostream& operator<<(std::ostream& out, const record_locator_t& l) +{ + return out << "record_locator_t(" + << "block_base=" << l.record_block_base + << ", " << l.write_result + << ")"; +} + +void scan_valid_records_cursor::emplace_record_group( + const record_group_header_t& header, ceph::bufferlist&& md_bl) +{ + auto new_committed_to = header.committed_to; + ceph_assert(last_committed == JOURNAL_SEQ_NULL || + last_committed <= new_committed_to); + last_committed = new_committed_to; + pending_record_groups.emplace_back( + seq.offset, + header, + std::move(md_bl)); + increment_seq(header.dlength + header.mdlength); + ceph_assert(new_committed_to == JOURNAL_SEQ_NULL || + new_committed_to < seq); +} + +std::ostream& operator<<(std::ostream& out, const scan_valid_records_cursor& c) +{ + return out << "cursor(last_valid_header_found=" << c.last_valid_header_found + << ", seq=" << c.seq + << ", last_committed=" << c.last_committed + << ", pending_record_groups=" << c.pending_record_groups.size() + << ", num_consumed_records=" << c.num_consumed_records + << ")"; +} + +} diff --git a/src/crimson/os/seastore/seastore_types.h b/src/crimson/os/seastore/seastore_types.h new file mode 100644 index 000000000..0b4ad8536 --- /dev/null +++ b/src/crimson/os/seastore/seastore_types.h @@ -0,0 +1,2254 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <limits> +#include <numeric> +#include <optional> +#include <iostream> +#include <vector> +#include <boost/core/ignore_unused.hpp> + +#include <seastar/core/lowres_clock.hh> + +#include "include/byteorder.h" +#include "include/denc.h" +#include "include/buffer.h" +#include "include/intarith.h" +#include "include/interval_set.h" +#include "include/uuid.h" + +namespace crimson::os::seastore { + +/* using a special xattr key "omap_header" to store omap header */ + const std::string OMAP_HEADER_XATTR_KEY = "omap_header"; + +using transaction_id_t = uint64_t; +constexpr transaction_id_t TRANS_ID_NULL = 0; + +/* + * Note: NULL value is usually the default and max value. + */ + +using depth_t = uint32_t; +using depth_le_t = ceph_le32; + +inline depth_le_t init_depth_le(uint32_t i) { + return ceph_le32(i); +} + +using checksum_t = uint32_t; + +// Immutable metadata for seastore to set at mkfs time +struct seastore_meta_t { + uuid_d seastore_id; + + DENC(seastore_meta_t, v, p) { + DENC_START(1, 1, p); + denc(v.seastore_id, p); + DENC_FINISH(p); + } +}; + +std::ostream& operator<<(std::ostream& out, const seastore_meta_t& meta); + +bool is_aligned(uint64_t offset, uint64_t alignment); + +// identifies a specific physical device within seastore +using device_id_t = uint8_t; + +constexpr auto DEVICE_ID_BITS = std::numeric_limits<device_id_t>::digits; + +constexpr device_id_t DEVICE_ID_MAX = std::numeric_limits<device_id_t>::max(); +constexpr device_id_t DEVICE_ID_NULL = DEVICE_ID_MAX; +constexpr device_id_t DEVICE_ID_RECORD_RELATIVE = DEVICE_ID_MAX - 1; +constexpr device_id_t DEVICE_ID_BLOCK_RELATIVE = DEVICE_ID_MAX - 2; +constexpr device_id_t DEVICE_ID_DELAYED = DEVICE_ID_MAX - 3; +// for tests which generate fake paddrs +constexpr device_id_t DEVICE_ID_FAKE = DEVICE_ID_MAX - 4; +constexpr device_id_t DEVICE_ID_ZERO = DEVICE_ID_MAX - 5; +constexpr device_id_t DEVICE_ID_ROOT = DEVICE_ID_MAX - 6; +constexpr device_id_t DEVICE_ID_MAX_VALID = DEVICE_ID_MAX - 7; +constexpr device_id_t DEVICE_ID_MAX_VALID_SEGMENT = DEVICE_ID_MAX >> 1; +constexpr device_id_t DEVICE_ID_SEGMENTED_MIN = 0; +constexpr device_id_t DEVICE_ID_RANDOM_BLOCK_MIN = + 1 << (std::numeric_limits<device_id_t>::digits - 1); + +struct device_id_printer_t { + device_id_t id; +}; + +std::ostream &operator<<(std::ostream &out, const device_id_printer_t &id); + +// 1 bit in paddr_t to identify the absolute physical address type +enum class paddr_types_t { + SEGMENT = 0, + RANDOM_BLOCK = 1, + RESERVED = 2 +}; + +constexpr paddr_types_t device_id_to_paddr_type(device_id_t id) { + if (id > DEVICE_ID_MAX_VALID) { + return paddr_types_t::RESERVED; + } else if ((id & 0x80) == 0) { + return paddr_types_t::SEGMENT; + } else { + return paddr_types_t::RANDOM_BLOCK; + } +} + +constexpr bool has_device_off(device_id_t id) { + return id == DEVICE_ID_RECORD_RELATIVE || + id == DEVICE_ID_BLOCK_RELATIVE || + id == DEVICE_ID_DELAYED || + id == DEVICE_ID_FAKE || + id == DEVICE_ID_ROOT; +} + +// internal segment id type of segment_id_t below, with the top +// "DEVICE_ID_BITS" bits representing the device id of the segment. +using internal_segment_id_t = uint32_t; +constexpr auto SEGMENT_ID_BITS = std::numeric_limits<internal_segment_id_t>::digits; + +// segment ids without a device id encapsulated +using device_segment_id_t = uint32_t; +constexpr auto DEVICE_SEGMENT_ID_BITS = SEGMENT_ID_BITS - DEVICE_ID_BITS; +constexpr device_segment_id_t DEVICE_SEGMENT_ID_MAX = (1 << DEVICE_SEGMENT_ID_BITS) - 1; + +// Identifies segment location on disk, see SegmentManager, +struct segment_id_t { +public: + // segment_id_t() == MAX_SEG_ID == NULL_SEG_ID + segment_id_t() + : segment_id_t(DEVICE_ID_MAX_VALID_SEGMENT, DEVICE_SEGMENT_ID_MAX) {} + + segment_id_t(device_id_t id, device_segment_id_t _segment) + : segment_id_t(make_internal(id, _segment)) {} + + segment_id_t(internal_segment_id_t _segment) + : segment(_segment) { + assert(device_id_to_paddr_type(device_id()) == paddr_types_t::SEGMENT); + } + + [[gnu::always_inline]] + constexpr device_id_t device_id() const { + return static_cast<device_id_t>(segment >> DEVICE_SEGMENT_ID_BITS); + } + + [[gnu::always_inline]] + constexpr device_segment_id_t device_segment_id() const { + constexpr internal_segment_id_t _SEGMENT_ID_MASK = (1u << DEVICE_SEGMENT_ID_BITS) - 1; + return segment & _SEGMENT_ID_MASK; + } + + bool operator==(const segment_id_t& other) const { + return segment == other.segment; + } + bool operator!=(const segment_id_t& other) const { + return segment != other.segment; + } + bool operator<(const segment_id_t& other) const { + return segment < other.segment; + } + bool operator<=(const segment_id_t& other) const { + return segment <= other.segment; + } + bool operator>(const segment_id_t& other) const { + return segment > other.segment; + } + bool operator>=(const segment_id_t& other) const { + return segment >= other.segment; + } + + DENC(segment_id_t, v, p) { + denc(v.segment, p); + } + + static constexpr segment_id_t create_const( + device_id_t id, device_segment_id_t segment) { + return segment_id_t(id, segment, const_t{}); + } + +private: + struct const_t {}; + constexpr segment_id_t(device_id_t id, device_segment_id_t _segment, const_t) + : segment(make_internal(id, _segment)) {} + + constexpr static inline internal_segment_id_t make_internal( + device_id_t d_id, + device_segment_id_t s_id) { + return static_cast<internal_segment_id_t>(s_id) | + (static_cast<internal_segment_id_t>(d_id) << DEVICE_SEGMENT_ID_BITS); + } + + internal_segment_id_t segment; + + friend struct segment_id_le_t; + friend struct paddr_t; +}; + +std::ostream &operator<<(std::ostream &out, const segment_id_t&); + +// ondisk type of segment_id_t +struct __attribute((packed)) segment_id_le_t { + ceph_le32 segment = ceph_le32(segment_id_t().segment); + + segment_id_le_t(const segment_id_t id) : + segment(ceph_le32(id.segment)) {} + + operator segment_id_t() const { + return segment_id_t(segment); + } +}; + +constexpr segment_id_t MIN_SEG_ID = segment_id_t::create_const(0, 0); +// segment_id_t() == MAX_SEG_ID == NULL_SEG_ID +constexpr segment_id_t MAX_SEG_ID = + segment_id_t::create_const(DEVICE_ID_MAX_VALID_SEGMENT, DEVICE_SEGMENT_ID_MAX); +constexpr segment_id_t NULL_SEG_ID = MAX_SEG_ID; + +/* Monotonically increasing segment seq, uniquely identifies + * the incarnation of a segment */ +using segment_seq_t = uint64_t; +static constexpr segment_seq_t MAX_SEG_SEQ = + std::numeric_limits<segment_seq_t>::max(); +static constexpr segment_seq_t NULL_SEG_SEQ = MAX_SEG_SEQ; + +enum class segment_type_t : uint8_t { + JOURNAL = 0, + OOL, + NULL_SEG, +}; + +std::ostream& operator<<(std::ostream& out, segment_type_t t); + +struct segment_seq_printer_t { + segment_seq_t seq; +}; + +std::ostream& operator<<(std::ostream& out, segment_seq_printer_t seq); + +/** + * segment_map_t + * + * Compact templated mapping from a segment_id_t to a value type. + */ +template <typename T> +class segment_map_t { +public: + segment_map_t() { + // initializes top vector with 0 length vectors to indicate that they + // are not yet present + device_to_segments.resize(DEVICE_ID_MAX_VALID); + } + void add_device(device_id_t device, std::size_t segments, const T& init) { + ceph_assert(device <= DEVICE_ID_MAX_VALID); + ceph_assert(device_to_segments[device].size() == 0); + ceph_assert(segments > 0); + device_to_segments[device].resize(segments, init); + total_segments += segments; + } + void clear() { + device_to_segments.clear(); + device_to_segments.resize(DEVICE_ID_MAX_VALID); + total_segments = 0; + } + + T& operator[](segment_id_t id) { + assert(id.device_segment_id() < device_to_segments[id.device_id()].size()); + return device_to_segments[id.device_id()][id.device_segment_id()]; + } + const T& operator[](segment_id_t id) const { + assert(id.device_segment_id() < device_to_segments[id.device_id()].size()); + return device_to_segments[id.device_id()][id.device_segment_id()]; + } + + bool contains(segment_id_t id) { + bool b = id.device_id() < device_to_segments.size(); + if (!b) { + return b; + } + b = id.device_segment_id() < device_to_segments[id.device_id()].size(); + return b; + } + + auto begin() { + return iterator<false>::lower_bound(*this, 0, 0); + } + auto begin() const { + return iterator<true>::lower_bound(*this, 0, 0); + } + + auto end() { + return iterator<false>::end_iterator(*this); + } + auto end() const { + return iterator<true>::end_iterator(*this); + } + + auto device_begin(device_id_t id) { + auto ret = iterator<false>::lower_bound(*this, id, 0); + assert(ret->first.device_id() == id); + return ret; + } + auto device_end(device_id_t id) { + return iterator<false>::lower_bound(*this, id + 1, 0); + } + + size_t size() const { + return total_segments; + } + +private: + template <bool is_const = false> + class iterator { + /// points at set being iterated over + std::conditional_t< + is_const, + const segment_map_t &, + segment_map_t &> parent; + + /// points at current device, or DEVICE_ID_MAX_VALID if is_end() + device_id_t device_id; + + /// segment at which we are pointing, 0 if is_end() + device_segment_id_t device_segment_id; + + /// holds referent for operator* and operator-> when !is_end() + std::optional< + std::pair< + const segment_id_t, + std::conditional_t<is_const, const T&, T&> + >> current; + + bool is_end() const { + return device_id == DEVICE_ID_MAX_VALID; + } + + void find_valid() { + assert(!is_end()); + auto &device_vec = parent.device_to_segments[device_id]; + if (device_vec.size() == 0 || + device_segment_id == device_vec.size()) { + while (++device_id < DEVICE_ID_MAX_VALID && + parent.device_to_segments[device_id].size() == 0); + device_segment_id = 0; + } + if (is_end()) { + current = std::nullopt; + } else { + current.emplace( + segment_id_t{device_id, device_segment_id}, + parent.device_to_segments[device_id][device_segment_id] + ); + } + } + + iterator( + decltype(parent) &parent, + device_id_t device_id, + device_segment_id_t device_segment_id) + : parent(parent), device_id(device_id), + device_segment_id(device_segment_id) {} + + public: + static iterator lower_bound( + decltype(parent) &parent, + device_id_t device_id, + device_segment_id_t device_segment_id) { + if (device_id == DEVICE_ID_MAX_VALID) { + return end_iterator(parent); + } else { + auto ret = iterator{parent, device_id, device_segment_id}; + ret.find_valid(); + return ret; + } + } + + static iterator end_iterator( + decltype(parent) &parent) { + return iterator{parent, DEVICE_ID_MAX_VALID, 0}; + } + + iterator<is_const>& operator++() { + assert(!is_end()); + ++device_segment_id; + find_valid(); + return *this; + } + + bool operator==(iterator<is_const> rit) { + return (device_id == rit.device_id && + device_segment_id == rit.device_segment_id); + } + + bool operator!=(iterator<is_const> rit) { + return !(*this == rit); + } + + template <bool c = is_const, std::enable_if_t<c, int> = 0> + const std::pair<const segment_id_t, const T&> *operator->() { + assert(!is_end()); + return &*current; + } + template <bool c = is_const, std::enable_if_t<!c, int> = 0> + std::pair<const segment_id_t, T&> *operator->() { + assert(!is_end()); + return &*current; + } + + using reference = std::conditional_t< + is_const, const std::pair<const segment_id_t, const T&>&, + std::pair<const segment_id_t, T&>&>; + reference operator*() { + assert(!is_end()); + return *current; + } + }; + + /** + * device_to_segments + * + * device -> segment -> T mapping. device_to_segments[d].size() > 0 iff + * device <d> has been added. + */ + std::vector<std::vector<T>> device_to_segments; + + /// total number of added segments + size_t total_segments = 0; +}; + +/** + * paddr_t + * + * <segment, offset> offset on disk, see SegmentManager + * + * May be absolute, record_relative, or block_relative. + * + * Blocks get read independently of the surrounding record, + * so paddrs embedded directly within a block need to refer + * to other blocks within the same record by a block_relative + * addr relative to the block's own offset. By contrast, + * deltas to existing blocks need to use record_relative + * addrs relative to the first block of the record. + * + * Fresh extents during a transaction are refered to by + * record_relative paddrs. + */ + +using internal_paddr_t = uint64_t; +constexpr auto PADDR_BITS = std::numeric_limits<internal_paddr_t>::digits; + +/** + * device_off_t + * + * Offset within a device, may be negative for relative offsets. + */ +using device_off_t = int64_t; +using u_device_off_t = uint64_t; +constexpr auto DEVICE_OFF_BITS = PADDR_BITS - DEVICE_ID_BITS; +constexpr auto DEVICE_OFF_MAX = + std::numeric_limits<device_off_t>::max() >> DEVICE_ID_BITS; +constexpr auto DEVICE_OFF_MIN = -(DEVICE_OFF_MAX + 1); + +/** + * segment_off_t + * + * Offset within a segment on disk, may be negative for relative offsets. + */ +using segment_off_t = int32_t; +using u_segment_off_t = uint32_t; +constexpr auto SEGMENT_OFF_MAX = std::numeric_limits<segment_off_t>::max(); +constexpr auto SEGMENT_OFF_MIN = std::numeric_limits<segment_off_t>::min(); +constexpr auto SEGMENT_OFF_BITS = std::numeric_limits<u_segment_off_t>::digits; +static_assert(PADDR_BITS == SEGMENT_ID_BITS + SEGMENT_OFF_BITS); + +constexpr auto DEVICE_ID_MASK = + ((internal_paddr_t(1) << DEVICE_ID_BITS) - 1) << DEVICE_OFF_BITS; +constexpr auto DEVICE_OFF_MASK = + std::numeric_limits<u_device_off_t>::max() >> DEVICE_ID_BITS; +constexpr auto SEGMENT_ID_MASK = + ((internal_paddr_t(1) << SEGMENT_ID_BITS) - 1) << SEGMENT_OFF_BITS; +constexpr auto SEGMENT_OFF_MASK = + (internal_paddr_t(1) << SEGMENT_OFF_BITS) - 1; + +constexpr internal_paddr_t encode_device_off(device_off_t off) { + return static_cast<internal_paddr_t>(off) & DEVICE_OFF_MASK; +} + +constexpr device_off_t decode_device_off(internal_paddr_t addr) { + if (addr & (1ull << (DEVICE_OFF_BITS - 1))) { + return static_cast<device_off_t>(addr | DEVICE_ID_MASK); + } else { + return static_cast<device_off_t>(addr & DEVICE_OFF_MASK); + } +} + +struct seg_paddr_t; +struct blk_paddr_t; +struct res_paddr_t; +struct pladdr_t; +struct paddr_t { +public: + // P_ADDR_MAX == P_ADDR_NULL == paddr_t{} + paddr_t() : paddr_t(DEVICE_ID_MAX, device_off_t(0)) {} + + static paddr_t make_seg_paddr( + segment_id_t seg, + segment_off_t offset) { + return paddr_t(seg, offset); + } + + static paddr_t make_seg_paddr( + device_id_t device, + device_segment_id_t seg, + segment_off_t offset) { + return paddr_t(segment_id_t(device, seg), offset); + } + + static paddr_t make_blk_paddr( + device_id_t device, + device_off_t offset) { + assert(device_id_to_paddr_type(device) == paddr_types_t::RANDOM_BLOCK); + return paddr_t(device, offset); + } + + static paddr_t make_res_paddr( + device_id_t device, + device_off_t offset) { + assert(device_id_to_paddr_type(device) == paddr_types_t::RESERVED); + return paddr_t(device, offset); + } + + void swap(paddr_t &other) { + std::swap(internal_paddr, other.internal_paddr); + } + + device_id_t get_device_id() const { + return static_cast<device_id_t>(internal_paddr >> DEVICE_OFF_BITS); + } + + paddr_types_t get_addr_type() const { + return device_id_to_paddr_type(get_device_id()); + } + + paddr_t add_offset(device_off_t o) const; + + paddr_t add_relative(paddr_t o) const; + + paddr_t add_block_relative(paddr_t o) const { + // special version mainly for documentation purposes + assert(o.is_block_relative()); + return add_relative(o); + } + + paddr_t add_record_relative(paddr_t o) const { + // special version mainly for documentation purposes + assert(o.is_record_relative()); + return add_relative(o); + } + + /** + * maybe_relative_to + * + * Helper for the case where an in-memory paddr_t may be + * either block_relative or absolute (not record_relative). + * + * base must be either absolute or record_relative. + */ + paddr_t maybe_relative_to(paddr_t base) const { + assert(!base.is_block_relative()); + if (is_block_relative()) { + return base.add_block_relative(*this); + } else { + return *this; + } + } + + /** + * block_relative_to + * + * Only defined for record_relative paddr_ts. Yields a + * block_relative address. + */ + paddr_t block_relative_to(paddr_t rhs) const; + + // To be compatible with laddr_t operator+ + paddr_t operator+(device_off_t o) const { + return add_offset(o); + } + + seg_paddr_t& as_seg_paddr(); + const seg_paddr_t& as_seg_paddr() const; + blk_paddr_t& as_blk_paddr(); + const blk_paddr_t& as_blk_paddr() const; + res_paddr_t& as_res_paddr(); + const res_paddr_t& as_res_paddr() const; + + bool is_delayed() const { + return get_device_id() == DEVICE_ID_DELAYED; + } + bool is_block_relative() const { + return get_device_id() == DEVICE_ID_BLOCK_RELATIVE; + } + bool is_record_relative() const { + return get_device_id() == DEVICE_ID_RECORD_RELATIVE; + } + bool is_relative() const { + return is_block_relative() || is_record_relative(); + } + /// Denotes special null addr + bool is_null() const { + return get_device_id() == DEVICE_ID_NULL; + } + /// Denotes special zero addr + bool is_zero() const { + return get_device_id() == DEVICE_ID_ZERO; + } + /// Denotes the root addr + bool is_root() const { + return get_device_id() == DEVICE_ID_ROOT; + } + + /** + * is_real + * + * indicates whether addr reflects a physical location, absolute, relative, + * or delayed. FAKE segments also count as real so as to reflect the way in + * which unit tests use them. + */ + bool is_real() const { + return !is_zero() && !is_null() && !is_root(); + } + + bool is_absolute() const { + return get_addr_type() != paddr_types_t::RESERVED; + } + + bool is_fake() const { + return get_device_id() == DEVICE_ID_FAKE; + } + + auto operator<=>(const paddr_t &) const = default; + + DENC(paddr_t, v, p) { + DENC_START(1, 1, p); + denc(v.internal_paddr, p); + DENC_FINISH(p); + } + + constexpr static paddr_t create_const( + device_id_t d_id, device_off_t offset) { + return paddr_t(d_id, offset, const_construct_t()); + } + +protected: + internal_paddr_t internal_paddr; + +private: + // as seg + paddr_t(segment_id_t seg, segment_off_t offset) + : paddr_t((static_cast<internal_paddr_t>(seg.segment) << SEGMENT_OFF_BITS) | + static_cast<u_segment_off_t>(offset)) {} + + // as blk or res + paddr_t(device_id_t d_id, device_off_t offset) + : paddr_t((static_cast<internal_paddr_t>(d_id) << DEVICE_OFF_BITS) | + encode_device_off(offset)) { + assert(offset >= DEVICE_OFF_MIN); + assert(offset <= DEVICE_OFF_MAX); + assert(get_addr_type() != paddr_types_t::SEGMENT); + } + + paddr_t(internal_paddr_t val); + + struct const_construct_t {}; + constexpr paddr_t(device_id_t d_id, device_off_t offset, const_construct_t) + : internal_paddr((static_cast<internal_paddr_t>(d_id) << DEVICE_OFF_BITS) | + static_cast<u_device_off_t>(offset)) {} + + friend struct paddr_le_t; + friend struct pladdr_le_t; + +}; + +std::ostream &operator<<(std::ostream &out, const paddr_t &rhs); + +struct seg_paddr_t : public paddr_t { + seg_paddr_t(const seg_paddr_t&) = delete; + seg_paddr_t(seg_paddr_t&) = delete; + seg_paddr_t& operator=(const seg_paddr_t&) = delete; + seg_paddr_t& operator=(seg_paddr_t&) = delete; + + segment_id_t get_segment_id() const { + return segment_id_t(static_cast<internal_segment_id_t>( + internal_paddr >> SEGMENT_OFF_BITS)); + } + + segment_off_t get_segment_off() const { + return segment_off_t(internal_paddr & SEGMENT_OFF_MASK); + } + + void set_segment_off(segment_off_t off) { + assert(off >= 0); + internal_paddr = (internal_paddr & SEGMENT_ID_MASK); + internal_paddr |= static_cast<u_segment_off_t>(off); + } + + paddr_t add_offset(device_off_t o) const { + device_off_t off = get_segment_off() + o; + assert(off >= 0); + assert(off <= SEGMENT_OFF_MAX); + return paddr_t::make_seg_paddr( + get_segment_id(), static_cast<segment_off_t>(off)); + } +}; + +struct blk_paddr_t : public paddr_t { + blk_paddr_t(const blk_paddr_t&) = delete; + blk_paddr_t(blk_paddr_t&) = delete; + blk_paddr_t& operator=(const blk_paddr_t&) = delete; + blk_paddr_t& operator=(blk_paddr_t&) = delete; + + device_off_t get_device_off() const { + return decode_device_off(internal_paddr); + } + + void set_device_off(device_off_t off) { + assert(off >= 0); + assert(off <= DEVICE_OFF_MAX); + internal_paddr = (internal_paddr & DEVICE_ID_MASK); + internal_paddr |= encode_device_off(off); + } + + paddr_t add_offset(device_off_t o) const { + assert(o >= DEVICE_OFF_MIN); + assert(o <= DEVICE_OFF_MAX); + auto off = get_device_off() + o; + return paddr_t::make_blk_paddr(get_device_id(), off); + } +}; + +struct res_paddr_t : public paddr_t { + res_paddr_t(const res_paddr_t&) = delete; + res_paddr_t(res_paddr_t&) = delete; + res_paddr_t& operator=(const res_paddr_t&) = delete; + res_paddr_t& operator=(res_paddr_t&) = delete; + + device_off_t get_device_off() const { + return decode_device_off(internal_paddr); + } + + void set_device_off(device_off_t off) { + assert(has_device_off(get_device_id())); + assert(off >= DEVICE_OFF_MIN); + assert(off <= DEVICE_OFF_MAX); + internal_paddr = (internal_paddr & DEVICE_ID_MASK); + internal_paddr |= encode_device_off(off); + } + + paddr_t add_offset(device_off_t o) const { + assert(has_device_off(get_device_id())); + assert(o >= DEVICE_OFF_MIN); + assert(o <= DEVICE_OFF_MAX); + auto off = get_device_off() + o; + return paddr_t::make_res_paddr(get_device_id(), off); + } + + paddr_t block_relative_to(const res_paddr_t &rhs) const { + assert(rhs.is_record_relative() && is_record_relative()); + auto off = get_device_off() - rhs.get_device_off(); + return paddr_t::make_res_paddr(DEVICE_ID_BLOCK_RELATIVE, off); + } +}; + +constexpr paddr_t P_ADDR_MIN = paddr_t::create_const(0, 0); +// P_ADDR_MAX == P_ADDR_NULL == paddr_t{} +constexpr paddr_t P_ADDR_MAX = paddr_t::create_const(DEVICE_ID_MAX, 0); +constexpr paddr_t P_ADDR_NULL = P_ADDR_MAX; +constexpr paddr_t P_ADDR_ZERO = paddr_t::create_const(DEVICE_ID_ZERO, 0); +constexpr paddr_t P_ADDR_ROOT = paddr_t::create_const(DEVICE_ID_ROOT, 0); + +inline paddr_t make_record_relative_paddr(device_off_t off) { + return paddr_t::make_res_paddr(DEVICE_ID_RECORD_RELATIVE, off); +} +inline paddr_t make_block_relative_paddr(device_off_t off) { + return paddr_t::make_res_paddr(DEVICE_ID_BLOCK_RELATIVE, off); +} +inline paddr_t make_fake_paddr(device_off_t off) { + return paddr_t::make_res_paddr(DEVICE_ID_FAKE, off); +} +inline paddr_t make_delayed_temp_paddr(device_off_t off) { + return paddr_t::make_res_paddr(DEVICE_ID_DELAYED, off); +} + +inline const seg_paddr_t& paddr_t::as_seg_paddr() const { + assert(get_addr_type() == paddr_types_t::SEGMENT); + return *static_cast<const seg_paddr_t*>(this); +} + +inline seg_paddr_t& paddr_t::as_seg_paddr() { + assert(get_addr_type() == paddr_types_t::SEGMENT); + return *static_cast<seg_paddr_t*>(this); +} + +inline const blk_paddr_t& paddr_t::as_blk_paddr() const { + assert(get_addr_type() == paddr_types_t::RANDOM_BLOCK); + return *static_cast<const blk_paddr_t*>(this); +} + +inline blk_paddr_t& paddr_t::as_blk_paddr() { + assert(get_addr_type() == paddr_types_t::RANDOM_BLOCK); + return *static_cast<blk_paddr_t*>(this); +} + +inline const res_paddr_t& paddr_t::as_res_paddr() const { + assert(get_addr_type() == paddr_types_t::RESERVED); + return *static_cast<const res_paddr_t*>(this); +} + +inline res_paddr_t& paddr_t::as_res_paddr() { + assert(get_addr_type() == paddr_types_t::RESERVED); + return *static_cast<res_paddr_t*>(this); +} + +inline paddr_t::paddr_t(internal_paddr_t val) : internal_paddr(val) { +#ifndef NDEBUG + auto type = get_addr_type(); + if (type == paddr_types_t::SEGMENT) { + assert(as_seg_paddr().get_segment_off() >= 0); + } else if (type == paddr_types_t::RANDOM_BLOCK) { + assert(as_blk_paddr().get_device_off() >= 0); + } else { + assert(type == paddr_types_t::RESERVED); + if (!has_device_off(get_device_id())) { + assert(as_res_paddr().get_device_off() == 0); + } + } +#endif +} + +#define PADDR_OPERATION(a_type, base, func) \ + if (get_addr_type() == a_type) { \ + return static_cast<const base*>(this)->func; \ + } + +inline paddr_t paddr_t::add_offset(device_off_t o) const { + PADDR_OPERATION(paddr_types_t::SEGMENT, seg_paddr_t, add_offset(o)) + PADDR_OPERATION(paddr_types_t::RANDOM_BLOCK, blk_paddr_t, add_offset(o)) + PADDR_OPERATION(paddr_types_t::RESERVED, res_paddr_t, add_offset(o)) + ceph_assert(0 == "not supported type"); + return P_ADDR_NULL; +} + +inline paddr_t paddr_t::add_relative(paddr_t o) const { + assert(o.is_relative()); + auto &res_o = o.as_res_paddr(); + return add_offset(res_o.get_device_off()); +} + +inline paddr_t paddr_t::block_relative_to(paddr_t rhs) const { + return as_res_paddr().block_relative_to(rhs.as_res_paddr()); +} + +struct __attribute((packed)) paddr_le_t { + ceph_le64 internal_paddr = + ceph_le64(P_ADDR_NULL.internal_paddr); + + using orig_type = paddr_t; + + paddr_le_t() = default; + paddr_le_t(const paddr_t &addr) : internal_paddr(ceph_le64(addr.internal_paddr)) {} + + operator paddr_t() const { + return paddr_t{internal_paddr}; + } +}; + +using objaddr_t = uint32_t; +constexpr objaddr_t OBJ_ADDR_MAX = std::numeric_limits<objaddr_t>::max(); +constexpr objaddr_t OBJ_ADDR_NULL = OBJ_ADDR_MAX; + +enum class placement_hint_t { + HOT = 0, // The default user hint that expects mutations or retirement + COLD, // Expect no mutations and no retirement in the near future + REWRITE, // Hint for the internal rewrites + NUM_HINTS // Constant for number of hints or as NULL +}; + +constexpr auto PLACEMENT_HINT_NULL = placement_hint_t::NUM_HINTS; + +std::ostream& operator<<(std::ostream& out, placement_hint_t h); + +enum class device_type_t : uint8_t { + NONE = 0, + HDD, + SSD, + ZBD, // ZNS SSD or SMR HDD + EPHEMERAL_COLD, + EPHEMERAL_MAIN, + RANDOM_BLOCK_SSD, + RANDOM_BLOCK_EPHEMERAL, + NUM_TYPES +}; + +std::ostream& operator<<(std::ostream& out, device_type_t t); + +bool can_delay_allocation(device_type_t type); +device_type_t string_to_device_type(std::string type); + +enum class backend_type_t { + SEGMENTED, // SegmentManager: SSD, ZBD, HDD + RANDOM_BLOCK // RBMDevice: RANDOM_BLOCK_SSD +}; + +std::ostream& operator<<(std::ostream& out, backend_type_t); +using journal_type_t = backend_type_t; + +constexpr backend_type_t get_default_backend_of_device(device_type_t dtype) { + assert(dtype != device_type_t::NONE && + dtype != device_type_t::NUM_TYPES); + if (dtype >= device_type_t::HDD && + dtype <= device_type_t::EPHEMERAL_MAIN) { + return backend_type_t::SEGMENTED; + } else { + return backend_type_t::RANDOM_BLOCK; + } +} + +/** + * Monotonically increasing identifier for the location of a + * journal_record. + */ +// JOURNAL_SEQ_NULL == JOURNAL_SEQ_MAX == journal_seq_t{} +struct journal_seq_t { + segment_seq_t segment_seq = NULL_SEG_SEQ; + paddr_t offset = P_ADDR_NULL; + + void swap(journal_seq_t &other) { + std::swap(segment_seq, other.segment_seq); + std::swap(offset, other.offset); + } + + // produces a pseudo journal_seq_t relative to this by offset + journal_seq_t add_offset( + journal_type_t type, + device_off_t off, + device_off_t roll_start, + device_off_t roll_size) const; + + device_off_t relative_to( + journal_type_t type, + const journal_seq_t& r, + device_off_t roll_start, + device_off_t roll_size) const; + + DENC(journal_seq_t, v, p) { + DENC_START(1, 1, p); + denc(v.segment_seq, p); + denc(v.offset, p); + DENC_FINISH(p); + } + + bool operator==(const journal_seq_t &o) const { return cmp(o) == 0; } + bool operator!=(const journal_seq_t &o) const { return cmp(o) != 0; } + bool operator<(const journal_seq_t &o) const { return cmp(o) < 0; } + bool operator<=(const journal_seq_t &o) const { return cmp(o) <= 0; } + bool operator>(const journal_seq_t &o) const { return cmp(o) > 0; } + bool operator>=(const journal_seq_t &o) const { return cmp(o) >= 0; } + +private: + int cmp(const journal_seq_t &other) const { + if (segment_seq > other.segment_seq) { + return 1; + } else if (segment_seq < other.segment_seq) { + return -1; + } + using ret_t = std::pair<device_off_t, segment_id_t>; + auto to_pair = [](const paddr_t &addr) -> ret_t { + if (addr.get_addr_type() == paddr_types_t::SEGMENT) { + auto &seg_addr = addr.as_seg_paddr(); + return ret_t(seg_addr.get_segment_off(), seg_addr.get_segment_id()); + } else if (addr.get_addr_type() == paddr_types_t::RANDOM_BLOCK) { + auto &blk_addr = addr.as_blk_paddr(); + return ret_t(blk_addr.get_device_off(), MAX_SEG_ID); + } else if (addr.get_addr_type() == paddr_types_t::RESERVED) { + auto &res_addr = addr.as_res_paddr(); + return ret_t(res_addr.get_device_off(), MAX_SEG_ID); + } else { + assert(0 == "impossible"); + return ret_t(0, MAX_SEG_ID); + } + }; + auto left = to_pair(offset); + auto right = to_pair(other.offset); + if (left > right) { + return 1; + } else if (left < right) { + return -1; + } else { + return 0; + } + } +}; + +std::ostream &operator<<(std::ostream &out, const journal_seq_t &seq); + +constexpr journal_seq_t JOURNAL_SEQ_MIN{ + 0, + P_ADDR_MIN +}; +constexpr journal_seq_t JOURNAL_SEQ_MAX{ + MAX_SEG_SEQ, + P_ADDR_MAX +}; +// JOURNAL_SEQ_NULL == JOURNAL_SEQ_MAX == journal_seq_t{} +constexpr journal_seq_t JOURNAL_SEQ_NULL = JOURNAL_SEQ_MAX; + +// logical addr, see LBAManager, TransactionManager +using laddr_t = uint64_t; +constexpr laddr_t L_ADDR_MIN = std::numeric_limits<laddr_t>::min(); +constexpr laddr_t L_ADDR_MAX = std::numeric_limits<laddr_t>::max(); +constexpr laddr_t L_ADDR_NULL = L_ADDR_MAX; +constexpr laddr_t L_ADDR_ROOT = L_ADDR_MAX - 1; +constexpr laddr_t L_ADDR_LBAT = L_ADDR_MAX - 2; + +struct __attribute((packed)) laddr_le_t { + ceph_le64 laddr = ceph_le64(L_ADDR_NULL); + + using orig_type = laddr_t; + + laddr_le_t() = default; + laddr_le_t(const laddr_le_t &) = default; + explicit laddr_le_t(const laddr_t &addr) + : laddr(ceph_le64(addr)) {} + + operator laddr_t() const { + return laddr_t(laddr); + } + laddr_le_t& operator=(laddr_t addr) { + ceph_le64 val; + val = addr; + laddr = val; + return *this; + } +}; + +constexpr uint64_t PL_ADDR_NULL = std::numeric_limits<uint64_t>::max(); + +struct pladdr_t { + std::variant<laddr_t, paddr_t> pladdr; + + pladdr_t() = default; + pladdr_t(const pladdr_t &) = default; + pladdr_t(laddr_t laddr) + : pladdr(laddr) {} + pladdr_t(paddr_t paddr) + : pladdr(paddr) {} + + bool is_laddr() const { + return pladdr.index() == 0; + } + + bool is_paddr() const { + return pladdr.index() == 1; + } + + pladdr_t& operator=(paddr_t paddr) { + pladdr = paddr; + return *this; + } + + pladdr_t& operator=(laddr_t laddr) { + pladdr = laddr; + return *this; + } + + bool operator==(const pladdr_t &) const = default; + + paddr_t get_paddr() const { + assert(pladdr.index() == 1); + return paddr_t(std::get<1>(pladdr)); + } + + laddr_t get_laddr() const { + assert(pladdr.index() == 0); + return laddr_t(std::get<0>(pladdr)); + } + +}; + +std::ostream &operator<<(std::ostream &out, const pladdr_t &pladdr); + +enum class addr_type_t : uint8_t { + PADDR=0, + LADDR=1, + MAX=2 // or NONE +}; + +struct __attribute((packed)) pladdr_le_t { + ceph_le64 pladdr = ceph_le64(PL_ADDR_NULL); + addr_type_t addr_type = addr_type_t::MAX; + + pladdr_le_t() = default; + pladdr_le_t(const pladdr_le_t &) = default; + explicit pladdr_le_t(const pladdr_t &addr) + : pladdr( + ceph_le64( + addr.is_laddr() ? + std::get<0>(addr.pladdr) : + std::get<1>(addr.pladdr).internal_paddr)), + addr_type( + addr.is_laddr() ? + addr_type_t::LADDR : + addr_type_t::PADDR) + {} + + operator pladdr_t() const { + if (addr_type == addr_type_t::LADDR) { + return pladdr_t(laddr_t(pladdr)); + } else { + assert(addr_type == addr_type_t::PADDR); + return pladdr_t(paddr_t(pladdr)); + } + } +}; + +template <typename T> +struct min_max_t {}; + +template <> +struct min_max_t<laddr_t> { + static constexpr laddr_t max = L_ADDR_MAX; + static constexpr laddr_t min = L_ADDR_MIN; + static constexpr laddr_t null = L_ADDR_NULL; +}; + +template <> +struct min_max_t<paddr_t> { + static constexpr paddr_t max = P_ADDR_MAX; + static constexpr paddr_t min = P_ADDR_MIN; + static constexpr paddr_t null = P_ADDR_NULL; +}; + +// logical offset, see LBAManager, TransactionManager +using extent_len_t = uint32_t; +constexpr extent_len_t EXTENT_LEN_MAX = + std::numeric_limits<extent_len_t>::max(); + +using extent_len_le_t = ceph_le32; +inline extent_len_le_t init_extent_len_le(extent_len_t len) { + return ceph_le32(len); +} + +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, extent_len_t>>(std::forward<T>(args)...) {} +}; +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, extent_len_t>>(std::forward<T>(args)...) {} +}; + +std::ostream &operator<<(std::ostream &out, const laddr_list_t &rhs); +std::ostream &operator<<(std::ostream &out, const paddr_list_t &rhs); + +/* identifies type of extent, used for interpretting deltas, managing + * writeback. + * + * Note that any new extent type needs to be added to + * Cache::get_extent_by_type in cache.cc + */ +enum class extent_types_t : uint8_t { + ROOT = 0, + LADDR_INTERNAL = 1, + LADDR_LEAF = 2, + DINK_LADDR_LEAF = 3, // should only be used for unitttests + OMAP_INNER = 4, + OMAP_LEAF = 5, + ONODE_BLOCK_STAGED = 6, + COLL_BLOCK = 7, + OBJECT_DATA_BLOCK = 8, + RETIRED_PLACEHOLDER = 9, + // the following two types are not extent types, + // they are just used to indicates paddr allocation deltas + ALLOC_INFO = 10, + JOURNAL_TAIL = 11, + // Test Block Types + TEST_BLOCK = 12, + TEST_BLOCK_PHYSICAL = 13, + BACKREF_INTERNAL = 14, + BACKREF_LEAF = 15, + // None and the number of valid extent_types_t + NONE = 16, +}; +using extent_types_le_t = uint8_t; +constexpr auto EXTENT_TYPES_MAX = static_cast<uint8_t>(extent_types_t::NONE); + +constexpr size_t BACKREF_NODE_SIZE = 4096; + +std::ostream &operator<<(std::ostream &out, extent_types_t t); + +constexpr bool is_logical_type(extent_types_t type) { + switch (type) { + case extent_types_t::ROOT: + case extent_types_t::LADDR_INTERNAL: + case extent_types_t::LADDR_LEAF: + case extent_types_t::BACKREF_INTERNAL: + case extent_types_t::BACKREF_LEAF: + return false; + default: + return true; + } +} + +constexpr bool is_retired_placeholder(extent_types_t type) +{ + return type == extent_types_t::RETIRED_PLACEHOLDER; +} + +constexpr bool is_lba_node(extent_types_t type) +{ + return type == extent_types_t::LADDR_INTERNAL || + type == extent_types_t::LADDR_LEAF || + type == extent_types_t::DINK_LADDR_LEAF; +} + +constexpr bool is_backref_node(extent_types_t type) +{ + return type == extent_types_t::BACKREF_INTERNAL || + type == extent_types_t::BACKREF_LEAF; +} + +constexpr bool is_lba_backref_node(extent_types_t type) +{ + return is_lba_node(type) || is_backref_node(type); +} + +std::ostream &operator<<(std::ostream &out, extent_types_t t); + +/** + * rewrite_gen_t + * + * The goal is to group the similar aged extents in the same segment for better + * bimodel utilization distribution, and also to the same device tier. For EPM, + * it has the flexibility to make placement decisions by re-assigning the + * generation. And each non-inline generation will be statically mapped to a + * writer in EPM. + * + * All the fresh and dirty extents start with INIT_GENERATION upon allocation, + * and they will be assigned to INLINE/OOL generation by EPM before the initial + * writes. After that, the generation can only be increased upon rewrite. + * + * Note, although EPM can re-assign the generations according to the tiering + * status, it cannot decrease the generation for the correctness of space + * reservation. It may choose to assign a larger generation if the extent is + * hinted cold, or if want to evict extents to the cold tier. And it may choose + * to not increase the generation if want to keep the hot tier as filled as + * possible. + */ +using rewrite_gen_t = uint8_t; + +// INIT_GENERATION requires EPM decision to INLINE/OOL_GENERATION +constexpr rewrite_gen_t INIT_GENERATION = 0; +constexpr rewrite_gen_t INLINE_GENERATION = 1; // to the journal +constexpr rewrite_gen_t OOL_GENERATION = 2; + +// All the rewritten extents start with MIN_REWRITE_GENERATION +constexpr rewrite_gen_t MIN_REWRITE_GENERATION = 3; +// without cold tier, the largest generation is less than MIN_COLD_GENERATION +constexpr rewrite_gen_t MIN_COLD_GENERATION = 5; +constexpr rewrite_gen_t MAX_REWRITE_GENERATION = 7; +constexpr rewrite_gen_t REWRITE_GENERATIONS = MAX_REWRITE_GENERATION + 1; +constexpr rewrite_gen_t NULL_GENERATION = + std::numeric_limits<rewrite_gen_t>::max(); + +struct rewrite_gen_printer_t { + rewrite_gen_t gen; +}; + +std::ostream &operator<<(std::ostream &out, rewrite_gen_printer_t gen); + +constexpr std::size_t generation_to_writer(rewrite_gen_t gen) { + // caller to assert the gen is in the reasonable range + return gen - OOL_GENERATION; +} + +// before EPM decision +constexpr bool is_target_rewrite_generation(rewrite_gen_t gen) { + return gen == INIT_GENERATION || + (gen >= MIN_REWRITE_GENERATION && + gen <= REWRITE_GENERATIONS); +} + +// after EPM decision +constexpr bool is_rewrite_generation(rewrite_gen_t gen) { + return gen >= INLINE_GENERATION && + gen < REWRITE_GENERATIONS; +} + +enum class data_category_t : uint8_t { + METADATA = 0, + DATA, + NUM +}; + +std::ostream &operator<<(std::ostream &out, data_category_t c); + +constexpr data_category_t get_extent_category(extent_types_t type) { + if (type == extent_types_t::OBJECT_DATA_BLOCK || + type == extent_types_t::TEST_BLOCK) { + return data_category_t::DATA; + } else { + return data_category_t::METADATA; + } +} + +// type for extent modification time, milliseconds since the epoch +using sea_time_point = seastar::lowres_system_clock::time_point; +using sea_duration = seastar::lowres_system_clock::duration; +using mod_time_point_t = int64_t; + +constexpr mod_time_point_t +timepoint_to_mod(const sea_time_point &t) { + return std::chrono::duration_cast<std::chrono::milliseconds>( + t.time_since_epoch()).count(); +} + +constexpr sea_time_point +mod_to_timepoint(mod_time_point_t t) { + return sea_time_point(std::chrono::duration_cast<sea_duration>( + std::chrono::milliseconds(t))); +} + +constexpr auto NULL_TIME = sea_time_point(); +constexpr auto NULL_MOD_TIME = timepoint_to_mod(NULL_TIME); + +struct sea_time_point_printer_t { + sea_time_point tp; +}; +std::ostream &operator<<(std::ostream &out, sea_time_point_printer_t tp); + +struct mod_time_point_printer_t { + mod_time_point_t tp; +}; +std::ostream &operator<<(std::ostream &out, mod_time_point_printer_t tp); + +constexpr sea_time_point +get_average_time(const sea_time_point& t1, std::size_t n1, + const sea_time_point& t2, std::size_t n2) { + assert(t1 != NULL_TIME); + assert(t2 != NULL_TIME); + auto new_size = n1 + n2; + assert(new_size > 0); + auto c1 = t1.time_since_epoch().count(); + auto c2 = t2.time_since_epoch().count(); + auto c_ret = c1 / new_size * n1 + c2 / new_size * n2; + return sea_time_point(sea_duration(c_ret)); +} + +/* description of a new physical extent */ +struct extent_t { + extent_types_t type; ///< type of extent + laddr_t addr; ///< laddr of extent (L_ADDR_NULL for non-logical) + ceph::bufferlist bl; ///< payload, bl.length() == length, aligned +}; + +using extent_version_t = uint32_t; + +/* description of a mutation to a physical extent */ +struct delta_info_t { + extent_types_t type = extent_types_t::NONE; ///< delta type + paddr_t paddr; ///< physical address + laddr_t laddr = L_ADDR_NULL; ///< logical address + uint32_t prev_crc = 0; + uint32_t final_crc = 0; + extent_len_t length = 0; ///< extent length + extent_version_t pversion; ///< prior version + segment_seq_t ext_seq; ///< seq of the extent's segment + segment_type_t seg_type; + ceph::bufferlist bl; ///< payload + + DENC(delta_info_t, v, p) { + DENC_START(1, 1, p); + denc(v.type, p); + denc(v.paddr, p); + denc(v.laddr, p); + denc(v.prev_crc, p); + denc(v.final_crc, p); + denc(v.length, p); + denc(v.pversion, p); + denc(v.ext_seq, p); + denc(v.seg_type, p); + denc(v.bl, p); + DENC_FINISH(p); + } + + bool operator==(const delta_info_t &rhs) const { + return ( + type == rhs.type && + paddr == rhs.paddr && + laddr == rhs.laddr && + prev_crc == rhs.prev_crc && + final_crc == rhs.final_crc && + length == rhs.length && + pversion == rhs.pversion && + ext_seq == rhs.ext_seq && + bl == rhs.bl + ); + } +}; + +std::ostream &operator<<(std::ostream &out, const delta_info_t &delta); + +/* contains the latest journal tail information */ +struct journal_tail_delta_t { + journal_seq_t alloc_tail; + journal_seq_t dirty_tail; + + DENC(journal_tail_delta_t, v, p) { + DENC_START(1, 1, p); + denc(v.alloc_tail, p); + denc(v.dirty_tail, p); + DENC_FINISH(p); + } +}; + +std::ostream &operator<<(std::ostream &out, const journal_tail_delta_t &delta); + +class object_data_t { + laddr_t reserved_data_base = L_ADDR_NULL; + extent_len_t reserved_data_len = 0; + + bool dirty = false; +public: + object_data_t( + laddr_t reserved_data_base, + extent_len_t reserved_data_len) + : reserved_data_base(reserved_data_base), + reserved_data_len(reserved_data_len) {} + + laddr_t get_reserved_data_base() const { + return reserved_data_base; + } + + extent_len_t get_reserved_data_len() const { + return reserved_data_len; + } + + bool is_null() const { + return reserved_data_base == L_ADDR_NULL; + } + + bool must_update() const { + return dirty; + } + + void update_reserved( + laddr_t base, + extent_len_t len) { + dirty = true; + reserved_data_base = base; + reserved_data_len = len; + } + + void update_len( + extent_len_t len) { + dirty = true; + reserved_data_len = len; + } + + void clear() { + dirty = true; + reserved_data_base = L_ADDR_NULL; + reserved_data_len = 0; + } +}; + +struct __attribute__((packed)) object_data_le_t { + laddr_le_t reserved_data_base = laddr_le_t(L_ADDR_NULL); + extent_len_le_t reserved_data_len = init_extent_len_le(0); + + void update(const object_data_t &nroot) { + reserved_data_base = nroot.get_reserved_data_base(); + reserved_data_len = init_extent_len_le(nroot.get_reserved_data_len()); + } + + object_data_t get() const { + return object_data_t( + reserved_data_base, + reserved_data_len); + } +}; + +struct omap_root_t { + laddr_t addr = L_ADDR_NULL; + depth_t depth = 0; + laddr_t hint = L_ADDR_MIN; + bool mutated = false; + + omap_root_t() = default; + omap_root_t(laddr_t addr, depth_t depth, laddr_t addr_min) + : addr(addr), + depth(depth), + hint(addr_min) {} + + omap_root_t(const omap_root_t &o) = default; + omap_root_t(omap_root_t &&o) = default; + omap_root_t &operator=(const omap_root_t &o) = default; + omap_root_t &operator=(omap_root_t &&o) = default; + + bool is_null() const { + return addr == L_ADDR_NULL; + } + + bool must_update() const { + return mutated; + } + + void update(laddr_t _addr, depth_t _depth, laddr_t _hint) { + mutated = true; + addr = _addr; + depth = _depth; + hint = _hint; + } + + laddr_t get_location() const { + return addr; + } + + depth_t get_depth() const { + return depth; + } + + laddr_t get_hint() const { + return hint; + } +}; +std::ostream &operator<<(std::ostream &out, const omap_root_t &root); + +class __attribute__((packed)) omap_root_le_t { + laddr_le_t addr = laddr_le_t(L_ADDR_NULL); + depth_le_t depth = init_depth_le(0); + +public: + omap_root_le_t() = default; + + omap_root_le_t(laddr_t addr, depth_t depth) + : addr(addr), depth(init_depth_le(depth)) {} + + omap_root_le_t(const omap_root_le_t &o) = default; + omap_root_le_t(omap_root_le_t &&o) = default; + omap_root_le_t &operator=(const omap_root_le_t &o) = default; + omap_root_le_t &operator=(omap_root_le_t &&o) = default; + + void update(const omap_root_t &nroot) { + addr = nroot.get_location(); + depth = init_depth_le(nroot.get_depth()); + } + + omap_root_t get(laddr_t hint) const { + return omap_root_t(addr, depth, hint); + } +}; + +/** + * phy_tree_root_t + */ +class __attribute__((packed)) phy_tree_root_t { + paddr_le_t root_addr; + depth_le_t depth = init_extent_len_le(0); + +public: + phy_tree_root_t() = default; + + phy_tree_root_t(paddr_t addr, depth_t depth) + : root_addr(addr), depth(init_depth_le(depth)) {} + + phy_tree_root_t(const phy_tree_root_t &o) = default; + phy_tree_root_t(phy_tree_root_t &&o) = default; + phy_tree_root_t &operator=(const phy_tree_root_t &o) = default; + phy_tree_root_t &operator=(phy_tree_root_t &&o) = default; + + paddr_t get_location() const { + return root_addr; + } + + void set_location(paddr_t location) { + root_addr = location; + } + + depth_t get_depth() const { + return depth; + } + + void set_depth(depth_t ndepth) { + depth = ndepth; + } + + void adjust_addrs_from_base(paddr_t base) { + paddr_t _root_addr = root_addr; + if (_root_addr.is_relative()) { + root_addr = base.add_record_relative(_root_addr); + } + } +}; + +class coll_root_t { + laddr_t addr = L_ADDR_NULL; + extent_len_t size = 0; + + bool mutated = false; + +public: + coll_root_t() = default; + coll_root_t(laddr_t addr, extent_len_t size) : addr(addr), size(size) {} + + coll_root_t(const coll_root_t &o) = default; + coll_root_t(coll_root_t &&o) = default; + coll_root_t &operator=(const coll_root_t &o) = default; + coll_root_t &operator=(coll_root_t &&o) = default; + + bool must_update() const { + return mutated; + } + + void update(laddr_t _addr, extent_len_t _s) { + mutated = true; + addr = _addr; + size = _s; + } + + laddr_t get_location() const { + return addr; + } + + extent_len_t get_size() const { + return size; + } +}; + +/** + * coll_root_le_t + * + * Information for locating CollectionManager information, to be embedded + * in root block. + */ +class __attribute__((packed)) coll_root_le_t { + laddr_le_t addr; + extent_len_le_t size = init_extent_len_le(0); + +public: + coll_root_le_t() = default; + + coll_root_le_t(laddr_t laddr, extent_len_t size) + : addr(laddr), size(init_extent_len_le(size)) {} + + + coll_root_le_t(const coll_root_le_t &o) = default; + coll_root_le_t(coll_root_le_t &&o) = default; + coll_root_le_t &operator=(const coll_root_le_t &o) = default; + coll_root_le_t &operator=(coll_root_le_t &&o) = default; + + void update(const coll_root_t &nroot) { + addr = nroot.get_location(); + size = init_extent_len_le(nroot.get_size()); + } + + coll_root_t get() const { + return coll_root_t(addr, size); + } +}; + +using lba_root_t = phy_tree_root_t; +using backref_root_t = phy_tree_root_t; + +/** + * root_t + * + * Contains information required to find metadata roots. + * TODO: generalize this to permit more than one lba_manager implementation + */ +struct __attribute__((packed)) root_t { + using meta_t = std::map<std::string, std::string>; + + static constexpr int MAX_META_LENGTH = 1024; + + backref_root_t backref_root; + lba_root_t lba_root; + laddr_le_t onode_root; + coll_root_le_t collection_root; + + char meta[MAX_META_LENGTH]; + + root_t() { + set_meta(meta_t{}); + } + + void adjust_addrs_from_base(paddr_t base) { + lba_root.adjust_addrs_from_base(base); + backref_root.adjust_addrs_from_base(base); + } + + meta_t get_meta() { + bufferlist bl; + bl.append(ceph::buffer::create_static(MAX_META_LENGTH, meta)); + meta_t ret; + auto iter = bl.cbegin(); + decode(ret, iter); + return ret; + } + + void set_meta(const meta_t &m) { + ceph::bufferlist bl; + encode(m, bl); + ceph_assert(bl.length() < MAX_META_LENGTH); + bl.rebuild(); + auto &bptr = bl.front(); + ::memset(meta, 0, MAX_META_LENGTH); + ::memcpy(meta, bptr.c_str(), bl.length()); + } +}; + +struct alloc_blk_t { + alloc_blk_t( + paddr_t paddr, + laddr_t laddr, + extent_len_t len, + extent_types_t type) + : paddr(paddr), laddr(laddr), len(len), type(type) + {} + + explicit alloc_blk_t() = default; + + paddr_t paddr = P_ADDR_NULL; + laddr_t laddr = L_ADDR_NULL; + extent_len_t len = 0; + extent_types_t type = extent_types_t::ROOT; + DENC(alloc_blk_t, v, p) { + DENC_START(1, 1, p); + denc(v.paddr, p); + denc(v.laddr, p); + denc(v.len, p); + denc(v.type, p); + DENC_FINISH(p); + } +}; + +// use absolute address +struct alloc_delta_t { + enum class op_types_t : uint8_t { + NONE = 0, + SET = 1, + CLEAR = 2 + }; + std::vector<alloc_blk_t> alloc_blk_ranges; + op_types_t op = op_types_t::NONE; + + alloc_delta_t() = default; + + DENC(alloc_delta_t, v, p) { + DENC_START(1, 1, p); + denc(v.alloc_blk_ranges, p); + denc(v.op, p); + DENC_FINISH(p); + } +}; + +struct extent_info_t { + extent_types_t type = extent_types_t::NONE; + laddr_t addr = L_ADDR_NULL; + extent_len_t len = 0; + + extent_info_t() = default; + extent_info_t(const extent_t &et) + : type(et.type), addr(et.addr), + len(et.bl.length()) + {} + + DENC(extent_info_t, v, p) { + DENC_START(1, 1, p); + denc(v.type, p); + denc(v.addr, p); + denc(v.len, p); + DENC_FINISH(p); + } +}; +std::ostream &operator<<(std::ostream &out, const extent_info_t &header); + +using segment_nonce_t = uint32_t; + +/** + * Segment header + * + * Every segment contains and encode segment_header_t in the first block. + * Our strategy for finding the journal replay point is: + * 1) Find the segment with the highest journal_segment_seq + * 2) Get dirty_tail and alloc_tail from the segment header + * 3) Scan forward to update tails from journal_tail_delta_t + * 4) Replay from the latest tails + */ +struct segment_header_t { + segment_seq_t segment_seq; + segment_id_t physical_segment_id; // debugging + + journal_seq_t dirty_tail; + journal_seq_t alloc_tail; + segment_nonce_t segment_nonce; + + segment_type_t type; + + data_category_t category; + rewrite_gen_t generation; + + segment_type_t get_type() const { + return type; + } + + DENC(segment_header_t, v, p) { + DENC_START(1, 1, p); + denc(v.segment_seq, p); + denc(v.physical_segment_id, p); + denc(v.dirty_tail, p); + denc(v.alloc_tail, p); + denc(v.segment_nonce, p); + denc(v.type, p); + denc(v.category, p); + denc(v.generation, p); + DENC_FINISH(p); + } +}; +std::ostream &operator<<(std::ostream &out, const segment_header_t &header); + +struct segment_tail_t { + segment_seq_t segment_seq; + segment_id_t physical_segment_id; // debugging + + segment_nonce_t segment_nonce; + + segment_type_t type; + + mod_time_point_t modify_time; + std::size_t num_extents; + + segment_type_t get_type() const { + return type; + } + + DENC(segment_tail_t, v, p) { + DENC_START(1, 1, p); + denc(v.segment_seq, p); + denc(v.physical_segment_id, p); + denc(v.segment_nonce, p); + denc(v.type, p); + denc(v.modify_time, p); + denc(v.num_extents, p); + DENC_FINISH(p); + } +}; +std::ostream &operator<<(std::ostream &out, const segment_tail_t &tail); + +enum class transaction_type_t : uint8_t { + MUTATE = 0, + READ, // including weak and non-weak read transactions + TRIM_DIRTY, + TRIM_ALLOC, + CLEANER_MAIN, + CLEANER_COLD, + MAX +}; + +static constexpr auto TRANSACTION_TYPE_NULL = transaction_type_t::MAX; + +static constexpr auto TRANSACTION_TYPE_MAX = static_cast<std::size_t>( + transaction_type_t::MAX); + +std::ostream &operator<<(std::ostream &os, transaction_type_t type); + +constexpr bool is_valid_transaction(transaction_type_t type) { + return type < transaction_type_t::MAX; +} + +constexpr bool is_background_transaction(transaction_type_t type) { + return (type >= transaction_type_t::TRIM_DIRTY && + type < transaction_type_t::MAX); +} + +constexpr bool is_trim_transaction(transaction_type_t type) { + return (type == transaction_type_t::TRIM_DIRTY || + type == transaction_type_t::TRIM_ALLOC); +} + +struct record_size_t { + extent_len_t plain_mdlength = 0; // mdlength without the record header + extent_len_t dlength = 0; + + extent_len_t get_raw_mdlength() const; + + bool is_empty() const { + return plain_mdlength == 0 && + dlength == 0; + } + + void account_extent(extent_len_t extent_len); + + void account(const extent_t& extent) { + account_extent(extent.bl.length()); + } + + void account(const delta_info_t& delta); + + bool operator==(const record_size_t &) const = default; +}; +std::ostream &operator<<(std::ostream&, const record_size_t&); + +struct record_t { + transaction_type_t type = TRANSACTION_TYPE_NULL; + std::vector<extent_t> extents; + std::vector<delta_info_t> deltas; + record_size_t size; + sea_time_point modify_time = NULL_TIME; + + record_t(transaction_type_t type) : type{type} { } + + // unit test only + record_t() { + type = transaction_type_t::MUTATE; + } + + // unit test only + record_t(std::vector<extent_t>&& _extents, + std::vector<delta_info_t>&& _deltas) { + auto modify_time = seastar::lowres_system_clock::now(); + for (auto& e: _extents) { + push_back(std::move(e), modify_time); + } + for (auto& d: _deltas) { + push_back(std::move(d)); + } + type = transaction_type_t::MUTATE; + } + + bool is_empty() const { + return extents.size() == 0 && + deltas.size() == 0; + } + + std::size_t get_delta_size() const { + auto delta_size = std::accumulate( + deltas.begin(), deltas.end(), 0, + [](uint64_t sum, auto& delta) { + return sum + delta.bl.length(); + } + ); + return delta_size; + } + + void push_back(extent_t&& extent, sea_time_point &t) { + ceph_assert(t != NULL_TIME); + if (extents.size() == 0) { + assert(modify_time == NULL_TIME); + modify_time = t; + } else { + modify_time = get_average_time(modify_time, extents.size(), t, 1); + } + size.account(extent); + extents.push_back(std::move(extent)); + } + + void push_back(delta_info_t&& delta) { + size.account(delta); + deltas.push_back(std::move(delta)); + } +}; +std::ostream &operator<<(std::ostream&, const record_t&); + +struct record_header_t { + transaction_type_t type; + uint32_t deltas; // number of deltas + uint32_t extents; // number of extents + mod_time_point_t modify_time; + + DENC(record_header_t, v, p) { + DENC_START(1, 1, p); + denc(v.type, p); + denc(v.deltas, p); + denc(v.extents, p); + denc(v.modify_time, p); + DENC_FINISH(p); + } +}; +std::ostream &operator<<(std::ostream&, const record_header_t&); + +struct record_group_header_t { + uint32_t records; + extent_len_t mdlength; // block aligned, length of metadata + extent_len_t dlength; // block aligned, length of data + segment_nonce_t segment_nonce;// nonce of containing segment + journal_seq_t committed_to; // records prior to committed_to have been + // fully written, maybe in another segment. + checksum_t data_crc; // crc of data payload + + + DENC(record_group_header_t, v, p) { + DENC_START(1, 1, p); + denc(v.records, p); + denc(v.mdlength, p); + denc(v.dlength, p); + denc(v.segment_nonce, p); + denc(v.committed_to, p); + denc(v.data_crc, p); + DENC_FINISH(p); + } +}; +std::ostream& operator<<(std::ostream&, const record_group_header_t&); + +struct record_group_size_t { + extent_len_t plain_mdlength = 0; // mdlength without the group header + extent_len_t dlength = 0; + extent_len_t block_size = 0; + + record_group_size_t() = default; + record_group_size_t( + const record_size_t& rsize, + extent_len_t block_size) { + account(rsize, block_size); + } + + extent_len_t get_raw_mdlength() const; + + extent_len_t get_mdlength() const { + assert(block_size > 0); + return p2roundup(get_raw_mdlength(), block_size); + } + + extent_len_t get_encoded_length() const { + assert(block_size > 0); + assert(dlength % block_size == 0); + return get_mdlength() + dlength; + } + + record_group_size_t get_encoded_length_after( + const record_size_t& rsize, + extent_len_t block_size) const { + record_group_size_t tmp = *this; + tmp.account(rsize, block_size); + return tmp; + } + + double get_fullness() const { + assert(block_size > 0); + return ((double)(get_raw_mdlength() + dlength) / + get_encoded_length()); + } + + void account(const record_size_t& rsize, + extent_len_t block_size); + + bool operator==(const record_group_size_t &) const = default; +}; +std::ostream& operator<<(std::ostream&, const record_group_size_t&); + +struct record_group_t { + std::vector<record_t> records; + record_group_size_t size; + + record_group_t() = default; + record_group_t( + record_t&& record, + extent_len_t block_size) { + push_back(std::move(record), block_size); + } + + std::size_t get_size() const { + return records.size(); + } + + void push_back( + record_t&& record, + extent_len_t block_size) { + size.account(record.size, block_size); + records.push_back(std::move(record)); + assert(size.get_encoded_length() < SEGMENT_OFF_MAX); + } + + void reserve(std::size_t limit) { + records.reserve(limit); + } + + void clear() { + records.clear(); + size = {}; + } +}; +std::ostream& operator<<(std::ostream&, const record_group_t&); + +ceph::bufferlist encode_record( + record_t&& record, + extent_len_t block_size, + const journal_seq_t& committed_to, + segment_nonce_t current_segment_nonce); + +ceph::bufferlist encode_records( + record_group_t& record_group, + const journal_seq_t& committed_to, + segment_nonce_t current_segment_nonce); + +std::optional<record_group_header_t> +try_decode_records_header( + const ceph::bufferlist& header_bl, + segment_nonce_t expected_nonce); + +bool validate_records_metadata( + const ceph::bufferlist& md_bl); + +bool validate_records_data( + const record_group_header_t& header, + const ceph::bufferlist& data_bl); + +struct record_extent_infos_t { + record_header_t header; + std::vector<extent_info_t> extent_infos; +}; +std::optional<std::vector<record_extent_infos_t> > +try_decode_extent_infos( + const record_group_header_t& header, + const ceph::bufferlist& md_bl); +std::optional<std::vector<record_header_t>> +try_decode_record_headers( + const record_group_header_t& header, + const ceph::bufferlist& md_bl); + +struct record_deltas_t { + paddr_t record_block_base; + std::vector<std::pair<sea_time_point, delta_info_t>> deltas; +}; +std::optional<std::vector<record_deltas_t> > +try_decode_deltas( + const record_group_header_t& header, + const ceph::bufferlist& md_bl, + paddr_t record_block_base); + +struct write_result_t { + journal_seq_t start_seq; + extent_len_t length; + + journal_seq_t get_end_seq() const { + return journal_seq_t{ + start_seq.segment_seq, + start_seq.offset.add_offset(length)}; + } +}; +std::ostream& operator<<(std::ostream&, const write_result_t&); + +struct record_locator_t { + paddr_t record_block_base; + write_result_t write_result; +}; +std::ostream& operator<<(std::ostream&, const record_locator_t&); + +/// scan segment for end incrementally +struct scan_valid_records_cursor { + bool last_valid_header_found = false; + journal_seq_t seq; + journal_seq_t last_committed; + std::size_t num_consumed_records = 0; + extent_len_t block_size = 0; + + struct found_record_group_t { + paddr_t offset; + record_group_header_t header; + bufferlist mdbuffer; + + found_record_group_t( + paddr_t offset, + const record_group_header_t &header, + const bufferlist &mdbuffer) + : offset(offset), header(header), mdbuffer(mdbuffer) {} + }; + std::deque<found_record_group_t> pending_record_groups; + + bool is_complete() const { + return last_valid_header_found && pending_record_groups.empty(); + } + + segment_id_t get_segment_id() const { + return seq.offset.as_seg_paddr().get_segment_id(); + } + + segment_off_t get_segment_offset() const { + return seq.offset.as_seg_paddr().get_segment_off(); + } + + extent_len_t get_block_size() const { + return block_size; + } + + void increment_seq(segment_off_t off) { + seq.offset = seq.offset.add_offset(off); + } + + void emplace_record_group(const record_group_header_t&, ceph::bufferlist&&); + + void pop_record_group() { + assert(!pending_record_groups.empty()); + ++num_consumed_records; + pending_record_groups.pop_front(); + } + + scan_valid_records_cursor( + journal_seq_t seq) + : seq(seq) {} +}; +std::ostream& operator<<(std::ostream&, const scan_valid_records_cursor&); + +} + +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::seastore_meta_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::segment_id_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::paddr_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::journal_seq_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::delta_info_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::journal_tail_delta_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::record_header_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::record_group_header_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::extent_info_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::segment_header_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::alloc_blk_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::alloc_delta_t) +WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::segment_tail_t) + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::data_category_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::delta_info_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::device_id_printer_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::extent_types_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::journal_seq_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::journal_tail_delta_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::laddr_list_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::omap_root_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::paddr_list_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::paddr_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::pladdr_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::placement_hint_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::device_type_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::record_group_header_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::record_group_size_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::record_header_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::record_locator_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::record_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::rewrite_gen_printer_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::scan_valid_records_cursor> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::sea_time_point_printer_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::segment_header_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::segment_id_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::segment_seq_printer_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::segment_tail_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::segment_type_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::transaction_type_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::write_result_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<ceph::buffer::list> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/segment_manager.cc b/src/crimson/os/seastore/segment_manager.cc new file mode 100644 index 000000000..1be9cce5f --- /dev/null +++ b/src/crimson/os/seastore/segment_manager.cc @@ -0,0 +1,107 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include "crimson/os/seastore/segment_manager.h" +#include "crimson/os/seastore/segment_manager/block.h" +#include "crimson/os/seastore/logging.h" + +#ifdef HAVE_ZNS +#include "crimson/os/seastore/segment_manager/zbd.h" +SET_SUBSYS(seastore_device); +#endif + + +namespace crimson::os::seastore { + +std::ostream& operator<<(std::ostream& out, const block_shard_info_t& sf) +{ + out << "(" + << "size=" << sf.size + << ", segments=" <<sf.segments + << ", tracker_offset=" <<sf.tracker_offset + << ", first_segment_offset=" <<sf.first_segment_offset + <<")"; + return out; +} + +std::ostream& operator<<(std::ostream& out, const block_sm_superblock_t& sb) +{ + out << "superblock(" + << "shard_num=" << sb.shard_num + << ", segment_size=" << sb.segment_size + << ", block_size=" << sb.block_size + << ", shard_info:"; + for (auto &sf : sb.shard_infos) { + out << sf + << ","; + } + out << "config=" << sb.config + << ")"; + return out; +} + +std::ostream& operator<<(std::ostream &out, Segment::segment_state_t s) +{ + using state_t = Segment::segment_state_t; + switch (s) { + case state_t::EMPTY: + return out << "EMPTY"; + case state_t::OPEN: + return out << "OPEN"; + case state_t::CLOSED: + return out << "CLOSED"; + default: + return out << "INVALID_SEGMENT_STATE!"; + } +} + +seastar::future<crimson::os::seastore::SegmentManagerRef> +SegmentManager::get_segment_manager( + const std::string &device, device_type_t dtype) +{ +#ifdef HAVE_ZNS +LOG_PREFIX(SegmentManager::get_segment_manager); + return seastar::do_with( + static_cast<size_t>(0), + [FNAME, + dtype, + device](auto &nr_zones) { + return seastar::open_file_dma( + device + "/block", + seastar::open_flags::rw + ).then([FNAME, + dtype, + device, + &nr_zones](auto file) { + return seastar::do_with( + file, + [&nr_zones](auto &f) -> seastar::future<int> { + ceph_assert(f); + return f.ioctl(BLKGETNRZONES, (void *)&nr_zones); + }); + }).then([FNAME, + dtype, + device, + &nr_zones](auto ret) -> crimson::os::seastore::SegmentManagerRef { + crimson::os::seastore::SegmentManagerRef sm; + INFO("Found {} zones.", nr_zones); + if (nr_zones != 0) { + return std::make_unique< + segment_manager::zbd::ZBDSegmentManager + >(device + "/block"); + } else { + return std::make_unique< + segment_manager::block::BlockSegmentManager + >(device + "/block", dtype); + } + }); + }); +#else + return seastar::make_ready_future<crimson::os::seastore::SegmentManagerRef>( + std::make_unique< + segment_manager::block::BlockSegmentManager + >(device + "/block", dtype)); +#endif +} + +} diff --git a/src/crimson/os/seastore/segment_manager.h b/src/crimson/os/seastore/segment_manager.h new file mode 100644 index 000000000..719fa6075 --- /dev/null +++ b/src/crimson/os/seastore/segment_manager.h @@ -0,0 +1,216 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <iosfwd> + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <boost/iterator/counting_iterator.hpp> +#include <seastar/core/future.hh> + +#include "include/buffer_fwd.h" +#include "include/ceph_assert.h" + +#include "crimson/common/config_proxy.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/osd/exceptions.h" +#include "device.h" + +namespace crimson::os::seastore { + +using std::vector; +struct block_shard_info_t { + std::size_t size; + std::size_t segments; + uint64_t tracker_offset; + uint64_t first_segment_offset; + + DENC(block_shard_info_t, v, p) { + DENC_START(1, 1, p); + denc(v.size, p); + denc(v.segments, p); + denc(v.tracker_offset, p); + denc(v.first_segment_offset, p); + DENC_FINISH(p); + } +}; + +struct block_sm_superblock_t { + unsigned int shard_num = 0; + size_t segment_size = 0; + size_t block_size = 0; + + std::vector<block_shard_info_t> shard_infos; + + device_config_t config; + + DENC(block_sm_superblock_t, v, p) { + DENC_START(1, 1, p); + denc(v.shard_num, p); + denc(v.segment_size, p); + denc(v.block_size, p); + denc(v.shard_infos, p); + denc(v.config, p); + DENC_FINISH(p); + } + + void validate() const { + ceph_assert(shard_num == seastar::smp::count); + ceph_assert(block_size > 0); + ceph_assert(segment_size > 0 && + segment_size % block_size == 0); + ceph_assert_always(segment_size <= SEGMENT_OFF_MAX); + for (unsigned int i = 0; i < seastar::smp::count; i ++) { + ceph_assert(shard_infos[i].size > segment_size && + shard_infos[i].size % block_size == 0); + ceph_assert_always(shard_infos[i].size <= DEVICE_OFF_MAX); + ceph_assert(shard_infos[i].segments > 0); + ceph_assert_always(shard_infos[i].segments <= DEVICE_SEGMENT_ID_MAX); + ceph_assert(shard_infos[i].tracker_offset > 0 && + shard_infos[i].tracker_offset % block_size == 0); + ceph_assert(shard_infos[i].first_segment_offset > shard_infos[i].tracker_offset && + shard_infos[i].first_segment_offset % block_size == 0); + } + ceph_assert(config.spec.magic != 0); + ceph_assert(get_default_backend_of_device(config.spec.dtype) == + backend_type_t::SEGMENTED); + ceph_assert(config.spec.id <= DEVICE_ID_MAX_VALID); + if (!config.major_dev) { + ceph_assert(config.secondary_devices.size() == 0); + } + for (const auto& [k, v] : config.secondary_devices) { + ceph_assert(k != config.spec.id); + ceph_assert(k <= DEVICE_ID_MAX_VALID); + ceph_assert(k == v.id); + ceph_assert(v.magic != 0); + ceph_assert(v.dtype > device_type_t::NONE); + ceph_assert(v.dtype < device_type_t::NUM_TYPES); + } + } +}; + +std::ostream& operator<<(std::ostream&, const block_shard_info_t&); +std::ostream& operator<<(std::ostream&, const block_sm_superblock_t&); + +class Segment : public boost::intrusive_ref_counter< + Segment, + boost::thread_unsafe_counter>{ +public: + + enum class segment_state_t : uint8_t { + EMPTY = 0, + OPEN = 1, + CLOSED = 2 + }; + + /** + * get_segment_id + */ + virtual segment_id_t get_segment_id() const = 0; + + /** + * min next write location + */ + virtual segment_off_t get_write_ptr() const = 0; + + /** + * max capacity + */ + virtual segment_off_t get_write_capacity() const = 0; + + /** + * close + * + * Closes segment for writes. Won't complete until + * outstanding writes to this segment are complete. + */ + using close_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent>; + virtual close_ertr::future<> close() = 0; + + + /** + * write + * + * @param offset offset of write, must be aligned to <> and >= write pointer, advances + * write pointer + * @param bl buffer to write, will be padded if not aligned + */ + using write_ertr = crimson::errorator< + crimson::ct_error::input_output_error, // media error or corruption + crimson::ct_error::invarg, // if offset is < write pointer or misaligned + crimson::ct_error::ebadf, // segment closed + crimson::ct_error::enospc // write exceeds segment size + >; + virtual write_ertr::future<> write( + segment_off_t offset, ceph::bufferlist bl) = 0; + + /** + * advance_wp + * + * advance the segment write pointer, + * needed when writing at wp is strictly implemented. ex: ZBD backed segments + * @param offset: advance write pointer till the given offset + */ + virtual write_ertr::future<> advance_wp( + segment_off_t offset) = 0; + + virtual ~Segment() {} +}; +using SegmentRef = boost::intrusive_ptr<Segment>; + +std::ostream& operator<<(std::ostream& out, Segment::segment_state_t); + +constexpr size_t PADDR_SIZE = sizeof(paddr_t); +class SegmentManager; + +using SegmentManagerRef = std::unique_ptr<SegmentManager>; + +class SegmentManager : public Device { +public: + backend_type_t get_backend_type() const final { + return backend_type_t::SEGMENTED; + } + + using open_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent>; + virtual open_ertr::future<SegmentRef> open(segment_id_t id) = 0; + + using release_ertr = crimson::errorator< + crimson::ct_error::input_output_error, + crimson::ct_error::invarg, + crimson::ct_error::enoent>; + virtual release_ertr::future<> release(segment_id_t id) = 0; + + /* Methods for discovering device geometry, segmentid set, etc */ + virtual segment_off_t get_segment_size() const = 0; + virtual device_segment_id_t get_num_segments() const { + ceph_assert(get_available_size() % get_segment_size() == 0); + return ((device_segment_id_t)(get_available_size() / get_segment_size())); + } + + virtual ~SegmentManager() {} + + static seastar::future<SegmentManagerRef> + get_segment_manager(const std::string &device, device_type_t dtype); +}; + +} + +WRITE_CLASS_DENC( + crimson::os::seastore::block_shard_info_t +) +WRITE_CLASS_DENC( + crimson::os::seastore::block_sm_superblock_t +) + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::block_shard_info_t> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<crimson::os::seastore::block_sm_superblock_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/segment_manager/block.cc b/src/crimson/os/seastore/segment_manager/block.cc new file mode 100644 index 000000000..4eb8d60b2 --- /dev/null +++ b/src/crimson/os/seastore/segment_manager/block.cc @@ -0,0 +1,810 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <sys/mman.h> +#include <string.h> + +#include <fmt/format.h> + +#include <seastar/core/metrics.hh> + +#include "include/buffer.h" + +#include "crimson/common/config_proxy.h" +#include "crimson/common/errorator-loop.h" + +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/segment_manager/block.h" + +SET_SUBSYS(seastore_device); +/* + * format: + * - D<device-id> S<segment-id> offset=<off>~<len> poffset=<off> information + * - D<device-id> poffset=<off>~<len> information + * + * levels: + * - INFO: major initiation, closing and segment operations + * - DEBUG: INFO details, major read and write operations + * - TRACE: DEBUG details + */ + +using segment_state_t = crimson::os::seastore::Segment::segment_state_t; + +template <> struct fmt::formatter<segment_state_t>: fmt::formatter<std::string_view> { + // parse is inherited from formatter<string_view>. + template <typename FormatContext> + auto format(segment_state_t s, FormatContext& ctx) { + std::string_view name = "unknown"; + switch (s) { + case segment_state_t::EMPTY: + name = "empty"; + break; + case segment_state_t::OPEN: + name = "open"; + break; + case segment_state_t::CLOSED: + name = "closed"; + break; + } + return formatter<string_view>::format(name, ctx); + } +}; + +namespace crimson::os::seastore::segment_manager::block { + +static write_ertr::future<> do_write( + device_id_t device_id, + seastar::file &device, + uint64_t offset, + bufferptr &bptr) +{ + LOG_PREFIX(block_do_write); + auto len = bptr.length(); + TRACE("{} poffset={}~{} ...", + device_id_printer_t{device_id}, offset, len); + return device.dma_write( + offset, + bptr.c_str(), + len + ).handle_exception( + [FNAME, device_id, offset, len](auto e) -> write_ertr::future<size_t> { + ERROR("{} poffset={}~{} got error -- {}", + device_id_printer_t{device_id}, offset, len, e); + return crimson::ct_error::input_output_error::make(); + }).then([FNAME, device_id, offset, len](auto result) -> write_ertr::future<> { + if (result != len) { + ERROR("{} poffset={}~{} write len={} inconsistent", + device_id_printer_t{device_id}, offset, len, result); + return crimson::ct_error::input_output_error::make(); + } + TRACE("{} poffset={}~{} done", device_id_printer_t{device_id}, offset, len); + return write_ertr::now(); + }); +} + +static write_ertr::future<> do_writev( + device_id_t device_id, + seastar::file &device, + uint64_t offset, + bufferlist&& bl, + size_t block_size) +{ + LOG_PREFIX(block_do_writev); + TRACE("{} poffset={}~{}, {} buffers", + device_id_printer_t{device_id}, offset, bl.length(), bl.get_num_buffers()); + + // writev requires each buffer to be aligned to the disks' block + // size, we need to rebuild here + bl.rebuild_aligned(block_size); + + return seastar::do_with( + bl.prepare_iovs(), + std::move(bl), + [&device, device_id, offset, FNAME](auto& iovs, auto& bl) + { + return write_ertr::parallel_for_each( + iovs, + [&device, device_id, offset, FNAME](auto& p) mutable + { + auto off = offset + p.offset; + auto len = p.length; + auto& iov = p.iov; + TRACE("{} poffset={}~{} dma_write ...", + device_id_printer_t{device_id}, off, len); + return device.dma_write(off, std::move(iov) + ).handle_exception( + [FNAME, device_id, off, len](auto e) -> write_ertr::future<size_t> + { + ERROR("{} poffset={}~{} dma_write got error -- {}", + device_id_printer_t{device_id}, off, len, e); + return crimson::ct_error::input_output_error::make(); + }).then([FNAME, device_id, off, len](size_t written) -> write_ertr::future<> { + if (written != len) { + ERROR("{} poffset={}~{} dma_write len={} inconsistent", + device_id_printer_t{device_id}, off, len, written); + return crimson::ct_error::input_output_error::make(); + } + TRACE("{} poffset={}~{} dma_write done", + device_id_printer_t{device_id}, off, len); + return write_ertr::now(); + }); + }); + }); +} + +static read_ertr::future<> do_read( + device_id_t device_id, + seastar::file &device, + uint64_t offset, + size_t len, + bufferptr &bptr) +{ + LOG_PREFIX(block_do_read); + TRACE("{} poffset={}~{} ...", device_id_printer_t{device_id}, offset, len); + assert(len <= bptr.length()); + return device.dma_read( + offset, + bptr.c_str(), + len + ).handle_exception( + //FIXME: this is a little bit tricky, since seastar::future<T>::handle_exception + // returns seastar::future<T>, to return an crimson::ct_error, we have to create + // a seastar::future<T> holding that crimson::ct_error. This is not necessary + // once seastar::future<T>::handle_exception() returns seastar::futurize_t<T> + [FNAME, device_id, offset, len](auto e) -> read_ertr::future<size_t> + { + ERROR("{} poffset={}~{} got error -- {}", + device_id_printer_t{device_id}, offset, len, e); + return crimson::ct_error::input_output_error::make(); + }).then([FNAME, device_id, offset, len](auto result) -> read_ertr::future<> { + if (result != len) { + ERROR("{} poffset={}~{} read len={} inconsistent", + device_id_printer_t{device_id}, offset, len, result); + return crimson::ct_error::input_output_error::make(); + } + TRACE("{} poffset={}~{} done", device_id_printer_t{device_id}, offset, len); + return read_ertr::now(); + }); +} + +write_ertr::future<> +SegmentStateTracker::write_out( + device_id_t device_id, + seastar::file &device, + uint64_t offset) +{ + LOG_PREFIX(SegmentStateTracker::write_out); + DEBUG("{} poffset={}~{}", + device_id_printer_t{device_id}, offset, bptr.length()); + return do_write(device_id, device, offset, bptr); +} + +write_ertr::future<> +SegmentStateTracker::read_in( + device_id_t device_id, + seastar::file &device, + uint64_t offset) +{ + LOG_PREFIX(SegmentStateTracker::read_in); + DEBUG("{} poffset={}~{}", + device_id_printer_t{device_id}, offset, bptr.length()); + return do_read( + device_id, + device, + offset, + bptr.length(), + bptr); +} +using std::vector; +static +block_sm_superblock_t make_superblock( + device_id_t device_id, + device_config_t sm_config, + const seastar::stat_data &data) +{ + LOG_PREFIX(block_make_superblock); + using crimson::common::get_conf; + + auto config_size = get_conf<Option::size_t>( + "seastore_device_size"); + + size_t size = (data.size == 0) ? config_size : data.size; + + auto config_segment_size = get_conf<Option::size_t>( + "seastore_segment_size"); + size_t raw_segments = size / config_segment_size; + size_t shard_tracker_size = SegmentStateTracker::get_raw_size( + raw_segments / seastar::smp::count, + data.block_size); + size_t total_tracker_size = shard_tracker_size * seastar::smp::count; + size_t tracker_off = data.block_size; //superblock + size_t segments = (size - tracker_off - total_tracker_size) / config_segment_size; + size_t segments_per_shard = segments / seastar::smp::count; + + vector<block_shard_info_t> shard_infos(seastar::smp::count); + for (unsigned int i = 0; i < seastar::smp::count; i++) { + shard_infos[i].size = segments_per_shard * config_segment_size; + shard_infos[i].segments = segments_per_shard; + shard_infos[i].tracker_offset = tracker_off + i * shard_tracker_size; + shard_infos[i].first_segment_offset = tracker_off + total_tracker_size + + i * segments_per_shard * config_segment_size; + } + + INFO("{} disk_size={}, segment_size={}, block_size={}", + device_id_printer_t{device_id}, + size, + uint64_t(config_segment_size), + data.block_size); + for (unsigned int i = 0; i < seastar::smp::count; i++) { + INFO("shard {} infos:", i, shard_infos[i]); + } + + return block_sm_superblock_t{ + seastar::smp::count, + config_segment_size, + data.block_size, + shard_infos, + std::move(sm_config) + }; +} + +using check_create_device_ertr = BlockSegmentManager::access_ertr; +using check_create_device_ret = check_create_device_ertr::future<>; +static check_create_device_ret check_create_device( + const std::string &path, + size_t size) +{ + LOG_PREFIX(block_check_create_device); + INFO("path={}, size={}", path, size); + return seastar::open_file_dma( + path, + seastar::open_flags::exclusive | + seastar::open_flags::rw | + seastar::open_flags::create + ).then([size, FNAME, &path](auto file) { + return seastar::do_with( + file, + [size, FNAME, &path](auto &f) -> seastar::future<> + { + DEBUG("path={} created, truncating to {}", path, size); + ceph_assert(f); + return f.truncate( + size + ).then([&f, size] { + return f.allocate(0, size); + }).finally([&f] { + return f.close(); + }); + }); + }).then_wrapped([&path, FNAME](auto f) -> check_create_device_ret { + if (f.failed()) { + try { + f.get(); + return seastar::now(); + } catch (const std::system_error &e) { + if (e.code().value() == EEXIST) { + ERROR("path={} exists", path); + return seastar::now(); + } else { + ERROR("path={} creation error -- {}", path, e); + return crimson::ct_error::input_output_error::make(); + } + } catch (...) { + ERROR("path={} creation error", path); + return crimson::ct_error::input_output_error::make(); + } + } + + DEBUG("path={} complete", path); + std::ignore = f.discard_result(); + return seastar::now(); + }); +} + +using open_device_ret = + BlockSegmentManager::access_ertr::future< + std::pair<seastar::file, seastar::stat_data> + >; +static +open_device_ret open_device( + const std::string &path) +{ + LOG_PREFIX(block_open_device); + return seastar::file_stat(path, seastar::follow_symlink::yes + ).then([&path, FNAME](auto stat) mutable { + return seastar::open_file_dma( + path, + seastar::open_flags::rw | seastar::open_flags::dsync + ).then([stat, &path, FNAME](auto file) mutable { + return file.size().then([stat, file, &path, FNAME](auto size) mutable { + stat.size = size; + INFO("path={} successful, size={}, block_size={}", + path, stat.size, stat.block_size); + return std::make_pair(file, stat); + }); + }); + }).handle_exception([FNAME, &path](auto e) -> open_device_ret { + ERROR("path={} got error -- {}", path, e); + return crimson::ct_error::input_output_error::make(); + }); +} + + +static +BlockSegmentManager::access_ertr::future<> +write_superblock( + device_id_t device_id, + seastar::file &device, + block_sm_superblock_t sb) +{ + LOG_PREFIX(block_write_superblock); + DEBUG("{} write {}", device_id_printer_t{device_id}, sb); + sb.validate(); + assert(ceph::encoded_sizeof<block_sm_superblock_t>(sb) < + sb.block_size); + return seastar::do_with( + bufferptr(ceph::buffer::create_page_aligned(sb.block_size)), + [=, &device](auto &bp) + { + bufferlist bl; + encode(sb, bl); + auto iter = bl.begin(); + assert(bl.length() < sb.block_size); + iter.copy(bl.length(), bp.c_str()); + return do_write(device_id, device, 0, bp); + }); +} + +static +BlockSegmentManager::access_ertr::future<block_sm_superblock_t> +read_superblock(seastar::file &device, seastar::stat_data sd) +{ + LOG_PREFIX(block_read_superblock); + DEBUG("reading superblock ..."); + return seastar::do_with( + bufferptr(ceph::buffer::create_page_aligned(sd.block_size)), + [=, &device](auto &bp) + { + return do_read( + DEVICE_ID_NULL, // unknown + device, + 0, + bp.length(), + bp + ).safe_then([=, &bp] { + bufferlist bl; + bl.push_back(bp); + block_sm_superblock_t ret; + auto bliter = bl.cbegin(); + try { + decode(ret, bliter); + } catch (...) { + ERROR("got decode error!"); + ceph_assert(0 == "invalid superblock"); + } + assert(ceph::encoded_sizeof<block_sm_superblock_t>(ret) < + sd.block_size); + return BlockSegmentManager::access_ertr::future<block_sm_superblock_t>( + BlockSegmentManager::access_ertr::ready_future_marker{}, + ret); + }); + }); +} + +BlockSegment::BlockSegment( + BlockSegmentManager &manager, segment_id_t id) + : manager(manager), id(id) {} + +segment_off_t BlockSegment::get_write_capacity() const +{ + return manager.get_segment_size(); +} + +Segment::close_ertr::future<> BlockSegment::close() +{ + return manager.segment_close(id, write_pointer); +} + +Segment::write_ertr::future<> BlockSegment::write( + segment_off_t offset, ceph::bufferlist bl) +{ + LOG_PREFIX(BlockSegment::write); + auto paddr = paddr_t::make_seg_paddr(id, offset); + DEBUG("{} offset={}~{} poffset={} ...", + id, offset, bl.length(), manager.get_offset(paddr)); + + if (offset < write_pointer || + offset % manager.superblock.block_size != 0 || + bl.length() % manager.superblock.block_size != 0) { + ERROR("{} offset={}~{} poffset={} invalid write", + id, offset, bl.length(), manager.get_offset(paddr)); + return crimson::ct_error::invarg::make(); + } + + if (offset + bl.length() > manager.superblock.segment_size) { + ERROR("{} offset={}~{} poffset={} write out of the range {}", + id, offset, bl.length(), manager.get_offset(paddr), + manager.superblock.segment_size); + return crimson::ct_error::enospc::make(); + } + + write_pointer = offset + bl.length(); + return manager.segment_write(paddr, bl); +} + +Segment::write_ertr::future<> BlockSegment::advance_wp( + segment_off_t offset) { + return write_ertr::now(); +} + +Segment::close_ertr::future<> BlockSegmentManager::segment_close( + segment_id_t id, segment_off_t write_pointer) +{ + LOG_PREFIX(BlockSegmentManager::segment_close); + auto s_id = id.device_segment_id(); + int unused_bytes = get_segment_size() - write_pointer; + INFO("{} unused_bytes={} ...", id, unused_bytes); + + assert(unused_bytes >= 0); + assert(id.device_id() == get_device_id()); + assert(tracker); + + tracker->set(s_id, segment_state_t::CLOSED); + ++stats.closed_segments; + stats.closed_segments_unused_bytes += unused_bytes; + stats.metadata_write.increment(tracker->get_size()); + return tracker->write_out( + get_device_id(), device, + shard_info.tracker_offset); +} + +Segment::write_ertr::future<> BlockSegmentManager::segment_write( + paddr_t addr, + ceph::bufferlist bl, + bool ignore_check) +{ + assert(addr.get_device_id() == get_device_id()); + assert((bl.length() % superblock.block_size) == 0); + stats.data_write.increment(bl.length()); + return do_writev( + get_device_id(), + device, + get_offset(addr), + std::move(bl), + superblock.block_size); +} + +BlockSegmentManager::~BlockSegmentManager() +{ +} + +BlockSegmentManager::mount_ret BlockSegmentManager::mount() +{ + return shard_devices.invoke_on_all([](auto &local_device) { + return local_device.shard_mount( + ).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in BlockSegmentManager::mount" + }); + }); +} + +BlockSegmentManager::mount_ret BlockSegmentManager::shard_mount() +{ + LOG_PREFIX(BlockSegmentManager::shard_mount); + return open_device( + device_path + ).safe_then([=, this](auto p) { + device = std::move(p.first); + auto sd = p.second; + return read_superblock(device, sd); + }).safe_then([=, this](auto sb) { + set_device_id(sb.config.spec.id); + shard_info = sb.shard_infos[seastar::this_shard_id()]; + INFO("{} read {}", device_id_printer_t{get_device_id()}, shard_info); + sb.validate(); + superblock = sb; + stats.data_read.increment( + ceph::encoded_sizeof<block_sm_superblock_t>(superblock)); + tracker = std::make_unique<SegmentStateTracker>( + shard_info.segments, + superblock.block_size); + stats.data_read.increment(tracker->get_size()); + return tracker->read_in( + get_device_id(), + device, + shard_info.tracker_offset + ).safe_then([this] { + for (device_segment_id_t i = 0; i < tracker->get_capacity(); ++i) { + if (tracker->get(i) == segment_state_t::OPEN) { + tracker->set(i, segment_state_t::CLOSED); + } + } + stats.metadata_write.increment(tracker->get_size()); + return tracker->write_out( + get_device_id(), device, + shard_info.tracker_offset); + }); + }).safe_then([this, FNAME] { + INFO("{} complete", device_id_printer_t{get_device_id()}); + register_metrics(); + }); +} + +BlockSegmentManager::mkfs_ret BlockSegmentManager::mkfs( + device_config_t sm_config) +{ + return shard_devices.local().primary_mkfs(sm_config + ).safe_then([this] { + return shard_devices.invoke_on_all([](auto &local_device) { + return local_device.shard_mkfs( + ).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in BlockSegmentManager::mkfs" + }); + }); + }); +} + +BlockSegmentManager::mkfs_ret BlockSegmentManager::primary_mkfs( + device_config_t sm_config) +{ + LOG_PREFIX(BlockSegmentManager::primary_mkfs); + ceph_assert(sm_config.spec.dtype == superblock.config.spec.dtype); + set_device_id(sm_config.spec.id); + INFO("{} path={}, {}", + device_id_printer_t{get_device_id()}, device_path, sm_config); + return seastar::do_with( + seastar::file{}, + seastar::stat_data{}, + block_sm_superblock_t{}, + std::unique_ptr<SegmentStateTracker>(), + [=, this](auto &device, auto &stat, auto &sb, auto &tracker) + { + check_create_device_ret maybe_create = check_create_device_ertr::now(); + using crimson::common::get_conf; + if (get_conf<bool>("seastore_block_create")) { + auto size = get_conf<Option::size_t>("seastore_device_size"); + maybe_create = check_create_device(device_path, size); + } + + return maybe_create.safe_then([this] { + return open_device(device_path); + }).safe_then([&, sm_config](auto p) { + device = p.first; + stat = p.second; + sb = make_superblock(get_device_id(), sm_config, stat); + stats.metadata_write.increment( + ceph::encoded_sizeof<block_sm_superblock_t>(sb)); + return write_superblock(get_device_id(), device, sb); + }).finally([&] { + return device.close(); + }).safe_then([FNAME, this] { + INFO("{} complete", device_id_printer_t{get_device_id()}); + return mkfs_ertr::now(); + }); + }); +} + +BlockSegmentManager::mkfs_ret BlockSegmentManager::shard_mkfs() +{ + LOG_PREFIX(BlockSegmentManager::shard_mkfs); + return open_device( + device_path + ).safe_then([this](auto p) { + device = std::move(p.first); + auto sd = p.second; + return read_superblock(device, sd); + }).safe_then([this, FNAME](auto sb) { + set_device_id(sb.config.spec.id); + shard_info = sb.shard_infos[seastar::this_shard_id()]; + INFO("{} read {}", device_id_printer_t{get_device_id()}, shard_info); + sb.validate(); + tracker.reset(new SegmentStateTracker( + shard_info.segments, sb.block_size)); + stats.metadata_write.increment(tracker->get_size()); + return tracker->write_out( + get_device_id(), device, + shard_info.tracker_offset); + }).finally([this] { + return device.close(); + }).safe_then([FNAME, this] { + INFO("{} complete", device_id_printer_t{get_device_id()}); + return mkfs_ertr::now(); + }); +} + +BlockSegmentManager::close_ertr::future<> BlockSegmentManager::close() +{ + LOG_PREFIX(BlockSegmentManager::close); + INFO("{}", device_id_printer_t{get_device_id()}); + metrics.clear(); + return device.close(); +} + +SegmentManager::open_ertr::future<SegmentRef> BlockSegmentManager::open( + segment_id_t id) +{ + LOG_PREFIX(BlockSegmentManager::open); + auto s_id = id.device_segment_id(); + INFO("{} ...", id); + + assert(id.device_id() == get_device_id()); + + if (s_id >= get_num_segments()) { + ERROR("{} segment-id out of range {}", id, get_num_segments()); + return crimson::ct_error::invarg::make(); + } + + if (tracker->get(s_id) != segment_state_t::EMPTY) { + ERROR("{} invalid state {} != EMPTY", id, tracker->get(s_id)); + return crimson::ct_error::invarg::make(); + } + + tracker->set(s_id, segment_state_t::OPEN); + stats.metadata_write.increment(tracker->get_size()); + return tracker->write_out( + get_device_id(), device, + shard_info.tracker_offset + ).safe_then([this, id, FNAME] { + ++stats.opened_segments; + DEBUG("{} done", id); + return open_ertr::future<SegmentRef>( + open_ertr::ready_future_marker{}, + SegmentRef(new BlockSegment(*this, id))); + }); +} + +SegmentManager::release_ertr::future<> BlockSegmentManager::release( + segment_id_t id) +{ + LOG_PREFIX(BlockSegmentManager::release); + auto s_id = id.device_segment_id(); + INFO("{} ...", id); + + assert(id.device_id() == get_device_id()); + + if (s_id >= get_num_segments()) { + ERROR("{} segment-id out of range {}", id, get_num_segments()); + return crimson::ct_error::invarg::make(); + } + + if (tracker->get(s_id) != segment_state_t::CLOSED) { + ERROR("{} invalid state {} != CLOSED", id, tracker->get(s_id)); + return crimson::ct_error::invarg::make(); + } + + tracker->set(s_id, segment_state_t::EMPTY); + ++stats.released_segments; + stats.metadata_write.increment(tracker->get_size()); + return tracker->write_out( + get_device_id(), device, + shard_info.tracker_offset); +} + +SegmentManager::read_ertr::future<> BlockSegmentManager::read( + paddr_t addr, + size_t len, + ceph::bufferptr &out) +{ + LOG_PREFIX(BlockSegmentManager::read); + auto& seg_addr = addr.as_seg_paddr(); + auto id = seg_addr.get_segment_id(); + auto s_id = id.device_segment_id(); + auto s_off = seg_addr.get_segment_off(); + auto p_off = get_offset(addr); + DEBUG("{} offset={}~{} poffset={} ...", id, s_off, len, p_off); + + assert(addr.get_device_id() == get_device_id()); + + if (s_off % superblock.block_size != 0 || + len % superblock.block_size != 0) { + ERROR("{} offset={}~{} poffset={} invalid read", id, s_off, len, p_off); + return crimson::ct_error::invarg::make(); + } + + if (s_id >= get_num_segments()) { + ERROR("{} offset={}~{} poffset={} segment-id out of range {}", + id, s_off, len, p_off, get_num_segments()); + return crimson::ct_error::invarg::make(); + } + + if (s_off + len > superblock.segment_size) { + ERROR("{} offset={}~{} poffset={} read out of range {}", + id, s_off, len, p_off, superblock.segment_size); + return crimson::ct_error::invarg::make(); + } + + if (tracker->get(s_id) == segment_state_t::EMPTY) { + // XXX: not an error during scanning, + // might need refactor to increase the log level + DEBUG("{} offset={}~{} poffset={} invalid state {}", + id, s_off, len, p_off, tracker->get(s_id)); + return crimson::ct_error::enoent::make(); + } + + stats.data_read.increment(len); + return do_read( + get_device_id(), + device, + p_off, + len, + out); +} + +void BlockSegmentManager::register_metrics() +{ + LOG_PREFIX(BlockSegmentManager::register_metrics); + DEBUG("{}", device_id_printer_t{get_device_id()}); + namespace sm = seastar::metrics; + std::vector<sm::label_instance> label_instances; + label_instances.push_back(sm::label_instance("device_id", get_device_id())); + stats.reset(); + metrics.add_group( + "segment_manager", + { + sm::make_counter( + "data_read_num", + stats.data_read.num, + sm::description("total number of data read"), + label_instances + ), + sm::make_counter( + "data_read_bytes", + stats.data_read.bytes, + sm::description("total bytes of data read"), + label_instances + ), + sm::make_counter( + "data_write_num", + stats.data_write.num, + sm::description("total number of data write"), + label_instances + ), + sm::make_counter( + "data_write_bytes", + stats.data_write.bytes, + sm::description("total bytes of data write"), + label_instances + ), + sm::make_counter( + "metadata_write_num", + stats.metadata_write.num, + sm::description("total number of metadata write"), + label_instances + ), + sm::make_counter( + "metadata_write_bytes", + stats.metadata_write.bytes, + sm::description("total bytes of metadata write"), + label_instances + ), + sm::make_counter( + "opened_segments", + stats.opened_segments, + sm::description("total segments opened"), + label_instances + ), + sm::make_counter( + "closed_segments", + stats.closed_segments, + sm::description("total segments closed"), + label_instances + ), + sm::make_counter( + "closed_segments_unused_bytes", + stats.closed_segments_unused_bytes, + sm::description("total unused bytes of closed segments"), + label_instances + ), + sm::make_counter( + "released_segments", + stats.released_segments, + sm::description("total segments released"), + label_instances + ), + } + ); +} + +} diff --git a/src/crimson/os/seastore/segment_manager/block.h b/src/crimson/os/seastore/segment_manager/block.h new file mode 100644 index 000000000..495d0d104 --- /dev/null +++ b/src/crimson/os/seastore/segment_manager/block.h @@ -0,0 +1,262 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include <seastar/core/file.hh> +#include <seastar/core/future.hh> +#include <seastar/core/reactor.hh> + +#include "crimson/common/layout.h" + +#include "crimson/os/seastore/segment_manager.h" + +namespace crimson::os::seastore::segment_manager::block { + +using write_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; +using read_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + +/** + * SegmentStateTracker + * + * Tracks lifecycle state of each segment using space at the beginning + * of the drive. + */ +class SegmentStateTracker { + using segment_state_t = Segment::segment_state_t; + + bufferptr bptr; + + using L = absl::container_internal::Layout<uint8_t>; + const L layout; + +public: + static size_t get_raw_size(size_t segments, size_t block_size) { + return p2roundup(segments, block_size); + } + + SegmentStateTracker(size_t segments, size_t block_size) + : bptr(ceph::buffer::create_page_aligned( + get_raw_size(segments, block_size))), + layout(bptr.length()) + { + ::memset( + bptr.c_str(), + static_cast<char>(segment_state_t::EMPTY), + bptr.length()); + } + + size_t get_size() const { + return bptr.length(); + } + + size_t get_capacity() const { + return bptr.length(); + } + + segment_state_t get(device_segment_id_t offset) const { + assert(offset < get_capacity()); + return static_cast<segment_state_t>( + layout.template Pointer<0>( + bptr.c_str())[offset]); + } + + void set(device_segment_id_t offset, segment_state_t state) { + assert(offset < get_capacity()); + layout.template Pointer<0>(bptr.c_str())[offset] = + static_cast<uint8_t>(state); + } + + write_ertr::future<> write_out( + device_id_t device_id, + seastar::file &device, + uint64_t offset); + + read_ertr::future<> read_in( + device_id_t device_id, + seastar::file &device, + uint64_t offset); +}; + +class BlockSegmentManager; +class BlockSegment final : public Segment { + friend class BlockSegmentManager; + BlockSegmentManager &manager; + const segment_id_t id; + segment_off_t write_pointer = 0; +public: + BlockSegment(BlockSegmentManager &manager, segment_id_t id); + + segment_id_t get_segment_id() const final { return id; } + segment_off_t get_write_capacity() const final; + segment_off_t get_write_ptr() const final { return write_pointer; } + close_ertr::future<> close() final; + write_ertr::future<> write(segment_off_t offset, ceph::bufferlist bl) final; + write_ertr::future<> advance_wp(segment_off_t offset) final; + + ~BlockSegment() {} +}; + +/** + * BlockSegmentManager + * + * Implements SegmentManager on a conventional block device. + * SegmentStateTracker uses space at the start of the device to store + * state analagous to that of the segments of a zns device. + */ +class BlockSegmentManager final : public SegmentManager { +// interfaces used by Device +public: + seastar::future<> start() { + return shard_devices.start(device_path, superblock.config.spec.dtype); + } + + seastar::future<> stop() { + return shard_devices.stop(); + } + + Device& get_sharded_device() final { + return shard_devices.local(); + } + mount_ret mount() final; + + mkfs_ret mkfs(device_config_t) final; +// interfaces used by each shard device +public: + close_ertr::future<> close(); + + BlockSegmentManager( + const std::string &path, + device_type_t dtype) + : device_path(path) { + ceph_assert(get_device_type() == device_type_t::NONE); + superblock.config.spec.dtype = dtype; + } + + ~BlockSegmentManager(); + + open_ertr::future<SegmentRef> open(segment_id_t id) final; + + release_ertr::future<> release(segment_id_t id) final; + + read_ertr::future<> read( + paddr_t addr, + size_t len, + ceph::bufferptr &out) final; + + device_type_t get_device_type() const final { + return superblock.config.spec.dtype; + } + size_t get_available_size() const final { + return shard_info.size; + } + extent_len_t get_block_size() const { + return superblock.block_size; + } + segment_off_t get_segment_size() const { + return superblock.segment_size; + } + + device_id_t get_device_id() const final { + assert(device_id <= DEVICE_ID_MAX_VALID); + return device_id; + } + secondary_device_set_t& get_secondary_devices() final { + return superblock.config.secondary_devices; + } + // public so tests can bypass segment interface when simpler + Segment::write_ertr::future<> segment_write( + paddr_t addr, + ceph::bufferlist bl, + bool ignore_check=false); + + magic_t get_magic() const final { + return superblock.config.spec.magic; + } + +private: + friend class BlockSegment; + using segment_state_t = Segment::segment_state_t; + + struct effort_t { + uint64_t num = 0; + uint64_t bytes = 0; + + void increment(uint64_t read_bytes) { + ++num; + bytes += read_bytes; + } + }; + + struct { + effort_t data_read; + effort_t data_write; + effort_t metadata_write; + uint64_t opened_segments; + uint64_t closed_segments; + uint64_t closed_segments_unused_bytes; + uint64_t released_segments; + + void reset() { + data_read = {}; + data_write = {}; + metadata_write = {}; + opened_segments = 0; + closed_segments = 0; + closed_segments_unused_bytes = 0; + released_segments = 0; + } + } stats; + + void register_metrics(); + seastar::metrics::metric_group metrics; + + std::string device_path; + std::unique_ptr<SegmentStateTracker> tracker; + block_shard_info_t shard_info; + block_sm_superblock_t superblock; + seastar::file device; + + void set_device_id(device_id_t id) { + assert(id <= DEVICE_ID_MAX_VALID); + assert(device_id == DEVICE_ID_NULL || + device_id == id); + device_id = id; + } + device_id_t device_id = DEVICE_ID_NULL; + + size_t get_offset(paddr_t addr) { + auto& seg_addr = addr.as_seg_paddr(); + return shard_info.first_segment_offset + + (seg_addr.get_segment_id().device_segment_id() * superblock.segment_size) + + seg_addr.get_segment_off(); + } + + const seastore_meta_t &get_meta() const { + return superblock.config.meta; + } + + std::vector<segment_state_t> segment_state; + + char *buffer = nullptr; + + Segment::close_ertr::future<> segment_close( + segment_id_t id, segment_off_t write_pointer); + +private: + // shard 0 mkfs + mkfs_ret primary_mkfs(device_config_t); + // all shards mkfs + mkfs_ret shard_mkfs(); + // all shards mount + mount_ret shard_mount(); + + seastar::sharded<BlockSegmentManager> shard_devices; +}; + +} diff --git a/src/crimson/os/seastore/segment_manager/ephemeral.cc b/src/crimson/os/seastore/segment_manager/ephemeral.cc new file mode 100644 index 000000000..4a4873afb --- /dev/null +++ b/src/crimson/os/seastore/segment_manager/ephemeral.cc @@ -0,0 +1,294 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <sys/mman.h> +#include <string.h> + +#include "seastar/core/sleep.hh" + +#include "crimson/common/log.h" + +#include "include/buffer.h" +#include "crimson/os/seastore/segment_manager/ephemeral.h" + +namespace { + seastar::logger& logger() { + return crimson::get_logger(ceph_subsys_seastore_device); + } +} + +namespace crimson::os::seastore::segment_manager { + +std::ostream &operator<<(std::ostream &lhs, const ephemeral_config_t &c) { + return lhs << "ephemeral_config_t(size=" << c.size << ", block_size=" << c.block_size + << ", segment_size=" << c.segment_size << ")"; +} + +EphemeralSegmentManagerRef create_test_ephemeral() { + return EphemeralSegmentManagerRef( + new EphemeralSegmentManager(DEFAULT_TEST_EPHEMERAL)); +} + +device_config_t get_ephemeral_device_config( + std::size_t index, + std::size_t num_main_devices, + std::size_t num_cold_devices) +{ + auto num_devices = num_main_devices + num_cold_devices; + assert(num_devices > index); + auto get_sec_dtype = [num_main_devices](std::size_t idx) { + if (idx < num_main_devices) { + return device_type_t::EPHEMERAL_MAIN; + } else { + return device_type_t::EPHEMERAL_COLD; + } + }; + + magic_t magic = 0xabcd; + bool is_major_device; + secondary_device_set_t secondary_devices; + if (index == 0) { + is_major_device = true; + for (std::size_t secondary_index = index + 1; + secondary_index < num_devices; + ++secondary_index) { + device_id_t secondary_id = static_cast<device_id_t>(secondary_index); + secondary_devices.insert({ + secondary_index, + device_spec_t{ + magic, + get_sec_dtype(secondary_index), + secondary_id + } + }); + } + } else { // index > 0 + is_major_device = false; + } + + device_id_t id = static_cast<device_id_t>(index); + seastore_meta_t meta = {}; + return {is_major_device, + device_spec_t{ + magic, + get_sec_dtype(index), + id + }, + meta, + secondary_devices}; +} + +EphemeralSegment::EphemeralSegment( + EphemeralSegmentManager &manager, segment_id_t id) + : manager(manager), id(id) {} + +segment_off_t EphemeralSegment::get_write_capacity() const +{ + return manager.get_segment_size(); +} + +Segment::close_ertr::future<> EphemeralSegment::close() +{ + return manager.segment_close(id).safe_then([] { + return seastar::sleep(std::chrono::milliseconds(1)); + }); +} + +Segment::write_ertr::future<> EphemeralSegment::write( + segment_off_t offset, ceph::bufferlist bl) +{ + if (offset < write_pointer || offset % manager.config.block_size != 0) + return crimson::ct_error::invarg::make(); + + if (offset + bl.length() > (size_t)manager.get_segment_size()) + return crimson::ct_error::enospc::make(); + + return manager.segment_write(paddr_t::make_seg_paddr(id, offset), bl); +} + +Segment::write_ertr::future<> EphemeralSegment::advance_wp( + segment_off_t offset) +{ + return write_ertr::now(); +} + +Segment::close_ertr::future<> EphemeralSegmentManager::segment_close(segment_id_t id) +{ + auto s_id = id.device_segment_id(); + if (segment_state[s_id] != segment_state_t::OPEN) + return crimson::ct_error::invarg::make(); + + segment_state[s_id] = segment_state_t::CLOSED; + return Segment::close_ertr::now().safe_then([] { + return seastar::sleep(std::chrono::milliseconds(1)); + }); +} + +EphemeralSegmentManager::mkfs_ret +EphemeralSegmentManager::mkfs(device_config_t _config) +{ + logger().info( + "Mkfs ephemeral segment manager with {}", + _config); + device_config = _config; + return mkfs_ertr::now(); +} + +Segment::write_ertr::future<> EphemeralSegmentManager::segment_write( + paddr_t addr, + ceph::bufferlist bl, + bool ignore_check) +{ + auto& seg_addr = addr.as_seg_paddr(); + logger().debug( + "segment_write to segment {} at offset {}, physical offset {}, len {}, crc {}", + seg_addr.get_segment_id(), + seg_addr.get_segment_off(), + get_offset(addr), + bl.length(), + bl.crc32c(1)); + if (!ignore_check && segment_state[seg_addr.get_segment_id().device_segment_id()] + != segment_state_t::OPEN) + return crimson::ct_error::invarg::make(); + + bl.begin().copy(bl.length(), buffer + get_offset(addr)); + return Segment::write_ertr::now().safe_then([] { + return seastar::sleep(std::chrono::milliseconds(1)); + }); +} + +EphemeralSegmentManager::init_ertr::future<> EphemeralSegmentManager::init() +{ + logger().info( + "Initing ephemeral segment manager with config {}", + config); + + if (config.block_size % (4<<10) != 0) { + return crimson::ct_error::invarg::make(); + } + if (config.segment_size % config.block_size != 0) { + return crimson::ct_error::invarg::make(); + } + if (config.size % config.segment_size != 0) { + return crimson::ct_error::invarg::make(); + } + + void* addr = ::mmap( + nullptr, + config.size, + PROT_READ | PROT_WRITE, MAP_SHARED | MAP_ANONYMOUS, + -1, + 0); + + segment_state.resize(config.size / config.segment_size, segment_state_t::EMPTY); + + if (addr == MAP_FAILED) + return crimson::ct_error::enospc::make(); + + buffer = (char*)addr; + + ::memset(buffer, 0, config.size); + return init_ertr::now().safe_then([] { + return seastar::sleep(std::chrono::milliseconds(1)); + }); +} + +EphemeralSegmentManager::~EphemeralSegmentManager() +{ + if (buffer) { + ::munmap(buffer, config.size); + } +} + +void EphemeralSegmentManager::remount() +{ + for (auto &i : segment_state) { + if (i == Segment::segment_state_t::OPEN) + i = Segment::segment_state_t::CLOSED; + } +} + +SegmentManager::open_ertr::future<SegmentRef> EphemeralSegmentManager::open( + segment_id_t id) +{ + auto s_id = id.device_segment_id(); + if (s_id >= get_num_segments()) { + logger().error("EphemeralSegmentManager::open: invalid segment {}", id); + return crimson::ct_error::invarg::make(); + } + + if (segment_state[s_id] != segment_state_t::EMPTY) { + logger().error("EphemeralSegmentManager::open: segment {} not empty", id); + return crimson::ct_error::invarg::make(); + } + + segment_state[s_id] = segment_state_t::OPEN; + return open_ertr::make_ready_future<SegmentRef>(new EphemeralSegment(*this, id)); +} + +SegmentManager::release_ertr::future<> EphemeralSegmentManager::release( + segment_id_t id) +{ + auto s_id = id.device_segment_id(); + logger().debug("EphemeralSegmentManager::release: {}", id); + + if (s_id >= get_num_segments()) { + logger().error( + "EphemeralSegmentManager::release: invalid segment {}", + id); + return crimson::ct_error::invarg::make(); + } + + if (segment_state[s_id] != segment_state_t::CLOSED) { + logger().error( + "EphemeralSegmentManager::release: segment id {} not closed", + id); + return crimson::ct_error::invarg::make(); + } + + ::memset(buffer + get_offset(paddr_t::make_seg_paddr(id, 0)), 0, config.segment_size); + segment_state[s_id] = segment_state_t::EMPTY; + return release_ertr::now().safe_then([] { + return seastar::sleep(std::chrono::milliseconds(1)); + }); +} + +SegmentManager::read_ertr::future<> EphemeralSegmentManager::read( + paddr_t addr, + size_t len, + ceph::bufferptr &out) +{ + auto& seg_addr = addr.as_seg_paddr(); + if (seg_addr.get_segment_id().device_segment_id() >= get_num_segments()) { + logger().error( + "EphemeralSegmentManager::read: invalid segment {}", + addr); + return crimson::ct_error::invarg::make(); + } + + if (seg_addr.get_segment_off() + len > config.segment_size) { + logger().error( + "EphemeralSegmentManager::read: invalid offset {}~{}!", + addr, + len); + return crimson::ct_error::invarg::make(); + } + + out.copy_in(0, len, buffer + get_offset(addr)); + + bufferlist bl; + bl.push_back(out); + logger().debug( + "segment_read to segment {} at offset {}, physical offset {}, length {}, crc {}", + seg_addr.get_segment_id().device_segment_id(), + seg_addr.get_segment_off(), + get_offset(addr), + len, + bl.begin().crc32c(len, 1)); + + return read_ertr::now().safe_then([] { + return seastar::sleep(std::chrono::milliseconds(1)); + }); +} + +} diff --git a/src/crimson/os/seastore/segment_manager/ephemeral.h b/src/crimson/os/seastore/segment_manager/ephemeral.h new file mode 100644 index 000000000..d7a3eb4a7 --- /dev/null +++ b/src/crimson/os/seastore/segment_manager/ephemeral.h @@ -0,0 +1,166 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> +#include <seastar/core/future.hh> + +#include "crimson/os/seastore/segment_manager.h" + +#include "crimson/os/seastore/segment_manager/ephemeral.h" + +namespace crimson::os::seastore::segment_manager { + +class EphemeralSegmentManager; +using EphemeralSegmentManagerRef = std::unique_ptr<EphemeralSegmentManager>; + +struct ephemeral_config_t { + size_t size = 0; + size_t block_size = 0; + size_t segment_size = 0; + + void validate() const { + ceph_assert_always(size > 0); + ceph_assert_always(size <= DEVICE_OFF_MAX); + ceph_assert_always(segment_size > 0); + ceph_assert_always(segment_size <= SEGMENT_OFF_MAX); + ceph_assert_always(size / segment_size > 0); + ceph_assert_always(size / segment_size <= DEVICE_SEGMENT_ID_MAX); + } +}; + +constexpr ephemeral_config_t DEFAULT_TEST_EPHEMERAL = { + 1 << 30, + 4 << 10, + 8 << 20 +}; + +std::ostream &operator<<(std::ostream &, const ephemeral_config_t &); + +EphemeralSegmentManagerRef create_test_ephemeral(); + +device_config_t get_ephemeral_device_config( + std::size_t index, + std::size_t num_main_devices, + std::size_t num_cold_devices); + +class EphemeralSegment final : public Segment { + friend class EphemeralSegmentManager; + EphemeralSegmentManager &manager; + const segment_id_t id; + segment_off_t write_pointer = 0; +public: + EphemeralSegment(EphemeralSegmentManager &manager, segment_id_t id); + + segment_id_t get_segment_id() const final { return id; } + segment_off_t get_write_capacity() const final; + segment_off_t get_write_ptr() const final { return write_pointer; } + close_ertr::future<> close() final; + write_ertr::future<> write(segment_off_t offset, ceph::bufferlist bl) final; + write_ertr::future<> advance_wp(segment_off_t offset) final; + + ~EphemeralSegment() {} +}; + +class EphemeralSegmentManager final : public SegmentManager { + friend class EphemeralSegment; + using segment_state_t = Segment::segment_state_t; + + const ephemeral_config_t config; + std::optional<device_config_t> device_config; + + device_type_t get_device_type() const final { + assert(device_config); + return device_config->spec.dtype; + } + + size_t get_offset(paddr_t addr) { + auto& seg_addr = addr.as_seg_paddr(); + return (seg_addr.get_segment_id().device_segment_id() * config.segment_size) + + seg_addr.get_segment_off(); + } + + std::vector<segment_state_t> segment_state; + + char *buffer = nullptr; + + Segment::close_ertr::future<> segment_close(segment_id_t id); + +public: + EphemeralSegmentManager( + ephemeral_config_t config) + : config(config) { + config.validate(); + } + + ~EphemeralSegmentManager(); + + close_ertr::future<> close() final { + return close_ertr::now(); + } + + device_id_t get_device_id() const final { + assert(device_config); + return device_config->spec.id; + } + + mount_ret mount() final { + return mount_ertr::now(); + } + + mkfs_ret mkfs(device_config_t) final; + + open_ertr::future<SegmentRef> open(segment_id_t id) final; + + release_ertr::future<> release(segment_id_t id) final; + + read_ertr::future<> read( + paddr_t addr, + size_t len, + ceph::bufferptr &out) final; + + size_t get_available_size() const final { + return config.size; + } + extent_len_t get_block_size() const final { + return config.block_size; + } + segment_off_t get_segment_size() const final { + return config.segment_size; + } + + const seastore_meta_t &get_meta() const final { + assert(device_config); + return device_config->meta; + } + + secondary_device_set_t& get_secondary_devices() final { + assert(device_config); + return device_config->secondary_devices; + } + + magic_t get_magic() const final { + return device_config->spec.magic; + } + + using init_ertr = crimson::errorator< + crimson::ct_error::enospc, + crimson::ct_error::invarg>; + init_ertr::future<> init(); + + void remount(); + + // public so tests can bypass segment interface when simpler + Segment::write_ertr::future<> segment_write( + paddr_t addr, + ceph::bufferlist bl, + bool ignore_check=false); +}; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::segment_manager::ephemeral_config_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/segment_manager/zbd.cc b/src/crimson/os/seastore/segment_manager/zbd.cc new file mode 100644 index 000000000..88521a947 --- /dev/null +++ b/src/crimson/os/seastore/segment_manager/zbd.cc @@ -0,0 +1,823 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <sys/mman.h> +#include <string.h> +#include <linux/blkzoned.h> + +#include <fmt/format.h> +#include "crimson/os/seastore/segment_manager/zbd.h" +#include "crimson/common/config_proxy.h" +#include "crimson/os/seastore/logging.h" +#include "crimson/common/errorator-loop.h" +#include "include/buffer.h" + +SET_SUBSYS(seastore_device); + +#define SECT_SHIFT 9 +#define RESERVED_ZONES 1 +// limit the max padding buf size to 1MB +#define MAX_PADDING_SIZE 4194304 + +using z_op = crimson::os::seastore::segment_manager::zbd::zone_op; +template <> struct fmt::formatter<z_op>: fmt::formatter<std::string_view> { + template <typename FormatContext> + auto format(z_op s, FormatContext& ctx) { + std::string_view name = "Unknown"; + switch (s) { + using enum z_op; + case OPEN: + name = "BLKOPENZONE"; + break; + case FINISH: + name = "BLKFINISHZONE"; + break; + case CLOSE: + name = "BLKCLOSEZONE"; + break; + case RESET: + name = "BLKRESETZONE"; + break; + } + return formatter<string_view>::format(name, ctx); + } +}; + +namespace crimson::os::seastore::segment_manager::zbd { + +using open_device_ret = ZBDSegmentManager::access_ertr::future< + std::pair<seastar::file, seastar::stat_data>>; +static open_device_ret open_device( + const std::string &path, + seastar::open_flags mode) +{ + LOG_PREFIX(ZBDSegmentManager::open_device); + return seastar::file_stat( + path, seastar::follow_symlink::yes + ).then([FNAME, mode, &path](auto stat) mutable { + return seastar::open_file_dma(path, mode).then([=](auto file) { + DEBUG("open of device {} successful, size {}", + path, + stat.size); + return std::make_pair(file, stat); + }); + }).handle_exception( + [FNAME](auto e) -> open_device_ret { + ERROR("got error {}", + e); + return crimson::ct_error::input_output_error::make(); + } + ); +} + +static zbd_sm_metadata_t make_metadata( + uint64_t total_size, + seastore_meta_t meta, + const seastar::stat_data &data, + size_t zone_size_sectors, + size_t zone_capacity_sectors, + size_t nr_cnv_zones, + size_t num_zones) +{ + LOG_PREFIX(ZBDSegmentManager::make_metadata); + + // Using only SWR zones in a SMR drive, for now + auto skipped_zones = RESERVED_ZONES + nr_cnv_zones; + assert(num_zones > skipped_zones); + + // TODO: support Option::size_t seastore_segment_size + // to allow zones_per_segment > 1 with striping. + size_t zone_size = zone_size_sectors << SECT_SHIFT; + assert(total_size == num_zones * zone_size); + size_t zone_capacity = zone_capacity_sectors << SECT_SHIFT; + size_t segment_size = zone_size; + size_t zones_per_segment = segment_size / zone_size; + size_t segments = (num_zones - skipped_zones) / zones_per_segment; + size_t per_shard_segments = segments / seastar::smp::count; + size_t available_size = zone_capacity * segments; + size_t per_shard_available_size = zone_capacity * per_shard_segments; + + + WARN("Ignoring configuration values for device and segment size"); + INFO( + "device size: {}, available size: {}, block size: {}, allocated size: {}," + " total zones {}, zone size: {}, zone capacity: {}," + " total segments: {}, zones per segment: {}, segment size: {}" + " conv zones: {}, swr zones: {}, per shard segments: {}" + " per shard available size: {}", + total_size, + available_size, + data.block_size, + data.allocated_size, + num_zones, + zone_size, + zone_capacity, + segments, + zones_per_segment, + zone_capacity * zones_per_segment, + nr_cnv_zones, + num_zones - nr_cnv_zones, + per_shard_segments, + per_shard_available_size); + + std::vector<zbd_shard_info_t> shard_infos(seastar::smp::count); + for (unsigned int i = 0; i < seastar::smp::count; i++) { + shard_infos[i].size = per_shard_available_size; + shard_infos[i].segments = per_shard_segments; + shard_infos[i].first_segment_offset = zone_size * skipped_zones + + i * segment_size * per_shard_segments; + INFO("First segment offset for shard {} is: {}", + i, shard_infos[i].first_segment_offset); + } + + zbd_sm_metadata_t ret = zbd_sm_metadata_t{ + seastar::smp::count, + segment_size, + zone_capacity * zones_per_segment, + zones_per_segment, + zone_capacity, + data.block_size, + zone_size, + shard_infos, + meta}; + ret.validate(); + return ret; +} + +struct ZoneReport { + struct blk_zone_report *hdr; + ZoneReport(int nr_zones) + : hdr((blk_zone_report *)malloc( + sizeof(struct blk_zone_report) + nr_zones * sizeof(struct blk_zone))){;} + ~ZoneReport(){ + free(hdr); + } + ZoneReport(const ZoneReport &) = delete; + ZoneReport(ZoneReport &&rhs) : hdr(rhs.hdr) { + rhs.hdr = nullptr; + } +}; + +static seastar::future<size_t> get_blk_dev_size( + seastar::file &device) +{ + return seastar::do_with( + (uint64_t)0, + [&](auto& size_sects) { + return device.ioctl( + BLKGETSIZE, + (void *)&size_sects + ).then([&](int ret) { + ceph_assert(size_sects); + size_t size = size_sects << SECT_SHIFT; + return seastar::make_ready_future<size_t>(size); + }); + }); +} + +// zone_size should be in 512B sectors +static seastar::future<> reset_device( + seastar::file &device, + uint64_t zone_size_sects, + uint64_t nr_zones) +{ + return seastar::do_with( + blk_zone_range{}, + [&, nr_zones, zone_size_sects](auto &range) { + range.sector = 0; + range.nr_sectors = zone_size_sects * nr_zones; + return device.ioctl( + BLKRESETZONE, + &range + ).then([&](int ret){ + return seastar::now(); + }); + } + ); +} + +static seastar::future<size_t> get_zone_capacity( + seastar::file &device, + uint32_t nr_zones) +{ + return seastar::do_with( + ZoneReport(nr_zones), + [&](auto &zr) { + zr.hdr->sector = 0; + zr.hdr->nr_zones = nr_zones; + return device.ioctl( + BLKREPORTZONE, + zr.hdr + ).then([&](int ret) { + return seastar::make_ready_future<size_t>(zr.hdr->zones[0].capacity); + }); + } + ); +} + +// get the number of conventional zones of SMR HDD, +// they are randomly writable and don't respond to zone operations +static seastar::future<size_t> get_nr_cnv_zones( + seastar::file &device, + uint32_t nr_zones) +{ + return seastar::do_with( + ZoneReport(nr_zones), + [&](auto &zr) { + zr.hdr->sector = 0; + zr.hdr->nr_zones = nr_zones; + return device.ioctl( + BLKREPORTZONE, + zr.hdr + ).then([&, nr_zones](int ret) { + size_t cnv_zones = 0; + for (uint32_t i = 0; i < nr_zones; i++) { + if (zr.hdr->zones[i].type == BLK_ZONE_TYPE_CONVENTIONAL) + cnv_zones++; + } + return seastar::make_ready_future<size_t>(cnv_zones); + }); + } + ); +} + + +static write_ertr::future<> do_write( + seastar::file &device, + uint64_t offset, + bufferptr &bptr) +{ + LOG_PREFIX(ZBDSegmentManager::do_write); + DEBUG("offset {} len {}", + offset, + bptr.length()); + return device.dma_write( + offset, + bptr.c_str(), + bptr.length() + ).handle_exception( + [FNAME](auto e) -> write_ertr::future<size_t> { + ERROR("dma_write got error {}", + e); + return crimson::ct_error::input_output_error::make(); + } + ).then([length = bptr.length()](auto result) -> write_ertr::future<> { + if (result != length) { + return crimson::ct_error::input_output_error::make(); + } + return write_ertr::now(); + }); +} + +static write_ertr::future<> do_writev( + device_id_t device_id, + seastar::file &device, + uint64_t offset, + bufferlist&& bl, + size_t block_size) +{ + LOG_PREFIX(ZBDSegmentManager::do_writev); + DEBUG("{} offset {} len {}", + device_id_printer_t{device_id}, offset, bl.length()); + // writev requires each buffer to be aligned to the disks' block + // size, we need to rebuild here + bl.rebuild_aligned(block_size); + + return seastar::do_with( + bl.prepare_iovs(), + std::move(bl), + [&device, device_id, offset, FNAME](auto& iovs, auto& bl) + { + return write_ertr::parallel_for_each( + iovs, + [&device, device_id, offset, FNAME](auto& p) + { + auto off = offset + p.offset; + auto len = p.length; + auto& iov = p.iov; + DEBUG("{} poffset={}~{} dma_write ...", + device_id_printer_t{device_id}, + off, len); + return device.dma_write(off, std::move(iov) + ).handle_exception( + [FNAME, device_id, off, len](auto e) -> write_ertr::future<size_t> + { + ERROR("{} poffset={}~{} dma_write got error -- {}", + device_id_printer_t{device_id}, off, len, e); + return crimson::ct_error::input_output_error::make(); + }).then([FNAME, device_id, off, len](size_t written) -> write_ertr::future<> { + if (written != len) { + ERROR("{} poffset={}~{} dma_write len={} inconsistent", + device_id_printer_t{device_id}, off, len, written); + return crimson::ct_error::input_output_error::make(); + } + DEBUG("{} poffset={}~{} dma_write done", + device_id_printer_t{device_id}, + off, len); + return write_ertr::now(); + }); + }); + }); +} + +static ZBDSegmentManager::access_ertr::future<> +write_metadata(seastar::file &device, zbd_sm_metadata_t sb) +{ + assert(ceph::encoded_sizeof_bounded<zbd_sm_metadata_t>() < + sb.block_size); + return seastar::do_with( + bufferptr(ceph::buffer::create_page_aligned(sb.block_size)), + [=, &device](auto &bp) { + LOG_PREFIX(ZBDSegmentManager::write_metadata); + DEBUG("block_size {}", sb.block_size); + bufferlist bl; + encode(sb, bl); + auto iter = bl.begin(); + assert(bl.length() < sb.block_size); + DEBUG("buffer length {}", bl.length()); + iter.copy(bl.length(), bp.c_str()); + DEBUG("doing writeout"); + return do_write(device, 0, bp); + }); +} + +static read_ertr::future<> do_read( + seastar::file &device, + uint64_t offset, + size_t len, + bufferptr &bptr) +{ + LOG_PREFIX(ZBDSegmentManager::do_read); + assert(len <= bptr.length()); + DEBUG("offset {} len {}", + offset, + len); + return device.dma_read( + offset, + bptr.c_str(), + len + ).handle_exception( + [FNAME](auto e) -> read_ertr::future<size_t> { + ERROR("dma_read got error {}", + e); + return crimson::ct_error::input_output_error::make(); + } + ).then([len](auto result) -> read_ertr::future<> { + if (result != len) { + return crimson::ct_error::input_output_error::make(); + } + return read_ertr::now(); + }); +} + +static +ZBDSegmentManager::access_ertr::future<zbd_sm_metadata_t> +read_metadata(seastar::file &device, seastar::stat_data sd) +{ + assert(ceph::encoded_sizeof_bounded<zbd_sm_metadata_t>() < + sd.block_size); + return seastar::do_with( + bufferptr(ceph::buffer::create_page_aligned(sd.block_size)), + [=, &device](auto &bp) { + return do_read( + device, + 0, + bp.length(), + bp + ).safe_then([=, &bp] { + bufferlist bl; + bl.push_back(bp); + zbd_sm_metadata_t ret; + auto bliter = bl.cbegin(); + decode(ret, bliter); + ret.validate(); + return ZBDSegmentManager::access_ertr::future<zbd_sm_metadata_t>( + ZBDSegmentManager::access_ertr::ready_future_marker{}, + ret); + }); + }); +} + +ZBDSegmentManager::mount_ret ZBDSegmentManager::mount() +{ + return shard_devices.invoke_on_all([](auto &local_device) { + return local_device.shard_mount( + ).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in ZBDSegmentManager::mount" + }); + }); +} + +ZBDSegmentManager::mount_ret ZBDSegmentManager::shard_mount() +{ + return open_device( + device_path, seastar::open_flags::rw + ).safe_then([=, this](auto p) { + device = std::move(p.first); + auto sd = p.second; + return read_metadata(device, sd); + }).safe_then([=, this](auto meta){ + shard_info = meta.shard_infos[seastar::this_shard_id()]; + metadata = meta; + return mount_ertr::now(); + }); +} + +ZBDSegmentManager::mkfs_ret ZBDSegmentManager::mkfs( + device_config_t config) +{ + return shard_devices.local().primary_mkfs(config + ).safe_then([this] { + return shard_devices.invoke_on_all([](auto &local_device) { + return local_device.shard_mkfs( + ).handle_error( + crimson::ct_error::assert_all{ + "Invalid error in ZBDSegmentManager::mkfs" + }); + }); + }); +} + +ZBDSegmentManager::mkfs_ret ZBDSegmentManager::primary_mkfs( + device_config_t config) +{ + LOG_PREFIX(ZBDSegmentManager::primary_mkfs); + INFO("starting, device_path {}", device_path); + return seastar::do_with( + seastar::file{}, + seastar::stat_data{}, + zbd_sm_metadata_t{}, + size_t(), + size_t(), + size_t(), + size_t(), + [=, this] + (auto &device, + auto &stat, + auto &sb, + auto &zone_size_sects, + auto &nr_zones, + auto &size, + auto &nr_cnv_zones) { + return open_device( + device_path, + seastar::open_flags::rw + ).safe_then([=, this, &device, &stat, &sb, &zone_size_sects, &nr_zones, &size, &nr_cnv_zones](auto p) { + device = p.first; + stat = p.second; + return device.ioctl( + BLKGETNRZONES, + (void *)&nr_zones + ).then([&](int ret) { + if (nr_zones == 0) { + return seastar::make_exception_future<int>( + std::system_error(std::make_error_code(std::errc::io_error))); + } + return device.ioctl(BLKGETZONESZ, (void *)&zone_size_sects); + }).then([&](int ret) { + ceph_assert(zone_size_sects); + return reset_device(device, zone_size_sects, nr_zones); + }).then([&] { + return get_blk_dev_size(device); + }).then([&](auto devsize) { + size = devsize; + return get_nr_cnv_zones(device, nr_zones); + }).then([&](auto cnv_zones) { + DEBUG("Found {} conventional zones", cnv_zones); + nr_cnv_zones = cnv_zones; + return get_zone_capacity(device, nr_zones); + }).then([&, FNAME, config](auto zone_capacity_sects) { + ceph_assert(zone_capacity_sects); + DEBUG("zone_size in sectors {}, zone_capacity in sectors {}", + zone_size_sects, zone_capacity_sects); + sb = make_metadata( + size, + config.meta, + stat, + zone_size_sects, + zone_capacity_sects, + nr_cnv_zones, + nr_zones); + metadata = sb; + stats.metadata_write.increment( + ceph::encoded_sizeof_bounded<zbd_sm_metadata_t>()); + DEBUG("Wrote to stats."); + return write_metadata(device, sb); + }).finally([&, FNAME] { + DEBUG("Closing device."); + return device.close(); + }).safe_then([FNAME] { + DEBUG("Returning from mkfs."); + return mkfs_ertr::now(); + }); + }); + }); +} + +ZBDSegmentManager::mkfs_ret ZBDSegmentManager::shard_mkfs() +{ + LOG_PREFIX(ZBDSegmentManager::shard_mkfs); + INFO("starting, device_path {}", device_path); + return open_device( + device_path, seastar::open_flags::rw + ).safe_then([=, this](auto p) { + device = std::move(p.first); + auto sd = p.second; + return read_metadata(device, sd); + }).safe_then([=, this](auto meta){ + shard_info = meta.shard_infos[seastar::this_shard_id()]; + metadata = meta; + return device.close(); + }).safe_then([FNAME] { + DEBUG("Returning from shard_mkfs."); + return mkfs_ertr::now(); + }); +} + +// Return range of sectors to operate on. +struct blk_zone_range make_range( + segment_id_t id, + size_t segment_size, + size_t first_segment_offset) +{ + return blk_zone_range{ + (id.device_segment_id() * (segment_size >> SECT_SHIFT) + + (first_segment_offset >> SECT_SHIFT)), + (segment_size >> SECT_SHIFT) + }; +} + +using blk_zone_op_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; +using blk_zone_op_ret = blk_zone_op_ertr::future<>; +blk_zone_op_ret blk_zone_op(seastar::file &device, + blk_zone_range &range, + zone_op op) { + LOG_PREFIX(ZBDSegmentManager::blk_zone_op); + + unsigned long ioctl_op = 0; + switch (op) { + using enum zone_op; + case OPEN: + ioctl_op = BLKOPENZONE; + break; + case FINISH: + ioctl_op = BLKFINISHZONE; + break; + case RESET: + ioctl_op = BLKRESETZONE; + break; + case CLOSE: + ioctl_op = BLKCLOSEZONE; + break; + default: + ERROR("Invalid zone operation {}", op); + ceph_assert(ioctl_op); + } + + return device.ioctl( + ioctl_op, + &range + ).then_wrapped([=](auto f) -> blk_zone_op_ret { + if (f.failed()) { + ERROR("{} ioctl failed", op); + return crimson::ct_error::input_output_error::make(); + } else { + int ret = f.get(); + if (ret == 0) { + return seastar::now(); + } else { + ERROR("{} ioctl failed with return code {}", op, ret); + return crimson::ct_error::input_output_error::make(); + } + } + }); +} + +ZBDSegmentManager::open_ertr::future<SegmentRef> ZBDSegmentManager::open( + segment_id_t id) +{ + LOG_PREFIX(ZBDSegmentManager::open); + return seastar::do_with( + blk_zone_range{}, + [=, this](auto &range) { + range = make_range( + id, + metadata.segment_size, + shard_info.first_segment_offset); + return blk_zone_op( + device, + range, + zone_op::OPEN + ); + } + ).safe_then([=, this] { + DEBUG("segment {}, open successful", id); + return open_ertr::future<SegmentRef>( + open_ertr::ready_future_marker{}, + SegmentRef(new ZBDSegment(*this, id)) + ); + }); +} + +ZBDSegmentManager::release_ertr::future<> ZBDSegmentManager::release( + segment_id_t id) +{ + LOG_PREFIX(ZBDSegmentManager::release); + DEBUG("Resetting zone/segment {}", id); + return seastar::do_with( + blk_zone_range{}, + [=, this](auto &range) { + range = make_range( + id, + metadata.segment_size, + shard_info.first_segment_offset); + return blk_zone_op( + device, + range, + zone_op::RESET + ); + } + ).safe_then([=] { + DEBUG("segment release successful"); + return release_ertr::now(); + }); +} + +SegmentManager::read_ertr::future<> ZBDSegmentManager::read( + paddr_t addr, + size_t len, + ceph::bufferptr &out) +{ + LOG_PREFIX(ZBDSegmentManager::read); + auto& seg_addr = addr.as_seg_paddr(); + if (seg_addr.get_segment_id().device_segment_id() >= get_num_segments()) { + ERROR("invalid segment {}", + seg_addr.get_segment_id().device_segment_id()); + return crimson::ct_error::invarg::make(); + } + + if (seg_addr.get_segment_off() + len > metadata.segment_capacity) { + ERROR("invalid read offset {}, len {}", + addr, + len); + return crimson::ct_error::invarg::make(); + } + return do_read( + device, + get_offset(addr), + len, + out); +} + +Segment::close_ertr::future<> ZBDSegmentManager::segment_close( + segment_id_t id, segment_off_t write_pointer) +{ + LOG_PREFIX(ZBDSegmentManager::segment_close); + return seastar::do_with( + blk_zone_range{}, + [=, this](auto &range) { + range = make_range( + id, + metadata.segment_size, + shard_info.first_segment_offset); + return blk_zone_op( + device, + range, + zone_op::FINISH + ); + } + ).safe_then([=] { + DEBUG("zone finish successful"); + return Segment::close_ertr::now(); + }); +} + +Segment::write_ertr::future<> ZBDSegmentManager::segment_write( + paddr_t addr, + ceph::bufferlist bl, + bool ignore_check) +{ + LOG_PREFIX(ZBDSegmentManager::segment_write); + assert(addr.get_device_id() == get_device_id()); + assert((bl.length() % metadata.block_size) == 0); + auto& seg_addr = addr.as_seg_paddr(); + DEBUG("write to segment {} at offset {}, physical offset {}, len {}", + seg_addr.get_segment_id(), + seg_addr.get_segment_off(), + get_offset(addr), + bl.length()); + stats.data_write.increment(bl.length()); + return do_writev( + get_device_id(), + device, + get_offset(addr), + std::move(bl), + metadata.block_size); +} + +device_id_t ZBDSegmentManager::get_device_id() const +{ + return metadata.device_id; +}; + +secondary_device_set_t& ZBDSegmentManager::get_secondary_devices() +{ + return metadata.secondary_devices; +}; + +magic_t ZBDSegmentManager::get_magic() const +{ + return metadata.magic; +}; + +segment_off_t ZBDSegment::get_write_capacity() const +{ + return manager.get_segment_size(); +} + +SegmentManager::close_ertr::future<> ZBDSegmentManager::close() +{ + if (device) { + return device.close(); + } + return seastar::now(); +} + +Segment::close_ertr::future<> ZBDSegment::close() +{ + return manager.segment_close(id, write_pointer); +} + +Segment::write_ertr::future<> ZBDSegment::write( + segment_off_t offset, ceph::bufferlist bl) +{ + LOG_PREFIX(ZBDSegment::write); + if (offset != write_pointer || offset % manager.metadata.block_size != 0) { + ERROR("Segment offset and zone write pointer mismatch. " + "segment {} segment-offset {} write pointer {}", + id, offset, write_pointer); + return crimson::ct_error::invarg::make(); + } + if (offset + bl.length() > manager.metadata.segment_capacity) { + return crimson::ct_error::enospc::make(); + } + + write_pointer = offset + bl.length(); + return manager.segment_write(paddr_t::make_seg_paddr(id, offset), bl); +} + +Segment::write_ertr::future<> ZBDSegment::write_padding_bytes( + size_t padding_bytes) +{ + LOG_PREFIX(ZBDSegment::write_padding_bytes); + DEBUG("Writing {} padding bytes to segment {} at wp {}", + padding_bytes, id, write_pointer); + + return crimson::repeat([FNAME, padding_bytes, this] () mutable { + size_t bufsize = 0; + if (padding_bytes >= MAX_PADDING_SIZE) { + bufsize = MAX_PADDING_SIZE; + } else { + bufsize = padding_bytes; + } + + padding_bytes -= bufsize; + bufferptr bp(ceph::buffer::create_page_aligned(bufsize)); + bp.zero(); + bufferlist padd_bl; + padd_bl.append(bp); + return write(write_pointer, padd_bl).safe_then([FNAME, padding_bytes, this]() { + if (padding_bytes == 0) { + return write_ertr::make_ready_future<seastar::stop_iteration>(seastar::stop_iteration::yes); + } else { + return write_ertr::make_ready_future<seastar::stop_iteration>(seastar::stop_iteration::no); + } + }); + }); +} + +// Advance write pointer, to given offset. +Segment::write_ertr::future<> ZBDSegment::advance_wp( + segment_off_t offset) +{ + LOG_PREFIX(ZBDSegment::advance_wp); + + DEBUG("Advancing write pointer from {} to {}", write_pointer, offset); + if (offset < write_pointer) { + return crimson::ct_error::invarg::make(); + } + + size_t padding_bytes = offset - write_pointer; + + if (padding_bytes == 0) { + return write_ertr::now(); + } + + assert(padding_bytes % manager.metadata.block_size == 0); + + return write_padding_bytes(padding_bytes); +} + +} diff --git a/src/crimson/os/seastore/segment_manager/zbd.h b/src/crimson/os/seastore/segment_manager/zbd.h new file mode 100644 index 000000000..c18f46336 --- /dev/null +++ b/src/crimson/os/seastore/segment_manager/zbd.h @@ -0,0 +1,246 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab +#pragma once + +#include <linux/blkzoned.h> + +#include <boost/intrusive_ptr.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include <seastar/core/file.hh> +#include <seastar/core/future.hh> +#include <seastar/core/reactor.hh> + +#include "crimson/common/layout.h" + +#include "crimson/os/seastore/segment_manager.h" + +#include "include/uuid.h" + +namespace crimson::os::seastore::segment_manager::zbd { + + struct zbd_shard_info_t { + size_t size = 0; + size_t segments = 0; + size_t first_segment_offset = 0; + + DENC(zbd_shard_info_t, v, p) { + DENC_START(1, 1, p); + denc(v.size, p); + denc(v.segments, p); + denc(v.first_segment_offset, p); + DENC_FINISH(p); + } + }; + + struct zbd_sm_metadata_t { + unsigned int shard_num = 0; + size_t segment_size = 0; + size_t segment_capacity = 0; + size_t zones_per_segment = 0; + size_t zone_capacity = 0; + size_t block_size = 0; + size_t zone_size = 0; + + std::vector<zbd_shard_info_t> shard_infos; + + seastore_meta_t meta; + + bool major_dev = false; + magic_t magic = 0; + device_type_t dtype = device_type_t::NONE; + device_id_t device_id = 0; + secondary_device_set_t secondary_devices; + + DENC(zbd_sm_metadata_t, v, p) { + DENC_START(1, 1, p); + denc(v.shard_num, p); + denc(v.segment_size, p); + denc(v.segment_capacity, p); + denc(v.zones_per_segment, p); + denc(v.zone_capacity, p); + denc(v.block_size, p); + denc(v.zone_size, p); + denc(v.shard_infos, p); + denc(v.meta, p); + denc(v.magic, p); + denc(v.dtype, p); + denc(v.device_id, p); + if (v.major_dev) { + denc(v.secondary_devices, p); + } + DENC_FINISH(p); + } + + void validate() const { + ceph_assert_always(shard_num == seastar::smp::count); + for (unsigned int i = 0; i < seastar::smp::count; i++) { + ceph_assert_always(shard_infos[i].size > 0); + ceph_assert_always(shard_infos[i].size <= DEVICE_OFF_MAX); + ceph_assert_always(shard_infos[i].segments > 0); + ceph_assert_always(shard_infos[i].segments <= DEVICE_SEGMENT_ID_MAX); + } + ceph_assert_always(segment_capacity > 0); + ceph_assert_always(segment_capacity <= SEGMENT_OFF_MAX); + } + }; + + using write_ertr = crimson::errorator<crimson::ct_error::input_output_error>; + using read_ertr = crimson::errorator<crimson::ct_error::input_output_error>; + + enum class zone_op { + OPEN, + FINISH, + CLOSE, + RESET, + }; + + class ZBDSegmentManager; + + class ZBDSegment final : public Segment { + public: + ZBDSegment(ZBDSegmentManager &man, segment_id_t i) : manager(man), id(i){}; + + segment_id_t get_segment_id() const final { return id; } + segment_off_t get_write_capacity() const final; + segment_off_t get_write_ptr() const final { return write_pointer; } + close_ertr::future<> close() final; + write_ertr::future<> write(segment_off_t offset, ceph::bufferlist bl) final; + write_ertr::future<> advance_wp(segment_off_t offset) final; + + ~ZBDSegment() {} + private: + friend class ZBDSegmentManager; + ZBDSegmentManager &manager; + const segment_id_t id; + segment_off_t write_pointer = 0; + write_ertr::future<> write_padding_bytes(size_t padding_bytes); + }; + + class ZBDSegmentManager final : public SegmentManager{ + // interfaces used by Device + public: + seastar::future<> start() { + return shard_devices.start(device_path); + } + + seastar::future<> stop() { + return shard_devices.stop(); + } + + Device& get_sharded_device() final { + return shard_devices.local(); + } + + mount_ret mount() final; + mkfs_ret mkfs(device_config_t meta) final; + + ZBDSegmentManager(const std::string &path) : device_path(path) {} + + ~ZBDSegmentManager() final = default; + + //interfaces used by each shard device + public: + open_ertr::future<SegmentRef> open(segment_id_t id) final; + close_ertr::future<> close() final; + + release_ertr::future<> release(segment_id_t id) final; + + read_ertr::future<> read( + paddr_t addr, + size_t len, + ceph::bufferptr &out) final; + + device_type_t get_device_type() const final { + return device_type_t::ZBD; + } + + size_t get_available_size() const final { + return shard_info.size; + }; + + extent_len_t get_block_size() const final { + return metadata.block_size; + }; + + segment_off_t get_segment_size() const final { + return metadata.segment_capacity; + }; + + const seastore_meta_t &get_meta() const { + return metadata.meta; + }; + + device_id_t get_device_id() const final; + + secondary_device_set_t& get_secondary_devices() final; + + magic_t get_magic() const final; + + Segment::write_ertr::future<> segment_write( + paddr_t addr, + ceph::bufferlist bl, + bool ignore_check=false); + + private: + friend class ZBDSegment; + std::string device_path; + zbd_shard_info_t shard_info; + zbd_sm_metadata_t metadata; + seastar::file device; + uint32_t nr_zones; + struct effort_t { + uint64_t num = 0; + uint64_t bytes = 0; + + void increment(uint64_t read_bytes) { + ++num; + bytes += read_bytes; + } + }; + + struct zbd_sm_stats { + effort_t data_read = {}; + effort_t data_write = {}; + effort_t metadata_write = {}; + uint64_t opened_segments = 0; + uint64_t closed_segments = 0; + uint64_t closed_segments_unused_bytes = 0; + uint64_t released_segments = 0; + + void reset() { + *this = zbd_sm_stats{}; + } + } stats; + + void register_metrics(); + seastar::metrics::metric_group metrics; + + Segment::close_ertr::future<> segment_close( + segment_id_t id, segment_off_t write_pointer); + + uint64_t get_offset(paddr_t addr) { + auto& seg_addr = addr.as_seg_paddr(); + return (shard_info.first_segment_offset + + (seg_addr.get_segment_id().device_segment_id() * + metadata.segment_size)) + seg_addr.get_segment_off(); + } + private: + // shard 0 mkfs + mkfs_ret primary_mkfs(device_config_t meta); + // all shards mkfs + mkfs_ret shard_mkfs(); + + mount_ret shard_mount(); + + seastar::sharded<ZBDSegmentManager> shard_devices; + }; + +} + +WRITE_CLASS_DENC_BOUNDED( + crimson::os::seastore::segment_manager::zbd::zbd_shard_info_t +) +WRITE_CLASS_DENC_BOUNDED( + crimson::os::seastore::segment_manager::zbd::zbd_sm_metadata_t +) diff --git a/src/crimson/os/seastore/segment_manager_group.cc b/src/crimson/os/seastore/segment_manager_group.cc new file mode 100644 index 000000000..332b794b7 --- /dev/null +++ b/src/crimson/os/seastore/segment_manager_group.cc @@ -0,0 +1,171 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#include "crimson/os/seastore/segment_manager_group.h" + +#include "crimson/os/seastore/logging.h" + +SET_SUBSYS(seastore_journal); + +namespace crimson::os::seastore { + +SegmentManagerGroup::read_segment_tail_ret +SegmentManagerGroup::read_segment_tail(segment_id_t segment) +{ + assert(has_device(segment.device_id())); + auto& segment_manager = *segment_managers[segment.device_id()]; + return segment_manager.read( + paddr_t::make_seg_paddr( + segment, + segment_manager.get_segment_size() - get_rounded_tail_length()), + get_rounded_tail_length() + ).handle_error( + read_segment_header_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in SegmentManagerGroup::read_segment_tail" + } + ).safe_then([=, &segment_manager](bufferptr bptr) -> read_segment_tail_ret { + LOG_PREFIX(SegmentManagerGroup::read_segment_tail); + DEBUG("segment {} bptr size {}", segment, bptr.length()); + + segment_tail_t tail; + bufferlist bl; + bl.push_back(bptr); + + DEBUG("segment {} block crc {}", + segment, + bl.begin().crc32c(segment_manager.get_block_size(), 0)); + + auto bp = bl.cbegin(); + try { + decode(tail, bp); + } catch (ceph::buffer::error &e) { + DEBUG("segment {} unable to decode tail, skipping -- {}", + segment, e.what()); + return crimson::ct_error::enodata::make(); + } + DEBUG("segment {} tail {}", segment, tail); + return read_segment_tail_ret( + read_segment_tail_ertr::ready_future_marker{}, + tail); + }); +} + +SegmentManagerGroup::read_segment_header_ret +SegmentManagerGroup::read_segment_header(segment_id_t segment) +{ + assert(has_device(segment.device_id())); + auto& segment_manager = *segment_managers[segment.device_id()]; + return segment_manager.read( + paddr_t::make_seg_paddr(segment, 0), + get_rounded_header_length() + ).handle_error( + read_segment_header_ertr::pass_further{}, + crimson::ct_error::assert_all{ + "Invalid error in SegmentManagerGroup::read_segment_header" + } + ).safe_then([=, &segment_manager](bufferptr bptr) -> read_segment_header_ret { + LOG_PREFIX(SegmentManagerGroup::read_segment_header); + DEBUG("segment {} bptr size {}", segment, bptr.length()); + + segment_header_t header; + bufferlist bl; + bl.push_back(bptr); + + DEBUG("segment {} block crc {}", + segment, + bl.begin().crc32c(segment_manager.get_block_size(), 0)); + + auto bp = bl.cbegin(); + try { + decode(header, bp); + } catch (ceph::buffer::error &e) { + DEBUG("segment {} unable to decode header, skipping -- {}", + segment, e.what()); + return crimson::ct_error::enodata::make(); + } + DEBUG("segment {} header {}", segment, header); + return read_segment_header_ret( + read_segment_header_ertr::ready_future_marker{}, + header); + }); +} + +void SegmentManagerGroup::initialize_cursor( + scan_valid_records_cursor &cursor) +{ + LOG_PREFIX(SegmentManagerGroup::initialize_cursor); + assert(has_device(cursor.get_segment_id().device_id())); + auto& segment_manager = + *segment_managers[cursor.get_segment_id().device_id()]; + if (cursor.get_segment_offset() == 0) { + INFO("start to scan segment {}", cursor.get_segment_id()); + cursor.increment_seq(segment_manager.get_block_size()); + } + cursor.block_size = segment_manager.get_block_size(); +} + +SegmentManagerGroup::read_ret +SegmentManagerGroup::read(paddr_t start, size_t len) +{ + LOG_PREFIX(SegmentManagerGroup::read); + assert(has_device(start.get_device_id())); + auto& segment_manager = *segment_managers[start.get_device_id()]; + TRACE("reading data {}~{}", start, len); + return segment_manager.read( + start, + len + ).safe_then([](auto bptr) { + return read_ret( + read_ertr::ready_future_marker{}, + std::move(bptr) + ); + }); +} + +SegmentManagerGroup::find_journal_segment_headers_ret +SegmentManagerGroup::find_journal_segment_headers() +{ + return seastar::do_with( + get_segment_managers(), + find_journal_segment_headers_ret_bare{}, + [this](auto &sms, auto& ret) -> find_journal_segment_headers_ret + { + return crimson::do_for_each(sms, + [this, &ret](SegmentManager *sm) + { + LOG_PREFIX(SegmentManagerGroup::find_journal_segment_headers); + auto device_id = sm->get_device_id(); + auto num_segments = sm->get_num_segments(); + DEBUG("processing {} with {} segments", + device_id_printer_t{device_id}, num_segments); + return crimson::do_for_each( + boost::counting_iterator<device_segment_id_t>(0), + boost::counting_iterator<device_segment_id_t>(num_segments), + [this, &ret, device_id](device_segment_id_t d_segment_id) + { + segment_id_t segment_id{device_id, d_segment_id}; + return read_segment_header(segment_id + ).safe_then([segment_id, &ret](auto &&header) { + if (header.get_type() == segment_type_t::JOURNAL) { + ret.emplace_back(std::make_pair(segment_id, std::move(header))); + } + }).handle_error( + crimson::ct_error::enoent::handle([](auto) { + return find_journal_segment_headers_ertr::now(); + }), + crimson::ct_error::enodata::handle([](auto) { + return find_journal_segment_headers_ertr::now(); + }), + crimson::ct_error::input_output_error::pass_further{} + ); + }); + }).safe_then([&ret]() mutable { + return find_journal_segment_headers_ret{ + find_journal_segment_headers_ertr::ready_future_marker{}, + std::move(ret)}; + }); + }); +} + +} // namespace crimson::os::seastore diff --git a/src/crimson/os/seastore/segment_manager_group.h b/src/crimson/os/seastore/segment_manager_group.h new file mode 100644 index 000000000..f193b5eed --- /dev/null +++ b/src/crimson/os/seastore/segment_manager_group.h @@ -0,0 +1,150 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*- +// vim: ts=8 sw=2 smarttab expandtab + +#pragma once + +#include <set> + +#include "crimson/common/errorator.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/segment_manager.h" +#include "crimson/os/seastore/record_scanner.h" + +namespace crimson::os::seastore { + +class SegmentManagerGroup : public RecordScanner { +public: + SegmentManagerGroup() { + segment_managers.resize(DEVICE_ID_MAX, nullptr); + } + + const std::set<device_id_t>& get_device_ids() const { + return device_ids; + } + + std::vector<SegmentManager*> get_segment_managers() const { + assert(device_ids.size()); + std::vector<SegmentManager*> ret; + for (auto& device_id : device_ids) { + auto segment_manager = segment_managers[device_id]; + assert(segment_manager->get_device_id() == device_id); + ret.emplace_back(segment_manager); + } + return ret; + } + + void add_segment_manager(SegmentManager* segment_manager) { + auto device_id = segment_manager->get_device_id(); + ceph_assert(!has_device(device_id)); + if (!device_ids.empty()) { + auto existing_id = *device_ids.begin(); + ceph_assert(segment_managers[existing_id]->get_device_type() + == segment_manager->get_device_type()); + } + segment_managers[device_id] = segment_manager; + device_ids.insert(device_id); + } + + void reset() { + segment_managers.clear(); + segment_managers.resize(DEVICE_ID_MAX, nullptr); + device_ids.clear(); + } + + /** + * get device info + * + * Assume all segment managers share the same following information. + */ + extent_len_t get_block_size() const { + assert(device_ids.size()); + return segment_managers[*device_ids.begin()]->get_block_size(); + } + + segment_off_t get_segment_size() const { + assert(device_ids.size()); + return segment_managers[*device_ids.begin()]->get_segment_size(); + } + + const seastore_meta_t &get_meta() const { + assert(device_ids.size()); + return segment_managers[*device_ids.begin()]->get_meta(); + } + + std::size_t get_rounded_header_length() const { + return p2roundup( + ceph::encoded_sizeof_bounded<segment_header_t>(), + (std::size_t)get_block_size()); + } + + std::size_t get_rounded_tail_length() const { + return p2roundup( + ceph::encoded_sizeof_bounded<segment_tail_t>(), + (std::size_t)get_block_size()); + } + + using read_segment_header_ertr = crimson::errorator< + crimson::ct_error::enoent, + crimson::ct_error::enodata, + crimson::ct_error::input_output_error + >; + using read_segment_header_ret = read_segment_header_ertr::future< + segment_header_t>; + read_segment_header_ret read_segment_header(segment_id_t segment); + + using read_segment_tail_ertr = read_segment_header_ertr; + using read_segment_tail_ret = read_segment_tail_ertr::future< + segment_tail_t>; + read_segment_tail_ret read_segment_tail(segment_id_t segment); + + /* + * read journal segment headers + */ + using find_journal_segment_headers_ertr = crimson::errorator< + crimson::ct_error::input_output_error>; + using find_journal_segment_headers_ret_bare = std::vector< + std::pair<segment_id_t, segment_header_t>>; + using find_journal_segment_headers_ret = find_journal_segment_headers_ertr::future< + find_journal_segment_headers_ret_bare>; + find_journal_segment_headers_ret find_journal_segment_headers(); + + using open_ertr = SegmentManager::open_ertr; + open_ertr::future<SegmentRef> open(segment_id_t id) { + assert(has_device(id.device_id())); + return segment_managers[id.device_id()]->open(id); + } + + using release_ertr = SegmentManager::release_ertr; + release_ertr::future<> release_segment(segment_id_t id) { + assert(has_device(id.device_id())); + return segment_managers[id.device_id()]->release(id); + } + +private: + bool has_device(device_id_t id) const { + assert(id <= DEVICE_ID_MAX_VALID); + return device_ids.count(id) >= 1; + } + + void initialize_cursor(scan_valid_records_cursor &cursor) final; + + read_ret read(paddr_t start, size_t len) final; + + bool is_record_segment_seq_invalid(scan_valid_records_cursor &cursor, + record_group_header_t &header) final { + return false; + } + + int64_t get_segment_end_offset(paddr_t addr) final { + auto& seg_addr = addr.as_seg_paddr(); + auto& segment_manager = *segment_managers[seg_addr.get_segment_id().device_id()]; + return static_cast<int64_t>(segment_manager.get_segment_size()); + } + + std::vector<SegmentManager*> segment_managers; + std::set<device_id_t> device_ids; +}; + +using SegmentManagerGroupRef = std::unique_ptr<SegmentManagerGroup>; + +} // namespace crimson::os::seastore diff --git a/src/crimson/os/seastore/segment_seq_allocator.h b/src/crimson/os/seastore/segment_seq_allocator.h new file mode 100644 index 000000000..28c81bf32 --- /dev/null +++ b/src/crimson/os/seastore/segment_seq_allocator.h @@ -0,0 +1,50 @@ +// -*- 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/logging.h" +#include "crimson/os/seastore/seastore_types.h" + +namespace crimson::os::seastore { +class AsyncCleaner; +} + +namespace crimson::os::seastore::journal { +class SegmentedJournal; +} + +namespace crimson::os::seastore { + +class SegmentSeqAllocator { +public: + SegmentSeqAllocator(segment_type_t type) + : type(type) {} + segment_seq_t get_and_inc_next_segment_seq() { + return next_segment_seq++; + } +private: + void set_next_segment_seq(segment_seq_t seq) { + LOG_PREFIX(SegmentSeqAllocator::set_next_segment_seq); + SUBDEBUG( + seastore_journal, + "{}, next={}, cur={}", + type, + segment_seq_printer_t{seq}, + segment_seq_printer_t{next_segment_seq}); + assert(type == segment_type_t::JOURNAL + ? seq >= next_segment_seq + : true); + if (seq > next_segment_seq) + next_segment_seq = seq; + } + segment_seq_t next_segment_seq = 0; + segment_type_t type = segment_type_t::NULL_SEG; + friend class journal::SegmentedJournal; + friend class SegmentCleaner; +}; + +using SegmentSeqAllocatorRef = + std::unique_ptr<SegmentSeqAllocator>; + +}; diff --git a/src/crimson/os/seastore/transaction.cc b/src/crimson/os/seastore/transaction.cc new file mode 100644 index 000000000..4cab476c3 --- /dev/null +++ b/src/crimson/os/seastore/transaction.cc @@ -0,0 +1,8 @@ +#include "transaction.h" +#include "crimson/common/interruptible_future.h" + +namespace crimson::interruptible { +template +thread_local interrupt_cond_t<::crimson::os::seastore::TransactionConflictCondition> +interrupt_cond<::crimson::os::seastore::TransactionConflictCondition>; +} diff --git a/src/crimson/os/seastore/transaction.h b/src/crimson/os/seastore/transaction.h new file mode 100644 index 000000000..d423196fe --- /dev/null +++ b/src/crimson/os/seastore/transaction.h @@ -0,0 +1,653 @@ +// -*- 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 "crimson/common/log.h" +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/ordering_handle.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/cached_extent.h" +#include "crimson/os/seastore/root_block.h" + +namespace crimson::os::seastore { + +class SeaStore; +class Transaction; + +struct io_stat_t { + uint64_t num = 0; + uint64_t bytes = 0; + + bool is_clear() const { + return (num == 0 && bytes == 0); + } + + void increment(uint64_t _bytes) { + ++num; + bytes += _bytes; + } + + void increment_stat(const io_stat_t& stat) { + num += stat.num; + bytes += stat.bytes; + } +}; +inline std::ostream& operator<<(std::ostream& out, const io_stat_t& stat) { + return out << stat.num << "(" << stat.bytes << "B)"; +} + +struct version_stat_t { + uint64_t num = 0; + uint64_t version = 0; + + bool is_clear() const { + return (num == 0 && version == 0); + } + + void increment(extent_version_t v) { + ++num; + version += v; + } + + void increment_stat(const version_stat_t& stat) { + num += stat.num; + version += stat.version; + } +}; + +/** + * Transaction + * + * Representation of in-progress mutation. Used exclusively through Cache methods. + * + * Transaction log levels: + * seastore_t + * - DEBUG: transaction create, conflict, commit events + * - TRACE: DEBUG details + * - seastore_cache logs + */ +class Transaction { +public: + using Ref = std::unique_ptr<Transaction>; + using on_destruct_func_t = std::function<void(Transaction&)>; + enum class get_extent_ret { + PRESENT, + ABSENT, + RETIRED + }; + get_extent_ret get_extent(paddr_t addr, CachedExtentRef *out) { + LOG_PREFIX(Transaction::get_extent); + // it's possible that both write_set and retired_set contain + // this addr at the same time when addr is absolute and the + // corresponding extent is used to map existing extent on disk. + // So search write_set first. + if (auto iter = write_set.find_offset(addr); + iter != write_set.end()) { + if (out) + *out = CachedExtentRef(&*iter); + SUBTRACET(seastore_cache, "{} is present in write_set -- {}", + *this, addr, *iter); + assert((*out)->is_valid()); + return get_extent_ret::PRESENT; + } else if (retired_set.count(addr)) { + return get_extent_ret::RETIRED; + } else if ( + auto iter = read_set.find(addr); + iter != read_set.end()) { + // placeholder in read-set should be in the retired-set + // at the same time. + assert(iter->ref->get_type() != extent_types_t::RETIRED_PLACEHOLDER); + if (out) + *out = iter->ref; + SUBTRACET(seastore_cache, "{} is present in read_set -- {}", + *this, addr, *(iter->ref)); + return get_extent_ret::PRESENT; + } else { + return get_extent_ret::ABSENT; + } + } + + void add_to_retired_set(CachedExtentRef ref) { + ceph_assert(!is_weak()); + if (ref->is_exist_clean() || + ref->is_exist_mutation_pending()) { + existing_block_stats.dec(ref); + ref->set_invalid(*this); + write_set.erase(*ref); + } else if (ref->is_initial_pending()) { + ref->set_invalid(*this); + write_set.erase(*ref); + } else if (ref->is_mutation_pending()) { + ref->set_invalid(*this); + write_set.erase(*ref); + assert(ref->prior_instance); + retired_set.insert(ref->prior_instance); + assert(read_set.count(ref->prior_instance->get_paddr())); + ref->prior_instance.reset(); + } else { + // && retired_set.count(ref->get_paddr()) == 0 + // If it's already in the set, insert here will be a noop, + // which is what we want. + retired_set.insert(ref); + } + } + + void add_to_read_set(CachedExtentRef ref) { + if (is_weak()) return; + + assert(ref->is_valid()); + + auto it = ref->transactions.lower_bound( + this, read_set_item_t<Transaction>::trans_cmp_t()); + if (it != ref->transactions.end() && it->t == this) return; + + auto [iter, inserted] = read_set.emplace(this, ref); + ceph_assert(inserted); + ref->transactions.insert_before( + it, const_cast<read_set_item_t<Transaction>&>(*iter)); + } + + void add_fresh_extent( + CachedExtentRef ref) { + ceph_assert(!is_weak()); + if (ref->is_exist_clean()) { + existing_block_stats.inc(ref); + existing_block_list.push_back(ref); + } else if (ref->get_paddr().is_delayed()) { + assert(ref->get_paddr() == make_delayed_temp_paddr(0)); + assert(ref->is_logical()); + ref->set_paddr(make_delayed_temp_paddr(delayed_temp_offset)); + delayed_temp_offset += ref->get_length(); + delayed_alloc_list.emplace_back(ref->cast<LogicalCachedExtent>()); + fresh_block_stats.increment(ref->get_length()); + } else if (ref->get_paddr().is_absolute()) { + pre_alloc_list.emplace_back(ref->cast<LogicalCachedExtent>()); + fresh_block_stats.increment(ref->get_length()); + } else { + if (likely(ref->get_paddr() == make_record_relative_paddr(0))) { + ref->set_paddr(make_record_relative_paddr(offset)); + } else { + ceph_assert(ref->get_paddr().is_fake()); + } + offset += ref->get_length(); + inline_block_list.push_back(ref); + fresh_block_stats.increment(ref->get_length()); + } + write_set.insert(*ref); + if (is_backref_node(ref->get_type())) + fresh_backref_extents++; + } + + uint64_t get_num_fresh_backref() const { + return fresh_backref_extents; + } + + void mark_delayed_extent_inline(LogicalCachedExtentRef& ref) { + write_set.erase(*ref); + assert(ref->get_paddr().is_delayed()); + ref->set_paddr(make_record_relative_paddr(offset), + /* need_update_mapping: */ true); + offset += ref->get_length(); + inline_block_list.push_back(ref); + write_set.insert(*ref); + } + + void mark_delayed_extent_ool(LogicalCachedExtentRef& ref) { + written_ool_block_list.push_back(ref); + } + + void update_delayed_ool_extent_addr(LogicalCachedExtentRef& ref, + paddr_t final_addr) { + write_set.erase(*ref); + assert(ref->get_paddr().is_delayed()); + ref->set_paddr(final_addr, /* need_update_mapping: */ true); + assert(!ref->get_paddr().is_null()); + assert(!ref->is_inline()); + write_set.insert(*ref); + } + + void mark_allocated_extent_ool(LogicalCachedExtentRef& ref) { + assert(ref->get_paddr().is_absolute()); + assert(!ref->is_inline()); + written_ool_block_list.push_back(ref); + } + + void add_mutated_extent(CachedExtentRef ref) { + ceph_assert(!is_weak()); + assert(ref->is_exist_mutation_pending() || + read_set.count(ref->prior_instance->get_paddr())); + mutated_block_list.push_back(ref); + if (!ref->is_exist_mutation_pending()) { + write_set.insert(*ref); + } else { + assert(write_set.find_offset(ref->get_paddr()) != + write_set.end()); + } + } + + void replace_placeholder(CachedExtent& placeholder, CachedExtent& extent) { + ceph_assert(!is_weak()); + + assert(placeholder.get_type() == extent_types_t::RETIRED_PLACEHOLDER); + assert(extent.get_type() != extent_types_t::RETIRED_PLACEHOLDER); + assert(extent.get_type() != extent_types_t::ROOT); + assert(extent.get_paddr() == placeholder.get_paddr()); + { + auto where = read_set.find(placeholder.get_paddr()); + assert(where != read_set.end()); + assert(where->ref.get() == &placeholder); + where = read_set.erase(where); + auto it = read_set.emplace_hint(where, this, &extent); + extent.transactions.insert(const_cast<read_set_item_t<Transaction>&>(*it)); + } + { + auto where = retired_set.find(&placeholder); + assert(where != retired_set.end()); + assert(where->get() == &placeholder); + where = retired_set.erase(where); + retired_set.emplace_hint(where, &extent); + } + } + + auto get_delayed_alloc_list() { + std::list<LogicalCachedExtentRef> ret; + for (auto& extent : delayed_alloc_list) { + // delayed extents may be invalidated + if (extent->is_valid()) { + ret.push_back(std::move(extent)); + } else { + ++num_delayed_invalid_extents; + } + } + delayed_alloc_list.clear(); + return ret; + } + + auto get_valid_pre_alloc_list() { + std::list<LogicalCachedExtentRef> ret; + assert(num_allocated_invalid_extents == 0); + for (auto& extent : pre_alloc_list) { + if (extent->is_valid()) { + ret.push_back(extent); + } else { + ++num_allocated_invalid_extents; + } + } + return ret; + } + + const auto &get_inline_block_list() { + return inline_block_list; + } + + const auto &get_mutated_block_list() { + return mutated_block_list; + } + + const auto &get_existing_block_list() { + return existing_block_list; + } + + const auto &get_retired_set() { + return retired_set; + } + + bool is_retired(paddr_t paddr, extent_len_t len) { + if (retired_set.empty()) { + return false; + } + auto iter = retired_set.lower_bound(paddr); + if (iter == retired_set.end() || + (*iter)->get_paddr() > paddr) { + assert(iter != retired_set.begin()); + --iter; + } + auto retired_paddr = (*iter)->get_paddr(); + auto retired_length = (*iter)->get_length(); + return retired_paddr <= paddr && + retired_paddr.add_offset(retired_length) >= paddr.add_offset(len); + } + + template <typename F> + auto for_each_fresh_block(F &&f) const { + std::for_each(written_ool_block_list.begin(), written_ool_block_list.end(), f); + std::for_each(inline_block_list.begin(), inline_block_list.end(), f); + } + + const io_stat_t& get_fresh_block_stats() const { + return fresh_block_stats; + } + + using src_t = transaction_type_t; + src_t get_src() const { + return src; + } + + bool is_weak() const { + return weak; + } + + void test_set_conflict() { + conflicted = true; + } + + bool is_conflicted() const { + return conflicted; + } + + auto &get_handle() { + return handle; + } + + Transaction( + OrderingHandle &&handle, + bool weak, + src_t src, + journal_seq_t initiated_after, + on_destruct_func_t&& f, + transaction_id_t trans_id + ) : weak(weak), + handle(std::move(handle)), + on_destruct(std::move(f)), + src(src), + trans_id(trans_id) + {} + + void invalidate_clear_write_set() { + for (auto &&i: write_set) { + i.set_invalid(*this); + } + write_set.clear(); + } + + ~Transaction() { + on_destruct(*this); + invalidate_clear_write_set(); + } + + friend class crimson::os::seastore::SeaStore; + friend class TransactionConflictCondition; + + void reset_preserve_handle(journal_seq_t initiated_after) { + root.reset(); + offset = 0; + delayed_temp_offset = 0; + read_set.clear(); + fresh_backref_extents = 0; + invalidate_clear_write_set(); + mutated_block_list.clear(); + fresh_block_stats = {}; + num_delayed_invalid_extents = 0; + num_allocated_invalid_extents = 0; + delayed_alloc_list.clear(); + inline_block_list.clear(); + written_ool_block_list.clear(); + pre_alloc_list.clear(); + retired_set.clear(); + existing_block_list.clear(); + existing_block_stats = {}; + onode_tree_stats = {}; + omap_tree_stats = {}; + lba_tree_stats = {}; + backref_tree_stats = {}; + ool_write_stats = {}; + rewrite_version_stats = {}; + conflicted = false; + if (!has_reset) { + has_reset = true; + } + } + + bool did_reset() const { + return has_reset; + } + + struct tree_stats_t { + uint64_t depth = 0; + uint64_t num_inserts = 0; + uint64_t num_erases = 0; + uint64_t num_updates = 0; + int64_t extents_num_delta = 0; + + bool is_clear() const { + return (depth == 0 && + num_inserts == 0 && + num_erases == 0 && + num_updates == 0 && + extents_num_delta == 0); + } + }; + tree_stats_t& get_onode_tree_stats() { + return onode_tree_stats; + } + tree_stats_t& get_omap_tree_stats() { + return omap_tree_stats; + } + tree_stats_t& get_lba_tree_stats() { + return lba_tree_stats; + } + tree_stats_t& get_backref_tree_stats() { + return backref_tree_stats; + } + + struct ool_write_stats_t { + io_stat_t extents; + uint64_t md_bytes = 0; + uint64_t num_records = 0; + + uint64_t get_data_bytes() const { + return extents.bytes; + } + + bool is_clear() const { + return (extents.is_clear() && + md_bytes == 0 && + num_records == 0); + } + }; + ool_write_stats_t& get_ool_write_stats() { + return ool_write_stats; + } + version_stat_t& get_rewrite_version_stats() { + return rewrite_version_stats; + } + + struct existing_block_stats_t { + uint64_t valid_num = 0; + uint64_t clean_num = 0; + uint64_t mutated_num = 0; + void inc(const CachedExtentRef &ref) { + valid_num++; + if (ref->is_exist_clean()) { + clean_num++; + } else { + mutated_num++; + } + } + void dec(const CachedExtentRef &ref) { + valid_num--; + if (ref->is_exist_clean()) { + clean_num--; + } else { + mutated_num--; + } + } + }; + existing_block_stats_t& get_existing_block_stats() { + return existing_block_stats; + } + + transaction_id_t get_trans_id() const { + return trans_id; + } + +private: + friend class Cache; + friend Ref make_test_transaction(); + + /** + * If set, *this may not be used to perform writes and will not provide + * consistentency allowing operations using to avoid maintaining a read_set. + */ + const bool weak; + + RootBlockRef root; ///< ref to root if read or written by transaction + + device_off_t offset = 0; ///< relative offset of next block + device_off_t delayed_temp_offset = 0; + + /** + * read_set + * + * Holds a reference (with a refcount) to every extent read via *this. + * Submitting a transaction mutating any contained extent/addr will + * invalidate *this. + */ + read_set_t<Transaction> read_set; ///< set of extents read by paddr + + uint64_t fresh_backref_extents = 0; // counter of new backref extents + + /** + * write_set + * + * Contains a reference (without a refcount) to every extent mutated + * as part of *this. No contained extent may be referenced outside + * of *this. Every contained extent will be in one of inline_block_list, + * written_ool_block_list or/and pre_alloc_list, mutated_block_list, + * or delayed_alloc_list. + */ + ExtentIndex write_set; + + /** + * lists of fresh blocks, holds refcounts, subset of write_set + */ + io_stat_t fresh_block_stats; + uint64_t num_delayed_invalid_extents = 0; + uint64_t num_allocated_invalid_extents = 0; + /// blocks that will be committed with journal record inline + std::list<CachedExtentRef> inline_block_list; + /// blocks that will be committed with out-of-line record + std::list<CachedExtentRef> written_ool_block_list; + /// blocks with delayed allocation, may become inline or ool above + std::list<LogicalCachedExtentRef> delayed_alloc_list; + + /// Extents with pre-allocated addresses, + /// will be added to written_ool_block_list after write + std::list<LogicalCachedExtentRef> pre_alloc_list; + + /// list of mutated blocks, holds refcounts, subset of write_set + std::list<CachedExtentRef> mutated_block_list; + + /// partial blocks of extents on disk, with data and refcounts + std::list<CachedExtentRef> existing_block_list; + existing_block_stats_t existing_block_stats; + + /** + * retire_set + * + * Set of extents retired by *this. + */ + pextent_set_t retired_set; + + /// stats to collect when commit or invalidate + tree_stats_t onode_tree_stats; + tree_stats_t omap_tree_stats; // exclude omap tree depth + tree_stats_t lba_tree_stats; + tree_stats_t backref_tree_stats; + ool_write_stats_t ool_write_stats; + version_stat_t rewrite_version_stats; + + bool conflicted = false; + + bool has_reset = false; + + OrderingHandle handle; + + on_destruct_func_t on_destruct; + + const src_t src; + + transaction_id_t trans_id = TRANS_ID_NULL; +}; +using TransactionRef = Transaction::Ref; + +/// Should only be used with dummy staged-fltree node extent manager +inline TransactionRef make_test_transaction() { + static transaction_id_t next_id = 0; + return std::make_unique<Transaction>( + get_dummy_ordering_handle(), + false, + Transaction::src_t::MUTATE, + JOURNAL_SEQ_NULL, + [](Transaction&) {}, + ++next_id + ); +} + +struct TransactionConflictCondition { + class transaction_conflict final : public std::exception { + public: + const char* what() const noexcept final { + return "transaction conflict detected"; + } + }; + +public: + TransactionConflictCondition(Transaction &t) : t(t) {} + + template <typename Fut> + std::optional<Fut> may_interrupt() { + if (t.conflicted) { + return seastar::futurize<Fut>::make_exception_future( + transaction_conflict()); + } else { + return std::optional<Fut>(); + } + } + + template <typename T> + static constexpr bool is_interruption_v = + std::is_same_v<T, transaction_conflict>; + + + static bool is_interruption(std::exception_ptr& eptr) { + return *eptr.__cxa_exception_type() == typeid(transaction_conflict); + } + +private: + Transaction &t; +}; + +using trans_intr = crimson::interruptible::interruptor< + TransactionConflictCondition + >; + +template <typename E> +using trans_iertr = + crimson::interruptible::interruptible_errorator< + TransactionConflictCondition, + E + >; + +template <typename F, typename... Args> +auto with_trans_intr(Transaction &t, F &&f, Args&&... args) { + return trans_intr::with_interruption_to_error<crimson::ct_error::eagain>( + std::move(f), + TransactionConflictCondition(t), + t, + std::forward<Args>(args)...); +} + +template <typename T> +using with_trans_ertr = typename T::base_ertr::template extend<crimson::ct_error::eagain>; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<crimson::os::seastore::io_stat_t> : fmt::ostream_formatter {}; +#endif diff --git a/src/crimson/os/seastore/transaction_manager.cc b/src/crimson/os/seastore/transaction_manager.cc new file mode 100644 index 000000000..ad8e5f1a6 --- /dev/null +++ b/src/crimson/os/seastore/transaction_manager.cc @@ -0,0 +1,759 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab expandtab + +#include "include/denc.h" +#include "include/intarith.h" + +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/transaction_manager.h" +#include "crimson/os/seastore/journal.h" +#include "crimson/os/seastore/journal/circular_bounded_journal.h" +#include "crimson/os/seastore/lba_manager/btree/lba_btree_node.h" +#include "crimson/os/seastore/random_block_manager/rbm_device.h" + +/* + * TransactionManager logs + * + * levels: + * - INFO: major initiation, closing operations + * - DEBUG: major extent related operations, INFO details + * - TRACE: DEBUG details + * - seastore_t logs + */ +SET_SUBSYS(seastore_tm); + +namespace crimson::os::seastore { + +TransactionManager::TransactionManager( + JournalRef _journal, + CacheRef _cache, + LBAManagerRef _lba_manager, + ExtentPlacementManagerRef &&_epm, + BackrefManagerRef&& _backref_manager) + : cache(std::move(_cache)), + lba_manager(std::move(_lba_manager)), + journal(std::move(_journal)), + epm(std::move(_epm)), + backref_manager(std::move(_backref_manager)) +{ + epm->set_extent_callback(this); + journal->set_write_pipeline(&write_pipeline); +} + +TransactionManager::mkfs_ertr::future<> TransactionManager::mkfs() +{ + LOG_PREFIX(TransactionManager::mkfs); + INFO("enter"); + return epm->mount( + ).safe_then([this] { + return journal->open_for_mkfs(); + }).safe_then([this](auto start_seq) { + journal->get_trimmer().update_journal_tails(start_seq, start_seq); + journal->get_trimmer().set_journal_head(start_seq); + return epm->open_for_write(); + }).safe_then([this, FNAME]() { + return with_transaction_intr( + Transaction::src_t::MUTATE, + "mkfs_tm", + [this, FNAME](auto& t) + { + cache->init(); + return cache->mkfs(t + ).si_then([this, &t] { + return lba_manager->mkfs(t); + }).si_then([this, &t] { + return backref_manager->mkfs(t); + }).si_then([this, FNAME, &t] { + INFOT("submitting mkfs transaction", t); + return submit_transaction_direct(t); + }); + }).handle_error( + crimson::ct_error::eagain::handle([] { + ceph_assert(0 == "eagain impossible"); + return mkfs_ertr::now(); + }), + mkfs_ertr::pass_further{} + ); + }).safe_then([this] { + return close(); + }).safe_then([FNAME] { + INFO("completed"); + }); +} + +TransactionManager::mount_ertr::future<> TransactionManager::mount() +{ + LOG_PREFIX(TransactionManager::mount); + INFO("enter"); + cache->init(); + return epm->mount( + ).safe_then([this] { + return journal->replay( + [this]( + const auto &offsets, + const auto &e, + const journal_seq_t &dirty_tail, + const journal_seq_t &alloc_tail, + sea_time_point modify_time) + { + auto start_seq = offsets.write_result.start_seq; + return cache->replay_delta( + start_seq, + offsets.record_block_base, + e, + dirty_tail, + alloc_tail, + modify_time); + }); + }).safe_then([this] { + return journal->open_for_mount(); + }).safe_then([this](auto start_seq) { + journal->get_trimmer().set_journal_head(start_seq); + return with_transaction_weak( + "mount", + [this](auto &t) + { + return cache->init_cached_extents(t, [this](auto &t, auto &e) { + if (is_backref_node(e->get_type())) { + return backref_manager->init_cached_extent(t, e); + } else { + return lba_manager->init_cached_extent(t, e); + } + }).si_then([this, &t] { + epm->start_scan_space(); + return backref_manager->scan_mapped_space( + t, + [this]( + paddr_t paddr, + paddr_t backref_key, + extent_len_t len, + extent_types_t type, + laddr_t laddr) { + if (is_backref_node(type)) { + assert(laddr == L_ADDR_NULL); + assert(backref_key != P_ADDR_NULL); + backref_manager->cache_new_backref_extent(paddr, backref_key, type); + cache->update_tree_extents_num(type, 1); + epm->mark_space_used(paddr, len); + } else if (laddr == L_ADDR_NULL) { + assert(backref_key == P_ADDR_NULL); + cache->update_tree_extents_num(type, -1); + epm->mark_space_free(paddr, len); + } else { + assert(backref_key == P_ADDR_NULL); + cache->update_tree_extents_num(type, 1); + epm->mark_space_used(paddr, len); + } + }); + }); + }); + }).safe_then([this] { + return epm->open_for_write(); + }).safe_then([FNAME, this] { + epm->start_background(); + INFO("completed"); + }).handle_error( + mount_ertr::pass_further{}, + crimson::ct_error::all_same_way([] { + ceph_assert(0 == "unhandled error"); + return mount_ertr::now(); + }) + ); +} + +TransactionManager::close_ertr::future<> TransactionManager::close() { + LOG_PREFIX(TransactionManager::close); + INFO("enter"); + return epm->stop_background( + ).then([this] { + return cache->close(); + }).safe_then([this] { + cache->dump_contents(); + return journal->close(); + }).safe_then([this] { + return epm->close(); + }).safe_then([FNAME] { + INFO("completed"); + return seastar::now(); + }); +} + +TransactionManager::ref_ret TransactionManager::inc_ref( + Transaction &t, + LogicalCachedExtentRef &ref) +{ + LOG_PREFIX(TransactionManager::inc_ref); + TRACET("{}", t, *ref); + return lba_manager->incref_extent(t, ref->get_laddr() + ).si_then([FNAME, ref, &t](auto result) { + DEBUGT("extent refcount is incremented to {} -- {}", + t, result.refcount, *ref); + return result.refcount; + }).handle_error_interruptible( + ref_iertr::pass_further{}, + ct_error::all_same_way([](auto e) { + ceph_assert(0 == "unhandled error, TODO"); + })); +} + +TransactionManager::ref_ret TransactionManager::inc_ref( + Transaction &t, + laddr_t offset) +{ + LOG_PREFIX(TransactionManager::inc_ref); + TRACET("{}", t, offset); + return lba_manager->incref_extent(t, offset + ).si_then([FNAME, offset, &t](auto result) { + DEBUGT("extent refcount is incremented to {} -- {}~{}, {}", + t, result.refcount, offset, result.length, result.addr); + return result.refcount; + }); +} + +TransactionManager::ref_ret TransactionManager::dec_ref( + Transaction &t, + LogicalCachedExtentRef &ref) +{ + LOG_PREFIX(TransactionManager::dec_ref); + TRACET("{}", t, *ref); + return lba_manager->decref_extent(t, ref->get_laddr(), true + ).si_then([this, FNAME, &t, ref](auto result) { + DEBUGT("extent refcount is decremented to {} -- {}", + t, result.refcount, *ref); + if (result.refcount == 0) { + cache->retire_extent(t, ref); + } + return result.refcount; + }); +} + +TransactionManager::ref_ret TransactionManager::_dec_ref( + Transaction &t, + laddr_t offset, + bool cascade_remove) +{ + LOG_PREFIX(TransactionManager::_dec_ref); + TRACET("{}", t, offset); + return lba_manager->decref_extent(t, offset, cascade_remove + ).si_then([this, FNAME, offset, &t](auto result) -> ref_ret { + DEBUGT("extent refcount is decremented to {} -- {}~{}, {}", + t, result.refcount, offset, result.length, result.addr); + auto fut = ref_iertr::now(); + if (result.refcount == 0) { + if (result.addr.is_paddr() && + !result.addr.get_paddr().is_zero()) { + fut = cache->retire_extent_addr( + t, result.addr.get_paddr(), result.length); + } + } + + return fut.si_then([result=std::move(result)] { + return result.refcount; + }); + }); +} + +TransactionManager::refs_ret TransactionManager::dec_ref( + Transaction &t, + std::vector<laddr_t> offsets) +{ + LOG_PREFIX(TransactionManager::dec_ref); + DEBUG("{} offsets", offsets.size()); + return seastar::do_with(std::move(offsets), std::vector<unsigned>(), + [this, &t] (auto &&offsets, auto &refcnt) { + return trans_intr::do_for_each(offsets.begin(), offsets.end(), + [this, &t, &refcnt] (auto &laddr) { + return this->dec_ref(t, laddr).si_then([&refcnt] (auto ref) { + refcnt.push_back(ref); + return ref_iertr::now(); + }); + }).si_then([&refcnt] { + return ref_iertr::make_ready_future<std::vector<unsigned>>(std::move(refcnt)); + }); + }); +} + +TransactionManager::submit_transaction_iertr::future<> +TransactionManager::submit_transaction( + Transaction &t) +{ + LOG_PREFIX(TransactionManager::submit_transaction); + SUBTRACET(seastore_t, "start", t); + return trans_intr::make_interruptible( + t.get_handle().enter(write_pipeline.reserve_projected_usage) + ).then_interruptible([this, FNAME, &t] { + auto dispatch_result = epm->dispatch_delayed_extents(t); + auto projected_usage = dispatch_result.usage; + SUBTRACET(seastore_t, "waiting for projected_usage: {}", t, projected_usage); + return trans_intr::make_interruptible( + epm->reserve_projected_usage(projected_usage) + ).then_interruptible([this, &t, dispatch_result = std::move(dispatch_result)] { + return do_submit_transaction(t, std::move(dispatch_result)); + }).finally([this, FNAME, projected_usage, &t] { + SUBTRACET(seastore_t, "releasing projected_usage: {}", t, projected_usage); + epm->release_projected_usage(projected_usage); + }); + }); +} + +TransactionManager::submit_transaction_direct_ret +TransactionManager::submit_transaction_direct( + Transaction &tref, + std::optional<journal_seq_t> trim_alloc_to) +{ + return do_submit_transaction( + tref, + epm->dispatch_delayed_extents(tref), + trim_alloc_to); +} + +TransactionManager::submit_transaction_direct_ret +TransactionManager::do_submit_transaction( + Transaction &tref, + ExtentPlacementManager::dispatch_result_t dispatch_result, + std::optional<journal_seq_t> trim_alloc_to) +{ + LOG_PREFIX(TransactionManager::do_submit_transaction); + SUBTRACET(seastore_t, "start", tref); + return trans_intr::make_interruptible( + tref.get_handle().enter(write_pipeline.ool_writes) + ).then_interruptible([this, FNAME, &tref, + dispatch_result = std::move(dispatch_result)] { + return seastar::do_with(std::move(dispatch_result), + [this, FNAME, &tref](auto &dispatch_result) { + return epm->write_delayed_ool_extents(tref, dispatch_result.alloc_map + ).si_then([this, FNAME, &tref, &dispatch_result] { + SUBTRACET(seastore_t, "update delayed extent mappings", tref); + return lba_manager->update_mappings(tref, dispatch_result.delayed_extents); + }).handle_error_interruptible( + crimson::ct_error::input_output_error::pass_further(), + crimson::ct_error::assert_all("invalid error") + ); + }); + }).si_then([this, FNAME, &tref] { + auto allocated_extents = tref.get_valid_pre_alloc_list(); + auto num_extents = allocated_extents.size(); + SUBTRACET(seastore_t, "process {} allocated extents", tref, num_extents); + return epm->write_preallocated_ool_extents(tref, allocated_extents + ).handle_error_interruptible( + crimson::ct_error::input_output_error::pass_further(), + crimson::ct_error::assert_all("invalid error") + ); + }).si_then([this, FNAME, &tref] { + SUBTRACET(seastore_t, "about to prepare", tref); + return tref.get_handle().enter(write_pipeline.prepare); + }).si_then([this, FNAME, &tref, trim_alloc_to=std::move(trim_alloc_to)]() mutable + -> submit_transaction_iertr::future<> { + if (trim_alloc_to && *trim_alloc_to != JOURNAL_SEQ_NULL) { + cache->trim_backref_bufs(*trim_alloc_to); + } + + auto record = cache->prepare_record( + tref, + journal->get_trimmer().get_journal_head(), + journal->get_trimmer().get_dirty_tail()); + + tref.get_handle().maybe_release_collection_lock(); + + SUBTRACET(seastore_t, "about to submit to journal", tref); + return journal->submit_record(std::move(record), tref.get_handle() + ).safe_then([this, FNAME, &tref](auto submit_result) mutable { + SUBDEBUGT(seastore_t, "committed with {}", tref, submit_result); + auto start_seq = submit_result.write_result.start_seq; + journal->get_trimmer().set_journal_head(start_seq); + cache->complete_commit( + tref, + submit_result.record_block_base, + start_seq); + + std::vector<CachedExtentRef> lba_to_clear; + std::vector<CachedExtentRef> backref_to_clear; + lba_to_clear.reserve(tref.get_retired_set().size()); + backref_to_clear.reserve(tref.get_retired_set().size()); + for (auto &e: tref.get_retired_set()) { + if (e->is_logical() || is_lba_node(e->get_type())) + lba_to_clear.push_back(e); + else if (is_backref_node(e->get_type())) + backref_to_clear.push_back(e); + } + + journal->get_trimmer().update_journal_tails( + cache->get_oldest_dirty_from().value_or(start_seq), + cache->get_oldest_backref_dirty_from().value_or(start_seq)); + return journal->finish_commit(tref.get_src() + ).then([&tref] { + return tref.get_handle().complete(); + }); + }).handle_error( + submit_transaction_iertr::pass_further{}, + crimson::ct_error::all_same_way([](auto e) { + ceph_assert(0 == "Hit error submitting to journal"); + }) + ); + }).finally([&tref]() { + tref.get_handle().exit(); + }); +} + +seastar::future<> TransactionManager::flush(OrderingHandle &handle) +{ + LOG_PREFIX(TransactionManager::flush); + SUBDEBUG(seastore_t, "H{} start", (void*)&handle); + return handle.enter(write_pipeline.reserve_projected_usage + ).then([this, &handle] { + return handle.enter(write_pipeline.ool_writes); + }).then([this, &handle] { + return handle.enter(write_pipeline.prepare); + }).then([this, &handle] { + handle.maybe_release_collection_lock(); + return journal->flush(handle); + }).then([FNAME, &handle] { + SUBDEBUG(seastore_t, "H{} completed", (void*)&handle); + }); +} + +TransactionManager::get_next_dirty_extents_ret +TransactionManager::get_next_dirty_extents( + Transaction &t, + journal_seq_t seq, + size_t max_bytes) +{ + LOG_PREFIX(TransactionManager::get_next_dirty_extents); + DEBUGT("max_bytes={}B, seq={}", t, max_bytes, seq); + return cache->get_next_dirty_extents(t, seq, max_bytes); +} + +TransactionManager::rewrite_extent_ret +TransactionManager::rewrite_logical_extent( + Transaction& t, + LogicalCachedExtentRef extent) +{ + LOG_PREFIX(TransactionManager::rewrite_logical_extent); + if (extent->has_been_invalidated()) { + ERRORT("extent has been invalidated -- {}", t, *extent); + ceph_abort(); + } + TRACET("rewriting extent -- {}", t, *extent); + + auto lextent = extent->cast<LogicalCachedExtent>(); + cache->retire_extent(t, extent); + auto nlextent = cache->alloc_new_extent_by_type( + t, + lextent->get_type(), + lextent->get_length(), + lextent->get_user_hint(), + // get target rewrite generation + lextent->get_rewrite_generation())->cast<LogicalCachedExtent>(); + lextent->get_bptr().copy_out( + 0, + lextent->get_length(), + nlextent->get_bptr().c_str()); + nlextent->set_laddr(lextent->get_laddr()); + nlextent->set_modify_time(lextent->get_modify_time()); + + DEBUGT("rewriting logical extent -- {} to {}", t, *lextent, *nlextent); + + /* This update_mapping is, strictly speaking, unnecessary for delayed_alloc + * extents since we're going to do it again once we either do the ool write + * or allocate a relative inline addr. TODO: refactor AsyncCleaner to + * avoid this complication. */ + return lba_manager->update_mapping( + t, + lextent->get_laddr(), + lextent->get_paddr(), + nlextent->get_paddr(), + nlextent.get()); +} + +TransactionManager::rewrite_extent_ret TransactionManager::rewrite_extent( + Transaction &t, + CachedExtentRef extent, + rewrite_gen_t target_generation, + sea_time_point modify_time) +{ + LOG_PREFIX(TransactionManager::rewrite_extent); + + { + auto updated = cache->update_extent_from_transaction(t, extent); + if (!updated) { + DEBUGT("extent is already retired, skipping -- {}", t, *extent); + return rewrite_extent_iertr::now(); + } + extent = updated; + ceph_assert(!extent->is_pending_io()); + } + + assert(extent->is_valid() && !extent->is_initial_pending()); + if (extent->is_dirty()) { + extent->set_target_rewrite_generation(INIT_GENERATION); + } else { + extent->set_target_rewrite_generation(target_generation); + ceph_assert(modify_time != NULL_TIME); + extent->set_modify_time(modify_time); + } + + t.get_rewrite_version_stats().increment(extent->get_version()); + + if (is_backref_node(extent->get_type())) { + DEBUGT("rewriting backref extent -- {}", t, *extent); + return backref_manager->rewrite_extent(t, extent); + } + + if (extent->get_type() == extent_types_t::ROOT) { + DEBUGT("rewriting root extent -- {}", t, *extent); + cache->duplicate_for_write(t, extent); + return rewrite_extent_iertr::now(); + } + + if (extent->is_logical()) { + return rewrite_logical_extent(t, extent->cast<LogicalCachedExtent>()); + } else { + DEBUGT("rewriting physical extent -- {}", t, *extent); + return lba_manager->rewrite_extent(t, extent); + } +} + +TransactionManager::get_extents_if_live_ret +TransactionManager::get_extents_if_live( + Transaction &t, + extent_types_t type, + paddr_t paddr, + laddr_t laddr, + extent_len_t len) +{ + LOG_PREFIX(TransactionManager::get_extent_if_live); + TRACET("{} {}~{} {}", t, type, laddr, len, paddr); + + // This only works with segments to check if alive, + // as parallel transactions may split the extent at the same time. + ceph_assert(paddr.get_addr_type() == paddr_types_t::SEGMENT); + + return cache->get_extent_if_cached(t, paddr, type + ).si_then([=, this, &t](auto extent) + -> get_extents_if_live_ret { + if (extent && extent->get_length() == len) { + DEBUGT("{} {}~{} {} is live in cache -- {}", + t, type, laddr, len, paddr, *extent); + std::list<CachedExtentRef> res; + res.emplace_back(std::move(extent)); + return get_extents_if_live_ret( + interruptible::ready_future_marker{}, + res); + } + + if (is_logical_type(type)) { + return lba_manager->get_mappings( + t, + laddr, + len + ).si_then([=, this, &t](lba_pin_list_t pin_list) { + return seastar::do_with( + std::list<CachedExtentRef>(), + [=, this, &t, pin_list=std::move(pin_list)]( + std::list<CachedExtentRef> &list) mutable + { + auto paddr_seg_id = paddr.as_seg_paddr().get_segment_id(); + return trans_intr::parallel_for_each( + pin_list, + [=, this, &list, &t]( + LBAMappingRef &pin) -> Cache::get_extent_iertr::future<> + { + auto pin_paddr = pin->get_val(); + auto &pin_seg_paddr = pin_paddr.as_seg_paddr(); + auto pin_paddr_seg_id = pin_seg_paddr.get_segment_id(); + auto pin_len = pin->get_length(); + if (pin_paddr_seg_id != paddr_seg_id) { + return seastar::now(); + } + // Only extent split can happen during the lookup + ceph_assert(pin_seg_paddr >= paddr && + pin_seg_paddr.add_offset(pin_len) <= paddr.add_offset(len)); + return read_pin_by_type(t, std::move(pin), type + ).si_then([&list](auto ret) { + list.emplace_back(std::move(ret)); + return seastar::now(); + }); + }).si_then([&list] { + return get_extents_if_live_ret( + interruptible::ready_future_marker{}, + std::move(list)); + }); + }); + }).handle_error_interruptible(crimson::ct_error::enoent::handle([] { + return get_extents_if_live_ret( + interruptible::ready_future_marker{}, + std::list<CachedExtentRef>()); + }), crimson::ct_error::pass_further_all{}); + } else { + return lba_manager->get_physical_extent_if_live( + t, + type, + paddr, + laddr, + len + ).si_then([=, &t](auto ret) { + std::list<CachedExtentRef> res; + if (ret) { + DEBUGT("{} {}~{} {} is live as physical extent -- {}", + t, type, laddr, len, paddr, *ret); + res.emplace_back(std::move(ret)); + } else { + DEBUGT("{} {}~{} {} is not live as physical extent", + t, type, laddr, len, paddr); + } + return get_extents_if_live_ret( + interruptible::ready_future_marker{}, + std::move(res)); + }); + } + }); +} + +TransactionManager::~TransactionManager() {} + +TransactionManagerRef make_transaction_manager( + Device *primary_device, + const std::vector<Device*> &secondary_devices, + bool is_test) +{ + auto epm = std::make_unique<ExtentPlacementManager>(); + auto cache = std::make_unique<Cache>(*epm); + auto lba_manager = lba_manager::create_lba_manager(*cache); + auto sms = std::make_unique<SegmentManagerGroup>(); + auto rbs = std::make_unique<RBMDeviceGroup>(); + auto backref_manager = create_backref_manager(*cache); + SegmentManagerGroupRef cold_sms = nullptr; + std::vector<SegmentProvider*> segment_providers_by_id{DEVICE_ID_MAX, nullptr}; + + auto p_backend_type = primary_device->get_backend_type(); + + if (p_backend_type == backend_type_t::SEGMENTED) { + auto dtype = primary_device->get_device_type(); + ceph_assert(dtype != device_type_t::HDD && + dtype != device_type_t::EPHEMERAL_COLD); + sms->add_segment_manager(static_cast<SegmentManager*>(primary_device)); + } else { + auto rbm = std::make_unique<BlockRBManager>( + static_cast<RBMDevice*>(primary_device), "", is_test); + rbs->add_rb_manager(std::move(rbm)); + } + + for (auto &p_dev : secondary_devices) { + if (p_dev->get_backend_type() == backend_type_t::SEGMENTED) { + if (p_dev->get_device_type() == primary_device->get_device_type()) { + sms->add_segment_manager(static_cast<SegmentManager*>(p_dev)); + } else { + if (!cold_sms) { + cold_sms = std::make_unique<SegmentManagerGroup>(); + } + cold_sms->add_segment_manager(static_cast<SegmentManager*>(p_dev)); + } + } else { + auto rbm = std::make_unique<BlockRBManager>( + static_cast<RBMDevice*>(p_dev), "", is_test); + rbs->add_rb_manager(std::move(rbm)); + } + } + + auto journal_type = p_backend_type; + device_off_t roll_size; + device_off_t roll_start; + if (journal_type == journal_type_t::SEGMENTED) { + roll_size = static_cast<SegmentManager*>(primary_device)->get_segment_size(); + roll_start = 0; + } else { + roll_size = static_cast<random_block_device::RBMDevice*>(primary_device) + ->get_journal_size() - primary_device->get_block_size(); + // see CircularBoundedJournal::get_records_start() + roll_start = static_cast<random_block_device::RBMDevice*>(primary_device) + ->get_shard_journal_start() + primary_device->get_block_size(); + ceph_assert_always(roll_size <= DEVICE_OFF_MAX); + ceph_assert_always((std::size_t)roll_size + roll_start <= + primary_device->get_available_size()); + } + ceph_assert(roll_size % primary_device->get_block_size() == 0); + ceph_assert(roll_start % primary_device->get_block_size() == 0); + + bool cleaner_is_detailed; + SegmentCleaner::config_t cleaner_config; + JournalTrimmerImpl::config_t trimmer_config; + if (is_test) { + cleaner_is_detailed = true; + cleaner_config = SegmentCleaner::config_t::get_test(); + trimmer_config = JournalTrimmerImpl::config_t::get_test( + roll_size, journal_type); + } else { + cleaner_is_detailed = false; + cleaner_config = SegmentCleaner::config_t::get_default(); + trimmer_config = JournalTrimmerImpl::config_t::get_default( + roll_size, journal_type); + } + + auto journal_trimmer = JournalTrimmerImpl::create( + *backref_manager, trimmer_config, + journal_type, roll_start, roll_size); + + AsyncCleanerRef cleaner; + JournalRef journal; + + SegmentCleanerRef cold_segment_cleaner = nullptr; + + if (cold_sms) { + cold_segment_cleaner = SegmentCleaner::create( + cleaner_config, + std::move(cold_sms), + *backref_manager, + epm->get_ool_segment_seq_allocator(), + cleaner_is_detailed, + /* is_cold = */ true); + if (journal_type == journal_type_t::SEGMENTED) { + for (auto id : cold_segment_cleaner->get_device_ids()) { + segment_providers_by_id[id] = + static_cast<SegmentProvider*>(cold_segment_cleaner.get()); + } + } + } + + if (journal_type == journal_type_t::SEGMENTED) { + cleaner = SegmentCleaner::create( + cleaner_config, + std::move(sms), + *backref_manager, + epm->get_ool_segment_seq_allocator(), + cleaner_is_detailed); + auto segment_cleaner = static_cast<SegmentCleaner*>(cleaner.get()); + for (auto id : segment_cleaner->get_device_ids()) { + segment_providers_by_id[id] = + static_cast<SegmentProvider*>(segment_cleaner); + } + segment_cleaner->set_journal_trimmer(*journal_trimmer); + journal = journal::make_segmented( + *segment_cleaner, + *journal_trimmer); + } else { + cleaner = RBMCleaner::create( + std::move(rbs), + *backref_manager, + cleaner_is_detailed); + journal = journal::make_circularbounded( + *journal_trimmer, + static_cast<random_block_device::RBMDevice*>(primary_device), + ""); + } + + cache->set_segment_providers(std::move(segment_providers_by_id)); + + epm->init(std::move(journal_trimmer), + std::move(cleaner), + std::move(cold_segment_cleaner)); + epm->set_primary_device(primary_device); + + return std::make_unique<TransactionManager>( + std::move(journal), + std::move(cache), + std::move(lba_manager), + std::move(epm), + std::move(backref_manager)); +} + +} diff --git a/src/crimson/os/seastore/transaction_manager.h b/src/crimson/os/seastore/transaction_manager.h new file mode 100644 index 000000000..dd1898ba7 --- /dev/null +++ b/src/crimson/os/seastore/transaction_manager.h @@ -0,0 +1,928 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#pragma once + +#include <iostream> +#include <optional> +#include <vector> +#include <utility> +#include <functional> + +#include <boost/intrusive_ptr.hpp> +#include <boost/iterator/counting_iterator.hpp> +#include <boost/smart_ptr/intrusive_ref_counter.hpp> + +#include <seastar/core/future.hh> + +#include "include/ceph_assert.h" +#include "include/buffer.h" + +#include "crimson/osd/exceptions.h" + +#include "crimson/os/seastore/logging.h" +#include "crimson/os/seastore/seastore_types.h" +#include "crimson/os/seastore/cache.h" +#include "crimson/os/seastore/lba_manager.h" +#include "crimson/os/seastore/backref_manager.h" +#include "crimson/os/seastore/journal.h" +#include "crimson/os/seastore/extent_placement_manager.h" +#include "crimson/os/seastore/device.h" + +namespace crimson::os::seastore { +class Journal; + +template <typename F> +auto repeat_eagain(F &&f) { + return seastar::do_with( + std::forward<F>(f), + [](auto &f) + { + return crimson::repeat([&f] { + return std::invoke(f + ).safe_then([] { + return seastar::stop_iteration::yes; + }).handle_error( + [](const crimson::ct_error::eagain &e) { + return seastar::stop_iteration::no; + }, + crimson::ct_error::pass_further_all{} + ); + }); + }); +} + +/** + * TransactionManager + * + * Abstraction hiding reading and writing to persistence. + * Exposes transaction based interface with read isolation. + */ +class TransactionManager : public ExtentCallbackInterface { +public: + TransactionManager( + JournalRef journal, + CacheRef cache, + LBAManagerRef lba_manager, + ExtentPlacementManagerRef &&epm, + BackrefManagerRef&& backref_manager); + + /// Writes initial metadata to disk + using mkfs_ertr = base_ertr; + mkfs_ertr::future<> mkfs(); + + /// Reads initial metadata from disk + using mount_ertr = base_ertr; + mount_ertr::future<> mount(); + + /// Closes transaction_manager + using close_ertr = base_ertr; + close_ertr::future<> close(); + + /// Resets transaction + void reset_transaction_preserve_handle(Transaction &t) { + return cache->reset_transaction_preserve_handle(t); + } + + /** + * get_pin + * + * Get the logical pin at offset + */ + using get_pin_iertr = LBAManager::get_mapping_iertr; + using get_pin_ret = LBAManager::get_mapping_iertr::future<LBAMappingRef>; + get_pin_ret get_pin( + Transaction &t, + laddr_t offset) { + LOG_PREFIX(TransactionManager::get_pin); + SUBTRACET(seastore_tm, "{}", t, offset); + return lba_manager->get_mapping(t, offset); + } + + /** + * get_pins + * + * Get logical pins overlapping offset~length + */ + using get_pins_iertr = LBAManager::get_mappings_iertr; + using get_pins_ret = get_pins_iertr::future<lba_pin_list_t>; + get_pins_ret get_pins( + Transaction &t, + laddr_t offset, + extent_len_t length) { + LOG_PREFIX(TransactionManager::get_pins); + SUBDEBUGT(seastore_tm, "{}~{}", t, offset, length); + return lba_manager->get_mappings( + t, offset, length); + } + + /** + * read_extent + * + * Read extent of type T at offset~length + */ + using read_extent_iertr = get_pin_iertr; + template <typename T> + using read_extent_ret = read_extent_iertr::future< + TCachedExtentRef<T>>; + template <typename T> + read_extent_ret<T> read_extent( + Transaction &t, + laddr_t offset, + extent_len_t length) { + LOG_PREFIX(TransactionManager::read_extent); + SUBTRACET(seastore_tm, "{}~{}", t, offset, length); + return get_pin( + t, offset + ).si_then([this, FNAME, &t, offset, length] (auto pin) + -> read_extent_ret<T> { + if (length != pin->get_length() || !pin->get_val().is_real()) { + SUBERRORT(seastore_tm, + "offset {} len {} got wrong pin {}", + t, offset, length, *pin); + ceph_assert(0 == "Should be impossible"); + } + return this->read_pin<T>(t, std::move(pin)); + }); + } + + /** + * read_extent + * + * Read extent of type T at offset + */ + template <typename T> + read_extent_ret<T> read_extent( + Transaction &t, + laddr_t offset) { + LOG_PREFIX(TransactionManager::read_extent); + SUBTRACET(seastore_tm, "{}", t, offset); + return get_pin( + t, offset + ).si_then([this, FNAME, &t, offset] (auto pin) + -> read_extent_ret<T> { + if (!pin->get_val().is_real()) { + SUBERRORT(seastore_tm, + "offset {} got wrong pin {}", + t, offset, *pin); + ceph_assert(0 == "Should be impossible"); + } + return this->read_pin<T>(t, std::move(pin)); + }); + } + + template <typename T> + base_iertr::future<TCachedExtentRef<T>> read_pin( + Transaction &t, + LBAMappingRef pin) + { + auto v = pin->get_logical_extent(t); + if (v.has_child()) { + return v.get_child_fut().safe_then([pin=std::move(pin)](auto extent) { +#ifndef NDEBUG + auto lextent = extent->template cast<LogicalCachedExtent>(); + auto pin_laddr = pin->get_key(); + if (pin->is_indirect()) { + pin_laddr = pin->get_intermediate_base(); + } + assert(lextent->get_laddr() == pin_laddr); +#endif + return extent->template cast<T>(); + }); + } else { + return pin_to_extent<T>(t, std::move(pin)); + } + } + + base_iertr::future<LogicalCachedExtentRef> read_pin_by_type( + Transaction &t, + LBAMappingRef pin, + extent_types_t type) + { + auto v = pin->get_logical_extent(t); + if (v.has_child()) { + return std::move(v.get_child_fut()); + } else { + return pin_to_extent_by_type(t, std::move(pin), type); + } + } + + /// Obtain mutable copy of extent + LogicalCachedExtentRef get_mutable_extent(Transaction &t, LogicalCachedExtentRef ref) { + LOG_PREFIX(TransactionManager::get_mutable_extent); + auto ret = cache->duplicate_for_write( + t, + ref)->cast<LogicalCachedExtent>(); + if (!ret->has_laddr()) { + SUBDEBUGT(seastore_tm, + "duplicating extent for write -- {} -> {}", + t, + *ref, + *ret); + ret->set_laddr(ref->get_laddr()); + } else { + SUBTRACET(seastore_tm, + "extent is already duplicated -- {}", + t, + *ref); + assert(ref->is_mutable()); + assert(&*ref == &*ret); + } + return ret; + } + + + using ref_iertr = LBAManager::ref_iertr; + using ref_ret = ref_iertr::future<unsigned>; + + /// Add refcount for ref + ref_ret inc_ref( + Transaction &t, + LogicalCachedExtentRef &ref); + + /// Add refcount for offset + ref_ret inc_ref( + Transaction &t, + laddr_t offset); + + /// Remove refcount for ref + ref_ret dec_ref( + Transaction &t, + LogicalCachedExtentRef &ref); + + /// Remove refcount for offset + ref_ret dec_ref( + Transaction &t, + laddr_t offset) { + return _dec_ref(t, offset, true); + } + + /// remove refcount for list of offset + using refs_ret = ref_iertr::future<std::vector<unsigned>>; + refs_ret dec_ref( + Transaction &t, + std::vector<laddr_t> offsets); + + /** + * alloc_extent + * + * Allocates a new block of type T with the minimum lba range of size len + * greater than laddr_hint. + */ + using alloc_extent_iertr = LBAManager::alloc_extent_iertr; + template <typename T> + using alloc_extent_ret = alloc_extent_iertr::future<TCachedExtentRef<T>>; + template <typename T> + alloc_extent_ret<T> alloc_extent( + Transaction &t, + laddr_t laddr_hint, + extent_len_t len, + placement_hint_t placement_hint = placement_hint_t::HOT) { + LOG_PREFIX(TransactionManager::alloc_extent); + SUBTRACET(seastore_tm, "{} len={}, placement_hint={}, laddr_hint={}", + t, T::TYPE, len, placement_hint, laddr_hint); + ceph_assert(is_aligned(laddr_hint, epm->get_block_size())); + auto ext = cache->alloc_new_extent<T>( + t, + len, + placement_hint, + INIT_GENERATION); + return lba_manager->alloc_extent( + t, + laddr_hint, + len, + ext->get_paddr(), + *ext + ).si_then([ext=std::move(ext), laddr_hint, &t](auto &&) mutable { + LOG_PREFIX(TransactionManager::alloc_extent); + SUBDEBUGT(seastore_tm, "new extent: {}, laddr_hint: {}", t, *ext, laddr_hint); + return alloc_extent_iertr::make_ready_future<TCachedExtentRef<T>>( + std::move(ext)); + }); + } + + /** + * remap_pin + * + * Remap original extent to new extents. + * Return the pins of new extent. + */ + struct remap_entry { + extent_len_t offset; + extent_len_t len; + remap_entry(extent_len_t _offset, extent_len_t _len) { + offset = _offset; + len = _len; + } + }; + using remap_pin_iertr = base_iertr; + template <std::size_t N> + using remap_pin_ret = remap_pin_iertr::future<std::array<LBAMappingRef, N>>; + template <typename T, std::size_t N> + remap_pin_ret<N> remap_pin( + Transaction &t, + LBAMappingRef &&pin, + std::array<remap_entry, N> remaps) { + +#ifndef NDEBUG + std::sort(remaps.begin(), remaps.end(), + [](remap_entry x, remap_entry y) { + return x.offset < y.offset; + }); + auto original_len = pin->get_length(); + extent_len_t total_remap_len = 0; + extent_len_t last_offset = 0; + extent_len_t last_len = 0; + + for (auto &remap : remaps) { + auto remap_offset = remap.offset; + auto remap_len = remap.len; + total_remap_len += remap.len; + ceph_assert(remap_offset >= (last_offset + last_len)); + last_offset = remap_offset; + last_len = remap_len; + } + ceph_assert(total_remap_len < original_len); +#endif + + // FIXME: paddr can be absolute and pending + ceph_assert(pin->get_val().is_absolute()); + return cache->get_extent_if_cached( + t, pin->get_val(), T::TYPE + ).si_then([this, &t, remaps, + original_laddr = pin->get_key(), + intermediate_base = pin->is_indirect() + ? pin->get_intermediate_base() + : L_ADDR_NULL, + intermediate_key = pin->is_indirect() + ? pin->get_intermediate_key() + : L_ADDR_NULL, + original_paddr = pin->get_val(), + original_len = pin->get_length()](auto ext) mutable { + std::optional<ceph::bufferptr> original_bptr; + LOG_PREFIX(TransactionManager::remap_pin); + SUBDEBUGT(seastore_tm, + "original laddr: {}, original paddr: {}, original length: {}," + " intermediate_base: {}, intermediate_key: {}," + " remap to {} extents", + t, original_laddr, original_paddr, original_len, + intermediate_base, intermediate_key, remaps.size()); + ceph_assert( + (intermediate_base == L_ADDR_NULL) + == (intermediate_key == L_ADDR_NULL)); + if (ext) { + // FIXME: cannot and will not remap a dirty extent for now. + ceph_assert(!ext->is_dirty()); + ceph_assert(!ext->is_mutable()); + ceph_assert(ext->get_length() >= original_len); + ceph_assert(ext->get_paddr() == original_paddr); + original_bptr = ext->get_bptr(); + } + return seastar::do_with( + std::array<LBAMappingRef, N>(), + 0, + std::move(original_bptr), + std::vector<remap_entry>(remaps.begin(), remaps.end()), + [this, &t, original_laddr, original_paddr, + original_len, intermediate_base, intermediate_key] + (auto &ret, auto &count, auto &original_bptr, auto &remaps) { + return _dec_ref(t, original_laddr, false + ).si_then([this, &t, &original_bptr, &ret, &count, + &remaps, intermediate_base, intermediate_key, + original_laddr, original_paddr, original_len](auto) { + return trans_intr::do_for_each( + remaps.begin(), + remaps.end(), + [this, &t, &original_bptr, &ret, + &count, intermediate_base, intermediate_key, + original_laddr, original_paddr, original_len](auto &remap) { + LOG_PREFIX(TransactionManager::remap_pin); + auto remap_offset = remap.offset; + auto remap_len = remap.len; + auto remap_laddr = original_laddr + remap_offset; + auto remap_paddr = original_paddr.add_offset(remap_offset); + ceph_assert(remap_len < original_len); + ceph_assert(remap_offset + remap_len <= original_len); + ceph_assert(remap_len != 0); + ceph_assert(remap_offset % cache->get_block_size() == 0); + ceph_assert(remap_len % cache->get_block_size() == 0); + SUBDEBUGT(seastore_tm, + "remap laddr: {}, remap paddr: {}, remap length: {}", t, + remap_laddr, remap_paddr, remap_len); + auto remapped_intermediate_key = intermediate_key; + if (remapped_intermediate_key != L_ADDR_NULL) { + assert(intermediate_base != L_ADDR_NULL); + remapped_intermediate_key += remap_offset; + } + return alloc_remapped_extent<T>( + t, + remap_laddr, + remap_paddr, + remap_len, + original_laddr, + intermediate_base, + remapped_intermediate_key, + std::move(original_bptr) + ).si_then([&ret, &count, remap_laddr](auto &&npin) { + ceph_assert(npin->get_key() == remap_laddr); + ret[count++] = std::move(npin); + }); + }); + }).si_then([this, &t, intermediate_base, intermediate_key] { + if (N > 1 && intermediate_key != L_ADDR_NULL) { + return lba_manager->incref_extent( + t, intermediate_base, N - 1 + ).si_then([](auto) { + return seastar::now(); + }); + } + return LBAManager::ref_iertr::now(); + }).handle_error_interruptible( + remap_pin_iertr::pass_further{}, + crimson::ct_error::assert_all{ + "TransactionManager::remap_pin hit invalid error" + } + ).si_then([&ret, &count] { + ceph_assert(count == N); + return remap_pin_iertr::make_ready_future< + std::array<LBAMappingRef, N>>(std::move(ret)); + }); + }); + }); + } + + using reserve_extent_iertr = alloc_extent_iertr; + using reserve_extent_ret = reserve_extent_iertr::future<LBAMappingRef>; + reserve_extent_ret reserve_region( + Transaction &t, + laddr_t hint, + extent_len_t len) { + LOG_PREFIX(TransactionManager::reserve_region); + SUBDEBUGT(seastore_tm, "len={}, laddr_hint={}", t, len, hint); + ceph_assert(is_aligned(hint, epm->get_block_size())); + return lba_manager->reserve_region( + t, + hint, + len); + } + + /* + * clone_pin + * + * create an indirect lba mapping pointing to the physical + * lba mapping whose key is intermediate_key. Resort to btree_lba_manager.h + * for the definition of "indirect lba mapping" and "physical lba mapping" + * + */ + using clone_extent_iertr = alloc_extent_iertr; + using clone_extent_ret = clone_extent_iertr::future<LBAMappingRef>; + clone_extent_ret clone_pin( + Transaction &t, + laddr_t hint, + const LBAMapping &mapping) { + auto intermediate_key = + mapping.is_indirect() + ? mapping.get_intermediate_key() + : mapping.get_key(); + auto intermediate_base = + mapping.is_indirect() + ? mapping.get_intermediate_base() + : mapping.get_key(); + + LOG_PREFIX(TransactionManager::clone_pin); + SUBDEBUGT(seastore_tm, "len={}, laddr_hint={}, clone_offset {}", + t, mapping.get_length(), hint, intermediate_key); + ceph_assert(is_aligned(hint, epm->get_block_size())); + return lba_manager->clone_extent( + t, + hint, + mapping.get_length(), + intermediate_key, + mapping.get_val(), + intermediate_key + ).si_then([this, &t, intermediate_base](auto pin) { + return inc_ref(t, intermediate_base + ).si_then([pin=std::move(pin)](auto) mutable { + return std::move(pin); + }).handle_error_interruptible( + crimson::ct_error::input_output_error::pass_further(), + crimson::ct_error::assert_all("not possible") + ); + }); + } + + /* alloc_extents + * + * allocates more than one new blocks of type T. + */ + using alloc_extents_iertr = alloc_extent_iertr; + template<class T> + alloc_extents_iertr::future<std::vector<TCachedExtentRef<T>>> + alloc_extents( + Transaction &t, + laddr_t hint, + extent_len_t len, + int num) { + LOG_PREFIX(TransactionManager::alloc_extents); + SUBDEBUGT(seastore_tm, "len={}, laddr_hint={}, num={}", + t, len, hint, num); + return seastar::do_with(std::vector<TCachedExtentRef<T>>(), + [this, &t, hint, len, num] (auto &extents) { + return trans_intr::do_for_each( + boost::make_counting_iterator(0), + boost::make_counting_iterator(num), + [this, &t, len, hint, &extents] (auto i) { + return alloc_extent<T>(t, hint, len).si_then( + [&extents](auto &&node) { + extents.push_back(node); + }); + }).si_then([&extents] { + return alloc_extents_iertr::make_ready_future + <std::vector<TCachedExtentRef<T>>>(std::move(extents)); + }); + }); + } + + /** + * submit_transaction + * + * Atomically submits transaction to persistence + */ + using submit_transaction_iertr = base_iertr; + submit_transaction_iertr::future<> submit_transaction(Transaction &); + + /** + * flush + * + * Block until all outstanding IOs on handle are committed. + * Note, flush() machinery must go through the same pipeline + * stages and locks as submit_transaction. + */ + seastar::future<> flush(OrderingHandle &handle); + + /* + * ExtentCallbackInterface + */ + + /// weak transaction should be type READ + TransactionRef create_transaction( + Transaction::src_t src, + const char* name, + bool is_weak=false) final { + return cache->create_transaction(src, name, is_weak); + } + + using ExtentCallbackInterface::submit_transaction_direct_ret; + submit_transaction_direct_ret submit_transaction_direct( + Transaction &t, + std::optional<journal_seq_t> seq_to_trim = std::nullopt) final; + + using ExtentCallbackInterface::get_next_dirty_extents_ret; + get_next_dirty_extents_ret get_next_dirty_extents( + Transaction &t, + journal_seq_t seq, + size_t max_bytes) final; + + using ExtentCallbackInterface::rewrite_extent_ret; + rewrite_extent_ret rewrite_extent( + Transaction &t, + CachedExtentRef extent, + rewrite_gen_t target_generation, + sea_time_point modify_time) final; + + using ExtentCallbackInterface::get_extents_if_live_ret; + get_extents_if_live_ret get_extents_if_live( + Transaction &t, + extent_types_t type, + paddr_t paddr, + laddr_t laddr, + extent_len_t len) final; + + /** + * read_root_meta + * + * Read root block meta entry for key. + */ + using read_root_meta_iertr = base_iertr; + using read_root_meta_bare = std::optional<std::string>; + using read_root_meta_ret = read_root_meta_iertr::future< + read_root_meta_bare>; + read_root_meta_ret read_root_meta( + Transaction &t, + const std::string &key) { + return cache->get_root( + t + ).si_then([&key, &t](auto root) { + LOG_PREFIX(TransactionManager::read_root_meta); + auto meta = root->root.get_meta(); + auto iter = meta.find(key); + if (iter == meta.end()) { + SUBDEBUGT(seastore_tm, "{} -> nullopt", t, key); + return seastar::make_ready_future<read_root_meta_bare>(std::nullopt); + } else { + SUBDEBUGT(seastore_tm, "{} -> {}", t, key, iter->second); + return seastar::make_ready_future<read_root_meta_bare>(iter->second); + } + }); + } + + /** + * update_root_meta + * + * Update root block meta entry for key to value. + */ + using update_root_meta_iertr = base_iertr; + using update_root_meta_ret = update_root_meta_iertr::future<>; + update_root_meta_ret update_root_meta( + Transaction& t, + const std::string& key, + const std::string& value) { + LOG_PREFIX(TransactionManager::update_root_meta); + SUBDEBUGT(seastore_tm, "seastore_tm, {} -> {}", t, key, value); + return cache->get_root( + t + ).si_then([this, &t, &key, &value](RootBlockRef root) { + root = cache->duplicate_for_write(t, root)->cast<RootBlock>(); + + auto meta = root->root.get_meta(); + meta[key] = value; + + root->root.set_meta(meta); + return seastar::now(); + }); + } + + /** + * read_onode_root + * + * Get onode-tree root logical address + */ + using read_onode_root_iertr = base_iertr; + using read_onode_root_ret = read_onode_root_iertr::future<laddr_t>; + read_onode_root_ret read_onode_root(Transaction &t) { + return cache->get_root(t).si_then([&t](auto croot) { + LOG_PREFIX(TransactionManager::read_onode_root); + laddr_t ret = croot->get_root().onode_root; + SUBTRACET(seastore_tm, "{}", t, ret); + return ret; + }); + } + + /** + * write_onode_root + * + * Write onode-tree root logical address, must be called after read. + */ + void write_onode_root(Transaction &t, laddr_t addr) { + LOG_PREFIX(TransactionManager::write_onode_root); + SUBDEBUGT(seastore_tm, "{}", t, addr); + auto croot = cache->get_root_fast(t); + croot = cache->duplicate_for_write(t, croot)->cast<RootBlock>(); + croot->get_root().onode_root = addr; + } + + /** + * read_collection_root + * + * Get collection root addr + */ + using read_collection_root_iertr = base_iertr; + using read_collection_root_ret = read_collection_root_iertr::future< + coll_root_t>; + read_collection_root_ret read_collection_root(Transaction &t) { + return cache->get_root(t).si_then([&t](auto croot) { + LOG_PREFIX(TransactionManager::read_collection_root); + auto ret = croot->get_root().collection_root.get(); + SUBTRACET(seastore_tm, "{}~{}", + t, ret.get_location(), ret.get_size()); + return ret; + }); + } + + /** + * write_collection_root + * + * Update collection root addr + */ + void write_collection_root(Transaction &t, coll_root_t cmroot) { + LOG_PREFIX(TransactionManager::write_collection_root); + SUBDEBUGT(seastore_tm, "{}~{}", + t, cmroot.get_location(), cmroot.get_size()); + auto croot = cache->get_root_fast(t); + croot = cache->duplicate_for_write(t, croot)->cast<RootBlock>(); + croot->get_root().collection_root.update(cmroot); + } + + extent_len_t get_block_size() const { + return epm->get_block_size(); + } + + store_statfs_t store_stat() const { + return epm->get_stat(); + } + + ~TransactionManager(); + +private: + friend class Transaction; + + CacheRef cache; + LBAManagerRef lba_manager; + JournalRef journal; + ExtentPlacementManagerRef epm; + BackrefManagerRef backref_manager; + + WritePipeline write_pipeline; + + rewrite_extent_ret rewrite_logical_extent( + Transaction& t, + LogicalCachedExtentRef extent); + + submit_transaction_direct_ret do_submit_transaction( + Transaction &t, + ExtentPlacementManager::dispatch_result_t dispatch_result, + std::optional<journal_seq_t> seq_to_trim = std::nullopt); + + /// Remove refcount for offset + ref_ret _dec_ref( + Transaction &t, + laddr_t offset, + bool cascade_remove); + + /** + * pin_to_extent + * + * Get extent mapped at pin. + */ + using pin_to_extent_iertr = base_iertr; + template <typename T> + using pin_to_extent_ret = pin_to_extent_iertr::future< + TCachedExtentRef<T>>; + template <typename T> + pin_to_extent_ret<T> pin_to_extent( + Transaction &t, + LBAMappingRef pin) { + LOG_PREFIX(TransactionManager::pin_to_extent); + SUBTRACET(seastore_tm, "getting extent {}", t, *pin); + static_assert(is_logical_type(T::TYPE)); + using ret = pin_to_extent_ret<T>; + auto &pref = *pin; + return cache->get_absent_extent<T>( + t, + pref.get_val(), + pref.is_indirect() ? + pref.get_intermediate_length() : + pref.get_length(), + [pin=std::move(pin)] + (T &extent) mutable { + assert(!extent.has_laddr()); + assert(!extent.has_been_invalidated()); + assert(!pin->has_been_invalidated()); + assert(pin->get_parent()); + pin->link_child(&extent); + extent.maybe_set_intermediate_laddr(*pin); + } + ).si_then([FNAME, &t](auto ref) mutable -> ret { + SUBTRACET(seastore_tm, "got extent -- {}", t, *ref); + assert(ref->is_fully_loaded()); + return pin_to_extent_ret<T>( + interruptible::ready_future_marker{}, + std::move(ref)); + }); + } + + /** + * pin_to_extent_by_type + * + * Get extent mapped at pin. + */ + using pin_to_extent_by_type_ret = pin_to_extent_iertr::future< + LogicalCachedExtentRef>; + pin_to_extent_by_type_ret pin_to_extent_by_type( + Transaction &t, + LBAMappingRef pin, + extent_types_t type) + { + LOG_PREFIX(TransactionManager::pin_to_extent_by_type); + SUBTRACET(seastore_tm, "getting extent {} type {}", t, *pin, type); + assert(is_logical_type(type)); + auto &pref = *pin; + return cache->get_absent_extent_by_type( + t, + type, + pref.get_val(), + pref.get_key(), + pref.is_indirect() ? + pref.get_intermediate_length() : + pref.get_length(), + [pin=std::move(pin)](CachedExtent &extent) mutable { + auto &lextent = static_cast<LogicalCachedExtent&>(extent); + assert(!lextent.has_laddr()); + assert(!lextent.has_been_invalidated()); + assert(!pin->has_been_invalidated()); + assert(pin->get_parent()); + assert(!pin->get_parent()->is_pending()); + pin->link_child(&lextent); + lextent.maybe_set_intermediate_laddr(*pin); + } + ).si_then([FNAME, &t](auto ref) { + SUBTRACET(seastore_tm, "got extent -- {}", t, *ref); + assert(ref->is_fully_loaded()); + return pin_to_extent_by_type_ret( + interruptible::ready_future_marker{}, + std::move(ref->template cast<LogicalCachedExtent>())); + }); + } + + /** + * alloc_remapped_extent + * + * Allocates a new extent at given remap_paddr that must be absolute and + * use the buffer to fill the new extent if buffer exists. Otherwise, will + * not read disk to fill the new extent. + * Returns the new extent. + * + * Should make sure the end laddr of remap extent <= the end laddr of + * original extent when using this method. + */ + using alloc_remapped_extent_iertr = + alloc_extent_iertr::extend_ertr<Device::read_ertr>; + using alloc_remapped_extent_ret = + alloc_remapped_extent_iertr::future<LBAMappingRef>; + template <typename T> + alloc_remapped_extent_ret alloc_remapped_extent( + Transaction &t, + laddr_t remap_laddr, + paddr_t remap_paddr, + extent_len_t remap_length, + laddr_t original_laddr, + laddr_t intermediate_base, + laddr_t intermediate_key, + std::optional<ceph::bufferptr> &&original_bptr) { + LOG_PREFIX(TransactionManager::alloc_remapped_extent); + SUBDEBUG(seastore_tm, "alloc remapped extent: remap_laddr: {}, " + "remap_paddr: {}, remap_length: {}, has data in cache: {} ", + remap_laddr, remap_paddr, remap_length, + original_bptr.has_value() ? "true":"false"); + TCachedExtentRef<T> ext; + auto fut = LBAManager::alloc_extent_iertr::make_ready_future< + LBAMappingRef>(); + assert((intermediate_key == L_ADDR_NULL) + == (intermediate_base == L_ADDR_NULL)); + if (intermediate_key == L_ADDR_NULL) { + // remapping direct mapping + ext = cache->alloc_remapped_extent<T>( + t, + remap_laddr, + remap_paddr, + remap_length, + original_laddr, + std::move(original_bptr)); + fut = lba_manager->alloc_extent( + t, remap_laddr, remap_length, remap_paddr, *ext); + } else { + fut = lba_manager->clone_extent( + t, + remap_laddr, + remap_length, + intermediate_key, + remap_paddr, + intermediate_base); + } + return fut.si_then([remap_laddr, remap_length, remap_paddr](auto &&ref) { + assert(ref->get_key() == remap_laddr); + assert(ref->get_val() == remap_paddr); + assert(ref->get_length() == remap_length); + return alloc_remapped_extent_iertr::make_ready_future + <LBAMappingRef>(std::move(ref)); + }); + } + +public: + // Testing interfaces + auto get_epm() { + return epm.get(); + } + + auto get_lba_manager() { + return lba_manager.get(); + } + + auto get_backref_manager() { + return backref_manager.get(); + } + + auto get_cache() { + return cache.get(); + } + auto get_journal() { + return journal.get(); + } +}; +using TransactionManagerRef = std::unique_ptr<TransactionManager>; + +TransactionManagerRef make_transaction_manager( + Device *primary_device, + const std::vector<Device*> &secondary_devices, + bool is_test); +} |