collection_manager/collection_flat_node.cc
object_data_handler.cc
seastore.cc
- randomblock_manager.cc
- nvmedevice/nvmedevice.cc
+ random_block_manager/nvme_manager.cc
+ random_block_manager/nvmedevice.cc
../../../test/crimson/seastore/test_block.cc
${PROJECT_SOURCE_DIR}/src/os/Transaction.cc
)
+++ /dev/null
-// -*- 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 "include/buffer.h"
-#include "crimson/os/seastore/nvmedevice/nvmedevice.h"
-
-namespace {
- seastar::logger& logger() {
- return crimson::get_logger(ceph_subsys_filestore);
- }
-}
-
-namespace crimson::os::seastore::nvme_device {
-
-open_ertr::future<> PosixNVMeDevice::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) {
- this->block_size = stat.block_size;
- this->size = stat.size;
- return seastar::open_file_dma(in_path, mode).then([=](auto file) {
- this->device = file;
- logger().debug("open");
- return seastar::now();
- });
- }).handle_exception([](auto e) -> open_ertr::future<> {
- logger().error("open: got error{}", e);
- return crimson::ct_error::input_output_error::make();
- });
- });
-}
-
-write_ertr::future<> PosixNVMeDevice::write(
- uint64_t offset,
- bufferptr &bptr,
- uint16_t stream) {
- logger().debug(
- "block: write offset {} len {}",
- offset,
- bptr.length());
- auto length = bptr.length();
-
- assert((length % block_size) == 0);
-
- return device.dma_write(offset, bptr.c_str(), length).handle_exception(
- [length](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<> PosixNVMeDevice::read(
- uint64_t offset,
- bufferptr &bptr) {
- logger().debug(
- "block: read offset {} len {}",
- offset,
- bptr.length());
- auto length = bptr.length();
-
- assert((length % block_size) == 0);
-
- return device.dma_read(offset, bptr.c_str(), length).handle_exception(
- [length](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();
- });
-}
-
-seastar::future<> PosixNVMeDevice::close() {
- logger().debug(" close ");
- return device.close();
-}
-
-open_ertr::future<> TestMemory::open(
- const std::string &in_path,
- seastar::open_flags mode) {
- if (buf) {
- return open_ertr::now();
- }
-
- logger().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<> TestMemory::write(
- uint64_t offset,
- bufferptr &bptr,
- uint16_t stream) {
- ceph_assert(buf);
- logger().debug(
- "TestMemory: write offset {} len {}",
- offset,
- bptr.length());
-
- ::memcpy(buf + offset, bptr.c_str(), bptr.length());
-
- return write_ertr::now();
-}
-
-read_ertr::future<> TestMemory::read(
- uint64_t offset,
- bufferptr &bptr) {
- ceph_assert(buf);
- logger().debug(
- "TestMemory: read offset {} len {}",
- offset,
- bptr.length());
-
- bptr.copy_in(0, bptr.length(), buf + offset);
- return read_ertr::now();
-}
-
-seastar::future<> TestMemory::close() {
- logger().debug(" close ");
- return seastar::now();
-}
-}
+++ /dev/null
-//-*- 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 <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 <linux/nvme_ioctl.h>
-#include <libaio.h>
-
-#include "crimson/osd/exceptions.h"
-#include "crimson/common/layout.h"
-
-namespace ceph {
- namespace buffer {
- class bufferptr;
- }
-}
-
-namespace crimson::os::seastore::nvme_device {
-
-/*
- * 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 cntroller_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_cmd;
- nvme_identify_command_t identify_cmd;
- };
-
- static const uint8_t OPCODE_IDENTIFY = 0x06;
-};
-
-struct nvme_version_t {
- uint32_t major_ver : 16;
- uint32_t minor_ver : 8;
- uint32_t tertiary_ver : 8;
-};
-
-struct admin_command_support_t {
- uint16_t unused : 5;
- uint16_t support_directives : 1;
- uint16_t unused2 : 10;
-};
-
-struct identify_controller_data_t {
- union
- {
- struct
- {
- uint8_t raw[1024];
- };
- struct
- {
- uint8_t unused[80];
- nvme_version_t version;
- uint8_t unused2[172];
- admin_command_support_t oacs;
- };
- };
-};
-
-struct identify_namespace_data_t {
- union
- {
- struct
- {
- uint8_t raw[4096];
- };
- struct
- {
- uint8_t unused[64];
- uint16_t npwg;
- uint16_t npwa;
- };
- };
-};
-
-using NVMePassThroughCommand = nvme_passthru_cmd;
-
-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>;
-
-struct io_context_t {
- iocb cb;
- bool done = false;
-};
-
-/*
- * Interface between NVMe SSD and its user.
- *
- * NVMeBlockDevice provides not only the basic APIs for IO, but also helper APIs
- * to accelerate SSD IO performance and reduce system overhead. By aggresively
- * utilizing and abstract useful features of latest NVMe SSD, it helps user ease
- * to get high performance of NVMe SSD and low system overhead.
- *
- * Various implementations with different interfaces such as POSIX APIs, Seastar,
- * and SPDK, are available.
- */
-class NVMeBlockDevice {
-protected:
- uint64_t size = 0;
- uint64_t block_size = 4096;
-
- uint64_t write_granularity = 4096;
- uint64_t write_alignment = 4096;
-
-public:
- NVMeBlockDevice() {}
- virtual ~NVMeBlockDevice() = default;
-
- template <typename T>
- static std::unique_ptr<T> create() {
- return std::make_unique<T>();
- }
-
- /*
- * 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.
- */
- uint64_t get_size() const { return size; }
- uint64_t get_block_size() const { return block_size; }
-
- uint64_t get_preffered_write_granularity() const { return write_granularity; }
- uint64_t get_preffered_write_alignment() const { return write_alignment; }
-
- 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;
-
- // TODO
- 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 seastar::future<> close() = 0;
-
- /*
- * For passsing through nvme IO or Admin command to SSD
- * Caller can construct and execute its own nvme command
- */
- virtual nvme_command_ertr::future<> pass_through_io(
- NVMePassThroughCommand& command) { return nvme_command_ertr::now(); }
- virtual nvme_command_ertr::future<> pass_admin(
- nvme_admin_command_t& command) { return nvme_command_ertr::now(); }
-
- /*
- * 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
- */
- virtual write_ertr::future<> protected_write(
- uint64_t offset,
- bufferptr &bptr,
- uint16_t stream = 0) { return write_ertr::now(); }
-
- /*
- * 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<> get_data_health(
- std::list<uint64_t>& fragile_lbas) { return nvme_command_ertr::now(); }
-
- /*
- * 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(); }
-
- /*
- * Predictable Latency
- *
- * NVMe device can guarantee IO latency within pre-defined time window. This
- * functionality will be analyzed soon.
- */
-};
-
-/*
- * Implementation of NVMeBlockDevice with POSIX APIs
- *
- * PosixNVMeDevice provides NVMe SSD interfaces through POSIX APIs which is generally
- * available at most operating environment.
- */
-class PosixNVMeDevice : public NVMeBlockDevice {
-public:
- PosixNVMeDevice() {}
- ~PosixNVMeDevice() = 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;
-
- read_ertr::future<> read(
- uint64_t offset,
- bufferptr &bptr) override;
-
- seastar::future<> close() override;
-
- // TODO Servicing NVMe features (multi-stream, protected write etc..) should
- // be followed by upstreaming ioctl to seastar.
-
-private:
- seastar::file device;
-};
-
-
-class TestMemory : public NVMeBlockDevice {
-public:
-
- TestMemory(size_t size) : buf(nullptr), size(size) {}
- ~TestMemory() {
- if (buf) {
- ::munmap(buf, size);
- buf = nullptr;
- }
- }
-
- 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;
-
- read_ertr::future<> read(
- uint64_t offset,
- bufferptr &bptr) override;
-
- seastar::future<> close() override;
-
- char *buf;
- size_t size;
-};
-
-}
--- /dev/null
+// -*- 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 "include/uuid.h"
+
+
+namespace crimson::os::seastore {
+
+class RandomBlockManager {
+public:
+
+ struct mkfs_config_t {
+ std::string path;
+ blk_paddr_t start;
+ blk_paddr_t end;
+ size_t block_size = 0;
+ size_t total_size = 0;
+ seastore_meta_t meta;
+ };
+ using mkfs_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error,
+ crimson::ct_error::invarg
+ >;
+ virtual mkfs_ertr::future<> mkfs(mkfs_config_t) = 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(uint64_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(uint64_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(const std::string &path, blk_paddr_t start) = 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
+ >;
+ virtual allocate_ertr::future<> alloc_extent(Transaction &t, size_t size) = 0; // allocator, return blocks
+
+ using free_block_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error,
+ crimson::ct_error::invarg
+ >;
+ // TODO: will include trim if necessary
+ virtual free_block_ertr::future<> free_extent(Transaction &t, blk_paddr_t from, blk_paddr_t to) = 0;
+
+ using abort_allocation_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error,
+ crimson::ct_error::invarg
+ >;
+ virtual abort_allocation_ertr::future<> abort_allocation(Transaction &t) = 0;
+
+ using complete_allocation_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error,
+ crimson::ct_error::invarg,
+ crimson::ct_error::enoent,
+ crimson::ct_error::erange
+ >;
+ virtual write_ertr::future<> complete_allocation(Transaction &t) = 0;
+
+ virtual size_t get_size() const = 0;
+ virtual size_t get_block_size() const = 0;
+ virtual uint64_t get_free_blocks() const = 0;
+ virtual ~RandomBlockManager() {}
+};
+using RandomBlockManagerRef = std::unique_ptr<RandomBlockManager>;
+
+}
--- /dev/null
+// -*- 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/common/log.h"
+
+#include "include/buffer.h"
+#include "nvmedevice.h"
+#include "include/interval_set.h"
+#include "include/intarith.h"
+#include "nvme_manager.h"
+
+namespace {
+ seastar::logger& logger() {
+ return crimson::get_logger(ceph_subsys_filestore);
+ }
+}
+
+namespace crimson::os::seastore {
+
+NVMeManager::write_ertr::future<> NVMeManager::rbm_sync_block_bitmap(
+ rbm_bitmap_block_t &block, blk_id_t block_no)
+{
+ bufferptr bptr;
+ try {
+ bptr = bufferptr(ceph::buffer::create_page_aligned(block.get_size()));
+ bufferlist bl;
+ encode(block, bl);
+ auto iter = bl.cbegin();
+ iter.copy(block.get_size(), bptr.c_str());
+ } catch (const std::exception &e) {
+ logger().error(
+ "rmb_sync_block_bitmap: "
+ "exception creating aligned buffer {}",
+ e
+ );
+ ceph_assert(0 == "unhandled exception");
+ }
+ uint64_t bitmap_block_no = convert_block_no_to_bitmap_block(block_no);
+ return device->write(super.start_alloc_area +
+ bitmap_block_no * super.block_size,
+ bptr);
+}
+
+NVMeManager::mkfs_ertr::future<> NVMeManager::initialize_blk_alloc_area() {
+ auto start = super.start_data_area / super.block_size;
+ logger().debug("initialize_alloc_area: start to read at {} ", start);
+
+ /* write allocated bitmap info to rbm meta block */
+ rbm_bitmap_block_t b_block(super.block_size);
+ alloc_rbm_bitmap_block_buf(b_block);
+ for (uint64_t i = 0; i < start; i++) {
+ b_block.set_bit(i);
+ }
+ b_block.set_crc();
+
+ return rbm_sync_block_bitmap(b_block,
+ super.start_alloc_area / super.block_size
+ ).safe_then([this, b_block, start] () mutable {
+
+ /* initialize bitmap blocks as unused */
+ auto max = max_block_by_bitmap_block();
+ auto max_block = super.size / super.block_size;
+ blk_id_t end = round_up_to(max_block, max) - 1;
+ logger().debug(" init start {} end {} ", start, end);
+ return rbm_sync_block_bitmap_by_range(
+ start,
+ end,
+ bitmap_op_types_t::ALL_CLEAR
+ ).safe_then([this, b_block, start]() mutable {
+ /*
+ * Set rest of the block bitmap, which is not used, to 1
+ * To do so, we only mark 1 to empty bitmap blocks
+ */
+ uint64_t na_block_no = super.size/super.block_size;
+ uint64_t remain_block = na_block_no % max_block_by_bitmap_block();
+ logger().debug(" na_block_no: {}, remain_block: {} ",
+ na_block_no, remain_block);
+ if (remain_block) {
+ logger().debug(" try to remained write alloc info ");
+ if (na_block_no > max_block_by_bitmap_block()) {
+ b_block.buf.clear();
+ alloc_rbm_bitmap_block_buf(b_block);
+ }
+ for (uint64_t i = remain_block; i < max_block_by_bitmap_block(); i++) {
+ b_block.set_bit(i);
+ }
+ b_block.set_crc();
+ return rbm_sync_block_bitmap(b_block, na_block_no
+ ).handle_error(
+ mkfs_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error rbm_sync_block_bitmap to update \
+ last bitmap block in NVMeManager::initialize_blk_alloc_area"
+ }
+ );
+ }
+ return mkfs_ertr::now();
+ }).handle_error(
+ mkfs_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error rbm_sync_block_bitmap \
+ in NVMeManager::initialize_blk_alloc_area"
+ }
+ );
+ }).handle_error(
+ mkfs_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error rbm_sync_block_bitmap_by_range \
+ in NVMeManager::initialize_blk_alloc_area"
+ }
+ );
+
+}
+
+NVMeManager::mkfs_ertr::future<> NVMeManager::mkfs(mkfs_config_t config)
+{
+ logger().debug("path {}", path);
+ return _open_device(path).safe_then([this, &config]() {
+ return read_rbm_header(config.start
+ ).safe_then([this, &config](auto super) {
+ logger().debug(" already exists ");
+ return mkfs_ertr::now();
+ }).handle_error(
+ crimson::ct_error::enoent::handle([this, &config] (auto) {
+
+ super.uuid = uuid_d(); // TODO
+ super.magic = 0xFF; // TODO
+ super.start = config.start;
+ super.end = config.end;
+ super.block_size = config.block_size;
+ super.size = config.total_size;
+ super.free_block_count = config.total_size/config.block_size - 2;
+ super.alloc_area_size = get_alloc_area_size();
+ super.start_alloc_area = RBM_SUPERBLOCK_SIZE;
+ super.start_data_area =
+ super.start_alloc_area + super.alloc_area_size;
+ super.crc = 0;
+ super.feature |= RBM_BITMAP_BLOCK_CRC;
+
+ logger().debug(" super {} ", super);
+ // write super block
+ return write_rbm_header().safe_then([this] {
+ return initialize_blk_alloc_area();
+ }).handle_error(
+ mkfs_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error write_rbm_header in NVMeManager::mkfs"
+ });
+ }),
+ mkfs_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error read_rbm_header in NVMeManager::mkfs"
+ }
+ );
+ }).handle_error(
+ mkfs_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error open_device in NVMeManager::mkfs"
+ }
+ ).finally([this] {
+ if (device) {
+ return device->close().then([]() {
+ return mkfs_ertr::now();
+ });
+ }
+ return mkfs_ertr::now();
+ });
+
+}
+
+NVMeManager::find_block_ret NVMeManager::find_free_block(Transaction &t, size_t size)
+{
+ auto bp = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
+ return seastar::do_with(uint64_t(0),
+ uint64_t(super.start_alloc_area),
+ interval_set<blk_id_t>(),
+ bp,
+ [&, this] (auto &allocated, auto &addr, auto &alloc_extent, auto &bp) mutable {
+ return crimson::do_until(
+ [&, this] () mutable {
+ return device->read(
+ addr,
+ bp
+ ).safe_then(
+ [&bp, &addr, size, &allocated, &t, &alloc_extent, this]() mutable {
+ logger().debug("find_free_list: allocate {}, addr {}", allocated, addr);
+ rbm_bitmap_block_t b_block(super.block_size);
+ bufferlist bl_bitmap_block;
+ bl_bitmap_block.append(bp);
+ decode(b_block, bl_bitmap_block);
+ auto max = max_block_by_bitmap_block();
+ auto allocated_blocks = t.get_rbm_allocated_blocks();
+ for (uint64_t i = 0;
+ i < max && (uint64_t)size/super.block_size > allocated; i++) {
+ auto block_id = convert_bitmap_block_no_to_block_id(i, addr);
+ bool out = false;
+ for (auto b : allocated_blocks) {
+ if (b.alloc_blk_ids.intersects(block_id, 1)) {
+ out = true;
+ break;
+ }
+ }
+ if (out) {
+ continue;
+ }
+ if (b_block.is_allocated(i)) {
+ continue;
+ }
+ logger().debug("find_free_list: allocated block no {} i {}",
+ convert_bitmap_block_no_to_block_id(i, addr), i);
+ if (allocated != 0 && alloc_extent.range_end() != block_id) {
+ /*
+ * if not continous block, just restart to find continuous blocks
+ * at the next block.
+ * in-memory allocator can handle this efficiently.
+ */
+ allocated = 0;
+ alloc_extent.clear(); // a range of block allocation
+ logger().debug("find_free_list: rety to find continuous blocks");
+ continue;
+ }
+ allocated += 1;
+ alloc_extent.insert(block_id);
+ }
+ addr += super.block_size;
+ logger().debug("find_free_list: allocated: {} alloc_extent {}",
+ allocated, alloc_extent);
+ if (((uint64_t)size)/super.block_size == allocated) {
+ return find_block_ertr::make_ready_future<bool>(true);
+ } else if (addr >= super.start_data_area) {
+ alloc_extent.clear();
+ return find_block_ertr::make_ready_future<bool>(true);
+ }
+ return find_block_ertr::make_ready_future<bool>(false);
+ });
+ }).safe_then([&allocated, &alloc_extent, &t, size, this] () {
+ logger().debug(" allocated: {} size {} ",
+ allocated * super.block_size, size);
+ if (allocated * super.block_size < size) {
+ alloc_extent.clear();
+ }
+ return find_block_ret(
+ find_block_ertr::ready_future_marker{},
+ alloc_extent);
+ }).handle_error(
+ find_block_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error in NVMeManager::find_free_block"
+ }
+ );
+ });
+}
+
+/* TODO : block allocator */
+NVMeManager::allocate_ertr::future<> NVMeManager::alloc_extent(
+ Transaction &t, size_t size)
+{
+
+ /*
+ * 1. find free blocks using block allocator
+ * 2. add free blocks to transaction
+ * (the free block is reserved state, not stored)
+ * 3. link free blocks to onode
+ * Due to in-memory block allocator is the next work to do,
+ * just read the block bitmap directly to find free blocks.
+ *
+ */
+ return find_free_block(t, size
+ ).safe_then([this, &t, size] (auto alloc_extent) mutable
+ -> allocate_ertr::future<> {
+ logger().debug("after find_free_block: allocated {}", alloc_extent);
+ if (!alloc_extent.empty()) {
+ // add alloc info to delta
+ rbm_alloc_delta_t alloc_info {
+ extent_types_t::RBM_ALLOC_INFO,
+ alloc_extent,
+ rbm_alloc_delta_t::op_types_t::SET
+ };
+ t.add_rbm_allocated_blocks(alloc_info);
+ } else {
+ return crimson::ct_error::enospc::make();
+ }
+ return allocate_ertr::now();
+ }
+ ).handle_error(
+ allocate_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error find_free_block in NVMeManager::alloc_extent"
+ }
+ );
+}
+
+NVMeManager::free_block_ertr::future<> NVMeManager::free_extent(
+ Transaction &t, blk_paddr_t from, blk_paddr_t to)
+{
+ blk_id_t blk_id_start = from / super.block_size;
+ blk_id_t blk_id_end = to / super.block_size;
+
+ interval_set<blk_id_t> free_extent;
+ free_extent.insert(blk_id_start, blk_id_end - blk_id_start + 1);
+ rbm_alloc_delta_t alloc_info {
+ extent_types_t::RBM_ALLOC_INFO,
+ free_extent,
+ rbm_alloc_delta_t::op_types_t::CLEAR
+ };
+ t.add_rbm_allocated_blocks(alloc_info);
+ return free_block_ertr::now();
+}
+
+NVMeManager::write_ertr::future<> NVMeManager::rbm_sync_block_bitmap_by_range(
+ blk_id_t start, blk_id_t end, bitmap_op_types_t op)
+{
+ auto addr = super.start_alloc_area +
+ (start / max_block_by_bitmap_block())
+ * super.block_size;
+ // aligned write
+ if (start % max_block_by_bitmap_block() == 0 &&
+ end % (max_block_by_bitmap_block() - 1) == 0) {
+ auto num_block = num_block_between_blk_ids(start, end);
+ bufferlist bl_bitmap_block;
+ add_cont_bitmap_blocks_to_buf(bl_bitmap_block, num_block, op);
+ return write(
+ addr,
+ bl_bitmap_block);
+ }
+ auto bp = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
+ // try to read first block, then check the block is aligned
+ return device->read(
+ addr,
+ bp).safe_then([bp, start, end, op, addr, this]() {
+ rbm_bitmap_block_t b_block(super.block_size);
+ bufferlist bl_bitmap_block;
+ bl_bitmap_block.append(bp);
+ decode(b_block, bl_bitmap_block);
+ auto max = max_block_by_bitmap_block();
+ auto loop_end = end < (start / max + 1) * max ?
+ end % max : max - 1;
+ for (uint64_t i = (start % max); i <= loop_end; i++) {
+ if (op == bitmap_op_types_t::ALL_SET) {
+ b_block.set_bit(i);
+ } else {
+ b_block.clear_bit(i);
+ }
+ }
+ auto num_block = num_block_between_blk_ids(start, end);
+ logger().debug("rbm_sync_block_bitmap_by_range: start {}, end {}, \
+ loop_end {}, num_block {}",
+ start, end, loop_end, num_block);
+
+ bl_bitmap_block.clear();
+ encode(b_block, bl_bitmap_block);
+ if (num_block == 1) {
+ // | front (unaligned) |
+ return write(
+ addr,
+ bl_bitmap_block);
+ } else if (!((end + 1) % max)) {
+ // | front (unaligned) | middle (aligned) |
+ add_cont_bitmap_blocks_to_buf(bl_bitmap_block, num_block - 1, op);
+ logger().debug("partially aligned write: addr {} length {}",
+ addr, bl_bitmap_block.length());
+ return write(
+ addr,
+ bl_bitmap_block);
+ } else if (num_block > 2) {
+ // | front (unaligned) | middle | end (unaligned) |
+ // fill up the middle
+ add_cont_bitmap_blocks_to_buf(bl_bitmap_block, num_block - 2, op);
+ }
+
+ auto next_addr = super.start_alloc_area +
+ (end / max_block_by_bitmap_block())
+ * super.block_size;
+ auto bptr = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
+ // | front (unaligned) | middle | end (unaligned) | or
+ // | front (unaligned) | end (unaligned) |
+ return device->read(
+ next_addr,
+ bptr).safe_then(
+ [bptr, bl_bitmap_block, start, end, op, addr, this]() mutable {
+ rbm_bitmap_block_t b_block(super.block_size);
+ bufferlist block;
+ block.append(bptr);
+ decode(b_block, block);
+ auto max = max_block_by_bitmap_block();
+ for (uint64_t i = (end - (end % max)) % max;
+ i <= (end % max); i++) {
+ if (op == bitmap_op_types_t::ALL_SET) {
+ b_block.set_bit(i);
+ } else {
+ b_block.clear_bit(i);
+ }
+ }
+ logger().debug("start {} end {} ", end - (end % max), end);
+ bl_bitmap_block.claim_append(block);
+ return write(
+ addr,
+ bl_bitmap_block);
+ }).handle_error(
+ write_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error in NVMeManager::rbm_sync_block_bitmap_by_range"
+ }
+ );
+ }).handle_error(
+ write_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error in NVMeManager::rbm_sync_block_bitmap_by_range"
+ }
+ );
+}
+
+NVMeManager::abort_allocation_ertr::future<> NVMeManager::abort_allocation(
+ Transaction &t)
+{
+ /*
+ * TODO: clear all allocation infos associated with transaction in in-memory allocator
+ */
+ t.clear_rbm_allocated_blocks();
+ return abort_allocation_ertr::now();
+}
+
+NVMeManager::write_ertr::future<> NVMeManager::complete_allocation(
+ Transaction &t)
+{
+ auto alloc_blocks = t.get_rbm_allocated_blocks();
+ if (alloc_blocks.empty()) {
+ return write_ertr::now();
+ }
+ return seastar::do_with(move(alloc_blocks),
+ [&, this] (auto &alloc_blocks) mutable {
+ return crimson::do_for_each(
+ alloc_blocks,
+ [this, &alloc_blocks](auto &alloc) {
+ return crimson::do_for_each(
+ alloc.alloc_blk_ids,
+ [this, &alloc] (auto &range) -> write_ertr::future<> {
+ logger().debug("range {} ~ {}", range.first, range.second);
+ bitmap_op_types_t op =
+ (alloc.op == rbm_alloc_delta_t::op_types_t::SET) ?
+ bitmap_op_types_t::ALL_SET :
+ bitmap_op_types_t::ALL_CLEAR;
+ return rbm_sync_block_bitmap_by_range(
+ range.first,
+ range.first + range.second - 1,
+ op);
+ });
+ }).safe_then([this, &alloc_blocks]() mutable {
+ int alloc_block_count = 0;
+ for (const auto b : alloc_blocks) {
+ for (interval_set<blk_id_t>::const_iterator r = b.alloc_blk_ids.begin();
+ r != b.alloc_blk_ids.end(); ++r) {
+ if (b.op == rbm_alloc_delta_t::op_types_t::SET) {
+ alloc_block_count += r.get_len();
+ logger().debug(" complete alloc block: start {} len {} ",
+ r.get_start(), r.get_len());
+ } else {
+ alloc_block_count -= r.get_len();
+ logger().debug(" complete free block: start {} len {} ",
+ r.get_start(), r.get_len());
+ }
+ }
+ }
+ logger().debug("complete_alloction: complete to allocate {} blocks",
+ alloc_block_count);
+ super.free_block_count -= alloc_block_count;
+ return write_ertr::now();
+ });
+ });
+}
+
+NVMeManager::open_ertr::future<> NVMeManager::open(
+ const std::string &path, blk_paddr_t addr)
+{
+ logger().debug("open: path{}", path);
+ return _open_device(path
+ ).safe_then([this, addr]() {
+ return read_rbm_header(addr).safe_then([&](auto s)
+ -> open_ertr::future<> {
+ if (s.magic != 0xFF) {
+ return crimson::ct_error::enoent::make();
+ }
+ super = s;
+ return check_bitmap_blocks().safe_then([]() {
+ return open_ertr::now();
+ });
+ }
+ ).handle_error(
+ open_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error read_rbm_header in NVMeManager::open"
+ }
+ );
+ });
+}
+
+NVMeManager::write_ertr::future<> NVMeManager::write(
+ blk_paddr_t addr,
+ bufferptr &bptr)
+{
+ ceph_assert(device);
+ if (addr > super.end || addr < super.start ||
+ bptr.length() > super.end - super.start) {
+ return crimson::ct_error::erange::make();
+ }
+ return device->write(
+ addr,
+ bptr);
+}
+
+NVMeManager::read_ertr::future<> NVMeManager::read(
+ blk_paddr_t addr,
+ bufferptr &bptr)
+{
+ ceph_assert(device);
+ if (addr > super.end || addr < super.start ||
+ bptr.length() > super.end - super.start) {
+ return crimson::ct_error::erange::make();
+ }
+ return device->read(
+ addr,
+ bptr);
+}
+
+NVMeManager::close_ertr::future<> NVMeManager::close()
+{
+ ceph_assert(device);
+ return device->close();
+}
+
+NVMeManager::open_ertr::future<> NVMeManager::_open_device(
+ const std::string path)
+{
+ ceph_assert(device);
+ return device->open(path, seastar::open_flags::rw
+ ).safe_then([this] {
+ return open_ertr::now();
+ });
+}
+
+NVMeManager::write_ertr::future<> NVMeManager::write_rbm_header()
+{
+ bufferlist meta_b_header;
+ super.crc = 0;
+ encode(super, meta_b_header);
+ 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 device->write(super.start, bp
+ ).safe_then([this]() {
+ return write_ertr::now();
+ });
+}
+
+NVMeManager::read_ertr::future<rbm_metadata_header_t> NVMeManager::read_rbm_header(
+ blk_paddr_t addr)
+{
+ ceph_assert(device);
+ bufferptr bptr =
+ bufferptr(ceph::buffer::create_page_aligned(RBM_SUPERBLOCK_SIZE));
+ bptr.zero();
+ return device->read(
+ addr,
+ bptr
+ ).safe_then([length=bptr.length(), this, bptr]()
+ -> 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) {
+ logger().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);
+ if (meta_b_header.crc32c(-1) != crc) {
+ logger().debug(" bad crc on super block, expected {} != actual {} ",
+ meta_b_header.crc32c(-1), crc);
+ return crimson::ct_error::input_output_error::make();
+ }
+ logger().debug(" got {} ", super);
+ return read_ertr::future<rbm_metadata_header_t>(
+ read_ertr::ready_future_marker{},
+ super_block
+ );
+
+ }).handle_error(
+ read_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error in NVMeManager::read_rbm_header"
+ }
+ );
+}
+
+NVMeManager::check_bitmap_blocks_ertr::future<> NVMeManager::check_bitmap_blocks()
+{
+ auto bp = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
+ return seastar::do_with(uint64_t(super.start_alloc_area), uint64_t(0), bp,
+ [&, this] (auto &addr, auto &free_blocks, auto &bp) mutable {
+ return crimson::do_until(
+ [&, this] () mutable {
+ return device->read(
+ addr,
+ bp
+ ).safe_then([&bp, &addr, &free_blocks, this]() mutable {
+ logger().debug("verify_bitmap_blocks: addr {}", addr);
+ rbm_bitmap_block_t b_block(super.block_size);
+ bufferlist bl_bitmap_block;
+ bl_bitmap_block.append(bp);
+ decode(b_block, bl_bitmap_block);
+ auto max = max_block_by_bitmap_block();
+ for (uint64_t i = 0; i < max; i++) {
+ if (!b_block.is_allocated(i)) {
+ free_blocks++;
+ }
+ }
+ addr += super.block_size;
+ if (addr >= super.start_data_area) {
+ return find_block_ertr::make_ready_future<bool>(true);
+ }
+ return find_block_ertr::make_ready_future<bool>(false);
+ });
+ }).safe_then([&free_blocks, this] () {
+ logger().debug(" free_blocks: {} ", free_blocks);
+ super.free_block_count = free_blocks;
+ return check_bitmap_blocks_ertr::now();
+ }).handle_error(
+ check_bitmap_blocks_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error in NVMeManager::find_free_block"
+ }
+ );
+
+ });
+}
+
+NVMeManager::write_ertr::future<> NVMeManager::write(
+ blk_paddr_t addr,
+ bufferlist &bl)
+{
+ 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) {
+ logger().error(
+ "write: "
+ "exception creating aligned buffer {}",
+ e
+ );
+ ceph_assert(0 == "unhandled exception");
+ }
+ return device->write(
+ addr,
+ 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
+ << ", start=" << header.start
+ << ", end=" << header.end
+ << ", magic=" << header.magic
+ << ", uuid=" << header.uuid
+ << ", free_block_count=" << header.free_block_count
+ << ", alloc_area_size=" << header.alloc_area_size
+ << ", start_alloc_area=" << header.start_alloc_area
+ << ", start_data_area=" << header.start_data_area
+ << ", flag=" << header.flag
+ << ", feature=" << header.feature
+ << ", crc=" << header.crc;
+ return out << ")";
+}
+
+std::ostream &operator<<(std::ostream &out,
+ const rbm_bitmap_block_header_t &header)
+{
+ out << " rbm_bitmap_block_header_t(size=" << header.size
+ << ", checksum=" << header.checksum;
+ return out << ")";
+}
+
+}
--- /dev/null
+// -*- 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 "nvmedevice.h"
+#include "crimson/os/seastore/random_block_manager.h"
+
+#include "crimson/common/layout.h"
+#include "include/buffer.h"
+#include "include/uuid.h"
+
+namespace crimson::os::seastore {
+
+constexpr uint32_t RBM_SUPERBLOCK_SIZE = 4096;
+
+using NVMeBlockDevice = nvme_device::NVMeBlockDevice;
+using NVMeBlockDeviceRef = std::unique_ptr<NVMeBlockDevice>;
+
+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,
+};
+
+constexpr uint32_t BITS_PER_CHAR = 8;
+inline char BIT_CHAR_MASK(uint64_t nr)
+{
+ return (char(1) << (nr % BITS_PER_CHAR));
+}
+
+struct rbm_metadata_header_t {
+ size_t size = 0;
+ size_t block_size = 0;
+ uint64_t start; // start location of the device
+ uint64_t end; // end location of the device
+ uint64_t magic; // to indicate randomblock_manager
+ uuid_d uuid;
+ uint64_t free_block_count;
+ uint64_t alloc_area_size; // bitmap
+ uint32_t start_alloc_area; // block number
+ uint32_t start_data_area;
+ uint64_t flag; // reserved
+ uint64_t feature;
+ checksum_t crc;
+
+ DENC(rbm_metadata_header_t, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.size, p);
+ denc(v.block_size, p);
+ denc(v.start, p);
+ denc(v.end, p);
+ denc(v.magic, p);
+ denc(v.uuid, p);
+ denc(v.free_block_count, p);
+ denc(v.alloc_area_size, p);
+ denc(v.start_alloc_area, p);
+ denc(v.start_data_area, p);
+ denc(v.flag, p);
+ denc(v.feature, p);
+
+ denc(v.crc, p);
+ DENC_FINISH(p);
+ }
+
+};
+
+struct rbm_bitmap_block_header_t {
+ uint32_t size;
+ checksum_t checksum;
+ DENC(rbm_bitmap_block_header_t, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.size, p);
+ denc(v.checksum, p);
+ DENC_FINISH(p);
+ }
+};
+
+std::ostream &operator<<(std::ostream &out, const rbm_metadata_header_t &header);
+std::ostream &operator<<(std::ostream &out, const rbm_bitmap_block_header_t &header);
+
+enum class bitmap_op_types_t : uint8_t {
+ ALL_CLEAR = 1,
+ ALL_SET = 2
+};
+
+struct rbm_bitmap_block_t {
+ rbm_bitmap_block_header_t header;
+ bufferlist buf;
+
+ uint64_t get_size() {
+ return header.size;
+ }
+ void set_crc() {
+ header.checksum = buf.crc32c(-1);
+ }
+
+ bool is_correct_crc() {
+ ceph_assert(buf.length());
+ return buf.crc32c(-1) == header.checksum;
+ }
+
+ void set_bit(uint64_t nr) {
+ ceph_assert(buf.length());
+ char mask = BIT_CHAR_MASK(nr);
+ char *p = buf.c_str() + (nr / BITS_PER_CHAR);
+ *p |= mask;
+ }
+
+ void set_all_bits() {
+ ceph_assert(buf.length());
+ ::memset(buf.c_str(), std::numeric_limits<unsigned char>::max(), buf.length());
+ }
+
+ void set_clear_bits() {
+ ceph_assert(buf.length());
+ ::memset(buf.c_str(), 0, buf.length());
+ }
+
+ void clear_bit(uint64_t nr) {
+ ceph_assert(buf.length());
+ char mask = ~BIT_CHAR_MASK(nr);
+ char *p = buf.c_str() + (nr / BITS_PER_CHAR);
+ *p &= mask;
+ }
+
+ bool is_allocated(uint64_t nr) {
+ ceph_assert(buf.length());
+ char mask = BIT_CHAR_MASK(nr);
+ char *p = buf.c_str() + (nr / BITS_PER_CHAR);
+ return *p & mask;
+ }
+
+ rbm_bitmap_block_t(size_t size) {
+ header.size = size;
+ }
+
+ rbm_bitmap_block_t() = default;
+
+ DENC(rbm_bitmap_block_t, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.header, p);
+ denc(v.buf, p);
+ DENC_FINISH(p);
+ }
+};
+
+}
+
+WRITE_CLASS_DENC_BOUNDED(
+ crimson::os::seastore::rbm_metadata_header_t
+)
+WRITE_CLASS_DENC_BOUNDED(
+ crimson::os::seastore::rbm_bitmap_block_t
+)
+WRITE_CLASS_DENC_BOUNDED(
+ crimson::os::seastore::rbm_bitmap_block_header_t
+)
+
+namespace crimson::os::seastore {
+
+class NVMeManager final : public RandomBlockManager {
+public:
+ /*
+ * Ondisk layout
+ *
+ * ---------------------------------------------------------------------------
+ * | rbm_metadata_header_t | rbm_bitmap_block_t 1 | ... | data blocks |
+ * ---------------------------------------------------------------------------
+ */
+
+ mkfs_ertr::future<> mkfs(mkfs_config_t) final;
+ read_ertr::future<> read(uint64_t addr, bufferptr &buffer) final;
+ write_ertr::future<> write(uint64_t addr, bufferptr &buf) final;
+ open_ertr::future<> open(const std::string &path, blk_paddr_t start) 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.
+ * But, in-memory allocator is the future work, and is not implemented yet,
+ * we use freebitmap directly to allocate freeblocks for now.
+ *
+ * Each bit in freebitmap block represents whether a block is allocated or not.
+ *
+ * TODO: multiple allocation
+ *
+ */
+ allocate_ertr::future<> alloc_extent(
+ Transaction &t, size_t size) final; // allocator, return blocks
+
+ /*
+ * free_extent
+ *
+ * add a range of free blocks to transaction
+ *
+ */
+ // TODO: will include trim if necessary
+ free_block_ertr::future<> free_extent(
+ Transaction &t, blk_paddr_t from, blk_paddr_t to) final;
+ abort_allocation_ertr::future<> abort_allocation(Transaction &t) final;
+ write_ertr::future<> complete_allocation(Transaction &t) final;
+
+ open_ertr::future<> _open_device(const std::string path);
+ read_ertr::future<rbm_metadata_header_t> read_rbm_header(blk_paddr_t addr);
+ write_ertr::future<> write_rbm_header();
+
+ size_t get_size() const final { return super.size; };
+ size_t get_block_size() const final { return super.block_size; }
+
+ // max block number a block can represent using bitmap
+ uint64_t max_block_by_bitmap_block() {
+ return (super.block_size - ceph::encoded_sizeof_bounded<rbm_bitmap_block_t>()) * 8;
+ }
+
+ uint64_t convert_block_no_to_bitmap_block(blk_id_t block_no)
+ {
+ ceph_assert(super.block_size);
+ return block_no / max_block_by_bitmap_block();
+ }
+
+ /*
+ * convert_bitmap_block_no_to_block_id
+ *
+ * return block id using address where freebitmap is stored and offset
+ */
+ blk_id_t convert_bitmap_block_no_to_block_id(uint64_t offset, blk_paddr_t addr)
+ {
+ ceph_assert(super.block_size);
+ // freebitmap begins at block 1
+ return (addr / super.block_size - 1) * max_block_by_bitmap_block() + offset;
+ }
+
+ uint64_t get_alloc_area_size() {
+ ceph_assert(super.size);
+ ceph_assert(super.block_size);
+ uint64_t total_block_num = super.size / super.block_size;
+ uint64_t need_blocks = (total_block_num % max_block_by_bitmap_block()) ?
+ (total_block_num / max_block_by_bitmap_block() + 1) :
+ (total_block_num / max_block_by_bitmap_block());
+ ceph_assert(need_blocks);
+ return need_blocks * super.block_size;
+ }
+
+ using find_block_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error,
+ crimson::ct_error::enoent>;
+ using find_block_ret = find_block_ertr::future<interval_set<blk_id_t>>;
+ /*
+ * find_free_block
+ *
+ * Try to find free blocks by reading bitmap blocks on the disk sequentially
+ * The free blocks will be added to allocated_blocks in Transaction.
+ * This needs to be improved after in-memory block allocation is introduced.
+ *
+ */
+ find_block_ret find_free_block(Transaction &t, size_t size);
+
+ /*
+ * rbm_sync_block_bitmap
+ *
+ * Write rbm_bitmap_block_t to the device
+ *
+ * @param rbm_bitmap_block_t
+ * @param uint64_t the block number the rbm_bitmap_block_t will be stored
+ *
+ */
+ write_ertr::future<> rbm_sync_block_bitmap(
+ rbm_bitmap_block_t &block, blk_id_t block_no);
+
+ using check_bitmap_blocks_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error,
+ crimson::ct_error::invarg>;
+ check_bitmap_blocks_ertr::future<> check_bitmap_blocks();
+ uint64_t get_free_blocks() const {
+ return super.free_block_count;
+ }
+ /*
+ * We will have mulitple partitions (circularjournals and randbomblockmanagers)
+ * on a device, so start and end location of the device are needed to
+ * support such case.
+ */
+ NVMeManager(NVMeBlockDevice * device, std::string path)
+ : device(device), path(path) {}
+
+ /*
+ * bitmap block area (freebitmap) layout
+ *
+ * -----------------------------------------------------------
+ * | header 1 | bitmap 1 | header 2 | bitmap 2 |
+ * -----------------------------------------------------------
+ * <-- 1 block --> <-- 1 block -->
+ *
+ * 1 block contains both bitmap header and bitmap.
+ * We use this layout as a default layout here.
+ * But, we'll consider to exploit end to end data protection.
+ * If we use the end to end data protection, which is a feature specified in NVMe,
+ * we can avoid any calculation for checksum. The checksum regarding the block
+ * will be managed by the NVMe device.
+ *
+ */
+ mkfs_ertr::future<> initialize_blk_alloc_area();
+ uint64_t get_start_block_alloc_area() {
+ return super.start_alloc_area;
+ }
+
+ void alloc_rbm_bitmap_block_buf(rbm_bitmap_block_t &b_block) {
+ auto bitmap_blk = ceph::bufferptr(buffer::create_page_aligned(
+ super.block_size -
+ ceph::encoded_sizeof_bounded<rbm_bitmap_block_t>()));
+ bitmap_blk.zero();
+ b_block.buf.append(bitmap_blk);
+ }
+
+ blk_paddr_t get_blk_paddr_by_block_no(blk_id_t id) {
+ return (id * super.block_size) + super.start;
+ }
+
+ int num_block_between_blk_ids(blk_id_t start, blk_id_t end) {
+ auto max = max_block_by_bitmap_block();
+ auto block_start = start / max;
+ auto block_end = end / max;
+ return block_end - block_start + 1;
+ }
+
+ write_ertr::future<> rbm_sync_block_bitmap_by_range(
+ blk_id_t start, blk_id_t end, bitmap_op_types_t op);
+ void add_cont_bitmap_blocks_to_buf(
+ bufferlist& buf, int num_block, bitmap_op_types_t op) {
+ rbm_bitmap_block_t b_block(super.block_size);
+ alloc_rbm_bitmap_block_buf(b_block);
+ if (op == bitmap_op_types_t::ALL_SET) {
+ b_block.set_all_bits();
+ } else {
+ b_block.set_clear_bits();
+ }
+ for (int i = 0; i < num_block; i++) {
+ encode(b_block, buf);
+ }
+ }
+
+ write_ertr::future<> write(blk_paddr_t addr, bufferlist &bl);
+
+private:
+ /*
+ * this contains the number of bitmap blocks, free blocks and
+ * rbm specific information
+ */
+ rbm_metadata_header_t super;
+ //FreelistManager free_manager; // TODO: block management
+ NVMeBlockDevice * device;
+ std::string path;
+ int stream_id; // for multi-stream
+};
+using NVMeManagerRef = std::unique_ptr<NVMeManager>;
+
+}
--- /dev/null
+// -*- 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 "include/buffer.h"
+#include "nvmedevice.h"
+
+namespace {
+ seastar::logger& logger() {
+ return crimson::get_logger(ceph_subsys_filestore);
+ }
+}
+
+namespace crimson::os::seastore::nvme_device {
+
+open_ertr::future<> PosixNVMeDevice::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) {
+ this->block_size = stat.block_size;
+ this->size = stat.size;
+ return seastar::open_file_dma(in_path, mode).then([=](auto file) {
+ this->device = file;
+ logger().debug("open");
+ return seastar::now();
+ });
+ }).handle_exception([](auto e) -> open_ertr::future<> {
+ logger().error("open: got error{}", e);
+ return crimson::ct_error::input_output_error::make();
+ });
+ });
+}
+
+write_ertr::future<> PosixNVMeDevice::write(
+ uint64_t offset,
+ bufferptr &bptr,
+ uint16_t stream) {
+ logger().debug(
+ "block: write offset {} len {}",
+ offset,
+ bptr.length());
+ auto length = bptr.length();
+
+ assert((length % block_size) == 0);
+
+ return device.dma_write(offset, bptr.c_str(), length).handle_exception(
+ [length](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<> PosixNVMeDevice::read(
+ uint64_t offset,
+ bufferptr &bptr) {
+ logger().debug(
+ "block: read offset {} len {}",
+ offset,
+ bptr.length());
+ auto length = bptr.length();
+
+ assert((length % block_size) == 0);
+
+ return device.dma_read(offset, bptr.c_str(), length).handle_exception(
+ [length](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();
+ });
+}
+
+seastar::future<> PosixNVMeDevice::close() {
+ logger().debug(" close ");
+ return device.close();
+}
+
+open_ertr::future<> TestMemory::open(
+ const std::string &in_path,
+ seastar::open_flags mode) {
+ if (buf) {
+ return open_ertr::now();
+ }
+
+ logger().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<> TestMemory::write(
+ uint64_t offset,
+ bufferptr &bptr,
+ uint16_t stream) {
+ ceph_assert(buf);
+ logger().debug(
+ "TestMemory: write offset {} len {}",
+ offset,
+ bptr.length());
+
+ ::memcpy(buf + offset, bptr.c_str(), bptr.length());
+
+ return write_ertr::now();
+}
+
+read_ertr::future<> TestMemory::read(
+ uint64_t offset,
+ bufferptr &bptr) {
+ ceph_assert(buf);
+ logger().debug(
+ "TestMemory: read offset {} len {}",
+ offset,
+ bptr.length());
+
+ bptr.copy_in(0, bptr.length(), buf + offset);
+ return read_ertr::now();
+}
+
+seastar::future<> TestMemory::close() {
+ logger().debug(" close ");
+ return seastar::now();
+}
+}
--- /dev/null
+//-*- 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 <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 <linux/nvme_ioctl.h>
+#include <libaio.h>
+
+#include "crimson/osd/exceptions.h"
+#include "crimson/common/layout.h"
+
+namespace ceph {
+ namespace buffer {
+ class bufferptr;
+ }
+}
+
+namespace crimson::os::seastore::nvme_device {
+
+/*
+ * 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 cntroller_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_cmd;
+ nvme_identify_command_t identify_cmd;
+ };
+
+ static const uint8_t OPCODE_IDENTIFY = 0x06;
+};
+
+struct nvme_version_t {
+ uint32_t major_ver : 16;
+ uint32_t minor_ver : 8;
+ uint32_t tertiary_ver : 8;
+};
+
+struct admin_command_support_t {
+ uint16_t unused : 5;
+ uint16_t support_directives : 1;
+ uint16_t unused2 : 10;
+};
+
+struct identify_controller_data_t {
+ union
+ {
+ struct
+ {
+ uint8_t raw[1024];
+ };
+ struct
+ {
+ uint8_t unused[80];
+ nvme_version_t version;
+ uint8_t unused2[172];
+ admin_command_support_t oacs;
+ };
+ };
+};
+
+struct identify_namespace_data_t {
+ union
+ {
+ struct
+ {
+ uint8_t raw[4096];
+ };
+ struct
+ {
+ uint8_t unused[64];
+ uint16_t npwg;
+ uint16_t npwa;
+ };
+ };
+};
+
+using NVMePassThroughCommand = nvme_passthru_cmd;
+
+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>;
+
+struct io_context_t {
+ iocb cb;
+ bool done = false;
+};
+
+/*
+ * Interface between NVMe SSD and its user.
+ *
+ * NVMeBlockDevice provides not only the basic APIs for IO, but also helper APIs
+ * to accelerate SSD IO performance and reduce system overhead. By aggresively
+ * utilizing and abstract useful features of latest NVMe SSD, it helps user ease
+ * to get high performance of NVMe SSD and low system overhead.
+ *
+ * Various implementations with different interfaces such as POSIX APIs, Seastar,
+ * and SPDK, are available.
+ */
+class NVMeBlockDevice {
+protected:
+ uint64_t size = 0;
+ uint64_t block_size = 4096;
+
+ uint64_t write_granularity = 4096;
+ uint64_t write_alignment = 4096;
+
+public:
+ NVMeBlockDevice() {}
+ virtual ~NVMeBlockDevice() = default;
+
+ template <typename T>
+ static std::unique_ptr<T> create() {
+ return std::make_unique<T>();
+ }
+
+ /*
+ * 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.
+ */
+ uint64_t get_size() const { return size; }
+ uint64_t get_block_size() const { return block_size; }
+
+ uint64_t get_preffered_write_granularity() const { return write_granularity; }
+ uint64_t get_preffered_write_alignment() const { return write_alignment; }
+
+ 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;
+
+ // TODO
+ 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 seastar::future<> close() = 0;
+
+ /*
+ * For passsing through nvme IO or Admin command to SSD
+ * Caller can construct and execute its own nvme command
+ */
+ virtual nvme_command_ertr::future<> pass_through_io(
+ NVMePassThroughCommand& command) { return nvme_command_ertr::now(); }
+ virtual nvme_command_ertr::future<> pass_admin(
+ nvme_admin_command_t& command) { return nvme_command_ertr::now(); }
+
+ /*
+ * 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
+ */
+ virtual write_ertr::future<> protected_write(
+ uint64_t offset,
+ bufferptr &bptr,
+ uint16_t stream = 0) { return write_ertr::now(); }
+
+ /*
+ * 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<> get_data_health(
+ std::list<uint64_t>& fragile_lbas) { return nvme_command_ertr::now(); }
+
+ /*
+ * 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(); }
+
+ /*
+ * Predictable Latency
+ *
+ * NVMe device can guarantee IO latency within pre-defined time window. This
+ * functionality will be analyzed soon.
+ */
+};
+
+/*
+ * Implementation of NVMeBlockDevice with POSIX APIs
+ *
+ * PosixNVMeDevice provides NVMe SSD interfaces through POSIX APIs which is
+ * generally available at most operating environment.
+ */
+class PosixNVMeDevice : public NVMeBlockDevice {
+public:
+ PosixNVMeDevice() {}
+ ~PosixNVMeDevice() = 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;
+
+ read_ertr::future<> read(
+ uint64_t offset,
+ bufferptr &bptr) override;
+
+ seastar::future<> close() override;
+
+ // TODO Servicing NVMe features (multi-stream, protected write etc..) should
+ // be followed by upstreaming ioctl to seastar.
+
+private:
+ seastar::file device;
+};
+
+
+class TestMemory : public NVMeBlockDevice {
+public:
+
+ TestMemory(size_t size) : buf(nullptr), size(size) {}
+ ~TestMemory() {
+ if (buf) {
+ ::munmap(buf, size);
+ buf = nullptr;
+ }
+ }
+
+ 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;
+
+ read_ertr::future<> read(
+ uint64_t offset,
+ bufferptr &bptr) override;
+
+ seastar::future<> close() override;
+
+ char *buf;
+ size_t size;
+};
+
+}
+++ /dev/null
-// -*- 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/common/log.h"
-
-#include "include/buffer.h"
-#include "randomblock_manager.h"
-#include "crimson/os/seastore/nvmedevice/nvmedevice.h"
-#include "include/interval_set.h"
-#include "include/intarith.h"
-
-namespace {
- seastar::logger& logger() {
- return crimson::get_logger(ceph_subsys_filestore);
- }
-}
-
-namespace crimson::os::seastore {
-
-RandomBlockManager::write_ertr::future<> RandomBlockManager::rbm_sync_block_bitmap(rbm_bitmap_block_t &block, blk_id_t block_no)
-{
- bufferptr bptr;
- try {
- bptr = bufferptr(ceph::buffer::create_page_aligned(block.get_size()));
- bufferlist bl;
- encode(block, bl);
- auto iter = bl.cbegin();
- iter.copy(block.get_size(), bptr.c_str());
- } catch (const std::exception &e) {
- logger().error(
- "rmb_sync_block_bitmap: "
- "exception creating aligned buffer {}",
- e
- );
- ceph_assert(0 == "unhandled exception");
- }
- uint64_t bitmap_block_no = convert_block_no_to_bitmap_block(block_no);
- return device->write(super.start_alloc_area +
- bitmap_block_no * super.block_size,
- bptr);
-}
-
-RandomBlockManager::mkfs_ertr::future<> RandomBlockManager::initialize_blk_alloc_area() {
- auto start = super.start_data_area / super.block_size;
- logger().debug("initialize_alloc_area: start to read at {} ", start);
-
- /* write allocated bitmap info to rbm meta block */
- rbm_bitmap_block_t b_block(super.block_size);
- alloc_rbm_bitmap_block_buf(b_block);
- for (uint64_t i = 0; i < start; i++) {
- b_block.set_bit(i);
- }
- b_block.set_crc();
-
- return rbm_sync_block_bitmap(b_block,
- super.start_alloc_area / super.block_size
- ).safe_then([this, b_block, start] () mutable {
-
- /* initialize bitmap blocks as unused */
- auto max = max_block_by_bitmap_block();
- auto max_block = super.size / super.block_size;
- blk_id_t end = round_up_to(max_block, max) - 1;
- logger().debug(" init start {} end {} ", start, end);
- return rbm_sync_block_bitmap_by_range(
- start,
- end,
- bitmap_op_types_t::ALL_CLEAR).safe_then([this, b_block, start]() mutable {
- /*
- * Set rest of the block bitmap, which is not used, to 1
- * To do so, we only mark 1 to empty bitmap blocks
- */
- uint64_t na_block_no = super.size/super.block_size;
- uint64_t remain_block = na_block_no % max_block_by_bitmap_block();
- logger().debug(" na_block_no: {}, remain_block: {} ",
- na_block_no, remain_block);
- if (remain_block) {
- logger().debug(" try to remained write alloc info ");
- if (na_block_no > max_block_by_bitmap_block()) {
- b_block.buf.clear();
- alloc_rbm_bitmap_block_buf(b_block);
- }
- for (uint64_t i = remain_block; i < max_block_by_bitmap_block(); i++) {
- b_block.set_bit(i);
- }
- b_block.set_crc();
- return rbm_sync_block_bitmap(b_block, na_block_no
- ).handle_error(
- mkfs_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error rbm_sync_block_bitmap to update last bitmap block \
- in RandomBlockManager::initialize_blk_alloc_area"
- }
- );
- }
- return mkfs_ertr::now();
- }).handle_error(
- mkfs_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error rbm_sync_block_bitmap in RandomBlockManager::initialize_blk_alloc_area"
- }
- );
- }).handle_error(
- mkfs_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error rbm_sync_block_bitmap_by_range in RandomBlockManager::initialize_blk_alloc_area"
- }
- );
-
-}
-
-RandomBlockManager::mkfs_ertr::future<> RandomBlockManager::mkfs(mkfs_config_t config)
-{
- logger().debug("path {}", path);
- return _open_device(path).safe_then([this, &config]() {
- return read_rbm_header(config.start
- ).safe_then([this, &config](auto super) {
- logger().debug(" already exists ");
- return mkfs_ertr::now();
- }).handle_error(
- crimson::ct_error::enoent::handle([this, &config] (auto) {
-
- super.uuid = uuid_d(); // TODO
- super.magic = 0xFF; // TODO
- super.start = config.start;
- super.end = config.end;
- super.block_size = config.block_size;
- super.size = config.total_size;
- super.free_block_count = config.total_size/config.block_size - 2;
- super.alloc_area_size = get_alloc_area_size();
- super.start_alloc_area = RBM_SUPERBLOCK_SIZE;
- super.start_data_area = super.start_alloc_area + super.alloc_area_size;
- super.crc = 0;
- super.feature |= RBM_BITMAP_BLOCK_CRC;
-
- logger().debug(" super {} ", super);
- // write super block
- return write_rbm_header().safe_then([this] {
- return initialize_blk_alloc_area();
- }).handle_error(
- mkfs_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error write_rbm_header in RandomBlockManager::mkfs"
- });
- }),
- mkfs_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error read_rbm_header in RandomBlockManager::mkfs"
- }
- );
- }).handle_error(
- mkfs_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error open_device in RandomBlockManager::mkfs"
- }
- ).finally([this] {
- if (device) {
- return device->close().then([]() {
- return mkfs_ertr::now();
- });
- }
- return mkfs_ertr::now();
- });
-
-}
-
-RandomBlockManager::find_block_ret RandomBlockManager::find_free_block(Transaction &t, size_t size)
-{
- auto bp = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
- return seastar::do_with(uint64_t(0), uint64_t(super.start_alloc_area), interval_set<blk_id_t>(), bp,
- [&, this] (auto &allocated, auto &addr, auto &alloc_extent, auto &bp) mutable {
- return crimson::do_until(
- [&, this] () mutable {
- return device->read(
- addr,
- bp
- ).safe_then([&bp, &addr, size, &allocated, &t, &alloc_extent, this]() mutable {
- logger().debug("find_free_list: allocate {}, addr {}", allocated, addr);
- rbm_bitmap_block_t b_block(super.block_size);
- bufferlist bl_bitmap_block;
- bl_bitmap_block.append(bp);
- decode(b_block, bl_bitmap_block);
- auto max = max_block_by_bitmap_block();
- auto allocated_blocks = t.get_rbm_allocated_blocks();
- for (uint64_t i = 0; i < max && (uint64_t)size/super.block_size > allocated; i++) {
- auto block_id = convert_bitmap_block_no_to_block_id(i, addr);
- bool out = false;
- for (auto b : allocated_blocks) {
- if (b.alloc_blk_ids.intersects(block_id, 1)) {
- out = true;
- break;
- }
- }
- if (out) {
- continue;
- }
- if (b_block.is_allocated(i)) {
- continue;
- }
- logger().debug("find_free_list: allocated block no {} i {}",
- convert_bitmap_block_no_to_block_id(i, addr), i);
- if (allocated != 0 && alloc_extent.range_end() != block_id) {
- /*
- * if not continous block, just restart to find continuous blocks at the next block.
- * in-memory allocator can handle this efficiently.
- */
- allocated = 0;
- alloc_extent.clear(); // a range of block allocation
- logger().debug("find_free_list: rety to find continuous blocks");
- continue;
- }
- allocated += 1;
- alloc_extent.insert(block_id);
- }
- addr += super.block_size;
- logger().debug("find_free_list: allocated: {} alloc_extent {}", allocated, alloc_extent);
- if (((uint64_t)size)/super.block_size == allocated) {
- return find_block_ertr::make_ready_future<bool>(true);
- } else if (addr >= super.start_data_area) {
- alloc_extent.clear();
- return find_block_ertr::make_ready_future<bool>(true);
- }
- return find_block_ertr::make_ready_future<bool>(false);
- });
- }).safe_then([&allocated, &alloc_extent, &t, size, this] () {
- logger().debug(" allocated: {} size {} ", allocated * super.block_size, size);
- if (allocated * super.block_size < size) {
- alloc_extent.clear();
- }
- return find_block_ret(
- find_block_ertr::ready_future_marker{},
- alloc_extent);
- }).handle_error(
- find_block_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error in RandomBlockManager::find_free_block"
- }
- );
- });
-}
-
-/* TODO : block allocator */
-RandomBlockManager::allocate_ertr::future<> RandomBlockManager::alloc_extent(Transaction &t, size_t size)
-{
-
- /*
- * 1. find free blocks using block allocator
- * 2. add free blocks to transaction (free block is reserved state, but not stored)
- * 3. link free blocks to onode
- * Due to in-memory block allocator is the next work to do,
- * just read the block bitmap directly to find free blocks here.
- *
- */
- return find_free_block(t, size
- ).safe_then([this, &t, size] (auto alloc_extent) mutable -> allocate_ertr::future<> {
- logger().debug("after find_free_block: allocated {}", alloc_extent);
- if (!alloc_extent.empty()) {
- // add alloc info to delta
- rbm_alloc_delta_t alloc_info {
- extent_types_t::RBM_ALLOC_INFO,
- alloc_extent,
- rbm_alloc_delta_t::op_types_t::SET
- };
- t.add_rbm_allocated_blocks(alloc_info);
- } else {
- return crimson::ct_error::enospc::make();
- }
- return allocate_ertr::now();
- }
- ).handle_error(
- allocate_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error find_free_block in RandomBlockManager::alloc_extent"
- }
- );
-}
-
-RandomBlockManager::free_block_ertr::future<> RandomBlockManager::free_extent(Transaction &t, blk_paddr_t from, blk_paddr_t to)
-{
- blk_id_t blk_id_start = from / super.block_size;
- blk_id_t blk_id_end = to / super.block_size;
-
- interval_set<blk_id_t> free_extent;
- free_extent.insert(blk_id_start, blk_id_end - blk_id_start + 1);
- rbm_alloc_delta_t alloc_info {
- extent_types_t::RBM_ALLOC_INFO,
- free_extent,
- rbm_alloc_delta_t::op_types_t::CLEAR
- };
- t.add_rbm_allocated_blocks(alloc_info);
- return free_block_ertr::now();
-}
-
-RandomBlockManager::write_ertr::future<> RandomBlockManager::rbm_sync_block_bitmap_by_range(blk_id_t start, blk_id_t end, bitmap_op_types_t op)
-{
- auto addr = super.start_alloc_area +
- (start / max_block_by_bitmap_block())
- * super.block_size;
- // aligned write
- if (start % max_block_by_bitmap_block() == 0 &&
- end % (max_block_by_bitmap_block() - 1) == 0) {
- auto num_block = num_block_between_blk_ids(start, end);
- bufferlist bl_bitmap_block;
- add_cont_bitmap_blocks_to_buf(bl_bitmap_block, num_block, op);
- return write(
- addr,
- bl_bitmap_block);
- }
- auto bp = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
- // try to read first block, then check the block is aligned
- return device->read(
- addr,
- bp).safe_then([bp, start, end, op, addr, this]() {
- rbm_bitmap_block_t b_block(super.block_size);
- bufferlist bl_bitmap_block;
- bl_bitmap_block.append(bp);
- decode(b_block, bl_bitmap_block);
- auto max = max_block_by_bitmap_block();
- auto loop_end = end < (start / max + 1) * max ?
- end % max : max - 1;
- for (uint64_t i = (start % max); i <= loop_end; i++) {
- if (op == bitmap_op_types_t::ALL_SET) {
- b_block.set_bit(i);
- } else {
- b_block.clear_bit(i);
- }
- }
- auto num_block = num_block_between_blk_ids(start, end);
- logger().debug("rbm_sync_block_bitmap_by_range: start {}, end {}, loop_end {}, num_block {}",
- start, end, loop_end, num_block);
-
- bl_bitmap_block.clear();
- encode(b_block, bl_bitmap_block);
- if (num_block == 1) {
- // | front (unaligned) |
- return write(
- addr,
- bl_bitmap_block);
- } else if (!((end + 1) % max)) {
- // | front (unaligned) | middle (aligned) |
- add_cont_bitmap_blocks_to_buf(bl_bitmap_block, num_block - 1, op);
- logger().debug("partially aligned write: addr {} length {}",
- addr, bl_bitmap_block.length());
- return write(
- addr,
- bl_bitmap_block);
- } else if (num_block > 2) {
- // | front (unaligned) | middle | end (unaligned) |
- // fill up the middle
- add_cont_bitmap_blocks_to_buf(bl_bitmap_block, num_block - 2, op);
- }
-
- auto next_addr = super.start_alloc_area +
- (end / max_block_by_bitmap_block())
- * super.block_size;
- auto bptr = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
- // | front (unaligned) | middle | end (unaligned) | or | front (unaligned) | end (unaligned) |
- return device->read(
- next_addr,
- bptr).safe_then([bptr, bl_bitmap_block, start, end, op, addr, this]() mutable {
- rbm_bitmap_block_t b_block(super.block_size);
- bufferlist block;
- block.append(bptr);
- decode(b_block, block);
- auto max = max_block_by_bitmap_block();
- for (uint64_t i = (end - (end % max)) % max; i <= (end % max); i++) {
- if (op == bitmap_op_types_t::ALL_SET) {
- b_block.set_bit(i);
- } else {
- b_block.clear_bit(i);
- }
- }
- logger().debug("start {} end {} ", end - (end % max), end);
- bl_bitmap_block.claim_append(block);
- return write(
- addr,
- bl_bitmap_block);
- }).handle_error(
- write_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error in RandomBlockManager::rbm_sync_block_bitmap_by_range"
- }
- );
- }).handle_error(
- write_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error in RandomBlockManager::rbm_sync_block_bitmap_by_range"
- }
- );
-}
-
-RandomBlockManager::abort_allocation_ertr::future<> RandomBlockManager::abort_allocation(Transaction &t)
-{
- /*
- * TODO: clear all allocation infos associated with transaction in in-memory allocator
- */
- t.clear_rbm_allocated_blocks();
- return abort_allocation_ertr::now();
-}
-
-RandomBlockManager::write_ertr::future<> RandomBlockManager::complete_allocation(Transaction &t)
-{
- auto alloc_blocks = t.get_rbm_allocated_blocks();
- if (alloc_blocks.empty()) {
- return write_ertr::now();
- }
- return seastar::do_with(move(alloc_blocks),
- [&, this] (auto &alloc_blocks) mutable {
- return crimson::do_for_each(
- alloc_blocks,
- [this, &alloc_blocks](auto &alloc) {
- return crimson::do_for_each(
- alloc.alloc_blk_ids,
- [this, &alloc] (auto &range) -> write_ertr::future<> {
- logger().debug("range {} ~ {}", range.first, range.second);
- bitmap_op_types_t op = (alloc.op == rbm_alloc_delta_t::op_types_t::SET) ?
- bitmap_op_types_t::ALL_SET : bitmap_op_types_t::ALL_CLEAR;
- return rbm_sync_block_bitmap_by_range(
- range.first,
- range.first + range.second - 1,
- op);
- });
- }).safe_then([this, &alloc_blocks]() mutable {
- int alloc_block_count = 0;
- for (const auto b : alloc_blocks) {
- for (interval_set<blk_id_t>::const_iterator r = b.alloc_blk_ids.begin();
- r != b.alloc_blk_ids.end(); ++r) {
- if (b.op == rbm_alloc_delta_t::op_types_t::SET) {
- alloc_block_count += r.get_len();
- logger().debug(" complete alloc block: start {} len {} ",
- r.get_start(), r.get_len());
- } else {
- alloc_block_count -= r.get_len();
- logger().debug(" complete free block: start {} len {} ",
- r.get_start(), r.get_len());
- }
- }
- }
- logger().debug("complete_alloction: complete to allocate {} blocks", alloc_block_count);
- super.free_block_count -= alloc_block_count;
- return write_ertr::now();
- });
- });
-}
-
-RandomBlockManager::open_ertr::future<> RandomBlockManager::open(const std::string &path, blk_paddr_t addr)
-{
- logger().debug("open: path{}", path);
- return _open_device(path
- ).safe_then([this, addr]() {
- return read_rbm_header(addr).safe_then([&](auto s) -> open_ertr::future<> {
- if (s.magic != 0xFF) {
- return crimson::ct_error::enoent::make();
- }
- super = s;
- return check_bitmap_blocks().safe_then([]() {
- return open_ertr::now();
- });
- }
- ).handle_error(
- open_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error read_rbm_header in RandomBlockManager::open"
- }
- );
- });
-}
-
-RandomBlockManager::write_ertr::future<> RandomBlockManager::write(
- blk_paddr_t addr,
- bufferptr &bptr)
-{
- ceph_assert(device);
- if (addr > super.end || addr < super.start ||
- bptr.length() > super.end - super.start) {
- return crimson::ct_error::erange::make();
- }
- return device->write(
- addr,
- bptr);
-}
-
-RandomBlockManager::read_ertr::future<> RandomBlockManager::read(
- blk_paddr_t addr,
- bufferptr &bptr)
-{
- ceph_assert(device);
- if (addr > super.end || addr < super.start ||
- bptr.length() > super.end - super.start) {
- return crimson::ct_error::erange::make();
- }
- return device->read(
- addr,
- bptr);
-}
-
-RandomBlockManager::close_ertr::future<> RandomBlockManager::close()
-{
- ceph_assert(device);
- return device->close();
-}
-
-RandomBlockManager::open_ertr::future<> RandomBlockManager::_open_device(const std::string path)
-{
- ceph_assert(device);
- return device->open(path, seastar::open_flags::rw
- ).safe_then([this] {
- return open_ertr::now();
- });
-}
-
-RandomBlockManager::write_ertr::future<> RandomBlockManager::write_rbm_header()
-{
- bufferlist meta_b_header;
- super.crc = 0;
- encode(super, meta_b_header);
- 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 device->write(super.start, bp
- ).safe_then([this]() {
- return write_ertr::now();
- });
-}
-
-RandomBlockManager::read_ertr::future<rbm_metadata_header_t> RandomBlockManager::read_rbm_header(blk_paddr_t addr)
-{
- ceph_assert(device);
- bufferptr bptr = bufferptr(ceph::buffer::create_page_aligned(RBM_SUPERBLOCK_SIZE));
- bptr.zero();
- return device->read(
- addr,
- bptr
- ).safe_then([length=bptr.length(), this, bptr]() -> 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) {
- logger().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);
- if (meta_b_header.crc32c(-1) != crc) {
- logger().debug(" bad crc on super block, expected {} != actual {} ", meta_b_header.crc32c(-1), crc);
- return crimson::ct_error::input_output_error::make();
- }
- logger().debug(" got {} ", super);
- return read_ertr::future<rbm_metadata_header_t>(
- read_ertr::ready_future_marker{},
- super_block
- );
-
- }).handle_error(
- read_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error in RandomBlockManager::read_rbm_header"
- }
- );
-}
-
-RandomBlockManager::check_bitmap_blocks_ertr::future<> RandomBlockManager::check_bitmap_blocks()
-{
- auto bp = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
- return seastar::do_with(uint64_t(super.start_alloc_area), uint64_t(0), bp,
- [&, this] (auto &addr, auto &free_blocks, auto &bp) mutable {
- return crimson::do_until(
- [&, this] () mutable {
- return device->read(
- addr,
- bp
- ).safe_then([&bp, &addr, &free_blocks, this]() mutable {
- logger().debug("verify_bitmap_blocks: addr {}", addr);
- rbm_bitmap_block_t b_block(super.block_size);
- bufferlist bl_bitmap_block;
- bl_bitmap_block.append(bp);
- decode(b_block, bl_bitmap_block);
- auto max = max_block_by_bitmap_block();
- for (uint64_t i = 0; i < max; i++) {
- if (!b_block.is_allocated(i)) {
- free_blocks++;
- }
- }
- addr += super.block_size;
- if (addr >= super.start_data_area) {
- return find_block_ertr::make_ready_future<bool>(true);
- }
- return find_block_ertr::make_ready_future<bool>(false);
- });
- }).safe_then([&free_blocks, this] () {
- logger().debug(" free_blocks: {} ", free_blocks);
- super.free_block_count = free_blocks;
- return check_bitmap_blocks_ertr::now();
- }).handle_error(
- check_bitmap_blocks_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error in RandomBlockManager::find_free_block"
- }
- );
-
- });
-}
-
-RandomBlockManager::write_ertr::future<> RandomBlockManager::write(
- blk_paddr_t addr,
- bufferlist &bl)
-{
- 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) {
- logger().error(
- "write: "
- "exception creating aligned buffer {}",
- e
- );
- ceph_assert(0 == "unhandled exception");
- }
- return device->write(
- addr,
- 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
- << ", start=" << header.start
- << ", end=" << header.end
- << ", magic=" << header.magic
- << ", uuid=" << header.uuid
- << ", free_block_count=" << header.free_block_count
- << ", alloc_area_size=" << header.alloc_area_size
- << ", start_alloc_area=" << header.start_alloc_area
- << ", start_data_area=" << header.start_data_area
- << ", flag=" << header.flag
- << ", feature=" << header.feature
- << ", crc=" << header.crc;
- return out << ")";
-}
-
-std::ostream &operator<<(std::ostream &out, const rbm_bitmap_block_header_t &header)
-{
- out << " rbm_bitmap_block_header_t(size=" << header.size
- << ", checksum=" << header.checksum;
- return out << ")";
-}
-
-}
+++ /dev/null
-// -*- 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/os/seastore/nvmedevice/block.h"
-#include "crimson/os/seastore/nvmedevice/nvmedevice.h"
-
-#include "crimson/common/layout.h"
-#include "include/buffer.h"
-#include "include/uuid.h"
-
-namespace crimson::os::seastore {
-
-constexpr uint32_t RBM_SUPERBLOCK_SIZE = 4096;
-
-using NVMeBlockDevice = nvme_device::NVMeBlockDevice;
-using NVMeBlockDeviceRef = std::unique_ptr<NVMeBlockDevice>;
-
-enum {
- RBM_NVME_END_TO_END_PROTECTION = 1, // TODO: This allows the device to manage crc on a block by itself
- RBM_BITMAP_BLOCK_CRC = 2,
-};
-
-constexpr uint32_t BITS_PER_CHAR = 8;
-inline char BIT_CHAR_MASK(uint64_t nr)
-{
- return (char(1) << (nr % BITS_PER_CHAR));
-}
-
-struct rbm_metadata_header_t {
- size_t size = 0;
- size_t block_size = 0;
- uint64_t start; // start location of the device
- uint64_t end; // end location of the device
- uint64_t magic; // to indicate randomblock_manager
- uuid_d uuid;
- uint64_t free_block_count;
- uint64_t alloc_area_size; // bitmap
- uint32_t start_alloc_area; // block number
- uint32_t start_data_area;
- uint64_t flag; // reserved
- uint64_t feature;
- checksum_t crc;
-
- DENC(rbm_metadata_header_t, v, p) {
- DENC_START(1, 1, p);
- denc(v.size, p);
- denc(v.block_size, p);
- denc(v.start, p);
- denc(v.end, p);
- denc(v.magic, p);
- denc(v.uuid, p);
- denc(v.free_block_count, p);
- denc(v.alloc_area_size, p);
- denc(v.start_alloc_area, p);
- denc(v.start_data_area, p);
- denc(v.flag, p);
- denc(v.feature, p);
-
- denc(v.crc, p);
- DENC_FINISH(p);
- }
-
-};
-
-struct rbm_bitmap_block_header_t {
- uint32_t size;
- checksum_t checksum;
- DENC(rbm_bitmap_block_header_t, v, p) {
- DENC_START(1, 1, p);
- denc(v.size, p);
- denc(v.checksum, p);
- DENC_FINISH(p);
- }
-};
-
-std::ostream &operator<<(std::ostream &out, const rbm_metadata_header_t &header);
-std::ostream &operator<<(std::ostream &out, const rbm_bitmap_block_header_t &header);
-
-enum class bitmap_op_types_t : uint8_t {
- ALL_CLEAR = 1,
- ALL_SET = 2
-};
-
-struct rbm_bitmap_block_t {
- rbm_bitmap_block_header_t header;
- bufferlist buf;
-
- uint64_t get_size() {
- return header.size;
- }
- void set_crc() {
- header.checksum = buf.crc32c(-1);
- }
-
- bool is_correct_crc() {
- ceph_assert(buf.length());
- return buf.crc32c(-1) == header.checksum;
- }
-
- void set_bit(uint64_t nr) {
- ceph_assert(buf.length());
- char mask = BIT_CHAR_MASK(nr);
- char *p = buf.c_str() + (nr / BITS_PER_CHAR);
- *p |= mask;
- }
-
- void set_all_bits() {
- ceph_assert(buf.length());
- ::memset(buf.c_str(), std::numeric_limits<unsigned char>::max(), buf.length());
- }
-
- void set_clear_bits() {
- ceph_assert(buf.length());
- ::memset(buf.c_str(), 0, buf.length());
- }
-
- void clear_bit(uint64_t nr) {
- ceph_assert(buf.length());
- char mask = ~BIT_CHAR_MASK(nr);
- char *p = buf.c_str() + (nr / BITS_PER_CHAR);
- *p &= mask;
- }
-
- bool is_allocated(uint64_t nr) {
- ceph_assert(buf.length());
- char mask = BIT_CHAR_MASK(nr);
- char *p = buf.c_str() + (nr / BITS_PER_CHAR);
- return *p & mask;
- }
-
- rbm_bitmap_block_t(size_t size) {
- header.size = size;
- }
-
- rbm_bitmap_block_t() = default;
-
- DENC(rbm_bitmap_block_t, v, p) {
- DENC_START(1, 1, p);
- denc(v.header, p);
- denc(v.buf, p);
- DENC_FINISH(p);
- }
-};
-
-}
-
-WRITE_CLASS_DENC_BOUNDED(
- crimson::os::seastore::rbm_metadata_header_t
-)
-WRITE_CLASS_DENC_BOUNDED(
- crimson::os::seastore::rbm_bitmap_block_t
-)
-WRITE_CLASS_DENC_BOUNDED(
- crimson::os::seastore::rbm_bitmap_block_header_t
-)
-
-namespace crimson::os::seastore {
-
-class RandomBlockManager {
-public:
-
- struct mkfs_config_t {
- std::string path;
- blk_paddr_t start;
- blk_paddr_t end;
- size_t block_size = 0;
- size_t total_size = 0;
- seastore_meta_t meta;
- };
- using mkfs_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::invarg
- >;
- /*
- * Ondisk layout
- *
- * -------------------------------------------------------------------------------------------------
- * | rbm_metadata_header_t | rbm_bitmap_block_t 1 | rbm_bitmap_block_t 2 | ... | data blocks |
- * -------------------------------------------------------------------------------------------------
- */
-
- mkfs_ertr::future<> mkfs(mkfs_config_t);
-
- using read_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::invarg,
- crimson::ct_error::enoent,
- crimson::ct_error::erange>;
- read_ertr::future<> read(uint64_t addr, bufferptr &buffer);
-
- 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
- >;
- write_ertr::future<> write(uint64_t addr, bufferptr &buf);
-
- using open_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::invarg,
- crimson::ct_error::enoent>;
- open_ertr::future<> open(const std::string &path, blk_paddr_t start);
-
- using close_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::invarg>;
- close_ertr::future<> close();
-
- 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<
- std::optional<bufferptr>
- >;
- /*
- * 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.
- * But, in-memory allocator is the future work, and is not implemented yet,
- * we use freebitmap directly to allocate freeblocks for now.
- *
- * Each bit in freebitmap block represents whether a block is allocated or not.
- *
- * TODO: multiple allocation
- *
- */
- allocate_ertr::future<> alloc_extent(Transaction &t, size_t size); // allocator, return blocks
-
- using free_block_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::invarg
- >;
- /*
- * free_extent
- *
- * add a range of free blocks to transaction
- *
- */
- free_block_ertr::future<> free_extent(Transaction &t, blk_paddr_t from, blk_paddr_t to); // TODO: will include trim if necessary
-
- using abort_allocation_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::invarg
- >;
- abort_allocation_ertr::future<> abort_allocation(Transaction &t);
-
- using complete_allocation_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::invarg,
- crimson::ct_error::enoent,
- crimson::ct_error::erange
- >;
- write_ertr::future<> complete_allocation(Transaction &t);
-
- open_ertr::future<> _open_device(const std::string path);
- read_ertr::future<rbm_metadata_header_t> read_rbm_header(blk_paddr_t addr);
- write_ertr::future<> write_rbm_header();
-
- size_t get_size() { return super.size; };
- size_t get_block_size() { return super.block_size; }
-
- // max block number a block can represent using bitmap
- uint64_t max_block_by_bitmap_block() {
- return (super.block_size - ceph::encoded_sizeof_bounded<rbm_bitmap_block_t>()) * 8;
- }
-
- uint64_t convert_block_no_to_bitmap_block(blk_id_t block_no)
- {
- ceph_assert(super.block_size);
- return block_no / max_block_by_bitmap_block();
- }
-
- /*
- * convert_bitmap_block_no_to_block_id
- *
- * return block id using address where freebitmap is stored and offset
- */
- blk_id_t convert_bitmap_block_no_to_block_id(uint64_t offset, blk_paddr_t addr)
- {
- ceph_assert(super.block_size);
- // freebitmap begins at block 1
- return (addr / super.block_size - 1) * max_block_by_bitmap_block() + offset;
- }
-
- uint64_t get_alloc_area_size() {
- ceph_assert(super.size);
- ceph_assert(super.block_size);
- uint64_t total_block_num = super.size / super.block_size;
- uint64_t need_blocks = (total_block_num % max_block_by_bitmap_block()) ?
- (total_block_num / max_block_by_bitmap_block() + 1) :
- (total_block_num / max_block_by_bitmap_block());
- ceph_assert(need_blocks);
- return need_blocks * super.block_size;
- }
-
- using find_block_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::enoent>;
- using find_block_ret = find_block_ertr::future<interval_set<blk_id_t>>;
- /*
- * find_free_block
- *
- * Try to find free blocks by reading bitmap blocks on the disk sequentially
- * The free blocks will be added to allocated_blocks in Transaction.
- * This needs to be improved after in-memory block allocation is introduced.
- *
- */
- find_block_ret find_free_block(Transaction &t, size_t size);
-
- /*
- * rbm_sync_block_bitmap
- *
- * Write rbm_bitmap_block_t to the device
- *
- * @param rbm_bitmap_block_t
- * @param uint64_t the block number the rbm_bitmap_block_t will be stored
- *
- */
- write_ertr::future<> rbm_sync_block_bitmap(rbm_bitmap_block_t &block, blk_id_t block_no);
-
- using check_bitmap_blocks_ertr = crimson::errorator<
- crimson::ct_error::input_output_error,
- crimson::ct_error::invarg>;
- check_bitmap_blocks_ertr::future<> check_bitmap_blocks();
- uint64_t get_free_blocks() {
- return super.free_block_count;
- }
- /*
- * We will have mulitple partitions (circularjournals and randbomblockmanagers) on a device,
- * so start and end location of the device are needed to support such case.
- *
- */
- RandomBlockManager(NVMeBlockDevice * device, std::string path)
- : device(device), path(path) {}
-
- /*
- * bitmap block area (freebitmap) layout
- *
- * -----------------------------------------------------------
- * | header 1 | bitmap 1 | header 2 | bitmap 2 |
- * -----------------------------------------------------------
- * <-- 1 block --> <-- 1 block -->
- *
- * 1 block contains both bitmap header and bitmap.
- * We use this layout as a default layout here.
- * But, we'll consider to exploit end to end data protection.
- * If we use the end to end data protection, which is a feature specified in NVMe,
- * we can avoid any calculation for checksum. The checksum regarding the block
- * will be managed by the NVMe device.
- *
- */
- mkfs_ertr::future<> initialize_blk_alloc_area();
- uint64_t get_start_block_alloc_area() {
- return super.start_alloc_area;
- }
-
- void alloc_rbm_bitmap_block_buf(rbm_bitmap_block_t &b_block) {
- auto bitmap_blk = ceph::bufferptr(buffer::create_page_aligned(
- super.block_size -
- ceph::encoded_sizeof_bounded<rbm_bitmap_block_t>()));
- bitmap_blk.zero();
- b_block.buf.append(bitmap_blk);
- }
-
- blk_paddr_t get_blk_paddr_by_block_no(blk_id_t id) {
- return (id * super.block_size) + super.start;
- }
-
- int num_block_between_blk_ids(blk_id_t start, blk_id_t end) {
- auto max = max_block_by_bitmap_block();
- auto block_start = start / max;
- auto block_end = end / max;
- return block_end - block_start + 1;
- }
-
- write_ertr::future<> rbm_sync_block_bitmap_by_range(blk_id_t start, blk_id_t end, bitmap_op_types_t op);
- void add_cont_bitmap_blocks_to_buf(bufferlist& buf, int num_block, bitmap_op_types_t op) {
- rbm_bitmap_block_t b_block(super.block_size);
- alloc_rbm_bitmap_block_buf(b_block);
- if (op == bitmap_op_types_t::ALL_SET) {
- b_block.set_all_bits();
- } else {
- b_block.set_clear_bits();
- }
- for (int i = 0; i < num_block; i++) {
- encode(b_block, buf);
- }
- }
-
- write_ertr::future<> write(blk_paddr_t addr, bufferlist &bl);
-
-private:
- rbm_metadata_header_t super; // this contains the number of bitmap blocks, free blocks and rbm specific information
- //FreelistManager free_manager; // TODO: block management
- NVMeBlockDevice * device;
- std::string path;
- int stream_id; // for multi-stream
-};
-using RandomBlockManagerRef = std::unique_ptr<RandomBlockManager>;
-
-
-}
// vim: ts=8 sw=2 smarttab
#include "include/buffer.h"
-#include "crimson/os/seastore/nvmedevice/nvmedevice.h"
+#include "crimson/os/seastore/random_block_manager/nvmedevice.h"
#include "test/crimson/gtest_seastar.h"
#include "include/stringify.h"
#include <random>
#include "crimson/common/log.h"
-#include "crimson/os/seastore/randomblock_manager.h"
-#include "crimson/os/seastore/nvmedevice/nvmedevice.h"
+#include "crimson/os/seastore/random_block_manager/nvme_manager.h"
+#include "crimson/os/seastore/random_block_manager/nvmedevice.h"
#include "test/crimson/seastore/transaction_manager_test_state.h"
#include "crimson/os/seastore/transaction_manager.h"
#include "crimson/os/seastore/cache.h"
TMTestState {
segment_manager::EphemeralSegmentManagerRef segment_manager; // Need to be deleted, just for Cache
Cache cache;
- std::unique_ptr<RandomBlockManager> rbm_manager;
+ std::unique_ptr<NVMeManager> rbm_manager;
nvme_device::NVMeBlockDevice *device;
std::default_random_engine generator;
cache(*segment_manager)
{
device = new nvme_device::TestMemory(DEFAULT_TEST_SIZE);
- rbm_manager.reset(new RandomBlockManager(device, std::string()));
+ rbm_manager.reset(new NVMeManager(device, std::string()));
config.start = 0;
config.end = DEFAULT_TEST_SIZE;
config.block_size = DEFAULT_BLOCK_SIZE;