namespace crimson::os::seastore {
-BlockRBManager::write_ertr::future<> BlockRBManager::rbm_sync_block_bitmap(
- rbm_bitmap_block_t &block, blk_no_t block_no)
-{
- LOG_PREFIX(BlockRBManager::rbm_sync_block_bitmap);
- 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) {
- DEBUG("rbm_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);
-}
-
-BlockRBManager::mkfs_ertr::future<> BlockRBManager::initialize_blk_alloc_area()
-{
- LOG_PREFIX(BlockRBManager::initialize_blk_alloc_area);
- auto start = super.start_data_area / super.block_size;
- 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);
- }
-
- // CRC calculation is offloaded to NVMeDevice if data protection is enabled.
- if (device->is_data_protection_enabled() == false) {
- b_block.set_crc();
- }
-
- return seastar::do_with(
- b_block,
- [this, start, FNAME](auto &b_block) {
- return rbm_sync_block_bitmap(b_block,
- super.start_alloc_area / super.block_size
- ).safe_then([this, &b_block, start, FNAME]() {
-
- /* initialize bitmap blocks as unused */
- auto max = max_block_by_bitmap_block();
- auto max_block = super.size / super.block_size;
- blk_no_t end = round_up_to(max_block, max) - 1;
- 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, FNAME]() {
- /*
- * 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();
- DEBUG("na_block_no: {}, remain_block: {} ",
- na_block_no, remain_block);
- if (remain_block) {
- 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 BlockRBManager::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 BlockRBManager::initialize_blk_alloc_area"
- }
- );
- }).handle_error(
- mkfs_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error rbm_sync_block_bitmap_by_range \
- in BlockRBManager::initialize_blk_alloc_area"
- }
- );
- });
-}
-
BlockRBManager::mkfs_ertr::future<> BlockRBManager::mkfs(mkfs_config_t config)
{
LOG_PREFIX(BlockRBManager::mkfs);
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.start_data_area = 0;
super.crc = 0;
super.feature |= RBM_BITMAP_BLOCK_CRC;
super.device_id = config.device_id;
DEBUG("super {} ", super);
// write super block
- return write_rbm_header().safe_then([this] {
- return initialize_blk_alloc_area();
+ return write_rbm_header(
+ ).safe_then([] {
+ return mkfs_ertr::now();
}).handle_error(
mkfs_ertr::pass_further{},
crimson::ct_error::assert_all{
});
}
-BlockRBManager::find_block_ret BlockRBManager::find_free_block(Transaction &t, size_t size)
-{
- LOG_PREFIX(BlockRBManager::find_free_block);
- 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_no_t>(),
- bp,
- [&, this, FNAME](auto &allocated, auto &addr, auto &alloc_extent, auto &bp) mutable {
- return crimson::repeat(
- [&, this, FNAME]() mutable {
- return device->read(
- addr,
- bp
- ).safe_then(
- [&bp, &addr, size, &allocated, &alloc_extent, this, FNAME]() mutable {
- 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();
- 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);
- if (b_block.is_allocated(i)) {
- continue;
- }
- 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
- DEBUG("find_free_list: rety to find continuous blocks");
- continue;
- }
- allocated += 1;
- alloc_extent.insert(block_id);
- }
- addr += super.block_size;
- DEBUG("find_free_list: allocated: {} alloc_extent {}",
- allocated, alloc_extent);
- if (((uint64_t)size)/super.block_size == allocated) {
- return seastar::stop_iteration::yes;
- } else if (addr >= super.start_data_area) {
- alloc_extent.clear();
- return seastar::stop_iteration::yes;
- }
- return seastar::stop_iteration::no;
- });
- }).safe_then([&allocated, &alloc_extent, size, this, FNAME]() {
- 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 BlockRBManager::find_free_block"
- }
- );
- });
-}
/* TODO : block allocator */
BlockRBManager::allocate_ret BlockRBManager::alloc_extent(
* just read the block bitmap directly to find free blocks.
*
*/
- LOG_PREFIX(BlockRBManager::alloc_extent);
- return find_free_block(t, size
- ).safe_then([this, FNAME](auto alloc_extent) mutable
- -> allocate_ertr::future<paddr_t> {
- DEBUG("after find_free_block: allocated {}", alloc_extent);
- if (alloc_extent.empty()) {
- return crimson::ct_error::enospc::make();
- }
- paddr_t paddr = convert_abs_addr_to_paddr(
- alloc_extent.range_start() * super.block_size,
- super.device_id);
- return allocate_ret(
- allocate_ertr::ready_future_marker{},
- paddr);
- }).handle_error(
- allocate_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error find_free_block in BlockRBManager::alloc_extent"
- }
- );
+ // TODO: block allocation using in-memory block allocator
+ return allocate_ret(
+ allocate_ertr::ready_future_marker{},
+ paddr_t{});
}
-void BlockRBManager::add_free_extent(
- std::vector<alloc_delta_t>& v, rbm_abs_addr from, size_t len)
-{
- ceph_assert(!(len % super.block_size));
- paddr_t paddr = convert_abs_addr_to_paddr(
- from,
- super.device_id);
- alloc_delta_t alloc_info;
- alloc_info.alloc_blk_ranges.emplace_back(
- paddr, L_ADDR_NULL, len, extent_types_t::ROOT);
- alloc_info.op = alloc_delta_t::op_types_t::CLEAR;
- v.push_back(alloc_info);
-}
-
-BlockRBManager::write_ertr::future<> BlockRBManager::rbm_sync_block_bitmap_by_range(
- blk_no_t start, blk_no_t end, bitmap_op_types_t op)
-{
- LOG_PREFIX(BlockRBManager::rbm_sync_block_bitmap_by_range);
- 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, FNAME]() {
- 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);
- 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);
- 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, end, op, addr, this, FNAME]() 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);
- }
- }
- 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 BlockRBManager::rbm_sync_block_bitmap_by_range"
- }
- );
- }).handle_error(
- write_ertr::pass_further{},
- crimson::ct_error::assert_all{
- "Invalid error in BlockRBManager::rbm_sync_block_bitmap_by_range"
- }
- );
-}
BlockRBManager::abort_allocation_ertr::future<> BlockRBManager::abort_allocation(
Transaction &t)
return write_ertr::now();
}
-BlockRBManager::write_ertr::future<> BlockRBManager::sync_allocation(
- std::vector<alloc_delta_t> &alloc_blocks)
-{
- LOG_PREFIX(BlockRBManager::sync_allocation);
- if (alloc_blocks.empty()) {
- return write_ertr::now();
- }
- return seastar::do_with(move(alloc_blocks),
- [&, this, FNAME](auto &alloc_blocks) mutable {
- return crimson::do_for_each(alloc_blocks,
- [this, FNAME](auto &alloc) {
- return crimson::do_for_each(alloc.alloc_blk_ranges,
- [this, &alloc, FNAME](auto &range) -> write_ertr::future<> {
- DEBUG("range {} ~ {}", range.paddr, range.len);
- bitmap_op_types_t op =
- (alloc.op == alloc_delta_t::op_types_t::SET) ?
- bitmap_op_types_t::ALL_SET :
- bitmap_op_types_t::ALL_CLEAR;
- rbm_abs_addr addr = convert_paddr_to_abs_addr(
- range.paddr);
- blk_no_t start = addr / super.block_size;
- blk_no_t end = start +
- (round_up_to(range.len, super.block_size)) / super.block_size
- - 1;
- return rbm_sync_block_bitmap_by_range(
- start,
- end,
- op);
- });
- }).safe_then([this, &alloc_blocks, FNAME]() mutable {
- int alloc_block_count = 0;
- for (const auto& b : alloc_blocks) {
- for (auto r : b.alloc_blk_ranges) {
- if (b.op == alloc_delta_t::op_types_t::SET) {
- alloc_block_count +=
- round_up_to(r.len, super.block_size) / super.block_size;
- DEBUG("complete alloc block: start {} len {} ",
- r.paddr, r.len);
- } else {
- alloc_block_count -=
- round_up_to(r.len, super.block_size) / super.block_size;
- DEBUG("complete alloc block: start {} len {} ",
- r.paddr, r.len);
- }
- }
- }
- DEBUG("complete_alloction: complete to allocate {} blocks",
- alloc_block_count);
- super.free_block_count -= alloc_block_count;
- return write_ertr::now();
- });
- });
-}
-
BlockRBManager::open_ertr::future<> BlockRBManager::open()
{
return read_rbm_header(RBM_START_ADDRESS
return crimson::ct_error::enoent::make();
}
super = s;
- return check_bitmap_blocks().safe_then([]() {
- return open_ertr::now();
- });
+ return open_ertr::now();
}).handle_error(
open_ertr::pass_further{},
crimson::ct_error::assert_all{
return device->close();
}
-BlockRBManager::open_ertr::future<> BlockRBManager::_open_device(
- const std::string path)
-{
- ceph_assert(device);
- return device->open(path, seastar::open_flags::rw);
-}
BlockRBManager::write_ertr::future<> BlockRBManager::write_rbm_header()
{
);
}
-BlockRBManager::check_bitmap_blocks_ertr::future<> BlockRBManager::check_bitmap_blocks()
-{
- LOG_PREFIX(BlockRBManager::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, FNAME](auto &addr, auto &free_blocks, auto &bp) mutable {
- return crimson::repeat([&, this, FNAME]() mutable {
- return device->read(addr, bp
- ).safe_then(
- [&bp, &addr, &free_blocks, this, FNAME]() mutable {
- 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 seastar::stop_iteration::yes;
- }
- return seastar::stop_iteration::no;
- });
- }).safe_then([&free_blocks, this, FNAME]() {
- 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 BlockRBManager::find_free_block"
- }
- );
- });
-}
-
BlockRBManager::write_ertr::future<> BlockRBManager::write(
rbm_abs_addr addr,
bufferlist &bl)
<< ", 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
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 << ")";
-}
-
}
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 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;
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);
};
-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 BlockRBManager final : public RandomBlockManager {
public:
/*
- * Ondisk layout
+ * Ondisk layout (TODO)
*
* ---------------------------------------------------------------------------
- * | rbm_metadata_header_t | rbm_bitmap_block_t 1 | ... | data blocks |
+ * | rbm_metadata_header_t | metadatas | ... | data blocks |
* ---------------------------------------------------------------------------
*/
* 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_ret alloc_extent(
Transaction &t, size_t size) final; // allocator, return blocks
- /*
- * free_extent
- *
- * add a range of free blocks to transaction
- *
- */
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(rbm_abs_addr addr);
write_ertr::future<> write_rbm_header();
size_t get_size() const final { return super.size; };
extent_len_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_no_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_no_t convert_bitmap_block_no_to_block_id(uint64_t offset, rbm_abs_addr 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_no_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_no_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
BlockRBManager(RBMDevice * 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);
- }
-
- rbm_abs_addr get_blk_paddr_by_block_no(blk_no_t id) {
- return (id * super.block_size) + super.start;
- }
-
- int num_block_between_blk_ids(blk_no_t start, blk_no_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_no_t start, blk_no_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(rbm_abs_addr addr, bufferlist &bl);
- write_ertr::future<> sync_allocation(
- std::vector<alloc_delta_t>& alloc_blocks);
- void add_free_extent(
- std::vector<alloc_delta_t>& v, rbm_abs_addr from, size_t len);
device_id_t get_device_id() const final {
return super.device_id;
}
+ uint64_t get_free_blocks() const final {
+ // TODO: return correct free blocks after block allocator is introduced
+ return super.size / super.block_size;
+ }
+
private:
/*
* this contains the number of bitmap blocks, free blocks and
}
auto open() {
+ device->mount().unsafe_get0();
return rbm_manager->open().unsafe_get0();
}
return rbm_manager->read(paddr, ptr).unsafe_get0();
}
- auto create_rbm_transaction() {
- return std::make_unique<rbm_transaction>();
- }
-
- auto alloc_extent(rbm_transaction &t, size_t size) {
- auto tt = make_test_transaction(); // dummy transaction
- auto extent = rbm_manager->find_free_block(*tt, size).unsafe_get0();
- if (!extent.empty()) {
- alloc_delta_t alloc_info;
- for (auto p : extent) {
- paddr_t paddr = convert_abs_addr_to_paddr(
- p.first * block_size,
- rbm_manager->get_device_id());
- size_t len = p.second * block_size;
- alloc_info.alloc_blk_ranges.emplace_back(
- paddr, L_ADDR_NULL, len, extent_types_t::ROOT);
- alloc_info.op = alloc_delta_t::op_types_t::SET;
- }
- t.add_rbm_allocated_blocks(alloc_info);
- }
- }
-
- void free_extent(rbm_transaction &t, interval_set<blk_no_t> range) {
- for (auto [off, len] : range) {
- logger().debug("free_extent: start {} len {}", off * DEFAULT_BLOCK_SIZE,
- len * DEFAULT_BLOCK_SIZE);
- rbm_manager->add_free_extent(t.allocated_blocks, off * DEFAULT_BLOCK_SIZE,
- len * DEFAULT_BLOCK_SIZE);
- }
- }
-
- interval_set<blk_no_t> get_allocated_blk_ids(rbm_transaction &t) {
- auto allocated_blocks = t.get_rbm_allocated_blocks();
- interval_set<blk_no_t> alloc_ids;
- for (auto p : allocated_blocks) {
- for (auto b : p.alloc_blk_ranges) {
- rbm_abs_addr addr =
- convert_paddr_to_abs_addr(b.paddr);
- alloc_ids.insert(addr / block_size, b.len / block_size);
- }
- }
- logger().debug(" get allocated blockid {}", alloc_ids);
- return alloc_ids;
- }
-
- bool check_ids_are_allocated(interval_set<blk_no_t> &ids, bool allocated = true) {
- bool ret = true;
- for (auto r : ids) {
- for (blk_no_t id = r.first; id < r.first + r.second; id++) {
- auto addr = rbm_manager->get_start_block_alloc_area() +
- (id / rbm_manager->max_block_by_bitmap_block())
- * DEFAULT_BLOCK_SIZE;
- logger().debug(" addr {} id {} ", addr, id);
- auto bp = bufferptr(ceph::buffer::create_page_aligned(DEFAULT_BLOCK_SIZE));
- paddr_t paddr = convert_abs_addr_to_paddr(
- addr,
- rbm_manager->get_device_id());
- rbm_manager->read(paddr, bp).unsafe_get0();
- rbm_bitmap_block_t b_block(DEFAULT_BLOCK_SIZE);
- bufferlist bl;
- bl.append(bp);
- auto b_bl = bl.cbegin();
- decode(b_block, b_bl);
- if (!b_block.is_allocated(id % rbm_manager->max_block_by_bitmap_block())) {
- logger().debug(" block id {} is not allocated", id);
- if (allocated) {
- ret = false;
- return ret;
- }
- } else {
- logger().debug(" block id {} allocated", id);
- if (!allocated) {
- ret = false;
- return ret;
- }
- }
- }
- }
- return ret;
- }
-
- auto complete_allocation(rbm_transaction &t) {
- auto alloc_blocks = t.get_rbm_allocated_blocks();
- return rbm_manager->sync_allocation(alloc_blocks).unsafe_get0();
- }
-
bufferptr generate_extent(size_t blocks) {
std::uniform_int_distribution<char> distribution(
std::numeric_limits<char>::min(),
return buffer::ptr(buffer::create(blocks * block_size, contents));
}
+ void close() {
+ rbm_manager->close().unsafe_get0();
+ return;
+ }
+
};
TEST_F(rbm_test_t, mkfs_test)
auto super = read_rbm_header();
ASSERT_TRUE(
super.block_size == DEFAULT_BLOCK_SIZE &&
- super.end == DEFAULT_TEST_SIZE &&
- super.start_alloc_area == DEFAULT_BLOCK_SIZE &&
- super.free_block_count == DEFAULT_TEST_SIZE / DEFAULT_BLOCK_SIZE - 2 &&
- super.alloc_area_size == DEFAULT_BLOCK_SIZE
+ super.end == DEFAULT_TEST_SIZE
+ );
+ config.block_size = 8196;
+ mkfs();
+ super = read_rbm_header();
+ ASSERT_TRUE(
+ super.block_size == 8196 &&
+ super.end == DEFAULT_TEST_SIZE
);
-
});
}
-TEST_F(rbm_test_t, open_test)
+TEST_F(rbm_test_t, open_read_write_test)
{
run_async([this] {
auto content = generate_extent(1);
- write(
- DEFAULT_BLOCK_SIZE,
- content
- );
- auto bp = bufferptr(ceph::buffer::create_page_aligned(DEFAULT_BLOCK_SIZE));
- read(
- DEFAULT_BLOCK_SIZE,
- bp
- );
- bufferlist bl;
- bufferlist block;
- bl.append(bp);
- block.append(content);
- ASSERT_EQ(
- bl.begin().crc32c(bl.length(), 1),
- block.begin().crc32c(block.length(), 1));
-
- });
-}
-
-TEST_F(rbm_test_t, block_alloc_test)
-{
- run_async([this] {
- auto t = create_rbm_transaction();
- alloc_extent(*t, DEFAULT_BLOCK_SIZE);
- auto alloc_ids = get_allocated_blk_ids(*t);
- complete_allocation(*t);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
-
- auto t2 = create_rbm_transaction();
- alloc_extent(*t2, DEFAULT_BLOCK_SIZE * 3);
- alloc_ids = get_allocated_blk_ids(*t2);
- complete_allocation(*t2);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- });
-}
-
-TEST_F(rbm_test_t, block_alloc_free_test)
-{
- run_async([this] {
- auto t = create_rbm_transaction();
- alloc_extent(*t, DEFAULT_BLOCK_SIZE);
- auto alloc_ids = get_allocated_blk_ids(*t);
- free_extent(*t, alloc_ids);
- complete_allocation(*t);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids, false));
-
- auto t2 = create_rbm_transaction();
- alloc_extent(*t2, DEFAULT_BLOCK_SIZE * 4);
- alloc_ids = get_allocated_blk_ids(*t2);
- free_extent(*t2, alloc_ids);
- complete_allocation(*t2);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids, false));
-
- auto t3 = create_rbm_transaction();
- alloc_extent(*t3, DEFAULT_BLOCK_SIZE * 8);
- alloc_ids = get_allocated_blk_ids(*t3);
- complete_allocation(*t3);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
-
- auto t4 = create_rbm_transaction();
- free_extent(*t4, alloc_ids);
- complete_allocation(*t4);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids, false));
- });
-}
-
-TEST_F(rbm_test_t, many_block_alloc)
-{
- run_async([this] {
- device_id_t d_id = 1 << (std::numeric_limits<device_id_t>::digits - 1);
- config.start = paddr_t::make_blk_paddr(d_id, 0);
- config.end = paddr_t::make_blk_paddr(d_id, (DEFAULT_TEST_SIZE * 1024));
- config.block_size = DEFAULT_BLOCK_SIZE;
- config.total_size = DEFAULT_TEST_SIZE * 1024;
- rbm_manager->close().unsafe_get0();
- mkfs();
+ {
+ write(
+ DEFAULT_BLOCK_SIZE,
+ content
+ );
+ auto bp = bufferptr(ceph::buffer::create_page_aligned(DEFAULT_BLOCK_SIZE));
+ read(
+ DEFAULT_BLOCK_SIZE,
+ bp
+ );
+ bufferlist bl;
+ bufferlist block;
+ bl.append(bp);
+ block.append(content);
+ ASSERT_EQ(
+ bl.begin().crc32c(bl.length(), 1),
+ block.begin().crc32c(block.length(), 1));
+ }
+ close();
open();
- auto max = rbm_manager->max_block_by_bitmap_block();
- rbm_manager->rbm_sync_block_bitmap_by_range(max + 10, max + 14, bitmap_op_types_t::ALL_SET).unsafe_get0();
- interval_set<blk_no_t> alloc_ids;
- alloc_ids.insert(max + 12, 2);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- alloc_ids.clear();
- alloc_ids.insert(max + 10, 4);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- rbm_manager->rbm_sync_block_bitmap_by_range(max + 10, max + 14, bitmap_op_types_t::ALL_CLEAR).unsafe_get0();
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids, false));
- rbm_manager->rbm_sync_block_bitmap_by_range(max + 10, max + max + 10, bitmap_op_types_t::ALL_SET).unsafe_get0();
- alloc_ids.clear();
- alloc_ids.insert(max + 10000, 10);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- alloc_ids.clear();
- alloc_ids.insert(max + max, 10);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- rbm_manager->rbm_sync_block_bitmap_by_range(max, max * 3, bitmap_op_types_t::ALL_SET).unsafe_get0();
- alloc_ids.clear();
- alloc_ids.insert(max * 3 - 1, 1);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- alloc_ids.clear();
- alloc_ids.insert(max * 3, 1);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- alloc_ids.clear();
- alloc_ids.insert(max, 1);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- rbm_manager->rbm_sync_block_bitmap_by_range(max, max * 6, bitmap_op_types_t::ALL_SET).unsafe_get0();
- alloc_ids.clear();
- alloc_ids.insert(max * 5, 10);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- alloc_ids.clear();
- alloc_ids.insert(max * 6, 1);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids));
- rbm_manager->rbm_sync_block_bitmap_by_range(max, max * 6, bitmap_op_types_t::ALL_CLEAR).unsafe_get0();
- alloc_ids.clear();
- alloc_ids.insert(max * 3, 10);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids, false));
- alloc_ids.clear();
- alloc_ids.insert(max * 5, 10);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids, false));
- alloc_ids.clear();
- alloc_ids.insert(max * 6, 1);
- ASSERT_TRUE(check_ids_are_allocated(alloc_ids, false));
+ {
+ auto bp = bufferptr(ceph::buffer::create_page_aligned(DEFAULT_BLOCK_SIZE));
+ read(
+ DEFAULT_BLOCK_SIZE,
+ bp
+ );
+ bufferlist bl;
+ bufferlist block;
+ bl.append(bp);
+ block.append(content);
+ ASSERT_EQ(
+ bl.begin().crc32c(bl.length(), 1),
+ block.begin().crc32c(block.length(), 1));
+ }
});
}
-TEST_F(rbm_test_t, check_free_blocks)
-{
- run_async([this] {
- rbm_manager->rbm_sync_block_bitmap_by_range(10, 12, bitmap_op_types_t::ALL_SET).unsafe_get0();
- rbm_manager->close().unsafe_get0();
- open();
- ASSERT_TRUE(rbm_manager->get_free_blocks() == DEFAULT_TEST_SIZE/DEFAULT_BLOCK_SIZE - 5);
- auto free = rbm_manager->get_free_blocks();
- interval_set<blk_no_t> alloc_ids;
- auto t = create_rbm_transaction();
- alloc_extent(*t, DEFAULT_BLOCK_SIZE * 4);
- alloc_ids = get_allocated_blk_ids(*t);
- complete_allocation(*t);
- ASSERT_TRUE(rbm_manager->get_free_blocks() == free - 4);
-
- free = rbm_manager->get_free_blocks();
- auto t2 = create_rbm_transaction();
- free_extent(*t2, alloc_ids);
- complete_allocation(*t2);
- ASSERT_TRUE(rbm_manager->get_free_blocks() == free + 4);
- });
-}