collection_manager/collection_flat_node.cc
object_data_handler.cc
seastore.cc
+ randomblock_manager.cc
+ nvmedevice/memory.cc
../../../test/crimson/seastore/test_block.cc
${PROJECT_SOURCE_DIR}/src/os/Transaction.cc
)
});
}
+ for (auto b : t.allocated_blocks) {
+ record.extents.push_back(
+ extent_t{
+ extent_types_t::RBM_ALLOC_INFO,
+ b.addr,
+ std::move(b.bl)
+ });
+ }
+
return std::make_optional<record_t>(std::move(record));
}
--- /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"
+
+namespace {
+ seastar::logger& logger() {
+ return crimson::get_logger(ceph_subsys_filestore);
+ }
+}
+
+namespace crimson::os::seastore::nvme_device {
+
+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 divice {}",
+ 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)
+{
+ 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 <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 {
+
+
+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>;
+
+/*
+ * 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 = 0;
+
+ uint64_t write_granularity = 4096;
+ uint64_t write_alignment = 4096;
+
+public:
+ NVMeBlockDevice() {}
+ virtual ~NVMeBlockDevice() = default;
+
+ template <typename T>
+ static NVMeBlockDevice *create() {
+ return new 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;
+
+ virtual write_ertr::future<> write(
+ uint64_t offset,
+ bufferptr &bptr) = 0;
+
+ // TODO
+ virtual int discard(uint64_t offset, uint64_t len) { return 0; }
+
+ virtual open_ertr::future<> open(
+ const std::string& path,
+ seastar::open_flags mode) = 0;
+
+ virtual seastar::future<> close() = 0;
+
+};
+
+
+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);
+
+ write_ertr::future<> write(
+ uint64_t offset,
+ bufferptr &bptr);
+
+ read_ertr::future<> read(
+ uint64_t offset,
+ bufferptr &bptr);
+
+ seastar::future<> close();
+
+ 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"
+
+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 bp = ceph::bufferptr(buffer::create_page_aligned(super.block_size));
+ bp.zero();
+ /* initialize alloc area to zero */
+ for (uint64_t i = 0; i < super.alloc_area_size / super.block_size; i++) {
+ device->write(super.start_alloc_area + (i * super.block_size), bp);
+ }
+
+ /* write allocated bitmap info to rbm meta block */
+ auto start = super.start_data_area / super.block_size;
+ rbm_bitmap_block_t b_block(super.block_size);
+ alloc_rbm_bitmap_block_buf(b_block);
+ logger().debug(" start {} ", start);
+ 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] () mutable {
+ /*
+ * Set rest of the block bitmap to 1
+ * To do so, we only mark 1 to the byte, which is the last byte of bitmap-blocks
+ */
+ uint64_t last_block_no = super.size/super.block_size - 1;
+ uint64_t remain_block = last_block_no % max_block_by_bitmap_block();
+ logger().debug(" last_block_no: {}, remain_block: {} ",
+ last_block_no, remain_block);
+ if (remain_block) {
+ logger().debug(" try to remained write alloc info ");
+ if (last_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, last_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"
+ }
+ );
+}
+
+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),
+ [&, this] (auto &allocated, auto addr) mutable {
+ return crimson::do_until(
+ [&, this] () mutable {
+ return device->read(
+ addr,
+ bp
+ ).safe_then([bp, addr, size, &allocated, &t, 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 prev_alloc_blocks = allocated;
+ auto allocated_blocks = t.get_rbm_allocated_blocks();
+ std::vector<blk_id_t> ids;
+ for (uint64_t i = 0; i < max; i++) {
+ bool alloc = false;
+ for (auto b : allocated_blocks) {
+ if (std::find(b.blk_ids.begin(), b.blk_ids.end(), convert_bitmap_block_no_to_block_id(i, addr))
+ != b.blk_ids.end()) {
+ alloc = true;
+ break;
+ }
+ }
+ if (b_block.is_allocated(i) || alloc) {
+ continue;
+ }
+ b_block.set_bit(i);
+ logger().debug("find_free_list: allocated block no {} i {}",
+ convert_bitmap_block_no_to_block_id(i, addr), i);
+ ids.push_back(convert_bitmap_block_no_to_block_id(i, addr));
+ allocated += 1;
+ if (((uint64_t)size)/super.block_size <= allocated ||
+ addr > super.end) {
+ break;
+ }
+ }
+
+ if (allocated - prev_alloc_blocks > 0) {
+ bufferlist bl;
+ encode(b_block, bl);
+ rbm_extent_t extent{
+ extent_types_t::RBM_ALLOC_INFO,
+ ids,
+ addr,
+ bl
+ };
+ t.add_rbm_allocated_blocks(extent);
+ }
+ if (((uint64_t)size)/super.block_size <= allocated ||
+ addr > super.end) {
+ logger().debug(" allocated: {} ", allocated);
+ return find_block_ertr::make_ready_future<bool>(true);
+ }
+
+ addr += super.block_size;
+ return find_block_ertr::make_ready_future<bool>(false);
+ });
+ }).safe_then([&allocated, &t] () {
+ logger().debug(" allocated: {} size {} ", allocated, t.get_rbm_allocated_blocks().size());
+ return find_block_ret(
+ find_block_ertr::ready_future_marker{},
+ allocated);
+ }).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 allocated) mutable -> allocate_ertr::future<> {
+ logger().debug("after find_free_block: allocated {}", allocated);
+ if (allocated * super.block_size < size) {
+ t.clear_rbm_allocated_blocks();
+ 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 blk_paddr)
+{
+
+ blk_id_t block_no = blk_paddr / super.block_size;
+ auto addr = (super.start_alloc_area +
+ block_no / max_block_by_bitmap_block())
+ * super.block_size;
+ auto bp = bufferptr(ceph::buffer::create_page_aligned(super.block_size));
+ return device->read(
+ addr,
+ bp
+ ).safe_then([&, this]() {
+ logger().debug("free_extent: addr {}", addr);
+ rbm_bitmap_block_t b_block(super.block_size);
+ bufferlist bl_bitmap_block;
+ bl_bitmap_block.append(bp);
+ encode(b_block, bl_bitmap_block);
+
+ auto remain_offset = block_no % max_block_by_bitmap_block();
+ b_block.clear_bit(remain_offset);
+ bufferlist bl;
+ encode(b_block, bl);
+ logger().debug("free_extent: free block no {}",
+ convert_bitmap_block_no_to_block_id(remain_offset, addr));
+ std::vector<blk_id_t> ids;
+ ids.push_back(convert_bitmap_block_no_to_block_id(remain_offset, addr));
+ rbm_extent_t extent{
+ extent_types_t::RBM_ALLOC_INFO,
+ ids,
+ addr,
+ bl
+ };
+ t.add_rbm_allocated_blocks(extent);
+ return free_block_ertr::now();
+ }
+ ).handle_error(
+ free_block_ertr::pass_further{},
+ crimson::ct_error::assert_all{
+ "Invalid error in RandomBlockManager::free_extent"
+ }
+ );
+}
+
+RandomBlockManager::write_ertr::future<>
+RandomBlockManager::complete_allocation(Transaction &t)
+{
+ const auto alloc_blocks = t.get_rbm_allocated_blocks();
+ return seastar::do_with(alloc_blocks,
+ [&, this] (auto &alloc_blocks) mutable {
+ return crimson::do_for_each(
+ alloc_blocks,
+ [this, &alloc_blocks](auto &alloc) {
+ logger().debug(
+ "complete_allocation: addr {}",
+ alloc.addr);
+ bufferptr bptr;
+ try {
+ bptr = bufferptr(ceph::buffer::create_page_aligned(alloc.bl.length()));
+ auto iter = alloc.bl.cbegin();
+ iter.copy(alloc.bl.length(), bptr.c_str());
+ } catch (const std::exception &e) {
+ logger().error(
+ "RandomBlockManager::complete_allocation: "
+ "exception creating aligned buffer {}",
+ e
+ );
+ ceph_assert(0 == "unhandled exception");
+ }
+ return device->write(
+ alloc.addr,
+ bptr);
+ }).safe_then([this, &alloc_blocks]() mutable {
+ auto alloc_block_count = 0;
+ for (const auto b : alloc_blocks) {
+ alloc_block_count += b.blk_ids.size();
+ }
+ 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 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 - super.start) {
+ return crimson::ct_error::erange::make();
+ }
+ return device->write(
+ addr,
+ bptr);
+}
+
+RandomBlockManager::read_ertr::future<>
+RandomBlockManager::read(
+ blk_paddr_t addr,
+ bufferptr &buffer)
+{
+ ceph_assert(device);
+ if (addr > super.end - super.start ||
+ buffer.length() > super.end - super.start) {
+ return crimson::ct_error::erange::make();
+ }
+ return device->read(
+ addr,
+ buffer);
+}
+
+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"
+ }
+ );
+}
+
+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);
+
+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 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_block_ertr::future<> free_extent(Transaction &t, blk_paddr_t blk_paddr); // TODO: will include trim if necessary
+
+ 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<uint32_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);
+
+ /*
+ * 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;
+ }
+
+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>;
+
+
+}
OBJECT_DATA_BLOCK = 8,
RETIRED_PLACEHOLDER = 9,
+ RBM_ALLOC_INFO = 0xE0,
// Test Block Types
TEST_BLOCK = 0xF0,
TEST_BLOCK_PHYSICAL = 0xF1,
}
};
+using blk_id_t = uint64_t;
+constexpr blk_id_t NULL_BLK_ID =
+ std::numeric_limits<blk_id_t>::max();
+
+// use absolute address
+using blk_paddr_t = uint64_t;
+struct rbm_extent_t {
+ extent_types_t type;
+ std::vector<blk_id_t> blk_ids;
+ blk_paddr_t addr;
+ ceph::bufferlist bl;
+};
+
}
WRITE_CLASS_DENC_BOUNDED(crimson::os::seastore::seastore_meta_t)
return weak;
}
+ void add_rbm_allocated_blocks(rbm_extent_t &extent) {
+ allocated_blocks.push_back(extent);
+ }
+ void clear_rbm_allocated_blocks() {
+ if (!allocated_blocks.empty()) {
+ allocated_blocks.clear();
+ }
+ }
+ const auto &get_rbm_allocated_blocks() {
+ return allocated_blocks;
+ }
+
private:
friend class Cache;
friend Ref make_test_transaction();
retired_extent_gate_t::token_t retired_gate_token;
+ std::vector<rbm_extent_t> allocated_blocks;
+
public:
Transaction(
OrderingHandle &&handle,
crimson-seastore
crimson-common)
+add_executable(unittest-seastore-randomblock-manager
+ test_randomblock_manager.cc)
+add_ceph_test(unittest-seastore-randomblock-manager
+ unittest-seastore-randomblock-manager --memory 256M --smp 1)
+target_link_libraries(
+ unittest-seastore-randomblock-manager
+ crimson::gtest
+ ${CMAKE_DL_LIBS}
+ crimson-seastore)
+
add_subdirectory(onode_tree)
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "test/crimson/gtest_seastar.h"
+
+#include <random>
+
+#include "crimson/common/log.h"
+#include "crimson/os/seastore/randomblock_manager.h"
+#include "crimson/os/seastore/nvmedevice/nvmedevice.h"
+#include "test/crimson/seastore/transaction_manager_test_state.h"
+#include "crimson/os/seastore/transaction_manager.h"
+#include "crimson/os/seastore/cache.h"
+#include "crimson/os/seastore/segment_manager/ephemeral.h"
+
+using namespace crimson;
+using namespace crimson::os;
+using namespace crimson::os::seastore;
+
+namespace {
+ [[maybe_unused]] seastar::logger& logger() {
+ return crimson::get_logger(ceph_subsys_test);
+ }
+}
+
+constexpr uint64_t DEFAULT_TEST_SIZE = 1 << 20;
+constexpr uint64_t DEFAULT_BLOCK_SIZE = 4096;
+
+struct rbm_test_t : public seastar_test_suite_t,
+ TMTestState {
+ segment_manager::EphemeralSegmentManagerRef segment_manager; // Need to be deleted, just for Cache
+ Cache cache;
+ std::unique_ptr<RandomBlockManager> rbm_manager;
+ nvme_device::NVMeBlockDevice *device;
+
+ std::default_random_engine generator;
+
+ const uint64_t block_size = DEFAULT_BLOCK_SIZE;
+
+ RandomBlockManager::mkfs_config_t config;
+ paddr_t current;
+
+ rbm_test_t() :
+ segment_manager(segment_manager::create_test_ephemeral()),
+ cache(*segment_manager)
+ {
+ device = new nvme_device::TestMemory(DEFAULT_TEST_SIZE);
+ rbm_manager.reset(new RandomBlockManager(device, std::string()));
+ config.start = 0;
+ config.end = DEFAULT_TEST_SIZE;
+ config.block_size = DEFAULT_BLOCK_SIZE;
+ config.total_size = DEFAULT_TEST_SIZE;
+ }
+
+ seastar::future<> set_up_fut() final {
+ return tm_setup();
+ }
+
+ seastar::future<> tear_down_fut() final {
+ if (device) {
+ delete device;
+ }
+ return tm_teardown();
+ }
+
+ auto mkfs() {
+ return rbm_manager->mkfs(config).unsafe_get0();
+ }
+
+ auto read_rbm_header() {
+ return rbm_manager->read_rbm_header(config.start).unsafe_get0();
+ }
+
+ auto open() {
+ return rbm_manager->open("", config.start).unsafe_get0();
+ }
+
+ auto write(uint64_t addr, bufferptr &ptr) {
+ return rbm_manager->write(addr, ptr).unsafe_get0();
+ }
+
+ auto read(uint64_t addr, bufferptr &ptr) {
+ return rbm_manager->read(addr, ptr).unsafe_get0();
+ }
+
+ auto create_transaction() {
+ return tm->create_transaction();
+ }
+
+ auto alloc_extent(Transaction &t, size_t size) {
+ return rbm_manager->alloc_extent(t, size).unsafe_get0();
+ }
+
+ auto submit_transaction(Transaction &t) {
+ auto record = cache.try_construct_record(t);
+ if (!record) {
+ return false;
+ }
+
+ auto allocated_blocks = t.get_rbm_allocated_blocks();
+
+ bufferlist bl;
+ uint32_t iter_cnt = 0;
+ for (auto &&block : record->extents) {
+ if (block.type == extent_types_t::RBM_ALLOC_INFO) {
+ rbm_bitmap_block_t b_block(DEFAULT_BLOCK_SIZE);
+ for (auto id : allocated_blocks[iter_cnt].blk_ids) {
+ logger().debug("submit allocated block id {}", id);
+ decode(b_block, block.bl);
+ iter_cnt++;
+ if (!b_block.is_allocated(id % rbm_manager->max_block_by_bitmap_block())) {
+ return false;
+ }
+ }
+ }
+ }
+ return true;
+ }
+
+ vector<blk_id_t> get_allocated_blk_ids(Transaction &t) {
+ vector<blk_id_t> ids;
+ auto allocated_blocks = t.get_rbm_allocated_blocks();
+ for (auto p : allocated_blocks) {
+ ids.insert(ids.end(), p.blk_ids.begin(), p.blk_ids.end());
+ }
+ logger().debug(" get allocated id {}", ids);
+ return ids;
+ }
+
+ bool check_ids_are_allocated(vector<blk_id_t> &ids) {
+
+ bool ret = true;
+ for (auto id : ids) {
+ auto addr = rbm_manager->get_start_block_alloc_area() +
+ (id / rbm_manager->max_block_by_bitmap_block())
+ * DEFAULT_BLOCK_SIZE;
+ logger().debug(" addr {} id {} ", addr, id);
+ auto bp = bufferptr(ceph::buffer::create_page_aligned(DEFAULT_BLOCK_SIZE));
+ rbm_manager->read(addr, bp).unsafe_get0();
+ rbm_bitmap_block_t b_block(DEFAULT_BLOCK_SIZE);
+ bufferlist bl;
+ bl.append(bp);
+ auto b_bl = bl.cbegin();
+ decode(b_block, b_bl);
+ logger().debug(" offset {} ", id % rbm_manager->max_block_by_bitmap_block());
+ if (!b_block.is_allocated(id % rbm_manager->max_block_by_bitmap_block())) {
+ ret = false;
+ break;
+ }
+
+ }
+ return ret;
+ }
+
+ auto complete_allocation(Transaction &t) {
+ return rbm_manager->complete_allocation(t).unsafe_get0();
+ }
+
+ bufferptr generate_extent(size_t blocks) {
+ std::uniform_int_distribution<char> distribution(
+ std::numeric_limits<char>::min(),
+ std::numeric_limits<char>::max()
+ );
+ char contents = distribution(generator);
+ return buffer::ptr(buffer::create(blocks * block_size, contents));
+ }
+
+};
+
+TEST_F(rbm_test_t, mkfs_test)
+{
+ run_async([this] {
+ mkfs();
+ open();
+ auto super = read_rbm_header();
+ ASSERT_TRUE(
+ super.block_size == DEFAULT_BLOCK_SIZE &&
+ super.end == DEFAULT_TEST_SIZE &&
+ super.start_alloc_area == DEFAULT_BLOCK_SIZE &&
+ super.free_block_count == DEFAULT_TEST_SIZE / DEFAULT_BLOCK_SIZE - 2 &&
+ super.alloc_area_size == DEFAULT_BLOCK_SIZE
+ );
+
+ });
+}
+
+TEST_F(rbm_test_t, open_test)
+{
+ run_async([this] {
+ mkfs();
+ open();
+ auto content = generate_extent(1);
+ write(
+ DEFAULT_BLOCK_SIZE,
+ content
+ );
+ auto bp = bufferptr(ceph::buffer::create_page_aligned(DEFAULT_BLOCK_SIZE));
+ read(
+ DEFAULT_BLOCK_SIZE,
+ bp
+ );
+ bufferlist bl;
+ bufferlist block;
+ bl.append(bp);
+ block.append(content);
+ ASSERT_EQ(
+ bl.begin().crc32c(bl.length(), 1),
+ block.begin().crc32c(block.length(), 1));
+
+ });
+}
+
+TEST_F(rbm_test_t, block_alloc_test)
+{
+ run_async([this] {
+ mkfs();
+ open();
+ auto t = tm->create_transaction();
+ alloc_extent(*t, DEFAULT_BLOCK_SIZE);
+ auto alloc_ids = get_allocated_blk_ids(*t);
+ ASSERT_TRUE(submit_transaction(*t));
+ complete_allocation(*t);
+ ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
+
+ auto t2 = tm->create_transaction();
+ alloc_extent(*t2, DEFAULT_BLOCK_SIZE);
+ alloc_ids = get_allocated_blk_ids(*t);
+ alloc_ids = get_allocated_blk_ids(*t2);
+ ASSERT_TRUE(submit_transaction(*t2));
+ complete_allocation(*t2);
+ });
+}