]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
seastore/rbm: relocate file layout like SegmentManager
authormyoungwon oh <ohmyoungwon@gmail.com>
Fri, 11 Jun 2021 12:00:16 +0000 (21:00 +0900)
committermyoungwon oh <ohmyoungwon@gmail.com>
Tue, 15 Jun 2021 02:09:06 +0000 (11:09 +0900)
Signed-off-by: Myoungwon Oh <myoungwon.oh@samsung.com>
12 files changed:
src/crimson/os/seastore/CMakeLists.txt
src/crimson/os/seastore/nvmedevice/nvmedevice.cc [deleted file]
src/crimson/os/seastore/nvmedevice/nvmedevice.h [deleted file]
src/crimson/os/seastore/random_block_manager.h [new file with mode: 0644]
src/crimson/os/seastore/random_block_manager/nvme_manager.cc [new file with mode: 0644]
src/crimson/os/seastore/random_block_manager/nvme_manager.h [new file with mode: 0644]
src/crimson/os/seastore/random_block_manager/nvmedevice.cc [new file with mode: 0644]
src/crimson/os/seastore/random_block_manager/nvmedevice.h [new file with mode: 0644]
src/crimson/os/seastore/randomblock_manager.cc [deleted file]
src/crimson/os/seastore/randomblock_manager.h [deleted file]
src/test/crimson/seastore/nvmedevice/test_nvmedevice.cc
src/test/crimson/seastore/test_randomblock_manager.cc

index ebedd99e131b1ec02d7c2be6f815614255e36b79..08d7605528c7add6b95fe8046009654e571e9f5b 100644 (file)
@@ -32,8 +32,8 @@ add_library(crimson-seastore STATIC
   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
        )
diff --git a/src/crimson/os/seastore/nvmedevice/nvmedevice.cc b/src/crimson/os/seastore/nvmedevice/nvmedevice.cc
deleted file mode 100644 (file)
index 3d79b41..0000000
+++ /dev/null
@@ -1,151 +0,0 @@
-// -*- 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();
-}
-}
diff --git a/src/crimson/os/seastore/nvmedevice/nvmedevice.h b/src/crimson/os/seastore/nvmedevice/nvmedevice.h
deleted file mode 100644 (file)
index 87ff562..0000000
+++ /dev/null
@@ -1,320 +0,0 @@
-//-*- 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;
-};
-
-}
diff --git a/src/crimson/os/seastore/random_block_manager.h b/src/crimson/os/seastore/random_block_manager.h
new file mode 100644 (file)
index 0000000..8efca62
--- /dev/null
@@ -0,0 +1,105 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#pragma once
+
+#include <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>;
+
+}
diff --git a/src/crimson/os/seastore/random_block_manager/nvme_manager.cc b/src/crimson/os/seastore/random_block_manager/nvme_manager.cc
new file mode 100644 (file)
index 0000000..f22b8e4
--- /dev/null
@@ -0,0 +1,702 @@
+// -*- 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 << ")";
+}
+
+}
diff --git a/src/crimson/os/seastore/random_block_manager/nvme_manager.h b/src/crimson/os/seastore/random_block_manager/nvme_manager.h
new file mode 100644 (file)
index 0000000..e8745dd
--- /dev/null
@@ -0,0 +1,372 @@
+// -*- 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>;
+
+}
diff --git a/src/crimson/os/seastore/random_block_manager/nvmedevice.cc b/src/crimson/os/seastore/random_block_manager/nvmedevice.cc
new file mode 100644 (file)
index 0000000..518fe93
--- /dev/null
@@ -0,0 +1,151 @@
+// -*- 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();
+}
+}
diff --git a/src/crimson/os/seastore/random_block_manager/nvmedevice.h b/src/crimson/os/seastore/random_block_manager/nvmedevice.h
new file mode 100644 (file)
index 0000000..61d07f3
--- /dev/null
@@ -0,0 +1,320 @@
+//-*- 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;
+};
+
+}
diff --git a/src/crimson/os/seastore/randomblock_manager.cc b/src/crimson/os/seastore/randomblock_manager.cc
deleted file mode 100644 (file)
index b500f56..0000000
+++ /dev/null
@@ -1,666 +0,0 @@
-// -*- 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 << ")";
-}
-
-}
diff --git a/src/crimson/os/seastore/randomblock_manager.h b/src/crimson/os/seastore/randomblock_manager.h
deleted file mode 100644 (file)
index 7a292ae..0000000
+++ /dev/null
@@ -1,421 +0,0 @@
-// -*- 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>;
-
-
-}
index 7dfaece9ef75fa99242783b1dc160d550e15154f..68010f53c4f350db5e8f33d7a0d5d5baa139d122 100644 (file)
@@ -2,7 +2,7 @@
 // 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"
 
index 66339d22e3aee6c422229ba54d8b23a1c67722b9..8ea9a502b5099293486c6647cf68ba954843712d 100644 (file)
@@ -6,8 +6,8 @@
 #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"
@@ -30,7 +30,7 @@ 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;
+  std::unique_ptr<NVMeManager> rbm_manager;
   nvme_device::NVMeBlockDevice *device;
 
   std::default_random_engine generator;
@@ -45,7 +45,7 @@ struct rbm_test_t : public  seastar_test_suite_t,
       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;