object_data_handler.cc
seastore.cc
random_block_manager/block_rb_manager.cc
- random_block_manager/nvmedevice.cc
+ random_block_manager/nvme_block_device.cc
journal/segmented_journal.cc
journal/segment_allocator.cc
journal.cc
}
JournalRef make_circularbounded(
- crimson::os::seastore::nvme_device::NVMeBlockDevice* device,
+ crimson::os::seastore::nvme_device::RBMDevice* device,
std::string path)
{
return std::make_unique<CircularBoundedJournal>(device, path);
namespace crimson::os::seastore {
namespace nvme_device {
-class NVMeBlockDevice;
+class RBMDevice;
}
class SegmentManagerGroup;
JournalRef make_segmented(SegmentProvider &provider);
JournalRef make_circularbounded(
- crimson::os::seastore::nvme_device::NVMeBlockDevice* device,
+ crimson::os::seastore::nvme_device::RBMDevice* device,
std::string path);
}
<< ")";
}
-CircularBoundedJournal::CircularBoundedJournal(NVMeBlockDevice* device,
+CircularBoundedJournal::CircularBoundedJournal(RBMDevice* device,
const std::string &path)
: device(device), path(path) {}
#include "crimson/os/seastore/journal.h"
#include "include/uuid.h"
#include "crimson/os/seastore/random_block_manager.h"
-#include "crimson/os/seastore/random_block_manager/nvmedevice.h"
+#include "crimson/os/seastore/random_block_manager/rbm_device.h"
#include <list>
constexpr rbm_abs_addr CBJOURNAL_START_ADDRESS = 0;
constexpr uint64_t CBJOURNAL_MAGIC = 0xCCCC;
-using NVMeBlockDevice = nvme_device::NVMeBlockDevice;
+using RBMDevice = nvme_device::RBMDevice;
/**
* CircularBoundedJournal
}
};
- CircularBoundedJournal(NVMeBlockDevice* device, const std::string &path);
+ CircularBoundedJournal(RBMDevice* device, const std::string &path);
~CircularBoundedJournal() {}
open_for_mkfs_ret open_for_mkfs() final;
rbm_abs_addr get_journal_end() const {
return get_start_addr() + header.size + get_block_size(); // journal size + header length
}
- void add_device(NVMeBlockDevice* dev) {
+ void add_device(RBMDevice* dev) {
device = dev;
}
private:
cbj_header_t header;
- NVMeBlockDevice* device;
+ RBMDevice* device;
std::string path;
WritePipeline *write_pipeline = nullptr;
/**
#include "crimson/os/seastore/logging.h"
#include "include/buffer.h"
-#include "nvmedevice.h"
+#include "rbm_device.h"
#include "include/interval_set.h"
#include "include/intarith.h"
#include "block_rb_manager.h"
#include "crimson/osd/exceptions.h"
#include "crimson/os/seastore/transaction.h"
-#include "nvmedevice.h"
+#include "rbm_device.h"
#include "crimson/os/seastore/random_block_manager.h"
#include "crimson/common/layout.h"
constexpr uint32_t RBM_SUPERBLOCK_SIZE = 4096;
-using NVMeBlockDevice = nvme_device::NVMeBlockDevice;
-using NVMeBlockDeviceRef = std::unique_ptr<NVMeBlockDevice>;
+using RBMDevice = nvme_device::RBMDevice;
+using RBMDeviceRef = std::unique_ptr<RBMDevice>;
enum {
// TODO: This allows the device to manage crc on a block by itself
* on a device, so start and end location of the device are needed to
* support such case.
*/
- BlockRBManager(NVMeBlockDevice * device, std::string path)
+ BlockRBManager(RBMDevice * device, std::string path)
: device(device), path(path) {}
/*
*/
rbm_metadata_header_t super;
//FreelistManager free_manager; // TODO: block management
- NVMeBlockDevice * device;
+ RBMDevice * device;
std::string path;
int stream_id; // for multi-stream
};
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include <sys/mman.h>
+#include <string.h>
+
+#include <fcntl.h>
+
+#include "crimson/common/log.h"
+#include "crimson/common/errorator-loop.h"
+
+#include "include/buffer.h"
+#include "rbm_device.h"
+
+namespace {
+ seastar::logger& logger() {
+ return crimson::get_logger(ceph_subsys_seastore_tm);
+ }
+}
+
+namespace crimson::os::seastore::nvme_device {
+
+open_ertr::future<> NVMeBlockDevice::open(
+ const std::string &in_path,
+ seastar::open_flags mode) {
+ return seastar::do_with(in_path, [this, mode](auto& in_path) {
+ return seastar::file_stat(in_path).then([this, mode, in_path](auto stat) {
+ size = stat.size;
+ return seastar::open_file_dma(in_path, mode).then([=](auto file) {
+ device = file;
+ logger().debug("open");
+ // Get SSD's features from identify_controller and namespace command.
+ // Do identify_controller first, and then identify_namespace.
+ return identify_controller().safe_then([this, in_path, mode](
+ auto id_controller_data) {
+ support_multistream = id_controller_data.oacs.support_directives;
+ if (support_multistream) {
+ stream_id_count = WRITE_LIFE_MAX;
+ }
+ awupf = id_controller_data.awupf + 1;
+ return identify_namespace().safe_then([this, in_path, mode] (
+ auto id_namespace_data) {
+ // LBA format provides LBA size which is power of 2. LBA is the
+ // minimum size of read and write.
+ block_size = (1 << id_namespace_data.lbaf0.lbads);
+ atomic_write_unit = awupf * block_size;
+ data_protection_type = id_namespace_data.dps.protection_type;
+ data_protection_enabled = (data_protection_type > 0);
+ if (id_namespace_data.nsfeat.opterf == 1){
+ // NPWG and NPWA is 0'based value
+ write_granularity = block_size * (id_namespace_data.npwg + 1);
+ write_alignment = block_size * (id_namespace_data.npwa + 1);
+ }
+ return open_for_io(in_path, mode);
+ });
+ }).handle_error(crimson::ct_error::input_output_error::handle([this, in_path, mode]{
+ logger().error("open: id ctrlr failed. open without ioctl");
+ return open_for_io(in_path, mode);
+ }), crimson::ct_error::pass_further_all{});
+ });
+ });
+ });
+}
+
+open_ertr::future<> NVMeBlockDevice::open_for_io(
+ const std::string& in_path,
+ seastar::open_flags mode) {
+ io_device.resize(stream_id_count);
+ return seastar::do_for_each(io_device, [=](auto &target_device) {
+ return seastar::open_file_dma(in_path, mode).then([this](
+ auto file) {
+ io_device[stream_index_to_open] = file;
+ return io_device[stream_index_to_open].fcntl(
+ F_SET_FILE_RW_HINT,
+ (uintptr_t)&stream_index_to_open).then([this](auto ret) {
+ stream_index_to_open++;
+ return seastar::now();
+ });
+ });
+ });
+}
+
+write_ertr::future<> NVMeBlockDevice::write(
+ uint64_t offset,
+ bufferptr &bptr,
+ uint16_t stream) {
+ logger().debug(
+ "block: write offset {} len {}",
+ offset,
+ bptr.length());
+ auto length = bptr.length();
+
+ assert((length % block_size) == 0);
+ uint16_t supported_stream = stream;
+ if (stream >= stream_id_count) {
+ supported_stream = WRITE_LIFE_NOT_SET;
+ }
+ return io_device[supported_stream].dma_write(
+ offset, bptr.c_str(), length).handle_exception(
+ [](auto e) -> write_ertr::future<size_t> {
+ logger().error("write: dma_write got error{}", e);
+ return crimson::ct_error::input_output_error::make();
+ }).then([length](auto result) -> write_ertr::future<> {
+ if (result != length) {
+ logger().error("write: dma_write got error with not proper length");
+ return crimson::ct_error::input_output_error::make();
+ }
+ return write_ertr::now();
+ });
+}
+
+read_ertr::future<> NVMeBlockDevice::read(
+ uint64_t offset,
+ bufferptr &bptr) {
+ logger().debug(
+ "block: read offset {} len {}",
+ offset,
+ bptr.length());
+ auto length = bptr.length();
+
+ assert((length % block_size) == 0);
+
+ return device.dma_read(offset, bptr.c_str(), length).handle_exception(
+ [](auto e) -> read_ertr::future<size_t> {
+ logger().error("read: dma_read got error{}", e);
+ return crimson::ct_error::input_output_error::make();
+ }).then([length](auto result) -> read_ertr::future<> {
+ if (result != length) {
+ logger().error("read: dma_read got error with not proper length");
+ return crimson::ct_error::input_output_error::make();
+ }
+ return read_ertr::now();
+ });
+}
+
+write_ertr::future<> NVMeBlockDevice::writev(
+ uint64_t offset,
+ ceph::bufferlist bl,
+ uint16_t stream) {
+ logger().debug(
+ "block: write offset {} len {}",
+ offset,
+ bl.length());
+
+ uint16_t supported_stream = stream;
+ if (stream >= stream_id_count) {
+ supported_stream = WRITE_LIFE_NOT_SET;
+ }
+ bl.rebuild_aligned(block_size);
+
+ return seastar::do_with(
+ bl.prepare_iovs(),
+ std::move(bl),
+ [this, supported_stream, offset](auto& iovs, auto& bl)
+ {
+ return write_ertr::parallel_for_each(
+ iovs,
+ [this, supported_stream, offset](auto& p) mutable
+ {
+ auto off = offset + p.offset;
+ auto len = p.length;
+ auto& iov = p.iov;
+ return io_device[supported_stream].dma_write(off, std::move(iov)
+ ).handle_exception(
+ [this, off, len](auto e) -> write_ertr::future<size_t>
+ {
+ logger().error("{} poffset={}~{} dma_write got error -- {}",
+ device_id_printer_t{get_device_id()}, off, len, e);
+ return crimson::ct_error::input_output_error::make();
+ }).then([this, off, len](size_t written) -> write_ertr::future<> {
+ if (written != len) {
+ logger().error("{} poffset={}~{} dma_write len={} inconsistent",
+ device_id_printer_t{get_device_id()}, off, len, written);
+ return crimson::ct_error::input_output_error::make();
+ }
+ return write_ertr::now();
+ });
+ });
+ });
+}
+
+Device::close_ertr::future<> NVMeBlockDevice::close() {
+ logger().debug(" close ");
+ return device.close().then([this]() {
+ return seastar::do_for_each(io_device, [](auto target_device) {
+ return target_device.close();
+ });
+ });
+}
+
+nvme_command_ertr::future<nvme_identify_controller_data_t>
+NVMeBlockDevice::identify_controller() {
+ return seastar::do_with(
+ nvme_admin_command_t(),
+ nvme_identify_controller_data_t(),
+ [this](auto &admin_command, auto &data) {
+ admin_command.common.opcode = nvme_admin_command_t::OPCODE_IDENTIFY;
+ admin_command.common.addr = (uint64_t)&data;
+ admin_command.common.data_len = sizeof(data);
+ admin_command.identify.cns = nvme_identify_command_t::CNS_CONTROLLER;
+
+ return pass_admin(admin_command).safe_then([&data](auto status) {
+ return seastar::make_ready_future<nvme_identify_controller_data_t>(
+ std::move(data));
+ });
+ });
+}
+
+discard_ertr::future<> NVMeBlockDevice::discard(uint64_t offset, uint64_t len) {
+ return device.discard(offset, len);
+}
+
+nvme_command_ertr::future<nvme_identify_namespace_data_t>
+NVMeBlockDevice::identify_namespace() {
+ return get_nsid().safe_then([this](auto nsid) {
+ return seastar::do_with(
+ nvme_admin_command_t(),
+ nvme_identify_namespace_data_t(),
+ [this, nsid](auto &admin_command, auto &data) {
+ admin_command.common.opcode = nvme_admin_command_t::OPCODE_IDENTIFY;
+ admin_command.common.addr = (uint64_t)&data;
+ admin_command.common.data_len = sizeof(data);
+ admin_command.common.nsid = nsid;
+ admin_command.identify.cns = nvme_identify_command_t::CNS_NAMESPACE;
+
+ return pass_admin(admin_command).safe_then([&data](auto status){
+ return seastar::make_ready_future<nvme_identify_namespace_data_t>(
+ std::move(data));
+ });
+ });
+ });
+}
+
+nvme_command_ertr::future<int> NVMeBlockDevice::get_nsid() {
+ return device.ioctl(NVME_IOCTL_ID, nullptr);
+}
+
+nvme_command_ertr::future<int> NVMeBlockDevice::pass_admin(
+ nvme_admin_command_t& admin_cmd) {
+ return device.ioctl(NVME_IOCTL_ADMIN_CMD, &admin_cmd).handle_exception(
+ [](auto e)->nvme_command_ertr::future<int> {
+ logger().error("pass_admin: ioctl failed");
+ return crimson::ct_error::input_output_error::make();
+ });
+}
+
+nvme_command_ertr::future<int> NVMeBlockDevice::pass_through_io(
+ nvme_io_command_t& io_cmd) {
+ return device.ioctl(NVME_IOCTL_IO_CMD, &io_cmd);
+}
+
+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();
+}
+
+Device::close_ertr::future<> TestMemory::close() {
+ logger().debug(" close ");
+ return close_ertr::now();
+}
+
+write_ertr::future<> TestMemory::writev(
+ uint64_t offset,
+ ceph::bufferlist bl,
+ uint16_t stream) {
+ ceph_assert(buf);
+ logger().debug(
+ "TestMemory: write offset {} len {}",
+ offset,
+ bl.length());
+
+ bl.begin().copy(bl.length(), buf + offset);
+ return write_ertr::now();
+}
+
+}
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include <sys/mman.h>
-#include <string.h>
-
-#include <fcntl.h>
-
-#include "crimson/common/log.h"
-#include "crimson/common/errorator-loop.h"
-
-#include "include/buffer.h"
-#include "nvmedevice.h"
-
-namespace {
- seastar::logger& logger() {
- return crimson::get_logger(ceph_subsys_seastore_tm);
- }
-}
-
-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) {
- size = stat.size;
- return seastar::open_file_dma(in_path, mode).then([=](auto file) {
- device = file;
- logger().debug("open");
- // Get SSD's features from identify_controller and namespace command.
- // Do identify_controller first, and then identify_namespace.
- return identify_controller().safe_then([this, in_path, mode](
- auto id_controller_data) {
- support_multistream = id_controller_data.oacs.support_directives;
- if (support_multistream) {
- stream_id_count = WRITE_LIFE_MAX;
- }
- awupf = id_controller_data.awupf + 1;
- return identify_namespace().safe_then([this, in_path, mode] (
- auto id_namespace_data) {
- // LBA format provides LBA size which is power of 2. LBA is the
- // minimum size of read and write.
- block_size = (1 << id_namespace_data.lbaf0.lbads);
- atomic_write_unit = awupf * block_size;
- data_protection_type = id_namespace_data.dps.protection_type;
- data_protection_enabled = (data_protection_type > 0);
- if (id_namespace_data.nsfeat.opterf == 1){
- // NPWG and NPWA is 0'based value
- write_granularity = block_size * (id_namespace_data.npwg + 1);
- write_alignment = block_size * (id_namespace_data.npwa + 1);
- }
- return open_for_io(in_path, mode);
- });
- }).handle_error(crimson::ct_error::input_output_error::handle([this, in_path, mode]{
- logger().error("open: id ctrlr failed. open without ioctl");
- return open_for_io(in_path, mode);
- }), crimson::ct_error::pass_further_all{});
- });
- });
- });
-}
-
-open_ertr::future<> PosixNVMeDevice::open_for_io(
- const std::string& in_path,
- seastar::open_flags mode) {
- io_device.resize(stream_id_count);
- return seastar::do_for_each(io_device, [=](auto &target_device) {
- return seastar::open_file_dma(in_path, mode).then([this](
- auto file) {
- io_device[stream_index_to_open] = file;
- return io_device[stream_index_to_open].fcntl(
- F_SET_FILE_RW_HINT,
- (uintptr_t)&stream_index_to_open).then([this](auto ret) {
- stream_index_to_open++;
- return seastar::now();
- });
- });
- });
-}
-
-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);
- uint16_t supported_stream = stream;
- if (stream >= stream_id_count) {
- supported_stream = WRITE_LIFE_NOT_SET;
- }
- return io_device[supported_stream].dma_write(
- offset, bptr.c_str(), length).handle_exception(
- [](auto e) -> write_ertr::future<size_t> {
- logger().error("write: dma_write got error{}", e);
- return crimson::ct_error::input_output_error::make();
- }).then([length](auto result) -> write_ertr::future<> {
- if (result != length) {
- logger().error("write: dma_write got error with not proper length");
- return crimson::ct_error::input_output_error::make();
- }
- return write_ertr::now();
- });
-}
-
-read_ertr::future<> 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(
- [](auto e) -> read_ertr::future<size_t> {
- logger().error("read: dma_read got error{}", e);
- return crimson::ct_error::input_output_error::make();
- }).then([length](auto result) -> read_ertr::future<> {
- if (result != length) {
- logger().error("read: dma_read got error with not proper length");
- return crimson::ct_error::input_output_error::make();
- }
- return read_ertr::now();
- });
-}
-
-write_ertr::future<> PosixNVMeDevice::writev(
- uint64_t offset,
- ceph::bufferlist bl,
- uint16_t stream) {
- logger().debug(
- "block: write offset {} len {}",
- offset,
- bl.length());
-
- uint16_t supported_stream = stream;
- if (stream >= stream_id_count) {
- supported_stream = WRITE_LIFE_NOT_SET;
- }
- bl.rebuild_aligned(block_size);
-
- return seastar::do_with(
- bl.prepare_iovs(),
- std::move(bl),
- [this, supported_stream, offset](auto& iovs, auto& bl)
- {
- return write_ertr::parallel_for_each(
- iovs,
- [this, supported_stream, offset](auto& p) mutable
- {
- auto off = offset + p.offset;
- auto len = p.length;
- auto& iov = p.iov;
- return io_device[supported_stream].dma_write(off, std::move(iov)
- ).handle_exception(
- [this, off, len](auto e) -> write_ertr::future<size_t>
- {
- logger().error("{} poffset={}~{} dma_write got error -- {}",
- device_id_printer_t{get_device_id()}, off, len, e);
- return crimson::ct_error::input_output_error::make();
- }).then([this, off, len](size_t written) -> write_ertr::future<> {
- if (written != len) {
- logger().error("{} poffset={}~{} dma_write len={} inconsistent",
- device_id_printer_t{get_device_id()}, off, len, written);
- return crimson::ct_error::input_output_error::make();
- }
- return write_ertr::now();
- });
- });
- });
-}
-
-Device::close_ertr::future<> PosixNVMeDevice::close() {
- logger().debug(" close ");
- return device.close().then([this]() {
- return seastar::do_for_each(io_device, [](auto target_device) {
- return target_device.close();
- });
- });
-}
-
-nvme_command_ertr::future<nvme_identify_controller_data_t>
-PosixNVMeDevice::identify_controller() {
- return seastar::do_with(
- nvme_admin_command_t(),
- nvme_identify_controller_data_t(),
- [this](auto &admin_command, auto &data) {
- admin_command.common.opcode = nvme_admin_command_t::OPCODE_IDENTIFY;
- admin_command.common.addr = (uint64_t)&data;
- admin_command.common.data_len = sizeof(data);
- admin_command.identify.cns = nvme_identify_command_t::CNS_CONTROLLER;
-
- return pass_admin(admin_command).safe_then([&data](auto status) {
- return seastar::make_ready_future<nvme_identify_controller_data_t>(
- std::move(data));
- });
- });
-}
-
-discard_ertr::future<> PosixNVMeDevice::discard(uint64_t offset, uint64_t len) {
- return device.discard(offset, len);
-}
-
-nvme_command_ertr::future<nvme_identify_namespace_data_t>
-PosixNVMeDevice::identify_namespace() {
- return get_nsid().safe_then([this](auto nsid) {
- return seastar::do_with(
- nvme_admin_command_t(),
- nvme_identify_namespace_data_t(),
- [this, nsid](auto &admin_command, auto &data) {
- admin_command.common.opcode = nvme_admin_command_t::OPCODE_IDENTIFY;
- admin_command.common.addr = (uint64_t)&data;
- admin_command.common.data_len = sizeof(data);
- admin_command.common.nsid = nsid;
- admin_command.identify.cns = nvme_identify_command_t::CNS_NAMESPACE;
-
- return pass_admin(admin_command).safe_then([&data](auto status){
- return seastar::make_ready_future<nvme_identify_namespace_data_t>(
- std::move(data));
- });
- });
- });
-}
-
-nvme_command_ertr::future<int> PosixNVMeDevice::get_nsid() {
- return device.ioctl(NVME_IOCTL_ID, nullptr);
-}
-
-nvme_command_ertr::future<int> PosixNVMeDevice::pass_admin(
- nvme_admin_command_t& admin_cmd) {
- return device.ioctl(NVME_IOCTL_ADMIN_CMD, &admin_cmd).handle_exception(
- [](auto e)->nvme_command_ertr::future<int> {
- logger().error("pass_admin: ioctl failed");
- return crimson::ct_error::input_output_error::make();
- });
-}
-
-nvme_command_ertr::future<int> PosixNVMeDevice::pass_through_io(
- nvme_io_command_t& io_cmd) {
- return device.ioctl(NVME_IOCTL_IO_CMD, &io_cmd);
-}
-
-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();
-}
-
-Device::close_ertr::future<> TestMemory::close() {
- logger().debug(" close ");
- return close_ertr::now();
-}
-
-write_ertr::future<> TestMemory::writev(
- uint64_t offset,
- ceph::bufferlist bl,
- uint16_t stream) {
- ceph_assert(buf);
- logger().debug(
- "TestMemory: write offset {} len {}",
- offset,
- bl.length());
-
- bl.begin().copy(bl.length(), buf + offset);
- return write_ertr::now();
-}
-
-}
+++ /dev/null
-//-*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#pragma once
-
-#include <memory>
-#include <vector>
-#include <boost/intrusive_ptr.hpp>
-#include <boost/smart_ptr/intrusive_ref_counter.hpp>
-
-#include <seastar/core/file.hh>
-#include <seastar/core/future.hh>
-#include <seastar/core/reactor.hh>
-#include <linux/nvme_ioctl.h>
-#include <libaio.h>
-
-#include "crimson/osd/exceptions.h"
-#include "crimson/common/layout.h"
-#include "crimson/os/seastore/seastore_types.h"
-#include "crimson/os/seastore/random_block_manager.h"
-#include "crimson/os/seastore/device.h"
-
-namespace ceph {
- namespace buffer {
- class bufferptr;
- }
-}
-
-namespace crimson::os::seastore::nvme_device {
-
-// from blk/BlockDevice.h
-#if defined(__linux__)
-#if !defined(F_SET_FILE_RW_HINT)
-#define F_LINUX_SPECIFIC_BASE 1024
-#define F_SET_FILE_RW_HINT (F_LINUX_SPECIFIC_BASE + 14)
-#endif
-// These values match Linux definition
-// https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/tree/include/uapi/linux/fcntl.h#n56
-#define WRITE_LIFE_NOT_SET 0 // No hint information set
-#define WRITE_LIFE_NONE 1 // No hints about write life time
-#define WRITE_LIFE_SHORT 2 // Data written has a short life time
-#define WRITE_LIFE_MEDIUM 3 // Data written has a medium life time
-#define WRITE_LIFE_LONG 4 // Data written has a long life time
-#define WRITE_LIFE_EXTREME 5 // Data written has an extremely long life time
-#define WRITE_LIFE_MAX 6
-#else
-// On systems don't have WRITE_LIFE_* only use one FD
-// And all files are created equal
-#define WRITE_LIFE_NOT_SET 0 // No hint information set
-#define WRITE_LIFE_NONE 0 // No hints about write life time
-#define WRITE_LIFE_SHORT 0 // Data written has a short life time
-#define WRITE_LIFE_MEDIUM 0 // Data written has a medium life time
-#define WRITE_LIFE_LONG 0 // Data written has a long life time
-#define WRITE_LIFE_EXTREME 0 // Data written has an extremely long life time
-#define WRITE_LIFE_MAX 1
-#endif
-
-/*
- * NVMe protocol structures (nvme_XX, identify_XX)
- *
- * All structures relative to NVMe protocol are following NVMe protocol v1.4
- * (latest). NVMe is protocol for fast interfacing between user and SSD device.
- * We selectively adopted features among various NVMe features to ease
- * implementation. And also, NVMeBlockDevice provides generic command submission
- * APIs for IO and Admin commands. Please use pass_through_io() and pass_admin()
- * to do it.
- *
- * For more information about NVMe protocol, refer https://nvmexpress.org/
- */
-struct nvme_identify_command_t {
- uint32_t common_dw[10];
-
- uint32_t cns : 8;
- uint32_t reserved : 8;
- uint32_t cnt_id : 16;
-
- static const uint8_t CNS_NAMESPACE = 0x00;
- static const uint8_t CNS_CONTROLLER = 0x01;
-};
-
-struct nvme_admin_command_t {
- union {
- nvme_passthru_cmd common;
- nvme_identify_command_t identify;
- };
-
- static const uint8_t OPCODE_IDENTIFY = 0x06;
-};
-
-// Optional Admin Command Support (OACS)
-// Indicates optional commands are supported by SSD or not
-struct oacs_t {
- uint16_t unused : 5;
- uint16_t support_directives : 1; // Support multi-stream
- uint16_t unused2 : 10;
-};
-
-struct nvme_identify_controller_data_t {
- union {
- struct {
- uint8_t unused[256]; // [255:0]
- oacs_t oacs; // [257:256]
- uint8_t unused2[270]; // [527:258]
- uint16_t awupf; // [529:528]
- };
- uint8_t raw[4096];
- };
-};
-
-// End-to-end Data Protection Capabilities (DPC)
-// Indicates type of E2E data protection supported by SSD
-struct dpc_t {
- uint8_t support_type1 : 1;
- uint8_t support_type2 : 1;
- uint8_t support_type3 : 1;
- uint8_t support_first_meta : 1;
- uint8_t support_last_meta : 1;
- uint8_t reserved : 3;
-};
-
-// End-to-end Data Protection Type Settings (DPS)
-// Indicates enabled type of E2E data protection
-struct dps_t {
- uint8_t protection_type : 3;
- uint8_t protection_info : 1;
- uint8_t reserved : 4;
-};
-
-// Namespace Features (NSFEAT)
-// Indicates features of namespace
-struct nsfeat_t {
- uint8_t thinp : 1;
- uint8_t nsabp : 1;
- uint8_t dae : 1;
- uint8_t uid_reuse : 1;
- uint8_t opterf : 1; // Support NPWG, NPWA
- uint8_t reserved : 3;
-};
-
-// LBA Format (LBAF)
-// Indicates LBA format (metadata size, data size, performance)
-struct lbaf_t {
- uint32_t ms : 16;
- uint32_t lbads : 8;
- uint32_t rp : 2;
- uint32_t reserved : 6;
-};
-
-struct nvme_identify_namespace_data_t {
- union {
- struct {
- uint8_t unused[24]; // [23:0]
- nsfeat_t nsfeat; // [24]
- uint8_t unused2[3]; // [27:25]
- dpc_t dpc; // [28]
- dps_t dps; // [29]
- uint8_t unused3[34]; // [63:30]
- uint16_t npwg; // [65:64]
- uint16_t npwa; // [67:66]
- uint8_t unused4[60]; // [127:68]
- lbaf_t lbaf0; // [131:128]
- };
- uint8_t raw[4096];
- };
-};
-
-struct nvme_rw_command_t {
- uint32_t common_dw[10];
-
- uint64_t s_lba;
-
- uint32_t nlb : 16; // 0's based value
- uint32_t reserved : 4;
- uint32_t d_type : 4;
- uint32_t reserved2 : 2;
- uint32_t prinfo_prchk : 3;
- uint32_t prinfo_pract : 1;
- uint32_t fua : 1;
- uint32_t lr : 1;
-
- uint32_t reserved3 : 16;
- uint32_t dspec : 16;
-
- static const uint32_t DTYPE_STREAM = 1;
-};
-
-struct nvme_io_command_t {
- union {
- nvme_passthru_cmd common;
- nvme_rw_command_t rw;
- };
- static const uint8_t OPCODE_WRITE = 0x01;
- static const uint8_t OPCODE_READ = 0x01;
-};
-
-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 : public Device {
-public:
- using Device::read;
- read_ertr::future<> read (
- paddr_t addr,
- size_t len,
- ceph::bufferptr &out) final {
- uint64_t rbm_addr = convert_paddr_to_abs_addr(addr);
- return read(rbm_addr, out);
- }
-protected:
- uint64_t size = 0;
-
- // LBA Size
- uint64_t block_size = 4096;
-
- uint64_t write_granularity = 4096;
- uint64_t write_alignment = 4096;
- uint32_t atomic_write_unit = 4096;
-
- bool data_protection_enabled = false;
- device_id_t device_id;
- seastore_meta_t meta;
- secondary_device_set_t devices;
-public:
- NVMeBlockDevice() {}
- virtual ~NVMeBlockDevice() = default;
-
- template <typename T>
- static std::unique_ptr<T> create() {
- return std::make_unique<T>();
- }
-
- device_id_t get_device_id() const {
- return device_id;
- }
- void set_device_id(device_id_t id) {
- device_id = id;
- }
-
- magic_t get_magic() const final {
- return magic_t();
- }
-
- device_type_t get_device_type() const final {
- return device_type_t::RANDOM_BLOCK;
- }
-
- const seastore_meta_t &get_meta() const final {
- return meta;
- }
-
- secondary_device_set_t& get_secondary_devices() final {
- return devices;
- }
-
- /*
- * Service NVMe device relative size
- *
- * size : total size of device in byte.
- *
- * block_size : IO unit size in byte. Caller should follow every IO command
- * aligned with block size.
- *
- * preffered_write_granularity(PWG), preffered_write_alignment(PWA) : IO unit
- * size for write in byte. Caller should request every write IO sized multiple
- * times of PWG and aligned starting address by PWA. Available only if NVMe
- * Device supports NVMe protocol 1.4 or later versions.
- * atomic_write_unit : The maximum size of write whose atomicity is guranteed
- * by SSD even on power failure. The write equal to or smaller than
- * atomic_write_unit does not require fsync().
- */
-
- std::size_t get_size() const { return size; }
- seastore_off_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; }
-
- uint64_t get_atomic_write_unit() const { return atomic_write_unit; }
-
- virtual read_ertr::future<> read(
- uint64_t offset,
- bufferptr &bptr) = 0;
-
- /*
- * Multi-stream write
- *
- * Give hint to device about classification of data whose life time is similar
- * with each other. Data with same stream value will be managed together in
- * SSD for better write performance.
- */
- virtual write_ertr::future<> write(
- uint64_t offset,
- bufferptr &bptr,
- uint16_t stream = 0) = 0;
-
- virtual discard_ertr::future<> discard(
- uint64_t offset,
- uint64_t len) { return seastar::now(); }
-
- virtual open_ertr::future<> open(
- const std::string& path,
- seastar::open_flags mode) = 0;
-
- virtual write_ertr::future<> writev(
- uint64_t offset,
- ceph::bufferlist bl,
- uint16_t stream = 0) = 0;
-
- /*
- * For passsing through nvme IO or Admin command to SSD
- * Caller can construct and execute its own nvme command
- */
- virtual nvme_command_ertr::future<int> pass_through_io(
- nvme_io_command_t& command) { return seastar::make_ready_future<int>(0); }
- virtual nvme_command_ertr::future<int> pass_admin(
- nvme_admin_command_t& command) { return seastar::make_ready_future<int>(0); }
-
- /*
- * End-to-End Data Protection
- *
- * NVMe device keeps track of data integrity similar with checksum. Client can
- * offload checksuming to NVMe device to reduce its CPU utilization. If data
- * protection is enabled, checksum is calculated on every write and used to
- * verify data on every read.
- */
- bool is_data_protection_enabled() const { return data_protection_enabled; }
-
- /*
- * Data Health
- *
- * Returns list of LBAs which have almost corrupted data. Data of the LBAs
- * will be corrupted very soon. Caller can overwrite, unmap or refresh data to
- * protect data
- */
- virtual nvme_command_ertr::future<std::list<uint64_t>> get_data_health() {
- std::list<uint64_t> fragile_lbas;
- return nvme_command_ertr::future<std::list<uint64_t>>(
- nvme_command_ertr::ready_future_marker{},
- fragile_lbas
- );
- }
-
- /*
- * Recovery Level
- *
- * Regulate magnitude of SSD-internal data recovery. Caller can get good read
- * latency with lower magnitude.
- */
- virtual nvme_command_ertr::future<> set_data_recovery_level(
- uint32_t level) { return nvme_command_ertr::now(); }
-
- /*
- * 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;
-
- using NVMeBlockDevice::read;
- read_ertr::future<> read(
- uint64_t offset,
- bufferptr &bptr) final;
-
- close_ertr::future<> close() override;
-
- discard_ertr::future<> discard(
- uint64_t offset,
- uint64_t len) override;
-
- mkfs_ret mkfs(device_config_t) final {
- return mkfs_ertr::now();
- }
-
- mount_ret mount() final {
- return mount_ertr::now();
- }
-
- write_ertr::future<> writev(
- uint64_t offset,
- ceph::bufferlist bl,
- uint16_t stream = 0) final;
-
- nvme_command_ertr::future<int> pass_admin(
- nvme_admin_command_t& admin_cmd) override;
- nvme_command_ertr::future<int> pass_through_io(
- nvme_io_command_t& io_cmd) override;
-
- bool support_multistream = false;
- uint8_t data_protection_type = 0;
-
-private:
- // identify_controller/namespace are used to get SSD internal information such
- // as supported features, NPWG and NPWA
- nvme_command_ertr::future<nvme_identify_controller_data_t> identify_controller();
- nvme_command_ertr::future<nvme_identify_namespace_data_t> identify_namespace();
- nvme_command_ertr::future<int> get_nsid();
- open_ertr::future<> open_for_io(
- const std::string& in_path,
- seastar::open_flags mode);
-
- seastar::file device;
- std::vector<seastar::file> io_device;
- uint32_t stream_index_to_open = WRITE_LIFE_NOT_SET;
- uint32_t stream_id_count = 1; // stream is disabled, defaultly.
- uint32_t awupf = 0;
-};
-
-
-class TestMemory : public NVMeBlockDevice {
-public:
-
- TestMemory(size_t size) : buf(nullptr), size(size) {}
- ~TestMemory() {
- if (buf) {
- ::munmap(buf, size);
- buf = nullptr;
- }
- }
-
- mkfs_ret mkfs(device_config_t) final {
- return mkfs_ertr::now();
- }
-
- mount_ret mount() final {
- return mount_ertr::now();
- }
-
- open_ertr::future<> open(
- const std::string &in_path,
- seastar::open_flags mode) override;
-
- write_ertr::future<> write(
- uint64_t offset,
- bufferptr &bptr,
- uint16_t stream = 0) override;
-
- using NVMeBlockDevice::read;
- read_ertr::future<> read(
- uint64_t offset,
- bufferptr &bptr) override;
-
- close_ertr::future<> close() override;
-
- write_ertr::future<> writev(
- uint64_t offset,
- ceph::bufferlist bl,
- uint16_t stream = 0) final;
-
- char *buf;
- size_t size;
-};
-}
--- /dev/null
+//-*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#pragma once
+
+#include <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"
+#include "crimson/os/seastore/seastore_types.h"
+#include "crimson/os/seastore/random_block_manager.h"
+#include "crimson/os/seastore/device.h"
+
+namespace ceph {
+ namespace buffer {
+ class bufferptr;
+ }
+}
+
+namespace crimson::os::seastore::nvme_device {
+
+// from blk/BlockDevice.h
+#if defined(__linux__)
+#if !defined(F_SET_FILE_RW_HINT)
+#define F_LINUX_SPECIFIC_BASE 1024
+#define F_SET_FILE_RW_HINT (F_LINUX_SPECIFIC_BASE + 14)
+#endif
+// These values match Linux definition
+// https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/tree/include/uapi/linux/fcntl.h#n56
+#define WRITE_LIFE_NOT_SET 0 // No hint information set
+#define WRITE_LIFE_NONE 1 // No hints about write life time
+#define WRITE_LIFE_SHORT 2 // Data written has a short life time
+#define WRITE_LIFE_MEDIUM 3 // Data written has a medium life time
+#define WRITE_LIFE_LONG 4 // Data written has a long life time
+#define WRITE_LIFE_EXTREME 5 // Data written has an extremely long life time
+#define WRITE_LIFE_MAX 6
+#else
+// On systems don't have WRITE_LIFE_* only use one FD
+// And all files are created equal
+#define WRITE_LIFE_NOT_SET 0 // No hint information set
+#define WRITE_LIFE_NONE 0 // No hints about write life time
+#define WRITE_LIFE_SHORT 0 // Data written has a short life time
+#define WRITE_LIFE_MEDIUM 0 // Data written has a medium life time
+#define WRITE_LIFE_LONG 0 // Data written has a long life time
+#define WRITE_LIFE_EXTREME 0 // Data written has an extremely long life time
+#define WRITE_LIFE_MAX 1
+#endif
+
+/*
+ * NVMe protocol structures (nvme_XX, identify_XX)
+ *
+ * All structures relative to NVMe protocol are following NVMe protocol v1.4
+ * (latest). NVMe is protocol for fast interfacing between user and SSD device.
+ * We selectively adopted features among various NVMe features to ease
+ * implementation. And also, NVMeBlockDevice provides generic command submission
+ * APIs for IO and Admin commands. Please use pass_through_io() and pass_admin()
+ * to do it.
+ *
+ * For more information about NVMe protocol, refer https://nvmexpress.org/
+ */
+struct nvme_identify_command_t {
+ uint32_t common_dw[10];
+
+ uint32_t cns : 8;
+ uint32_t reserved : 8;
+ uint32_t cnt_id : 16;
+
+ static const uint8_t CNS_NAMESPACE = 0x00;
+ static const uint8_t CNS_CONTROLLER = 0x01;
+};
+
+struct nvme_admin_command_t {
+ union {
+ nvme_passthru_cmd common;
+ nvme_identify_command_t identify;
+ };
+
+ static const uint8_t OPCODE_IDENTIFY = 0x06;
+};
+
+// Optional Admin Command Support (OACS)
+// Indicates optional commands are supported by SSD or not
+struct oacs_t {
+ uint16_t unused : 5;
+ uint16_t support_directives : 1; // Support multi-stream
+ uint16_t unused2 : 10;
+};
+
+struct nvme_identify_controller_data_t {
+ union {
+ struct {
+ uint8_t unused[256]; // [255:0]
+ oacs_t oacs; // [257:256]
+ uint8_t unused2[270]; // [527:258]
+ uint16_t awupf; // [529:528]
+ };
+ uint8_t raw[4096];
+ };
+};
+
+// End-to-end Data Protection Capabilities (DPC)
+// Indicates type of E2E data protection supported by SSD
+struct dpc_t {
+ uint8_t support_type1 : 1;
+ uint8_t support_type2 : 1;
+ uint8_t support_type3 : 1;
+ uint8_t support_first_meta : 1;
+ uint8_t support_last_meta : 1;
+ uint8_t reserved : 3;
+};
+
+// End-to-end Data Protection Type Settings (DPS)
+// Indicates enabled type of E2E data protection
+struct dps_t {
+ uint8_t protection_type : 3;
+ uint8_t protection_info : 1;
+ uint8_t reserved : 4;
+};
+
+// Namespace Features (NSFEAT)
+// Indicates features of namespace
+struct nsfeat_t {
+ uint8_t thinp : 1;
+ uint8_t nsabp : 1;
+ uint8_t dae : 1;
+ uint8_t uid_reuse : 1;
+ uint8_t opterf : 1; // Support NPWG, NPWA
+ uint8_t reserved : 3;
+};
+
+// LBA Format (LBAF)
+// Indicates LBA format (metadata size, data size, performance)
+struct lbaf_t {
+ uint32_t ms : 16;
+ uint32_t lbads : 8;
+ uint32_t rp : 2;
+ uint32_t reserved : 6;
+};
+
+struct nvme_identify_namespace_data_t {
+ union {
+ struct {
+ uint8_t unused[24]; // [23:0]
+ nsfeat_t nsfeat; // [24]
+ uint8_t unused2[3]; // [27:25]
+ dpc_t dpc; // [28]
+ dps_t dps; // [29]
+ uint8_t unused3[34]; // [63:30]
+ uint16_t npwg; // [65:64]
+ uint16_t npwa; // [67:66]
+ uint8_t unused4[60]; // [127:68]
+ lbaf_t lbaf0; // [131:128]
+ };
+ uint8_t raw[4096];
+ };
+};
+
+struct nvme_rw_command_t {
+ uint32_t common_dw[10];
+
+ uint64_t s_lba;
+
+ uint32_t nlb : 16; // 0's based value
+ uint32_t reserved : 4;
+ uint32_t d_type : 4;
+ uint32_t reserved2 : 2;
+ uint32_t prinfo_prchk : 3;
+ uint32_t prinfo_pract : 1;
+ uint32_t fua : 1;
+ uint32_t lr : 1;
+
+ uint32_t reserved3 : 16;
+ uint32_t dspec : 16;
+
+ static const uint32_t DTYPE_STREAM = 1;
+};
+
+struct nvme_io_command_t {
+ union {
+ nvme_passthru_cmd common;
+ nvme_rw_command_t rw;
+ };
+ static const uint8_t OPCODE_WRITE = 0x01;
+ static const uint8_t OPCODE_READ = 0x01;
+};
+
+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;
+};
+
+class RBMDevice : public Device {
+public:
+ using Device::read;
+ read_ertr::future<> read (
+ paddr_t addr,
+ size_t len,
+ ceph::bufferptr &out) final {
+ uint64_t rbm_addr = convert_paddr_to_abs_addr(addr);
+ return read(rbm_addr, out);
+ }
+protected:
+ uint64_t size = 0;
+
+ // LBA Size
+ uint64_t block_size = 4096;
+
+ device_id_t device_id;
+ seastore_meta_t meta;
+ secondary_device_set_t devices;
+public:
+ RBMDevice() {}
+ virtual ~RBMDevice() = default;
+
+ template <typename T>
+ static std::unique_ptr<T> create() {
+ return std::make_unique<T>();
+ }
+
+ device_id_t get_device_id() const {
+ return device_id;
+ }
+ void set_device_id(device_id_t id) {
+ device_id = id;
+ }
+
+ magic_t get_magic() const final {
+ return magic_t();
+ }
+
+ device_type_t get_device_type() const final {
+ return device_type_t::RANDOM_BLOCK;
+ }
+
+ const seastore_meta_t &get_meta() const final {
+ return meta;
+ }
+
+ secondary_device_set_t& get_secondary_devices() final {
+ return devices;
+ }
+ std::size_t get_size() const { return size; }
+ seastore_off_t get_block_size() const { return block_size; }
+
+ virtual read_ertr::future<> read(
+ uint64_t offset,
+ bufferptr &bptr) = 0;
+
+ /*
+ * Multi-stream write
+ *
+ * Give hint to device about classification of data whose life time is similar
+ * with each other. Data with same stream value will be managed together in
+ * SSD for better write performance.
+ */
+ virtual write_ertr::future<> write(
+ uint64_t offset,
+ bufferptr &bptr,
+ uint16_t stream = 0) = 0;
+
+ virtual discard_ertr::future<> discard(
+ uint64_t offset,
+ uint64_t len) { return seastar::now(); }
+
+ virtual open_ertr::future<> open(
+ const std::string& path,
+ seastar::open_flags mode) = 0;
+
+ virtual write_ertr::future<> writev(
+ uint64_t offset,
+ ceph::bufferlist bl,
+ uint16_t stream = 0) = 0;
+
+ bool is_data_protection_enabled() const { return false; }
+};
+
+/*
+ * Implementation of NVMeBlockDevice with POSIX APIs
+ *
+ * NVMeBlockDevice provides NVMe SSD interfaces through POSIX APIs which is
+ * generally available at most operating environment.
+ */
+class NVMeBlockDevice : public RBMDevice {
+public:
+
+ /*
+ * Service NVMe device relative size
+ *
+ * size : total size of device in byte.
+ *
+ * block_size : IO unit size in byte. Caller should follow every IO command
+ * aligned with block size.
+ *
+ * preffered_write_granularity(PWG), preffered_write_alignment(PWA) : IO unit
+ * size for write in byte. Caller should request every write IO sized multiple
+ * times of PWG and aligned starting address by PWA. Available only if NVMe
+ * Device supports NVMe protocol 1.4 or later versions.
+ * atomic_write_unit : The maximum size of write whose atomicity is guranteed
+ * by SSD even on power failure. The write equal to or smaller than
+ * atomic_write_unit does not require fsync().
+ */
+
+ NVMeBlockDevice() {}
+ ~NVMeBlockDevice() = default;
+
+ open_ertr::future<> open(
+ const std::string &in_path,
+ seastar::open_flags mode) override;
+
+ write_ertr::future<> write(
+ uint64_t offset,
+ bufferptr &bptr,
+ uint16_t stream = 0) override;
+
+ using RBMDevice::read;
+ read_ertr::future<> read(
+ uint64_t offset,
+ bufferptr &bptr) final;
+
+ close_ertr::future<> close() override;
+
+ discard_ertr::future<> discard(
+ uint64_t offset,
+ uint64_t len) override;
+
+ mkfs_ret mkfs(device_config_t) final {
+ return mkfs_ertr::now();
+ }
+
+ mount_ret mount() final {
+ return mount_ertr::now();
+ }
+
+ write_ertr::future<> writev(
+ uint64_t offset,
+ ceph::bufferlist bl,
+ uint16_t stream = 0) final;
+
+ uint64_t get_preffered_write_granularity() const { return write_granularity; }
+ uint64_t get_preffered_write_alignment() const { return write_alignment; }
+ uint64_t get_atomic_write_unit() const { return atomic_write_unit; }
+ /*
+ * End-to-End Data Protection
+ *
+ * NVMe device keeps track of data integrity similar with checksum. Client can
+ * offload checksuming to NVMe device to reduce its CPU utilization. If data
+ * protection is enabled, checksum is calculated on every write and used to
+ * verify data on every read.
+ */
+ bool is_data_protection_enabled() const { return data_protection_enabled; }
+
+ /*
+ * Data Health
+ *
+ * Returns list of LBAs which have almost corrupted data. Data of the LBAs
+ * will be corrupted very soon. Caller can overwrite, unmap or refresh data to
+ * protect data
+ */
+ virtual nvme_command_ertr::future<std::list<uint64_t>> get_data_health() {
+ std::list<uint64_t> fragile_lbas;
+ return nvme_command_ertr::future<std::list<uint64_t>>(
+ nvme_command_ertr::ready_future_marker{},
+ fragile_lbas
+ );
+ }
+
+ /*
+ * Recovery Level
+ *
+ * Regulate magnitude of SSD-internal data recovery. Caller can get good read
+ * latency with lower magnitude.
+ */
+ virtual nvme_command_ertr::future<> set_data_recovery_level(
+ uint32_t level) { return nvme_command_ertr::now(); }
+ /*
+ * For passsing through nvme IO or Admin command to SSD
+ * Caller can construct and execute its own nvme command
+ */
+ nvme_command_ertr::future<int> pass_admin(
+ nvme_admin_command_t& admin_cmd);
+ nvme_command_ertr::future<int> pass_through_io(
+ nvme_io_command_t& io_cmd);
+
+ bool support_multistream = false;
+ uint8_t data_protection_type = 0;
+
+ /*
+ * Predictable Latency
+ *
+ * NVMe device can guarantee IO latency within pre-defined time window. This
+ * functionality will be analyzed soon.
+ */
+
+private:
+ // identify_controller/namespace are used to get SSD internal information such
+ // as supported features, NPWG and NPWA
+ nvme_command_ertr::future<nvme_identify_controller_data_t> identify_controller();
+ nvme_command_ertr::future<nvme_identify_namespace_data_t> identify_namespace();
+ nvme_command_ertr::future<int> get_nsid();
+ open_ertr::future<> open_for_io(
+ const std::string& in_path,
+ seastar::open_flags mode);
+
+ seastar::file device;
+ std::vector<seastar::file> io_device;
+ uint32_t stream_index_to_open = WRITE_LIFE_NOT_SET;
+ uint32_t stream_id_count = 1; // stream is disabled, defaultly.
+ uint32_t awupf = 0;
+
+ uint64_t write_granularity = 4096;
+ uint64_t write_alignment = 4096;
+ uint32_t atomic_write_unit = 4096;
+
+ bool data_protection_enabled = false;
+};
+
+
+class TestMemory : public RBMDevice {
+public:
+
+ TestMemory(size_t size) : buf(nullptr), size(size) {}
+ ~TestMemory() {
+ if (buf) {
+ ::munmap(buf, size);
+ buf = nullptr;
+ }
+ }
+
+ mkfs_ret mkfs(device_config_t) final {
+ return mkfs_ertr::now();
+ }
+
+ mount_ret mount() final {
+ return mount_ertr::now();
+ }
+
+ open_ertr::future<> open(
+ const std::string &in_path,
+ seastar::open_flags mode) override;
+
+ write_ertr::future<> write(
+ uint64_t offset,
+ bufferptr &bptr,
+ uint16_t stream = 0) override;
+
+ using RBMDevice::read;
+ read_ertr::future<> read(
+ uint64_t offset,
+ bufferptr &bptr) override;
+
+ close_ertr::future<> close() override;
+
+ write_ertr::future<> writev(
+ uint64_t offset,
+ ceph::bufferlist bl,
+ uint16_t stream = 0) final;
+
+ char *buf;
+ size_t size;
+};
+}
// vim: ts=8 sw=2 smarttab
#include "include/buffer.h"
-#include "crimson/os/seastore/random_block_manager/nvmedevice.h"
+#include "crimson/os/seastore/random_block_manager/rbm_device.h"
#include "test/crimson/gtest_seastar.h"
#include "include/stringify.h"
using namespace nvme_device;
struct nvdev_test_t : seastar_test_suite_t {
- std::unique_ptr<NVMeBlockDevice> device;
+ std::unique_ptr<RBMDevice> device;
std::string dev_path;
static const uint64_t DEV_SIZE = 1024 * 1024 * 1024;
TEST_F(nvdev_test_t, write_and_verify_test)
{
run_async([this] {
- device = NVMeBlockDevice::create<PosixNVMeDevice>();
+ device = RBMDevice::create<NVMeBlockDevice>();
device->open(dev_path, seastar::open_flags::rw).unsafe_get();
nvdev_test_block_t original_data;
std::minstd_rand0 generator;
#include "crimson/os/seastore/journal.h"
#include "crimson/os/seastore/journal/circular_bounded_journal.h"
#include "crimson/os/seastore/random_block_manager.h"
-#include "crimson/os/seastore/random_block_manager/nvmedevice.h"
+#include "crimson/os/seastore/random_block_manager/rbm_device.h"
#include "test/crimson/seastore/transaction_manager_test_state.h"
using namespace crimson;
Cache cache;
std::vector<entry_validator_t> entries;
std::unique_ptr<CircularBoundedJournal> cbj;
- nvme_device::NVMeBlockDevice *device;
+ nvme_device::RBMDevice *device;
std::default_random_engine generator;
uint64_t block_size;
#include "crimson/common/log.h"
#include "crimson/os/seastore/random_block_manager/block_rb_manager.h"
-#include "crimson/os/seastore/random_block_manager/nvmedevice.h"
+#include "crimson/os/seastore/random_block_manager/rbm_device.h"
#include "test/crimson/seastore/transaction_manager_test_state.h"
using namespace crimson;
struct rbm_test_t :
public seastar_test_suite_t, TMTestState {
std::unique_ptr<BlockRBManager> rbm_manager;
- std::unique_ptr<nvme_device::NVMeBlockDevice> device;
+ std::unique_ptr<nvme_device::RBMDevice> device;
struct rbm_transaction {
void add_rbm_allocated_blocks(alloc_delta_t &d) {
#include "crimson/os/seastore/segment_manager.h"
#include "crimson/os/seastore/collection_manager/flat_collection_manager.h"
#include "crimson/os/seastore/onode_manager/staged-fltree/fltree_onode_manager.h"
-#include "crimson/os/seastore/random_block_manager/nvmedevice.h"
+#include "crimson/os/seastore/random_block_manager/rbm_device.h"
#include "crimson/os/seastore/journal/circular_bounded_journal.h"
using namespace crimson;
protected:
segment_manager::EphemeralSegmentManagerRef segment_manager;
std::list<segment_manager::EphemeralSegmentManagerRef> secondary_segment_managers;
- std::unique_ptr<nvme_device::NVMeBlockDevice> rb_device;
+ std::unique_ptr<nvme_device::RBMDevice> rb_device;
tm_make_config_t tm_config = tm_make_config_t::get_test_segmented_journal();
EphemeralTestState(std::size_t num_segment_managers) {