if(HAVE_LIBZBC)
list(APPEND libos_srcs
+ bluestore/zoned_types.cc
+ bluestore/ZonedFreelistManager.cc
bluestore/ZonedAllocator.cc)
endif()
#ifndef CEPH_OS_BLUESTORE_ALLOCATOR_H
#define CEPH_OS_BLUESTORE_ALLOCATOR_H
+#include <functional>
#include <ostream>
#include "include/ceph_assert.h"
-#include "os/bluestore/bluestore_types.h"
-#include <functional>
+#include "bluestore_types.h"
+#include "zoned_types.h"
class Allocator {
public:
virtual void dump() = 0;
virtual void dump(std::function<void(uint64_t offset, uint64_t length)> notify) = 0;
+ virtual void set_zone_states(std::vector<zone_state_t> &&_zone_states) {}
virtual void init_add_free(uint64_t offset, uint64_t length) = 0;
virtual void init_rm_free(uint64_t offset, uint64_t length) = 0;
const string PREFIX_ALLOC = "B"; // u64 offset -> u64 length (freelist)
const string PREFIX_ALLOC_BITMAP = "b";// (see BitmapFreelistManager)
const string PREFIX_SHARED_BLOB = "X"; // u64 offset -> shared_blob_t
+const string PREFIX_ZONED_META = "Z"; // (see ZonedFreelistManager)
+const string PREFIX_ZONED_INFO = "z"; // (see ZonedFreelistManager)
const string BLUESTORE_GLOBAL_STATFS_KEY = "bluestore_statfs";
if (r < 0) {
goto fail_close;
}
+
+ if (bdev->is_smr()) {
+ freelist_type = "zoned";
+ }
return 0;
fail_close:
// being able to allocate in units less than bdev block size
// seems to be a bad idea.
ceph_assert( cct->_conf->bdev_block_size <= (int64_t)min_alloc_size);
- fm->create(bdev->get_size(), (int64_t)min_alloc_size, t);
+
+ uint64_t alloc_size = min_alloc_size;
+ if (bdev->is_smr()) {
+ alloc_size = _piggyback_zoned_device_parameters_onto(alloc_size);
+ }
+
+ fm->create(bdev->get_size(), alloc_size, t);
// allocate superblock reserved space. note that we do not mark
// bluefs space as allocated in the freelist; we instead rely on
<< dendl;
}
+ uint64_t alloc_size = min_alloc_size;
if (bdev->is_smr()) {
+ alloc_size = _piggyback_zoned_device_parameters_onto(alloc_size);
if (cct->_conf->bluestore_allocator != "zoned") {
dout(1) << __func__ << " The drive is HM-SMR but "
<< cct->_conf->bluestore_allocator << " allocator is specified. "
<< "Please set to 0." << dendl;
return -EINVAL;
}
-
- // For now, to avoid interface changes we piggyback zone_size (in MiB) and
- // the first sequential zone number onto min_alloc_size and pass it to
- // Allocator::create.
- uint64_t zone_size = bdev->get_zone_size();
- uint64_t zone_size_mb = zone_size / (1024 * 1024);
- uint64_t first_seq_zone = bdev->get_conventional_region_size() / zone_size;
-
- min_alloc_size |= (zone_size_mb << 32);
- min_alloc_size |= (first_seq_zone << 48);
}
alloc = Allocator::create(cct, cct->_conf->bluestore_allocator,
bdev->get_size(),
- min_alloc_size, "block");
-
- if (bdev->is_smr()) {
- min_alloc_size &= 0x00000000ffffffff;
- }
+ alloc_size, "block");
if (!alloc) {
lderr(cct) << __func__ << " Allocator::unknown alloc type "
return -EINVAL;
}
+ if (bdev->is_smr()) {
+ alloc->set_zone_states(fm->get_zone_states(db));
+ }
+
uint64_t num = 0, bytes = 0;
dout(1) << __func__ << " opening allocation metadata" << dendl;
return -EIO;
}
- FreelistManager::setup_merge_operators(db);
+ FreelistManager::setup_merge_operators(db, freelist_type);
db->set_merge_operator(PREFIX_STAT, merge_op);
db->set_cache_size(cache_kv_ratio * cache_size);
return 0;
int _setup_block_symlink_or_file(std::string name, std::string path, uint64_t size,
bool create);
+ // Functions related to zoned storage.
+
+ // For now, to avoid interface changes we piggyback zone_size (in MiB) and the
+ // first sequential zone number onto min_alloc_size and pass it to functions
+ // Allocator::create and FreelistManager::create.
+ uint64_t _piggyback_zoned_device_parameters_onto(uint64_t min_alloc_size) {
+ uint64_t zone_size = bdev->get_zone_size();
+ uint64_t zone_size_mb = zone_size / (1024 * 1024);
+ uint64_t first_seq_zone = bdev->get_conventional_region_size() / zone_size;
+ min_alloc_size |= (zone_size_mb << 32);
+ min_alloc_size |= (first_seq_zone << 48);
+ return min_alloc_size;
+ }
+
public:
utime_t get_deferred_last_submitted() {
std::lock_guard l(deferred_lock);
#include "FreelistManager.h"
#include "BitmapFreelistManager.h"
+#ifdef HAVE_LIBZBC
+#include "ZonedFreelistManager.h"
+#endif
FreelistManager *FreelistManager::create(
CephContext* cct,
ceph_assert(prefix == "B");
if (type == "bitmap")
return new BitmapFreelistManager(cct, "B", "b");
+
+#ifdef HAVE_LIBZBC
+ // With zoned drives there is only one FreelistManager implementation that we
+ // can use, and we also know if a drive is zoned right after opening it
+ // (BlueStore::_open_bdev). Hence, we set freelist_type to "zoned" whenever
+ // we open the device and it turns out to be is zoned. We ignore |prefix|
+ // passed to create and use the prefixes defined for zoned devices at the top
+ // of BlueStore.cc.
+ if (type == "zoned")
+ return new ZonedFreelistManager(cct, "Z", "z");
+#endif
+
return NULL;
}
-void FreelistManager::setup_merge_operators(KeyValueDB *db)
+void FreelistManager::setup_merge_operators(KeyValueDB *db,
+ const std::string& type)
{
- BitmapFreelistManager::setup_merge_operator(db, "b");
+#ifdef HAVE_LIBZBC
+ if (type == "zoned")
+ ZonedFreelistManager::setup_merge_operator(db, "z");
+ else
+#endif
+ BitmapFreelistManager::setup_merge_operator(db, "b");
}
#include <ostream>
#include "kv/KeyValueDB.h"
#include "bluestore_types.h"
+#include "zoned_types.h"
class FreelistManager {
public:
std::string type,
std::string prefix);
- static void setup_merge_operators(KeyValueDB *db);
+ static void setup_merge_operators(KeyValueDB *db, const std::string &type);
virtual int create(uint64_t size, uint64_t granularity,
KeyValueDB::Transaction txn) = 0;
virtual void get_meta(uint64_t target_size,
std::vector<std::pair<string, string>>*) const = 0;
+
+ virtual std::vector<zone_state_t> get_zone_states(KeyValueDB *kvdb) const {
+ return {};
+ }
};
#include "ZonedAllocator.h"
#include "bluestore_types.h"
+#include "zoned_types.h"
#include "common/debug.h"
#define dout_context cct
cct(cct),
num_free(0),
size(size),
+ // To avoid interface changes, we piggyback zone size and the first
+ // sequential zone number onto the first 32 bits of 64-bit |block_size|.
+ // The last 32 bits of |block_size| is holding the actual block size.
block_size((block_size & 0x00000000ffffffff)),
zone_size(((block_size & 0x0000ffff00000000) >> 32) * 1024 * 1024),
- starting_zone((block_size & 0xffff000000000000) >> 48),
- nr_zones(size / zone_size),
- write_pointers(nr_zones) {
+ starting_zone_num((block_size & 0xffff000000000000) >> 48),
+ num_zones(size / zone_size) {
ldout(cct, 10) << __func__ << " size 0x" << std::hex << size
<< " zone size 0x" << zone_size << std::dec
- << " number of zones " << nr_zones
- << " first sequential zone " << starting_zone
+ << " number of zones " << num_zones
+ << " first sequential zone " << starting_zone_num
<< dendl;
ceph_assert(size % zone_size == 0);
}
ldout(cct, 10) << __func__ << " trying to allocate "
<< std::hex << want_size << dendl;
- uint64_t zone = starting_zone;
- for ( ; zone < nr_zones; ++zone) {
- if (fits(want_size, zone))
+ uint64_t zone_num = starting_zone_num;
+ for ( ; zone_num < num_zones; ++zone_num) {
+ if (fits(want_size, zone_num)) {
break;
- ldout(cct, 10) << __func__ << " skipping zone " << zone
+ }
+ ldout(cct, 10) << __func__ << " skipping zone " << zone_num
<< " because there is not enough space: "
<< " want_size = " << want_size
- << " available = " << zone_free_space(zone) << dendl;
+ << " available = " << get_remaining_space(zone_num)
+ << dendl;
}
- if (zone == nr_zones) {
+ if (zone_num == num_zones) {
ldout(cct, 10) << __func__ << " failed to allocate" << dendl;
return -ENOSPC;
}
- uint64_t offset = zone_offset(zone);
- ldout(cct, 10) << __func__ << " advancing zone " << zone
- << " write pointer from " << std::hex << offset
+ uint64_t offset = get_offset(zone_num);
+
+ ldout(cct, 10) << __func__ << " advancing zone " << std::hex
+ << zone_num << " write pointer from " << offset
<< " to " << offset + want_size << dendl;
- advance_wp(zone, want_size);
- if (zone_free_space(zone) == 0) {
- starting_zone = zone + 1;
+ advance_write_pointer(zone_num, want_size);
+ if (get_remaining_space(zone_num) == 0) {
+ starting_zone_num = zone_num + 1;
}
- ldout(cct, 10) << __func__ << " zone " << zone << " offset is now "
- << std::hex << zone_wp(zone) << dendl;
+ ldout(cct, 10) << __func__ << std::hex << " zone " << zone_num
+ << " offset is now " << get_write_pointer(zone_num) << dendl;
ldout(cct, 10) << __func__ << " allocated " << std::hex << want_size
<< " bytes at offset " << offset
- << " located at zone " << zone
+ << " located at zone " << zone_num
<< " and zone offset " << offset % zone_size << dendl;
extents->emplace_back(bluestore_pextent_t(offset, want_size));
std::lock_guard l(lock);
}
+// This just increments |num_free|. The actual free space is added by
+// set_zone_states, as it updates the write pointer for each zone.
void ZonedAllocator::init_add_free(uint64_t offset, uint64_t length) {
std::lock_guard l(lock);
ldout(cct, 10) << __func__ << " " << std::hex
<< offset << "~" << length << dendl;
num_free += length;
- uint64_t zone = offset / zone_size;
- offset %= zone_size;
- write_pointers[zone] = offset;
- ldout(cct, 10) << __func__ << " set zone " << std::hex
- << zone << " write pointer to 0x" << offset << dendl;
-
- if (length > zone_size - offset) {
- length -= zone_size - offset;
- for (++zone; length >= zone_size; length -= zone_size) {
- ldout(cct, 30) << __func__ << " set zone 0x" << std::hex
- << zone << " write pointer to 0x" << 0 << dendl;
- write_pointers[zone++] = 0;
- }
- if (length > 0) {
- ldout(cct, 20) << __func__ << " set zone 0x" << std::hex
- << zone << " write pointer to 0x" << 0 << dendl;
- write_pointers[zone] = length;
- }
- }
}
void ZonedAllocator::init_rm_free(uint64_t offset, uint64_t length) {
num_free -= length;
ceph_assert(num_free >= 0);
- uint64_t zone = offset / zone_size;
- offset %= zone_size;
- ceph_assert(write_pointers[zone] == offset);
- write_pointers[zone] = zone_size;
+ uint64_t zone_num = offset / zone_size;
+ uint64_t write_pointer = offset % zone_size;
+ uint64_t remaining_space = get_remaining_space(zone_num);
+
+ ceph_assert(get_write_pointer(zone_num) == write_pointer);
+ ceph_assert(remaining_space <= length);
+ advance_write_pointer(zone_num, remaining_space);
+
ldout(cct, 10) << __func__ << " set zone 0x" << std::hex
- << zone << " write pointer to 0x" << zone_size << dendl;
+ << zone_num << " write pointer to 0x" << zone_size << dendl;
- length -= zone_size - offset;
+ length -= remaining_space;
ceph_assert(length % zone_size == 0);
for ( ; length; length -= zone_size) {
- write_pointers[++zone] = zone_size;
+ advance_write_pointer(++zone_num, zone_size);
ldout(cct, 10) << __func__ << " set zone 0x" << std::hex
- << zone << " write pointer to 0x" << zone_size << dendl;
+ << zone_num << " write pointer to 0x" << zone_size << dendl;
}
}
+void ZonedAllocator::set_zone_states(std::vector<zone_state_t> &&_zone_states) {
+ std::lock_guard l(lock);
+ ldout(cct, 10) << __func__ << dendl;
+ zone_states = std::move(_zone_states);
+}
void ZonedAllocator::shutdown() {
ldout(cct, 1) << __func__ << dendl;
#include "include/btree_map.h"
#include "include/interval_set.h"
#include "include/mempool.h"
-#include "os/bluestore/bluestore_types.h"
+#include "bluestore_types.h"
+#include "zoned_types.h"
class ZonedAllocator : public Allocator {
CephContext* cct;
uint64_t size;
uint64_t block_size;
uint64_t zone_size;
- uint64_t starting_zone;
- uint64_t nr_zones;
- std::vector<uint64_t> write_pointers;
+ uint64_t starting_zone_num;
+ uint64_t num_zones;
+ std::vector<zone_state_t> zone_states;
- inline uint64_t zone_offset(uint64_t zone) {
- ceph_assert(zone < nr_zones);
- return zone * zone_size + zone_wp(zone);
+ inline uint64_t get_offset(uint64_t zone_num) const {
+ return zone_num * zone_size + get_write_pointer(zone_num);
}
- inline uint64_t zone_wp(uint64_t zone) {
- ceph_assert(zone < nr_zones);
- return write_pointers[zone];
+ inline uint64_t get_write_pointer(uint64_t zone_num) const {
+ return zone_states[zone_num].get_write_pointer();
}
- inline uint64_t zone_free_space(uint64_t zone) {
- ceph_assert(zone < nr_zones);
- return zone_size - zone_wp(zone);
+ inline uint64_t get_remaining_space(uint64_t zone_num) const {
+ return zone_size - get_write_pointer(zone_num);
}
- inline void advance_wp(uint64_t zone, uint64_t size) {
- ceph_assert(zone < nr_zones);
- write_pointers[zone] += size;
- ceph_assert(write_pointers[zone] <= zone_size);
+ inline void advance_write_pointer(uint64_t zone_num, uint64_t want_size) {
+ zone_states[zone_num].increment_write_pointer(want_size);
}
- inline bool fits(uint64_t want_size, uint64_t zone) {
- ceph_assert(zone < nr_zones);
- return want_size <= zone_free_space(zone);
+ inline bool fits(uint64_t want_size, uint64_t zone_num) const {
+ return want_size <= get_remaining_space(zone_num);
}
public:
void dump(std::function<void(uint64_t offset,
uint64_t length)> notify) override;
+ void set_zone_states(std::vector<zone_state_t> &&_zone_states) override;
void init_add_free(uint64_t offset, uint64_t length) override;
void init_rm_free(uint64_t offset, uint64_t length) override;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+//
+// A freelist manager for zoned devices. This iteration just keeps the write
+// pointer per zone. Following iterations will add enough information to enable
+// cleaning of zones.
+//
+// Copyright (C) 2020 Abutalib Aghayev
+//
+
+#include "ZonedFreelistManager.h"
+#include "bluestore_common.h"
+#include "include/stringify.h"
+#include "kv/KeyValueDB.h"
+#include "os/kv.h"
+#include "zoned_types.h"
+
+#include "common/debug.h"
+
+#define dout_context cct
+#define dout_subsys ceph_subsys_bluestore
+#undef dout_prefix
+#define dout_prefix *_dout << "zoned freelist "
+
+using std::string;
+
+using ceph::bufferlist;
+using ceph::bufferptr;
+using ceph::decode;
+using ceph::encode;
+
+void ZonedFreelistManager::write_zone_state_to_db(
+ uint64_t zone_num,
+ const zone_state_t &zone_state,
+ KeyValueDB::Transaction txn) {
+ string key;
+ _key_encode_u64(zone_num, &key);
+ bufferlist bl;
+ zone_state.encode(bl);
+ txn->merge(info_prefix, key, bl);
+}
+
+void ZonedFreelistManager::load_zone_state_from_db(
+ uint64_t zone_num,
+ zone_state_t &zone_state,
+ KeyValueDB::Iterator& it) const {
+ string k = it->key();
+ uint64_t zone_num_from_db;
+ _key_decode_u64(k.c_str(), &zone_num_from_db);
+ ceph_assert(zone_num_from_db == zone_num);
+
+ bufferlist bl = it->value();
+ auto p = bl.cbegin();
+ zone_state.decode(p);
+}
+
+void ZonedFreelistManager::init_zone_states(KeyValueDB::Transaction txn) {
+ dout(10) << __func__ << dendl;
+ for (uint64_t zone_num = 0; zone_num < num_zones; ++zone_num) {
+ zone_state_t zone_state;
+ write_zone_state_to_db(zone_num, zone_state, txn);
+ }
+}
+
+void ZonedFreelistManager::setup_merge_operator(KeyValueDB *db, string prefix) {
+ std::shared_ptr<Int64ArrayMergeOperator> merge_op(
+ new Int64ArrayMergeOperator);
+ db->set_merge_operator(prefix, merge_op);
+}
+
+ZonedFreelistManager::ZonedFreelistManager(
+ CephContext* cct,
+ string meta_prefix,
+ string info_prefix)
+ : FreelistManager(cct),
+ meta_prefix(meta_prefix),
+ info_prefix(info_prefix),
+ enumerate_zone_num(~0UL) {}
+
+int ZonedFreelistManager::create(
+ uint64_t new_size,
+ uint64_t granularity,
+ KeyValueDB::Transaction txn) {
+ // To avoid interface changes, we piggyback zone size and the first sequential
+ // zone number onto the first 32 bits of 64-bit |granularity|. The last 32
+ // bits of |granularity| is holding the actual allocation granularity, which
+ // is bytes_per_block.
+ size = new_size;
+ bytes_per_block = granularity & 0x00000000ffffffff;
+ zone_size = ((granularity & 0x0000ffff00000000) >> 32) * 1024 * 1024;
+ num_zones = size / zone_size;
+ starting_zone_num = (granularity & 0xffff000000000000) >> 48;
+ enumerate_zone_num = ~0UL;
+
+ ceph_assert(size % zone_size == 0);
+
+ dout(1) << __func__ << std::hex
+ << " size 0x" << size
+ << " bytes_per_block 0x" << bytes_per_block
+ << " zone size 0x " << zone_size
+ << " num_zones 0x" << num_zones
+ << " starting_zone 0x" << starting_zone_num << dendl;
+ {
+ bufferlist bl;
+ encode(size, bl);
+ txn->set(meta_prefix, "size", bl);
+ }
+ {
+ bufferlist bl;
+ encode(bytes_per_block, bl);
+ txn->set(meta_prefix, "bytes_per_block", bl);
+ }
+ {
+ bufferlist bl;
+ encode(zone_size, bl);
+ txn->set(meta_prefix, "zone_size", bl);
+ }
+ {
+ bufferlist bl;
+ encode(num_zones, bl);
+ txn->set(meta_prefix, "num_zones", bl);
+ }
+ {
+ bufferlist bl;
+ encode(starting_zone_num, bl);
+ txn->set(meta_prefix, "starting_zone_num", bl);
+ }
+
+ init_zone_states(txn);
+
+ return 0;
+}
+
+int ZonedFreelistManager::init(
+ KeyValueDB *kvdb,
+ bool db_in_read_only,
+ cfg_reader_t cfg_reader) {
+ dout(1) << __func__ << dendl;
+ int r = _read_cfg(cfg_reader);
+ if (r != 0) {
+ return r;
+ }
+
+ ceph_assert(num_zones == size / zone_size);
+
+ dout(10) << __func__ << std::hex
+ << " size 0x" << size
+ << " bytes_per_block 0x" << bytes_per_block
+ << " zone size 0x" << zone_size
+ << " num_zones 0x" << num_zones
+ << " starting_zone 0x" << starting_zone_num
+ << std::dec << dendl;
+ return 0;
+}
+
+void ZonedFreelistManager::sync(KeyValueDB* kvdb) {}
+
+void ZonedFreelistManager::shutdown() {
+ dout(1) << __func__ << dendl;
+}
+
+void ZonedFreelistManager::enumerate_reset() {
+ std::lock_guard l(lock);
+
+ dout(1) << __func__ << dendl;
+
+ enumerate_p.reset();
+ enumerate_zone_num = ~0UL;
+}
+
+// Currently, this just iterates over the list of zones and sets |offset| and
+// |length| to the write pointer and the number of remaining free bytes in a
+// given zone. Hence, it can set |length| to 0 if a zone is full, and it can
+// also return two contiguous empty zones in two calls. This does not violate
+// current semantics of the call and appears to work fine with the clients of
+// this call.
+bool ZonedFreelistManager::enumerate_next(
+ KeyValueDB *kvdb,
+ uint64_t *offset,
+ uint64_t *length) {
+ std::lock_guard l(lock);
+
+ // starting case
+ if (enumerate_zone_num == ~0UL) {
+ dout(30) << __func__ << " start" << dendl;
+ enumerate_p = kvdb->get_iterator(info_prefix);
+ enumerate_p->lower_bound(string());
+ ceph_assert(enumerate_p->valid());
+ enumerate_zone_num = 0;
+ } else {
+ enumerate_p->next();
+ if (!enumerate_p->valid()) {
+ dout(30) << __func__ << " end" << dendl;
+ return false;
+ }
+ ++enumerate_zone_num;
+ }
+
+ zone_state_t zone_state;
+ load_zone_state_from_db(enumerate_zone_num, zone_state, enumerate_p);
+
+ *offset = enumerate_zone_num * zone_size + zone_state.get_write_pointer();
+ *length = zone_size - zone_state.get_write_pointer();
+
+ dout(30) << __func__ << std::hex << " 0x" << *offset << "~" << *length
+ << std::dec << dendl;
+
+ return true;
+}
+
+void ZonedFreelistManager::dump(KeyValueDB *kvdb) {
+ enumerate_reset();
+ uint64_t offset, length;
+ while (enumerate_next(kvdb, &offset, &length)) {
+ dout(20) << __func__ << " 0x" << std::hex << offset << "~" << length
+ << std::dec << dendl;
+ }
+}
+
+// Advances the write pointer and writes the updated write pointer to database.
+void ZonedFreelistManager::allocate(
+ uint64_t offset,
+ uint64_t length,
+ KeyValueDB::Transaction txn) {
+ dout(10) << __func__ << " 0x" << std::hex << offset << "~" << length << dendl;
+ uint64_t zone_num = offset / zone_size;
+ zone_state_t zone_state;
+ zone_state.increment_write_pointer(length);
+ write_zone_state_to_db(zone_num, zone_state, txn);
+}
+
+// Increments the number of dead bytes in a zone and writes the updated value to
+// database. The dead bytes in the zone are not usable. The cleaner will later
+// copy live objects from the zone to another zone an make the zone writable
+// again. The number of dead bytes in a zone is used by the cleaner to select
+// which zones to clean -- the ones with most dead bytes are good candidates
+// since they require less I/O.
+void ZonedFreelistManager::release(
+ uint64_t offset,
+ uint64_t length,
+ KeyValueDB::Transaction txn) {
+ dout(10) << __func__ << " 0x" << std::hex << offset << "~" << length << dendl;
+ uint64_t zone_num = offset / zone_size;
+ zone_state_t zone_state;
+ zone_state.increment_num_dead_bytes(length);
+ write_zone_state_to_db(zone_num, zone_state, txn);
+}
+
+void ZonedFreelistManager::get_meta(
+ uint64_t target_size,
+ std::vector<std::pair<string, string>>* res) const {
+ // We do not support expanding devices for now.
+ ceph_assert(target_size == 0);
+ res->emplace_back("zfm_size", stringify(size));
+ res->emplace_back("zfm_bytes_per_block", stringify(bytes_per_block));
+ res->emplace_back("zfm_zone_size", stringify(zone_size));
+ res->emplace_back("zfm_num_zones", stringify(num_zones));
+ res->emplace_back("zfm_starting_zone_num", stringify(starting_zone_num));
+}
+
+std::vector<zone_state_t> ZonedFreelistManager::get_zone_states(
+ KeyValueDB *kvdb) const {
+ std::vector<zone_state_t> zone_states;
+ auto p = kvdb->get_iterator(info_prefix);
+ uint64_t zone_num = 0;
+ for (p->lower_bound(string()); p->valid(); p->next(), ++zone_num) {
+ zone_state_t zone_state;
+ load_zone_state_from_db(zone_num, zone_state, p);
+ zone_states.emplace_back(zone_state);
+ }
+ return zone_states;
+}
+
+// TODO: The following function is copied almost verbatim from
+// BitmapFreelistManager. Eliminate duplication.
+int ZonedFreelistManager::_read_cfg(cfg_reader_t cfg_reader) {
+ dout(1) << __func__ << dendl;
+
+ string err;
+
+ const size_t key_count = 5;
+ string keys[key_count] = {
+ "zfm_size",
+ "zfm_bytes_per_block",
+ "zfm_zone_size",
+ "zfm_num_zones",
+ "zfm_starting_zone_num"
+ };
+ uint64_t* vals[key_count] = {
+ &size,
+ &bytes_per_block,
+ &zone_size,
+ &num_zones,
+ &starting_zone_num};
+
+ for (size_t i = 0; i < key_count; i++) {
+ string val;
+ int r = cfg_reader(keys[i], &val);
+ if (r == 0) {
+ *(vals[i]) = strict_iecstrtoll(val.c_str(), &err);
+ if (!err.empty()) {
+ derr << __func__ << " Failed to parse - "
+ << keys[i] << ":" << val
+ << ", error: " << err << dendl;
+ return -EINVAL;
+ }
+ } else {
+ // this is expected for legacy deployed OSDs
+ dout(0) << __func__ << " " << keys[i] << " not found in bdev meta" << dendl;
+ return r;
+ }
+ }
+ return 0;
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+//
+// A freelist manager for zoned devices. This iteration just keeps the write
+// pointer per zone. Following iterations will add enough information to enable
+// cleaning of zones.
+//
+// Copyright (C) 2020 Abutalib Aghayev
+//
+
+#ifndef CEPH_OS_BLUESTORE_ZONEDFREELISTMANAGER_H
+#define CEPH_OS_BLUESTORE_ZONEDFREELISTMANAGER_H
+
+#include "FreelistManager.h"
+
+#include <string>
+#include <mutex>
+
+#include "common/ceph_mutex.h"
+#include "include/buffer.h"
+#include "kv/KeyValueDB.h"
+
+using cfg_reader_t = std::function<int(const std::string&, std::string*)>;
+
+class ZonedFreelistManager : public FreelistManager {
+ std::string meta_prefix; ///< device size, zone size, etc.
+ std::string info_prefix; ///< per zone write pointer, dead bytes
+ mutable ceph::mutex lock = ceph::make_mutex("ZonedFreelistManager::lock");
+
+ uint64_t size; ///< size of sequential region (bytes)
+ uint64_t bytes_per_block; ///< bytes per allocation unit (bytes)
+ uint64_t zone_size; ///< size of a single zone (bytes)
+ uint64_t num_zones; ///< number of sequential zones
+ uint64_t starting_zone_num; ///< the first sequential zone number
+
+ KeyValueDB::Iterator enumerate_p;
+ uint64_t enumerate_zone_num;
+
+ void write_zone_state_to_db(uint64_t zone_num,
+ const zone_state_t &zone_state,
+ KeyValueDB::Transaction txn);
+ void load_zone_state_from_db(uint64_t zone_num,
+ zone_state_t &zone_state,
+ KeyValueDB::Iterator &it) const;
+
+ void init_zone_states(KeyValueDB::Transaction txn);
+
+ void increment_write_pointer(
+ uint64_t zone, uint64_t length, KeyValueDB::Transaction txn);
+ void increment_num_dead_bytes(
+ uint64_t zone, uint64_t num_bytes, KeyValueDB::Transaction txn);
+
+ int _read_cfg(cfg_reader_t cfg_reader);
+
+public:
+ ZonedFreelistManager(CephContext* cct,
+ std::string meta_prefix,
+ std::string info_prefix);
+
+ static void setup_merge_operator(KeyValueDB *db, std::string prefix);
+
+ int create(uint64_t size,
+ uint64_t granularity,
+ KeyValueDB::Transaction txn) override;
+
+ int init(KeyValueDB *kvdb,
+ bool db_in_read_only,
+ cfg_reader_t cfg_reader) override;
+
+ void shutdown() override;
+ void sync(KeyValueDB* kvdb) override;
+ void dump(KeyValueDB *kvdb) override;
+
+ void enumerate_reset() override;
+ bool enumerate_next(KeyValueDB *kvdb,
+ uint64_t *offset,
+ uint64_t *length) override;
+
+ void allocate(uint64_t offset,
+ uint64_t length,
+ KeyValueDB::Transaction txn) override;
+
+ void release(uint64_t offset,
+ uint64_t length,
+ KeyValueDB::Transaction txn) override;
+
+ inline uint64_t get_size() const override {
+ return size;
+ }
+
+ inline uint64_t get_alloc_units() const override {
+ return size / bytes_per_block;
+ }
+
+ inline uint64_t get_alloc_size() const override {
+ return bytes_per_block;
+ }
+
+ void get_meta(uint64_t target_size,
+ std::vector<std::pair<string, string>>*) const override;
+
+ std::vector<zone_state_t> get_zone_states(KeyValueDB *kvdb) const override;
+};
+
+#endif
--- /dev/null
+#include "zoned_types.h"
+
+using ceph::decode;
+using ceph::encode;
+
+std::ostream& operator<<(std::ostream& out,
+ const zone_state_t& zone_state) {
+ return out << " zone: 0x" << std::hex
+ << " dead bytes: 0x" << zone_state.get_num_dead_bytes()
+ << " write pointer: 0x" << zone_state.get_write_pointer()
+ << " " << std::dec;
+}
+
+void zone_state_t::encode(ceph::buffer::list &bl) const {
+ uint64_t v = static_cast<uint64_t>(num_dead_bytes) << 32 | write_pointer;
+ ::encode(v, bl);
+}
+
+void zone_state_t::decode(ceph::buffer::list::const_iterator &p) {
+ uint64_t v;
+ ::decode(v, p);
+ num_dead_bytes = v >> 32;
+ write_pointer = v; // discard left-most 32 bits
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+#ifndef CEPH_OS_BLUESTORE_ZONED_TYPES_H
+#define CEPH_OS_BLUESTORE_ZONED_TYPES_H
+
+#include "include/types.h"
+#include "kv/KeyValueDB.h"
+#include "os/kv.h"
+
+// Tracks two bits of information about the state of a zone: (1) number of dead
+// bytes in a zone and (2) the write pointer. We assume that for now 32 bits is
+// enough for the zone capacity and represent these as uint32_t, and we store
+// them as a single 64-bit value in RocksDB so that we can use the existing
+// Int64ArrayMergeOperator for merge and avoid the cost of point queries.
+//
+// We use the same struct for an on-disk and in-memory representation of the
+// state.
+struct zone_state_t {
+ uint32_t num_dead_bytes = 0;
+ uint32_t write_pointer = 0;
+
+ void encode(ceph::buffer::list &bl) const;
+ void decode(ceph::buffer::list::const_iterator &p);
+
+ uint64_t get_num_dead_bytes() const {
+ return num_dead_bytes;
+ }
+
+ uint64_t get_write_pointer() const {
+ return write_pointer;
+ }
+
+ void increment_num_dead_bytes(uint64_t num_bytes) {
+ num_dead_bytes += num_bytes;
+ }
+
+ void increment_write_pointer(uint64_t num_bytes) {
+ write_pointer += num_bytes;
+ }
+};
+
+std::ostream& operator<<(std::ostream& out, const zone_state_t& zone_state);
+
+#endif