--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "Allocator.h"
+#include <bit>
+#include "StupidAllocator.h"
+#include "BitmapAllocator.h"
+#include "AvlAllocator.h"
+#include "BtreeAllocator.h"
+#include "Btree2Allocator.h"
+#include "HybridAllocator.h"
+#include "common/debug.h"
+#include "common/admin_socket.h"
+
+#define dout_subsys ceph_subsys_bluestore
+using TOPNSPC::common::cmd_getval;
+
+using std::string;
+using std::to_string;
+
+using ceph::bufferlist;
+using ceph::Formatter;
+
+class Allocator::SocketHook : public AdminSocketHook {
+ Allocator *alloc;
+
+ friend class Allocator;
+ std::string name;
+public:
+ SocketHook(Allocator *alloc, std::string_view _name) :
+ alloc(alloc), name(_name)
+ {
+ AdminSocket *admin_socket = g_ceph_context->get_admin_socket();
+ if (name.empty()) {
+ name = to_string((uintptr_t)this);
+ }
+ if (admin_socket) {
+ int r = admin_socket->register_command(
+ ("bluestore allocator dump " + name).c_str(),
+ this,
+ "dump allocator free regions");
+ if (r != 0)
+ alloc = nullptr; //some collision, disable
+ if (alloc) {
+ r = admin_socket->register_command(
+ ("bluestore allocator score " + name).c_str(),
+ this,
+ "give score on allocator fragmentation (0-no fragmentation, 1-absolute fragmentation)");
+ ceph_assert(r == 0);
+ r = admin_socket->register_command(
+ ("bluestore allocator fragmentation " + name).c_str(),
+ this,
+ "give allocator fragmentation (0-no fragmentation, 1-absolute fragmentation)");
+ ceph_assert(r == 0);
+ r = admin_socket->register_command(
+ ("bluestore allocator fragmentation histogram " + name +
+ " name=alloc_unit,type=CephInt,req=false" +
+ " name=num_buckets,type=CephInt,req=false").c_str(),
+ this,
+ "build allocator free regions state histogram");
+ ceph_assert(r == 0);
+ }
+ }
+ }
+ ~SocketHook()
+ {
+ AdminSocket *admin_socket = g_ceph_context->get_admin_socket();
+ if (admin_socket && alloc) {
+ admin_socket->unregister_commands(this);
+ }
+ }
+
+ int call(std::string_view command,
+ const cmdmap_t& cmdmap,
+ const bufferlist&,
+ Formatter *f,
+ std::ostream& ss,
+ bufferlist& out) override {
+ int r = 0;
+ if (command == "bluestore allocator dump " + name) {
+ f->open_object_section("allocator_dump");
+ f->dump_unsigned("capacity", alloc->get_capacity());
+ f->dump_unsigned("alloc_unit", alloc->get_block_size());
+ f->dump_string("alloc_type", alloc->get_type());
+ f->dump_string("alloc_name", name);
+
+ f->open_array_section("extents");
+ auto iterated_allocation = [&](size_t off, size_t len) {
+ ceph_assert(len > 0);
+ f->open_object_section("free");
+ char off_hex[30];
+ char len_hex[30];
+ snprintf(off_hex, sizeof(off_hex) - 1, "0x%zx", off);
+ snprintf(len_hex, sizeof(len_hex) - 1, "0x%zx", len);
+ f->dump_string("offset", off_hex);
+ f->dump_string("length", len_hex);
+ f->close_section();
+ };
+ alloc->foreach(iterated_allocation);
+ f->close_section();
+ f->close_section();
+ } else if (command == "bluestore allocator score " + name) {
+ f->open_object_section("fragmentation_score");
+ f->dump_float("fragmentation_rating", alloc->get_fragmentation_score());
+ f->close_section();
+ } else if (command == "bluestore allocator fragmentation " + name) {
+ f->open_object_section("fragmentation");
+ f->dump_float("fragmentation_rating", alloc->get_fragmentation());
+ f->close_section();
+ } else if (command == "bluestore allocator fragmentation histogram " + name) {
+ int64_t alloc_unit = alloc->get_block_size();
+ cmd_getval(cmdmap, "alloc_unit", alloc_unit);
+ if (alloc_unit <= 0 ||
+ p2align(alloc_unit, alloc->get_block_size()) != alloc_unit) {
+ ss << "Invalid allocation unit: '" << alloc_unit
+ << "', to be aligned with: '" << alloc->get_block_size()
+ << "'" << std::endl;
+ return -EINVAL;
+ }
+ int64_t num_buckets = 8;
+ cmd_getval(cmdmap, "num_buckets", num_buckets);
+ if (num_buckets < 2) {
+ ss << "Invalid amount of buckets (min=2): '" << num_buckets
+ << "'" << std::endl;
+ return -EINVAL;
+ }
+
+ Allocator::FreeStateHistogram hist(num_buckets);
+ alloc->foreach(
+ [&](size_t off, size_t len) {
+ hist.record_extent(uint64_t(alloc_unit), off, len);
+ });
+ f->open_array_section("extent_counts");
+ hist.foreach(
+ [&](uint64_t max_len, uint64_t total, uint64_t aligned, uint64_t units) {
+ f->open_object_section("c");
+ f->dump_unsigned("max_len", max_len);
+ f->dump_unsigned("total", total);
+ f->dump_unsigned("aligned", aligned);
+ f->dump_unsigned("units", units);
+ f->close_section();
+ }
+ );
+ f->close_section();
+ } else {
+ ss << "Invalid command" << std::endl;
+ r = -ENOSYS;
+ }
+ return r;
+ }
+
+};
+Allocator::Allocator(std::string_view name,
+ int64_t _capacity,
+ int64_t _block_size)
+ : device_size(_capacity),
+ block_size(_block_size)
+{
+ asok_hook = new SocketHook(this, name);
+}
+
+
+Allocator::~Allocator()
+{
+ delete asok_hook;
+}
+
+const string& Allocator::get_name() const {
+ return asok_hook->name;
+}
+
+Allocator *Allocator::create(
+ CephContext* cct,
+ std::string_view type,
+ int64_t size,
+ int64_t block_size,
+ std::string_view name)
+{
+ Allocator* alloc = nullptr;
+ if (type == "stupid") {
+ alloc = new StupidAllocator(cct, size, block_size, name);
+ } else if (type == "bitmap") {
+ alloc = new BitmapAllocator(cct, size, block_size, name);
+ } else if (type == "avl") {
+ return new AvlAllocator(cct, size, block_size, name);
+ } else if (type == "btree") {
+ return new BtreeAllocator(cct, size, block_size, name);
+ } else if (type == "hybrid") {
+ return new HybridAvlAllocator(cct, size, block_size,
+ cct->_conf.get_val<uint64_t>("bluestore_hybrid_alloc_mem_cap"),
+ name);
+ } else if (type == "hybrid_btree2") {
+ return new HybridBtree2Allocator(cct, size, block_size,
+ cct->_conf.get_val<uint64_t>("bluestore_hybrid_alloc_mem_cap"),
+ cct->_conf.get_val<double>("bluestore_btree2_alloc_weight_factor"),
+ name);
+ }
+ if (alloc == nullptr) {
+ lderr(cct) << "Allocator::" << __func__ << " unknown alloc type "
+ << type << dendl;
+ }
+ return alloc;
+}
+
+void Allocator::release(const PExtentVector& release_vec)
+{
+ release_set_t release_set;
+ for (auto e : release_vec) {
+ release_set.insert(e.offset, e.length);
+ }
+ release(release_set);
+}
+
+/**
+ * Gives fragmentation a numeric value.
+ *
+ * Following algorithm applies value to each existing free unallocated block.
+ * Value of single block is a multiply of size and per-byte-value.
+ * Per-byte-value is greater for larger blocks.
+ * Assume block size X has value per-byte p; then block size 2*X will have per-byte value 1.1*p.
+ *
+ * This could be expressed in logarithms, but for speed this is interpolated inside ranges.
+ * [1] [2..3] [4..7] [8..15] ...
+ * ^ ^ ^ ^
+ * 1.1 1.1^2 1.1^3 1.1^4 ...
+ *
+ * Final score is obtained by proportion between score that would have been obtained
+ * in condition of absolute fragmentation and score in no fragmentation at all.
+ */
+double Allocator::get_fragmentation_score()
+{
+ // this value represents how much worth is 2X bytes in one chunk then in X + X bytes
+ static const double double_size_worth_small = 1.2;
+ // chunks larger then 128MB are large enough that should be counted without penalty
+ static const double double_size_worth_huge = 1;
+ static const size_t small_chunk_p2 = 20; // 1MB
+ static const size_t huge_chunk_p2 = 27; // 128MB
+ // for chunks 1MB - 128MB penalty coeffs are linearly weighted 1.2 (at small) ... 1 (at huge)
+ static std::vector<double> scales{1};
+ double score_sum = 0;
+ size_t sum = 0;
+
+ auto get_score = [&](size_t v) -> double {
+ size_t sc = sizeof(v) * 8 - std::countl_zero(v) - 1; //assign to grade depending on log2(len)
+ while (scales.size() <= sc + 1) {
+ //unlikely expand scales vector
+ auto ss = scales.size();
+ double scale = double_size_worth_small;
+ if (ss >= huge_chunk_p2) {
+ scale = double_size_worth_huge;
+ } else if (ss > small_chunk_p2) {
+ // linear decrease 1.2 ... 1
+ scale = (double_size_worth_huge * (ss - small_chunk_p2) + double_size_worth_small * (huge_chunk_p2 - ss)) /
+ (huge_chunk_p2 - small_chunk_p2);
+ }
+ scales.push_back(scales[scales.size() - 1] * scale);
+ }
+ size_t sc_shifted = size_t(1) << sc;
+ double x = double(v - sc_shifted) / sc_shifted; //x is <0,1) in its scale grade
+ // linear extrapolation in its scale grade
+ double score = (sc_shifted ) * scales[sc] * (1-x) +
+ (sc_shifted * 2) * scales[sc+1] * x;
+ return score;
+ };
+
+ auto iterated_allocation = [&](size_t off, size_t len) {
+ ceph_assert(len > 0);
+ score_sum += get_score(len);
+ sum += len;
+ };
+ foreach(iterated_allocation);
+
+ double ideal = get_score(sum);
+ double terrible = (sum / block_size) * get_score(block_size);
+ return (ideal - score_sum) / (ideal - terrible);
+}
+
+/*************
+* Allocator::FreeStateHistogram
+*************/
+using std::function;
+
+void Allocator::FreeStateHistogram::record_extent(uint64_t alloc_unit,
+ uint64_t off,
+ uint64_t len)
+{
+ size_t idx = myTraits._get_bucket(len);
+ ceph_assert(idx < buckets.size());
+ ++buckets[idx].total;
+
+ // now calculate the bucket for the chunk after alignment,
+ // resulting chunks shorter than alloc_unit are discarded
+ auto delta = p2roundup(off, alloc_unit) - off;
+ if (len >= delta + alloc_unit) {
+ len -= delta;
+ idx = myTraits._get_bucket(len);
+ ceph_assert(idx < buckets.size());
+ ++buckets[idx].aligned;
+ buckets[idx].alloc_units += len / alloc_unit;
+ }
+}
+void Allocator::FreeStateHistogram::foreach(
+ function<void(uint64_t max_len,
+ uint64_t total,
+ uint64_t aligned,
+ uint64_t unit)> cb)
+{
+ size_t i = 0;
+ for (const auto& b : buckets) {
+ cb(myTraits._get_bucket_max(i),
+ b.total, b.aligned, b.alloc_units);
+ ++i;
+ }
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+#ifndef CEPH_OS_BLUESTORE_ALLOCATOR_H
+#define CEPH_OS_BLUESTORE_ALLOCATOR_H
+
+#include <functional>
+#include <ostream>
+#include "include/ceph_assert.h"
+#include "bluestore_types.h"
+#include "common/ceph_mutex.h"
+
+typedef interval_set<uint64_t> release_set_t;
+typedef release_set_t::value_type release_set_entry_t;
+
+class Allocator {
+protected:
+
+ /**
+ * This is a base set of traits for logical placing entries
+ * into limited collection of buckets depending on their sizes.
+ * Descandants should implement get_bucket(len) method to obtain
+ * bucket index using entry length.
+ */
+ struct LenPartitionedSetTraits {
+ size_t num_buckets;
+ size_t base_bits; // bits in min entry size
+ size_t base; // min entry size
+ size_t factor; // additional factor to be applied
+ // to entry size when calculating
+ // target bucket
+
+
+ LenPartitionedSetTraits(size_t _num_buckets,
+ size_t _base_bits = 12, //= 4096 bytes
+ size_t _factor = 1) :
+ num_buckets(_num_buckets),
+ base_bits(_base_bits),
+ base(1ull << base_bits),
+ factor(_factor)
+ {
+ ceph_assert(factor);
+ }
+ };
+
+ /**
+ * This extends LenPartitionedSetTraits to implement linear bucket indexing:
+ * bucket index to be determined as entry's size divided by (base * factor),
+ * i.e. buckets are:
+ * [0..base)
+ * [base, base+base*factor)
+ * [base+base*factor, base+base*factor*2)
+ * [base+base*factor*2, base+base*factor*3)
+ * ...
+ */
+ struct LenPartitionedSetTraitsLinear : public LenPartitionedSetTraits {
+ using LenPartitionedSetTraits::LenPartitionedSetTraits;
+ /*
+ * Determines bucket index for a given extent's length in a bucket set
+ * with linear (len / base / factor) indexing.
+ * The first bucket is targeted for lengths < base,
+ * the last bucket is used for lengths above the maximum
+ * detemined by bucket count.
+ */
+ inline size_t _get_bucket(uint64_t len) const {
+ size_t idx = (len / factor) >> base_bits;
+ idx = idx < num_buckets ? idx : num_buckets - 1;
+ return idx;
+ }
+ /*
+ * returns upper bound of a specific bucket
+ */
+ inline size_t _get_bucket_max(size_t bucket) const {
+ return
+ bucket < num_buckets - 1 ?
+ base * factor * (1 + bucket) :
+ std::numeric_limits<uint64_t>::max();
+ }
+ };
+
+ /**
+ * This extends LenPartitionedSetTraits to implement exponential bucket indexing:
+ * target bucket bounds are determined as
+ * [0, base]
+ * (base, base*2^factor]
+ * (base*2^factor, base*2^(factor*2)]
+ * (base*2^(factor*2), base*2^(factor*3)]
+ * ...
+ *
+ */
+ struct LenPartitionedSetTraitsPow2 : public LenPartitionedSetTraits {
+ /*
+ * Determines bucket index for a given extent's length in a bucket collection
+ * with log2(len) indexing.
+ * The first bucket is targeted for lengths < base,
+ * The last bucket index is used for lengths above the maximum
+ * detemined by bucket count.
+ */
+ using LenPartitionedSetTraits::LenPartitionedSetTraits;
+ inline size_t _get_bucket(uint64_t len) const {
+ size_t idx;
+ const size_t len_p2_max =
+ base << ((factor * (num_buckets - 2)));
+ if (len <= base) {
+ idx = 0;
+ } else if (len > len_p2_max) {
+ idx = num_buckets - 1;
+ } else {
+ size_t most_bit = cbits(uint64_t(len - 1)) - 1;
+ idx = 1 + ((most_bit - base_bits) / factor);
+ }
+ ceph_assert(idx < num_buckets);
+ return idx;
+ }
+ /*
+ * returns upper bound of the bucket with log2(len) indexing.
+ */
+ inline size_t _get_bucket_max(size_t bucket) const {
+ return
+ bucket < num_buckets - 1 ?
+ base << (factor * bucket) :
+ std::numeric_limits<uint64_t>::max();
+ }
+ };
+
+ /*
+ * Lockless stack implementation
+ * that permits put/get operation exclusively
+ * if no waiting is needed.
+ * Conflicting operations are omitted.
+ */
+ class LocklessOpportunisticStack {
+ std::atomic<size_t> ref = 0;
+ std::atomic<size_t> count = 0;
+ std::vector<uint64_t> data;
+ public:
+ void init(size_t size) {
+ data.resize(size);
+ }
+ bool try_put(uint64_t& v) {
+ bool done = ++ref == 1 && count < data.size();
+ if (done) {
+ data[count++] = v;
+ }
+ --ref;
+ return done;
+ }
+ bool try_get(uint64_t& v) {
+ bool done = ++ref == 1 && count > 0;
+ if (done) {
+ v = data[--count];
+ }
+ --ref;
+ return done;
+ }
+ void foreach(std::function<void(uint64_t)> notify) {
+ for (size_t i = 0; i < count; i++) {
+ notify(data[i]);
+ }
+ }
+ };
+ /*
+ * Concurrently accessed extent (offset,length) cache
+ * which permits put/get operation exclusively if no waiting is needed.
+ * Implemented via a set of independent buckets (aka LocklessOpportunisticStack).
+ * Each bucket keeps extents of specific size only: 4K, 8K, 12K...64K
+ * which allows to avoid individual extent size tracking.
+ * Each bucket permits a single operation at a given time only,
+ * additional operations against the bucket are rejected meaning relevant
+ * extents aren't not cached.
+ */
+ class OpportunisticExtentCache {
+ const LenPartitionedSetTraitsLinear myTraits;
+ enum {
+ BUCKET_COUNT = 16,
+ EXTENTS_PER_BUCKET = 16, // amount of entries per single bucket,
+ // total amount of entries will be
+ // BUCKET_COUNT * EXTENTS_PER_BUCKET.
+ };
+
+ std::vector<LocklessOpportunisticStack> buckets;
+ std::atomic<size_t> hits = 0;
+ ceph::shared_mutex lock{
+ ceph::make_shared_mutex(std::string(), false, false, false)
+ };
+ public:
+ OpportunisticExtentCache() :
+ myTraits(BUCKET_COUNT + 1), // 16 regular buckets + 1 "catch-all" pseudo
+ // one to be used for out-of-bound checking
+ // since _get_*_size_bucket() methods imply
+ // the last bucket usage for the entries
+ // exceeding the max length.
+ buckets(BUCKET_COUNT)
+ {
+ //buckets.resize(BUCKET_COUNT);
+ for(auto& b : buckets) {
+ b.init(EXTENTS_PER_BUCKET);
+ }
+ }
+ bool try_put(uint64_t offset, uint64_t len) {
+ if (!lock.try_lock_shared()) {
+ return false;
+ }
+ bool ret = false;
+ ceph_assert(p2aligned(offset, myTraits.base));
+ ceph_assert(p2aligned(len, myTraits.base));
+ auto idx = myTraits._get_bucket(len);
+ if (idx < buckets.size())
+ ret = buckets[idx].try_put(offset);
+ lock.unlock_shared();
+ return ret;
+ }
+ bool try_get(uint64_t* offset, uint64_t len) {
+ if (!lock.try_lock_shared()) {
+ return false;
+ }
+ bool ret = false;
+ ceph_assert(offset);
+ ceph_assert(p2aligned(len, myTraits.base));
+ size_t idx = len >> myTraits.base_bits;
+ if (idx < buckets.size()) {
+ ret = buckets[idx].try_get(*offset);
+ if (ret) {
+ ++hits;
+ }
+ }
+ lock.unlock_shared();
+ return ret;
+ }
+ size_t get_hit_count() const {
+ return hits.load();
+ }
+ void foreach(std::function<void(uint64_t offset, uint64_t length)> notify) {
+ std::unique_lock _lock(lock);
+ for (uint64_t i = 0; i < buckets.size(); i++) {
+ auto cb = [&](uint64_t o) {
+ notify(o, i << myTraits.base_bits);
+ };
+ buckets[i].foreach(cb);
+ }
+ }
+ };
+
+public:
+ Allocator(std::string_view name,
+ int64_t _capacity,
+ int64_t _block_size);
+ virtual ~Allocator();
+
+ /*
+ * returns allocator type name as per names in config
+ */
+ virtual const char* get_type() const = 0;
+
+ /*
+ * Allocate required number of blocks in n number of extents.
+ * Min and Max number of extents are limited by:
+ * a. alloc unit
+ * b. max_alloc_size.
+ * as no extent can be lesser than block_size and greater than max_alloc size.
+ * Apart from that extents can vary between these lower and higher limits according
+ * to free block search algorithm and availability of contiguous space.
+ */
+ virtual int64_t allocate(uint64_t want_size, uint64_t block_size,
+ uint64_t max_alloc_size, int64_t hint,
+ PExtentVector *extents) = 0;
+
+ int64_t allocate(uint64_t want_size, uint64_t block_size,
+ int64_t hint, PExtentVector *extents) {
+ return allocate(want_size, block_size, want_size, hint, extents);
+ }
+
+ /* Bulk release. Implementations may override this method to handle the whole
+ * set at once. This could save e.g. unnecessary mutex dance. */
+ virtual void release(const release_set_t& release_set) = 0;
+ void release(const PExtentVector& release_set);
+
+ virtual void dump() = 0;
+ virtual void foreach(
+ std::function<void(uint64_t offset, uint64_t length)> notify) = 0;
+
+ virtual void init_add_free(uint64_t offset, uint64_t length) = 0;
+ virtual void init_rm_free(uint64_t offset, uint64_t length) = 0;
+
+ virtual uint64_t get_free() = 0;
+ virtual double get_fragmentation()
+ {
+ return 0.0;
+ }
+ virtual double get_fragmentation_score();
+ virtual void shutdown() = 0;
+
+ static Allocator *create(
+ CephContext* cct,
+ std::string_view type,
+ int64_t size,
+ int64_t block_size,
+ const std::string_view name = ""
+ );
+
+
+ const std::string& get_name() const;
+ int64_t get_capacity() const
+ {
+ return device_size;
+ }
+ int64_t get_block_size() const
+ {
+ return block_size;
+ }
+
+ // The following class implements Allocator's free extents histogram.
+ // Which is a set of N buckets to track extents layout.
+ // Extent matches a bucket depending on its length using the following
+ // length spans:
+ // [0..4K] (4K..16K] (16K..64K] .. (4M..16M] (16M..]
+ // Each bucket tracks:
+ // - total amount of extents of specific lengths
+ // - amount of extents aligned with allocation boundary
+ // - amount of allocation units in aligned extents
+ //
+ class FreeStateHistogram {
+ const LenPartitionedSetTraitsPow2 myTraits;
+ enum {
+ BASE_BITS = 12, // 4096 bytes
+ FACTOR = 2,
+ };
+ struct free_state_hist_bucket {
+ size_t total = 0;
+ size_t aligned = 0;
+ size_t alloc_units = 0;
+ };
+ std::vector<free_state_hist_bucket> buckets;
+ public:
+
+ FreeStateHistogram(size_t num_buckets)
+ : myTraits(num_buckets, BASE_BITS, FACTOR) {
+ buckets.resize(num_buckets);
+ }
+
+ void record_extent(uint64_t alloc_unit, uint64_t off, uint64_t len);
+ void foreach(
+ std::function<void(uint64_t, uint64_t, uint64_t, uint64_t)> cb);
+ };
+
+private:
+ class SocketHook;
+ SocketHook* asok_hook = nullptr;
+protected:
+ const int64_t device_size = 0;
+ const int64_t block_size = 0;
+};
+
+#endif
\ No newline at end of file
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "Allocator.h"
-#include <bit>
-#include "StupidAllocator.h"
-#include "BitmapAllocator.h"
-#include "AvlAllocator.h"
-#include "BtreeAllocator.h"
-#include "Btree2Allocator.h"
-#include "HybridAllocator.h"
-#include "common/debug.h"
-#include "common/admin_socket.h"
-
-#define dout_subsys ceph_subsys_bluestore
-using TOPNSPC::common::cmd_getval;
-
-using std::string;
-using std::to_string;
-
-using ceph::bufferlist;
-using ceph::Formatter;
-
-class Allocator::SocketHook : public AdminSocketHook {
- Allocator *alloc;
-
- friend class Allocator;
- std::string name;
-public:
- SocketHook(Allocator *alloc, std::string_view _name) :
- alloc(alloc), name(_name)
- {
- AdminSocket *admin_socket = g_ceph_context->get_admin_socket();
- if (name.empty()) {
- name = to_string((uintptr_t)this);
- }
- if (admin_socket) {
- int r = admin_socket->register_command(
- ("bluestore allocator dump " + name).c_str(),
- this,
- "dump allocator free regions");
- if (r != 0)
- alloc = nullptr; //some collision, disable
- if (alloc) {
- r = admin_socket->register_command(
- ("bluestore allocator score " + name).c_str(),
- this,
- "give score on allocator fragmentation (0-no fragmentation, 1-absolute fragmentation)");
- ceph_assert(r == 0);
- r = admin_socket->register_command(
- ("bluestore allocator fragmentation " + name).c_str(),
- this,
- "give allocator fragmentation (0-no fragmentation, 1-absolute fragmentation)");
- ceph_assert(r == 0);
- r = admin_socket->register_command(
- ("bluestore allocator fragmentation histogram " + name +
- " name=alloc_unit,type=CephInt,req=false" +
- " name=num_buckets,type=CephInt,req=false").c_str(),
- this,
- "build allocator free regions state histogram");
- ceph_assert(r == 0);
- }
- }
- }
- ~SocketHook()
- {
- AdminSocket *admin_socket = g_ceph_context->get_admin_socket();
- if (admin_socket && alloc) {
- admin_socket->unregister_commands(this);
- }
- }
-
- int call(std::string_view command,
- const cmdmap_t& cmdmap,
- const bufferlist&,
- Formatter *f,
- std::ostream& ss,
- bufferlist& out) override {
- int r = 0;
- if (command == "bluestore allocator dump " + name) {
- f->open_object_section("allocator_dump");
- f->dump_unsigned("capacity", alloc->get_capacity());
- f->dump_unsigned("alloc_unit", alloc->get_block_size());
- f->dump_string("alloc_type", alloc->get_type());
- f->dump_string("alloc_name", name);
-
- f->open_array_section("extents");
- auto iterated_allocation = [&](size_t off, size_t len) {
- ceph_assert(len > 0);
- f->open_object_section("free");
- char off_hex[30];
- char len_hex[30];
- snprintf(off_hex, sizeof(off_hex) - 1, "0x%zx", off);
- snprintf(len_hex, sizeof(len_hex) - 1, "0x%zx", len);
- f->dump_string("offset", off_hex);
- f->dump_string("length", len_hex);
- f->close_section();
- };
- alloc->foreach(iterated_allocation);
- f->close_section();
- f->close_section();
- } else if (command == "bluestore allocator score " + name) {
- f->open_object_section("fragmentation_score");
- f->dump_float("fragmentation_rating", alloc->get_fragmentation_score());
- f->close_section();
- } else if (command == "bluestore allocator fragmentation " + name) {
- f->open_object_section("fragmentation");
- f->dump_float("fragmentation_rating", alloc->get_fragmentation());
- f->close_section();
- } else if (command == "bluestore allocator fragmentation histogram " + name) {
- int64_t alloc_unit = alloc->get_block_size();
- cmd_getval(cmdmap, "alloc_unit", alloc_unit);
- if (alloc_unit <= 0 ||
- p2align(alloc_unit, alloc->get_block_size()) != alloc_unit) {
- ss << "Invalid allocation unit: '" << alloc_unit
- << "', to be aligned with: '" << alloc->get_block_size()
- << "'" << std::endl;
- return -EINVAL;
- }
- int64_t num_buckets = 8;
- cmd_getval(cmdmap, "num_buckets", num_buckets);
- if (num_buckets < 2) {
- ss << "Invalid amount of buckets (min=2): '" << num_buckets
- << "'" << std::endl;
- return -EINVAL;
- }
-
- Allocator::FreeStateHistogram hist(num_buckets);
- alloc->foreach(
- [&](size_t off, size_t len) {
- hist.record_extent(uint64_t(alloc_unit), off, len);
- });
- f->open_array_section("extent_counts");
- hist.foreach(
- [&](uint64_t max_len, uint64_t total, uint64_t aligned, uint64_t units) {
- f->open_object_section("c");
- f->dump_unsigned("max_len", max_len);
- f->dump_unsigned("total", total);
- f->dump_unsigned("aligned", aligned);
- f->dump_unsigned("units", units);
- f->close_section();
- }
- );
- f->close_section();
- } else {
- ss << "Invalid command" << std::endl;
- r = -ENOSYS;
- }
- return r;
- }
-
-};
-Allocator::Allocator(std::string_view name,
- int64_t _capacity,
- int64_t _block_size)
- : device_size(_capacity),
- block_size(_block_size)
-{
- asok_hook = new SocketHook(this, name);
-}
-
-
-Allocator::~Allocator()
-{
- delete asok_hook;
-}
-
-const string& Allocator::get_name() const {
- return asok_hook->name;
-}
-
-Allocator *Allocator::create(
- CephContext* cct,
- std::string_view type,
- int64_t size,
- int64_t block_size,
- std::string_view name)
-{
- Allocator* alloc = nullptr;
- if (type == "stupid") {
- alloc = new StupidAllocator(cct, size, block_size, name);
- } else if (type == "bitmap") {
- alloc = new BitmapAllocator(cct, size, block_size, name);
- } else if (type == "avl") {
- return new AvlAllocator(cct, size, block_size, name);
- } else if (type == "btree") {
- return new BtreeAllocator(cct, size, block_size, name);
- } else if (type == "hybrid") {
- return new HybridAvlAllocator(cct, size, block_size,
- cct->_conf.get_val<uint64_t>("bluestore_hybrid_alloc_mem_cap"),
- name);
- } else if (type == "hybrid_btree2") {
- return new HybridBtree2Allocator(cct, size, block_size,
- cct->_conf.get_val<uint64_t>("bluestore_hybrid_alloc_mem_cap"),
- cct->_conf.get_val<double>("bluestore_btree2_alloc_weight_factor"),
- name);
- }
- if (alloc == nullptr) {
- lderr(cct) << "Allocator::" << __func__ << " unknown alloc type "
- << type << dendl;
- }
- return alloc;
-}
-
-void Allocator::release(const PExtentVector& release_vec)
-{
- release_set_t release_set;
- for (auto e : release_vec) {
- release_set.insert(e.offset, e.length);
- }
- release(release_set);
-}
-
-/**
- * Gives fragmentation a numeric value.
- *
- * Following algorithm applies value to each existing free unallocated block.
- * Value of single block is a multiply of size and per-byte-value.
- * Per-byte-value is greater for larger blocks.
- * Assume block size X has value per-byte p; then block size 2*X will have per-byte value 1.1*p.
- *
- * This could be expressed in logarithms, but for speed this is interpolated inside ranges.
- * [1] [2..3] [4..7] [8..15] ...
- * ^ ^ ^ ^
- * 1.1 1.1^2 1.1^3 1.1^4 ...
- *
- * Final score is obtained by proportion between score that would have been obtained
- * in condition of absolute fragmentation and score in no fragmentation at all.
- */
-double Allocator::get_fragmentation_score()
-{
- // this value represents how much worth is 2X bytes in one chunk then in X + X bytes
- static const double double_size_worth_small = 1.2;
- // chunks larger then 128MB are large enough that should be counted without penalty
- static const double double_size_worth_huge = 1;
- static const size_t small_chunk_p2 = 20; // 1MB
- static const size_t huge_chunk_p2 = 27; // 128MB
- // for chunks 1MB - 128MB penalty coeffs are linearly weighted 1.2 (at small) ... 1 (at huge)
- static std::vector<double> scales{1};
- double score_sum = 0;
- size_t sum = 0;
-
- auto get_score = [&](size_t v) -> double {
- size_t sc = sizeof(v) * 8 - std::countl_zero(v) - 1; //assign to grade depending on log2(len)
- while (scales.size() <= sc + 1) {
- //unlikely expand scales vector
- auto ss = scales.size();
- double scale = double_size_worth_small;
- if (ss >= huge_chunk_p2) {
- scale = double_size_worth_huge;
- } else if (ss > small_chunk_p2) {
- // linear decrease 1.2 ... 1
- scale = (double_size_worth_huge * (ss - small_chunk_p2) + double_size_worth_small * (huge_chunk_p2 - ss)) /
- (huge_chunk_p2 - small_chunk_p2);
- }
- scales.push_back(scales[scales.size() - 1] * scale);
- }
- size_t sc_shifted = size_t(1) << sc;
- double x = double(v - sc_shifted) / sc_shifted; //x is <0,1) in its scale grade
- // linear extrapolation in its scale grade
- double score = (sc_shifted ) * scales[sc] * (1-x) +
- (sc_shifted * 2) * scales[sc+1] * x;
- return score;
- };
-
- auto iterated_allocation = [&](size_t off, size_t len) {
- ceph_assert(len > 0);
- score_sum += get_score(len);
- sum += len;
- };
- foreach(iterated_allocation);
-
- double ideal = get_score(sum);
- double terrible = (sum / block_size) * get_score(block_size);
- return (ideal - score_sum) / (ideal - terrible);
-}
-
-/*************
-* Allocator::FreeStateHistogram
-*************/
-using std::function;
-
-void Allocator::FreeStateHistogram::record_extent(uint64_t alloc_unit,
- uint64_t off,
- uint64_t len)
-{
- size_t idx = myTraits._get_bucket(len);
- ceph_assert(idx < buckets.size());
- ++buckets[idx].total;
-
- // now calculate the bucket for the chunk after alignment,
- // resulting chunks shorter than alloc_unit are discarded
- auto delta = p2roundup(off, alloc_unit) - off;
- if (len >= delta + alloc_unit) {
- len -= delta;
- idx = myTraits._get_bucket(len);
- ceph_assert(idx < buckets.size());
- ++buckets[idx].aligned;
- buckets[idx].alloc_units += len / alloc_unit;
- }
-}
-void Allocator::FreeStateHistogram::foreach(
- function<void(uint64_t max_len,
- uint64_t total,
- uint64_t aligned,
- uint64_t unit)> cb)
-{
- size_t i = 0;
- for (const auto& b : buckets) {
- cb(myTraits._get_bucket_max(i),
- b.total, b.aligned, b.alloc_units);
- ++i;
- }
-}
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-#ifndef CEPH_OS_BLUESTORE_ALLOCATOR_H
-#define CEPH_OS_BLUESTORE_ALLOCATOR_H
-
-#include <functional>
-#include <ostream>
-#include "include/ceph_assert.h"
-#include "bluestore_types.h"
-#include "common/ceph_mutex.h"
-
-typedef interval_set<uint64_t> release_set_t;
-typedef release_set_t::value_type release_set_entry_t;
-
-class Allocator {
-protected:
-
- /**
- * This is a base set of traits for logical placing entries
- * into limited collection of buckets depending on their sizes.
- * Descandants should implement get_bucket(len) method to obtain
- * bucket index using entry length.
- */
- struct LenPartitionedSetTraits {
- size_t num_buckets;
- size_t base_bits; // bits in min entry size
- size_t base; // min entry size
- size_t factor; // additional factor to be applied
- // to entry size when calculating
- // target bucket
-
-
- LenPartitionedSetTraits(size_t _num_buckets,
- size_t _base_bits = 12, //= 4096 bytes
- size_t _factor = 1) :
- num_buckets(_num_buckets),
- base_bits(_base_bits),
- base(1ull << base_bits),
- factor(_factor)
- {
- ceph_assert(factor);
- }
- };
-
- /**
- * This extends LenPartitionedSetTraits to implement linear bucket indexing:
- * bucket index to be determined as entry's size divided by (base * factor),
- * i.e. buckets are:
- * [0..base)
- * [base, base+base*factor)
- * [base+base*factor, base+base*factor*2)
- * [base+base*factor*2, base+base*factor*3)
- * ...
- */
- struct LenPartitionedSetTraitsLinear : public LenPartitionedSetTraits {
- using LenPartitionedSetTraits::LenPartitionedSetTraits;
- /*
- * Determines bucket index for a given extent's length in a bucket set
- * with linear (len / base / factor) indexing.
- * The first bucket is targeted for lengths < base,
- * the last bucket is used for lengths above the maximum
- * detemined by bucket count.
- */
- inline size_t _get_bucket(uint64_t len) const {
- size_t idx = (len / factor) >> base_bits;
- idx = idx < num_buckets ? idx : num_buckets - 1;
- return idx;
- }
- /*
- * returns upper bound of a specific bucket
- */
- inline size_t _get_bucket_max(size_t bucket) const {
- return
- bucket < num_buckets - 1 ?
- base * factor * (1 + bucket) :
- std::numeric_limits<uint64_t>::max();
- }
- };
-
- /**
- * This extends LenPartitionedSetTraits to implement exponential bucket indexing:
- * target bucket bounds are determined as
- * [0, base]
- * (base, base*2^factor]
- * (base*2^factor, base*2^(factor*2)]
- * (base*2^(factor*2), base*2^(factor*3)]
- * ...
- *
- */
- struct LenPartitionedSetTraitsPow2 : public LenPartitionedSetTraits {
- /*
- * Determines bucket index for a given extent's length in a bucket collection
- * with log2(len) indexing.
- * The first bucket is targeted for lengths < base,
- * The last bucket index is used for lengths above the maximum
- * detemined by bucket count.
- */
- using LenPartitionedSetTraits::LenPartitionedSetTraits;
- inline size_t _get_bucket(uint64_t len) const {
- size_t idx;
- const size_t len_p2_max =
- base << ((factor * (num_buckets - 2)));
- if (len <= base) {
- idx = 0;
- } else if (len > len_p2_max) {
- idx = num_buckets - 1;
- } else {
- size_t most_bit = cbits(uint64_t(len - 1)) - 1;
- idx = 1 + ((most_bit - base_bits) / factor);
- }
- ceph_assert(idx < num_buckets);
- return idx;
- }
- /*
- * returns upper bound of the bucket with log2(len) indexing.
- */
- inline size_t _get_bucket_max(size_t bucket) const {
- return
- bucket < num_buckets - 1 ?
- base << (factor * bucket) :
- std::numeric_limits<uint64_t>::max();
- }
- };
-
- /*
- * Lockless stack implementation
- * that permits put/get operation exclusively
- * if no waiting is needed.
- * Conflicting operations are omitted.
- */
- class LocklessOpportunisticStack {
- std::atomic<size_t> ref = 0;
- std::atomic<size_t> count = 0;
- std::vector<uint64_t> data;
- public:
- void init(size_t size) {
- data.resize(size);
- }
- bool try_put(uint64_t& v) {
- bool done = ++ref == 1 && count < data.size();
- if (done) {
- data[count++] = v;
- }
- --ref;
- return done;
- }
- bool try_get(uint64_t& v) {
- bool done = ++ref == 1 && count > 0;
- if (done) {
- v = data[--count];
- }
- --ref;
- return done;
- }
- void foreach(std::function<void(uint64_t)> notify) {
- for (size_t i = 0; i < count; i++) {
- notify(data[i]);
- }
- }
- };
- /*
- * Concurrently accessed extent (offset,length) cache
- * which permits put/get operation exclusively if no waiting is needed.
- * Implemented via a set of independent buckets (aka LocklessOpportunisticStack).
- * Each bucket keeps extents of specific size only: 4K, 8K, 12K...64K
- * which allows to avoid individual extent size tracking.
- * Each bucket permits a single operation at a given time only,
- * additional operations against the bucket are rejected meaning relevant
- * extents aren't not cached.
- */
- class OpportunisticExtentCache {
- const LenPartitionedSetTraitsLinear myTraits;
- enum {
- BUCKET_COUNT = 16,
- EXTENTS_PER_BUCKET = 16, // amount of entries per single bucket,
- // total amount of entries will be
- // BUCKET_COUNT * EXTENTS_PER_BUCKET.
- };
-
- std::vector<LocklessOpportunisticStack> buckets;
- std::atomic<size_t> hits = 0;
- ceph::shared_mutex lock{
- ceph::make_shared_mutex(std::string(), false, false, false)
- };
- public:
- OpportunisticExtentCache() :
- myTraits(BUCKET_COUNT + 1), // 16 regular buckets + 1 "catch-all" pseudo
- // one to be used for out-of-bound checking
- // since _get_*_size_bucket() methods imply
- // the last bucket usage for the entries
- // exceeding the max length.
- buckets(BUCKET_COUNT)
- {
- //buckets.resize(BUCKET_COUNT);
- for(auto& b : buckets) {
- b.init(EXTENTS_PER_BUCKET);
- }
- }
- bool try_put(uint64_t offset, uint64_t len) {
- if (!lock.try_lock_shared()) {
- return false;
- }
- bool ret = false;
- ceph_assert(p2aligned(offset, myTraits.base));
- ceph_assert(p2aligned(len, myTraits.base));
- auto idx = myTraits._get_bucket(len);
- if (idx < buckets.size())
- ret = buckets[idx].try_put(offset);
- lock.unlock_shared();
- return ret;
- }
- bool try_get(uint64_t* offset, uint64_t len) {
- if (!lock.try_lock_shared()) {
- return false;
- }
- bool ret = false;
- ceph_assert(offset);
- ceph_assert(p2aligned(len, myTraits.base));
- size_t idx = len >> myTraits.base_bits;
- if (idx < buckets.size()) {
- ret = buckets[idx].try_get(*offset);
- if (ret) {
- ++hits;
- }
- }
- lock.unlock_shared();
- return ret;
- }
- size_t get_hit_count() const {
- return hits.load();
- }
- void foreach(std::function<void(uint64_t offset, uint64_t length)> notify) {
- std::unique_lock _lock(lock);
- for (uint64_t i = 0; i < buckets.size(); i++) {
- auto cb = [&](uint64_t o) {
- notify(o, i << myTraits.base_bits);
- };
- buckets[i].foreach(cb);
- }
- }
- };
-
-public:
- Allocator(std::string_view name,
- int64_t _capacity,
- int64_t _block_size);
- virtual ~Allocator();
-
- /*
- * returns allocator type name as per names in config
- */
- virtual const char* get_type() const = 0;
-
- /*
- * Allocate required number of blocks in n number of extents.
- * Min and Max number of extents are limited by:
- * a. alloc unit
- * b. max_alloc_size.
- * as no extent can be lesser than block_size and greater than max_alloc size.
- * Apart from that extents can vary between these lower and higher limits according
- * to free block search algorithm and availability of contiguous space.
- */
- virtual int64_t allocate(uint64_t want_size, uint64_t block_size,
- uint64_t max_alloc_size, int64_t hint,
- PExtentVector *extents) = 0;
-
- int64_t allocate(uint64_t want_size, uint64_t block_size,
- int64_t hint, PExtentVector *extents) {
- return allocate(want_size, block_size, want_size, hint, extents);
- }
-
- /* Bulk release. Implementations may override this method to handle the whole
- * set at once. This could save e.g. unnecessary mutex dance. */
- virtual void release(const release_set_t& release_set) = 0;
- void release(const PExtentVector& release_set);
-
- virtual void dump() = 0;
- virtual void foreach(
- std::function<void(uint64_t offset, uint64_t length)> notify) = 0;
-
- virtual void init_add_free(uint64_t offset, uint64_t length) = 0;
- virtual void init_rm_free(uint64_t offset, uint64_t length) = 0;
-
- virtual uint64_t get_free() = 0;
- virtual double get_fragmentation()
- {
- return 0.0;
- }
- virtual double get_fragmentation_score();
- virtual void shutdown() = 0;
-
- static Allocator *create(
- CephContext* cct,
- std::string_view type,
- int64_t size,
- int64_t block_size,
- const std::string_view name = ""
- );
-
-
- const std::string& get_name() const;
- int64_t get_capacity() const
- {
- return device_size;
- }
- int64_t get_block_size() const
- {
- return block_size;
- }
-
- // The following class implements Allocator's free extents histogram.
- // Which is a set of N buckets to track extents layout.
- // Extent matches a bucket depending on its length using the following
- // length spans:
- // [0..4K] (4K..16K] (16K..64K] .. (4M..16M] (16M..]
- // Each bucket tracks:
- // - total amount of extents of specific lengths
- // - amount of extents aligned with allocation boundary
- // - amount of allocation units in aligned extents
- //
- class FreeStateHistogram {
- const LenPartitionedSetTraitsPow2 myTraits;
- enum {
- BASE_BITS = 12, // 4096 bytes
- FACTOR = 2,
- };
- struct free_state_hist_bucket {
- size_t total = 0;
- size_t aligned = 0;
- size_t alloc_units = 0;
- };
- std::vector<free_state_hist_bucket> buckets;
- public:
-
- FreeStateHistogram(size_t num_buckets)
- : myTraits(num_buckets, BASE_BITS, FACTOR) {
- buckets.resize(num_buckets);
- }
-
- void record_extent(uint64_t alloc_unit, uint64_t off, uint64_t len);
- void foreach(
- std::function<void(uint64_t, uint64_t, uint64_t, uint64_t)> cb);
- };
-
-private:
- class SocketHook;
- SocketHook* asok_hook = nullptr;
-protected:
- const int64_t device_size = 0;
- const int64_t block_size = 0;
-};
-
-#endif
\ No newline at end of file
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "Allocator.h"
+#include <bit>
+#include "StupidAllocator.h"
+#include "BitmapAllocator.h"
+#include "AvlAllocator.h"
+#include "BtreeAllocator.h"
+#include "Btree2Allocator.h"
+#include "HybridAllocator.h"
+#include "common/debug.h"
+#include "common/admin_socket.h"
+
+#define dout_subsys ceph_subsys_bluestore
+using TOPNSPC::common::cmd_getval;
+
+using std::string;
+using std::to_string;
+
+using ceph::bufferlist;
+using ceph::Formatter;
+
+class Allocator::SocketHook : public AdminSocketHook {
+ Allocator *alloc;
+
+ friend class Allocator;
+ std::string name;
+public:
+ SocketHook(Allocator *alloc, std::string_view _name) :
+ alloc(alloc), name(_name)
+ {
+ AdminSocket *admin_socket = g_ceph_context->get_admin_socket();
+ if (name.empty()) {
+ name = to_string((uintptr_t)this);
+ }
+ if (admin_socket) {
+ int r = admin_socket->register_command(
+ ("bluestore allocator dump " + name).c_str(),
+ this,
+ "dump allocator free regions");
+ if (r != 0)
+ alloc = nullptr; //some collision, disable
+ if (alloc) {
+ r = admin_socket->register_command(
+ ("bluestore allocator score " + name).c_str(),
+ this,
+ "give score on allocator fragmentation (0-no fragmentation, 1-absolute fragmentation)");
+ ceph_assert(r == 0);
+ r = admin_socket->register_command(
+ ("bluestore allocator fragmentation " + name).c_str(),
+ this,
+ "give allocator fragmentation (0-no fragmentation, 1-absolute fragmentation)");
+ ceph_assert(r == 0);
+ r = admin_socket->register_command(
+ ("bluestore allocator fragmentation histogram " + name +
+ " name=alloc_unit,type=CephInt,req=false" +
+ " name=num_buckets,type=CephInt,req=false").c_str(),
+ this,
+ "build allocator free regions state histogram");
+ ceph_assert(r == 0);
+ }
+ }
+ }
+ ~SocketHook()
+ {
+ AdminSocket *admin_socket = g_ceph_context->get_admin_socket();
+ if (admin_socket && alloc) {
+ admin_socket->unregister_commands(this);
+ }
+ }
+
+ int call(std::string_view command,
+ const cmdmap_t& cmdmap,
+ const bufferlist&,
+ Formatter *f,
+ std::ostream& ss,
+ bufferlist& out) override {
+ int r = 0;
+ if (command == "bluestore allocator dump " + name) {
+ f->open_object_section("allocator_dump");
+ f->dump_unsigned("capacity", alloc->get_capacity());
+ f->dump_unsigned("alloc_unit", alloc->get_block_size());
+ f->dump_string("alloc_type", alloc->get_type());
+ f->dump_string("alloc_name", name);
+
+ f->open_array_section("extents");
+ auto iterated_allocation = [&](size_t off, size_t len) {
+ ceph_assert(len > 0);
+ f->open_object_section("free");
+ char off_hex[30];
+ char len_hex[30];
+ snprintf(off_hex, sizeof(off_hex) - 1, "0x%zx", off);
+ snprintf(len_hex, sizeof(len_hex) - 1, "0x%zx", len);
+ f->dump_string("offset", off_hex);
+ f->dump_string("length", len_hex);
+ f->close_section();
+ };
+ alloc->foreach(iterated_allocation);
+ f->close_section();
+ f->close_section();
+ } else if (command == "bluestore allocator score " + name) {
+ f->open_object_section("fragmentation_score");
+ f->dump_float("fragmentation_rating", alloc->get_fragmentation_score());
+ f->close_section();
+ } else if (command == "bluestore allocator fragmentation " + name) {
+ f->open_object_section("fragmentation");
+ f->dump_float("fragmentation_rating", alloc->get_fragmentation());
+ f->close_section();
+ } else if (command == "bluestore allocator fragmentation histogram " + name) {
+ int64_t alloc_unit = 4096;
+ cmd_getval(cmdmap, "alloc_unit", alloc_unit);
+ if (alloc_unit <= 0 ||
+ p2align(alloc_unit, alloc->get_block_size()) != alloc_unit) {
+ ss << "Invalid allocation unit: '" << alloc_unit
+ << ", to be aligned with: '" << alloc->get_block_size()
+ << std::endl;
+ return -EINVAL;
+ }
+ int64_t num_buckets = 8;
+ cmd_getval(cmdmap, "num_buckets", num_buckets);
+ if (num_buckets < 2) {
+ ss << "Invalid amount of buckets (min=2): '" << num_buckets
+ << std::endl;
+ return -EINVAL;
+ }
+
+ Allocator::FreeStateHistogram hist(num_buckets);
+ alloc->foreach(
+ [&](size_t off, size_t len) {
+ hist.record_extent(uint64_t(alloc_unit), off, len);
+ });
+ f->open_array_section("extent_counts");
+ hist.foreach(
+ [&](uint64_t max_len, uint64_t total, uint64_t aligned, uint64_t units) {
+ f->open_object_section("c");
+ f->dump_unsigned("max_len", max_len);
+ f->dump_unsigned("total", total);
+ f->dump_unsigned("aligned", aligned);
+ f->dump_unsigned("units", units);
+ f->close_section();
+ }
+ );
+ f->close_section();
+ } else {
+ ss << "Invalid command" << std::endl;
+ r = -ENOSYS;
+ }
+ return r;
+ }
+
+};
+Allocator::Allocator(std::string_view name,
+ int64_t _capacity,
+ int64_t _block_size)
+ : device_size(_capacity),
+ block_size(_block_size)
+{
+ asok_hook = new SocketHook(this, name);
+}
+
+
+Allocator::~Allocator()
+{
+ delete asok_hook;
+}
+
+const string& Allocator::get_name() const {
+ return asok_hook->name;
+}
+
+Allocator *Allocator::create(
+ CephContext* cct,
+ std::string_view type,
+ int64_t size,
+ int64_t block_size,
+ std::string_view name)
+{
+ Allocator* alloc = nullptr;
+ if (type == "stupid") {
+ alloc = new StupidAllocator(cct, size, block_size, name);
+ } else if (type == "bitmap") {
+ alloc = new BitmapAllocator(cct, size, block_size, name);
+ } else if (type == "avl") {
+ return new AvlAllocator(cct, size, block_size, name);
+ } else if (type == "btree") {
+ return new BtreeAllocator(cct, size, block_size, name);
+ } else if (type == "hybrid") {
+ return new HybridAvlAllocator(cct, size, block_size,
+ cct->_conf.get_val<uint64_t>("bluestore_hybrid_alloc_mem_cap"),
+ name);
+ } else if (type == "hybrid_btree2") {
+ return new HybridBtree2Allocator(cct, size, block_size,
+ cct->_conf.get_val<uint64_t>("bluestore_hybrid_alloc_mem_cap"),
+ cct->_conf.get_val<double>("bluestore_btree2_alloc_weight_factor"),
+ name);
+ }
+ if (alloc == nullptr) {
+ lderr(cct) << "Allocator::" << __func__ << " unknown alloc type "
+ << type << dendl;
+ }
+ return alloc;
+}
+
+void Allocator::release(const PExtentVector& release_vec)
+{
+ release_set_t release_set;
+ for (auto e : release_vec) {
+ release_set.insert(e.offset, e.length);
+ }
+ release(release_set);
+}
+
+/**
+ * Gives fragmentation a numeric value.
+ *
+ * Following algorithm applies value to each existing free unallocated block.
+ * Value of single block is a multiply of size and per-byte-value.
+ * Per-byte-value is greater for larger blocks.
+ * Assume block size X has value per-byte p; then block size 2*X will have per-byte value 1.1*p.
+ *
+ * This could be expressed in logarithms, but for speed this is interpolated inside ranges.
+ * [1] [2..3] [4..7] [8..15] ...
+ * ^ ^ ^ ^
+ * 1.1 1.1^2 1.1^3 1.1^4 ...
+ *
+ * Final score is obtained by proportion between score that would have been obtained
+ * in condition of absolute fragmentation and score in no fragmentation at all.
+ */
+double Allocator::get_fragmentation_score()
+{
+ // this value represents how much worth is 2X bytes in one chunk then in X + X bytes
+ static const double double_size_worth_small = 1.2;
+ // chunks larger then 128MB are large enough that should be counted without penalty
+ static const double double_size_worth_huge = 1;
+ static const size_t small_chunk_p2 = 20; // 1MB
+ static const size_t huge_chunk_p2 = 27; // 128MB
+ // for chunks 1MB - 128MB penalty coeffs are linearly weighted 1.2 (at small) ... 1 (at huge)
+ static std::vector<double> scales{1};
+ double score_sum = 0;
+ size_t sum = 0;
+
+ auto get_score = [&](size_t v) -> double {
+ size_t sc = sizeof(v) * 8 - std::countl_zero(v) - 1; //assign to grade depending on log2(len)
+ while (scales.size() <= sc + 1) {
+ //unlikely expand scales vector
+ auto ss = scales.size();
+ double scale = double_size_worth_small;
+ if (ss >= huge_chunk_p2) {
+ scale = double_size_worth_huge;
+ } else if (ss > small_chunk_p2) {
+ // linear decrease 1.2 ... 1
+ scale = (double_size_worth_huge * (ss - small_chunk_p2) + double_size_worth_small * (huge_chunk_p2 - ss)) /
+ (huge_chunk_p2 - small_chunk_p2);
+ }
+ scales.push_back(scales[scales.size() - 1] * scale);
+ }
+ size_t sc_shifted = size_t(1) << sc;
+ double x = double(v - sc_shifted) / sc_shifted; //x is <0,1) in its scale grade
+ // linear extrapolation in its scale grade
+ double score = (sc_shifted ) * scales[sc] * (1-x) +
+ (sc_shifted * 2) * scales[sc+1] * x;
+ return score;
+ };
+
+ auto iterated_allocation = [&](size_t off, size_t len) {
+ ceph_assert(len > 0);
+ score_sum += get_score(len);
+ sum += len;
+ };
+ foreach(iterated_allocation);
+
+ double ideal = get_score(sum);
+ double terrible = (sum / block_size) * get_score(block_size);
+ return (ideal - score_sum) / (ideal - terrible);
+}
+
+/*************
+* Allocator::FreeStateHistogram
+*************/
+using std::function;
+
+void Allocator::FreeStateHistogram::record_extent(uint64_t alloc_unit,
+ uint64_t off,
+ uint64_t len)
+{
+ size_t idx = myTraits._get_bucket(len);
+ ceph_assert(idx < buckets.size());
+ ++buckets[idx].total;
+
+ // now calculate the bucket for the chunk after alignment,
+ // resulting chunks shorter than alloc_unit are discarded
+ auto delta = p2roundup(off, alloc_unit) - off;
+ if (len >= delta + alloc_unit) {
+ len -= delta;
+ idx = myTraits._get_bucket(len);
+ ceph_assert(idx < buckets.size());
+ ++buckets[idx].aligned;
+ buckets[idx].alloc_units += len / alloc_unit;
+ }
+}
+void Allocator::FreeStateHistogram::foreach(
+ function<void(uint64_t max_len,
+ uint64_t total,
+ uint64_t aligned,
+ uint64_t unit)> cb)
+{
+ size_t i = 0;
+ for (const auto& b : buckets) {
+ cb(myTraits._get_bucket_max(i),
+ b.total, b.aligned, b.alloc_units);
+ ++i;
+ }
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+#ifndef CEPH_OS_BLUESTORE_ALLOCATOR_H
+#define CEPH_OS_BLUESTORE_ALLOCATOR_H
+
+#include <functional>
+#include <ostream>
+#include "include/ceph_assert.h"
+#include "bluestore_types.h"
+#include "common/ceph_mutex.h"
+
+typedef interval_set<uint64_t> release_set_t;
+typedef release_set_t::value_type release_set_entry_t;
+
+class Allocator {
+protected:
+
+ /**
+ * This is a base set of traits for logical placing entries
+ * into limited collection of buckets depending on their sizes.
+ * Descandants should implement get_bucket(len) method to obtain
+ * bucket index using entry length.
+ */
+ struct LenPartitionedSetTraits {
+ size_t num_buckets;
+ size_t base_bits; // bits in min entry size
+ size_t base; // min entry size
+ size_t factor; // additional factor to be applied
+ // to entry size when calculating
+ // target bucket
+
+
+ LenPartitionedSetTraits(size_t _num_buckets,
+ size_t _base_bits = 12, //= 4096 bytes
+ size_t _factor = 1) :
+ num_buckets(_num_buckets),
+ base_bits(_base_bits),
+ base(1ull << base_bits),
+ factor(_factor)
+ {
+ ceph_assert(factor);
+ }
+ };
+
+ /**
+ * This extends LenPartitionedSetTraits to implement linear bucket indexing:
+ * bucket index to be determined as entry's size divided by (base * factor),
+ * i.e. buckets are:
+ * [0..base)
+ * [base, base+base*factor)
+ * [base+base*factor, base+base*factor*2)
+ * [base+base*factor*2, base+base*factor*3)
+ * ...
+ */
+ struct LenPartitionedSetTraitsLinear : public LenPartitionedSetTraits {
+ using LenPartitionedSetTraits::LenPartitionedSetTraits;
+ /*
+ * Determines bucket index for a given extent's length in a bucket set
+ * with linear (len / base / factor) indexing.
+ * The first bucket is targeted for lengths < base,
+ * the last bucket is used for lengths above the maximum
+ * detemined by bucket count.
+ */
+ inline size_t _get_bucket(uint64_t len) const {
+ size_t idx = (len / factor) >> base_bits;
+ idx = idx < num_buckets ? idx : num_buckets - 1;
+ return idx;
+ }
+ /*
+ * returns upper bound of a specific bucket
+ */
+ inline size_t _get_bucket_max(size_t bucket) const {
+ return
+ bucket < num_buckets - 1 ?
+ base * factor * (1 + bucket) :
+ std::numeric_limits<uint64_t>::max();
+ }
+ };
+
+ /**
+ * This extends LenPartitionedSetTraits to implement exponential bucket indexing:
+ * target bucket bounds are determined as
+ * [0, base]
+ * (base, base*2^factor]
+ * (base*2^factor, base*2^(factor*2)]
+ * (base*2^(factor*2), base*2^(factor*3)]
+ * ...
+ *
+ */
+ struct LenPartitionedSetTraitsPow2 : public LenPartitionedSetTraits {
+ /*
+ * Determines bucket index for a given extent's length in a bucket collection
+ * with log2(len) indexing.
+ * The first bucket is targeted for lengths < base,
+ * The last bucket index is used for lengths above the maximum
+ * detemined by bucket count.
+ */
+ using LenPartitionedSetTraits::LenPartitionedSetTraits;
+ inline size_t _get_bucket(uint64_t len) const {
+ size_t idx;
+ const size_t len_p2_max =
+ base << ((factor * (num_buckets - 2)));
+ if (len <= base) {
+ idx = 0;
+ } else if (len > len_p2_max) {
+ idx = num_buckets - 1;
+ } else {
+ size_t most_bit = cbits(uint64_t(len - 1)) - 1;
+ idx = 1 + ((most_bit - base_bits) / factor);
+ }
+ ceph_assert(idx < num_buckets);
+ return idx;
+ }
+ /*
+ * returns upper bound of the bucket with log2(len) indexing.
+ */
+ inline size_t _get_bucket_max(size_t bucket) const {
+ return
+ bucket < num_buckets - 1 ?
+ base << (factor * bucket) :
+ std::numeric_limits<uint64_t>::max();
+ }
+ };
+
+ /*
+ * Lockless stack implementation
+ * that permits put/get operation exclusively
+ * if no waiting is needed.
+ * Conflicting operations are omitted.
+ */
+ class LocklessOpportunisticStack {
+ std::atomic<size_t> ref = 0;
+ std::atomic<size_t> count = 0;
+ std::vector<uint64_t> data;
+ public:
+ void init(size_t size) {
+ data.resize(size);
+ }
+ bool try_put(uint64_t& v) {
+ bool done = ++ref == 1 && count < data.size();
+ if (done) {
+ data[count++] = v;
+ }
+ --ref;
+ return done;
+ }
+ bool try_get(uint64_t& v) {
+ bool done = ++ref == 1 && count > 0;
+ if (done) {
+ v = data[--count];
+ }
+ --ref;
+ return done;
+ }
+ void foreach(std::function<void(uint64_t)> notify) {
+ for (size_t i = 0; i < count; i++) {
+ notify(data[i]);
+ }
+ }
+ };
+ /*
+ * Concurrently accessed extent (offset,length) cache
+ * which permits put/get operation exclusively if no waiting is needed.
+ * Implemented via a set of independent buckets (aka LocklessOpportunisticStack).
+ * Each bucket keeps extents of specific size only: 4K, 8K, 12K...64K
+ * which allows to avoid individual extent size tracking.
+ * Each bucket permits a single operation at a given time only,
+ * additional operations against the bucket are rejected meaning relevant
+ * extents aren't not cached.
+ */
+ class OpportunisticExtentCache {
+ const LenPartitionedSetTraitsLinear myTraits;
+ enum {
+ BUCKET_COUNT = 16,
+ EXTENTS_PER_BUCKET = 16, // amount of entries per single bucket,
+ // total amount of entries will be
+ // BUCKET_COUNT * EXTENTS_PER_BUCKET.
+ };
+
+ std::vector<LocklessOpportunisticStack> buckets;
+ std::atomic<size_t> hits = 0;
+ ceph::shared_mutex lock{
+ ceph::make_shared_mutex(std::string(), false, false, false)
+ };
+ public:
+ OpportunisticExtentCache() :
+ myTraits(BUCKET_COUNT + 1), // 16 regular buckets + 1 "catch-all" pseudo
+ // one to be used for out-of-bound checking
+ // since _get_*_size_bucket() methods imply
+ // the last bucket usage for the entries
+ // exceeding the max length.
+ buckets(BUCKET_COUNT)
+ {
+ //buckets.resize(BUCKET_COUNT);
+ for(auto& b : buckets) {
+ b.init(EXTENTS_PER_BUCKET);
+ }
+ }
+ bool try_put(uint64_t offset, uint64_t len) {
+ if (!lock.try_lock_shared()) {
+ return false;
+ }
+ bool ret = false;
+ ceph_assert(p2aligned(offset, myTraits.base));
+ ceph_assert(p2aligned(len, myTraits.base));
+ auto idx = myTraits._get_bucket(len);
+ if (idx < buckets.size())
+ ret = buckets[idx].try_put(offset);
+ lock.unlock_shared();
+ return ret;
+ }
+ bool try_get(uint64_t* offset, uint64_t len) {
+ if (!lock.try_lock_shared()) {
+ return false;
+ }
+ bool ret = false;
+ ceph_assert(offset);
+ ceph_assert(p2aligned(len, myTraits.base));
+ size_t idx = len >> myTraits.base_bits;
+ if (idx < buckets.size()) {
+ ret = buckets[idx].try_get(*offset);
+ if (ret) {
+ ++hits;
+ }
+ }
+ lock.unlock_shared();
+ return ret;
+ }
+ size_t get_hit_count() const {
+ return hits.load();
+ }
+ void foreach(std::function<void(uint64_t offset, uint64_t length)> notify) {
+ std::unique_lock _lock(lock);
+ for (uint64_t i = 0; i < buckets.size(); i++) {
+ auto cb = [&](uint64_t o) {
+ notify(o, i << myTraits.base_bits);
+ };
+ buckets[i].foreach(cb);
+ }
+ }
+ };
+
+public:
+ Allocator(std::string_view name,
+ int64_t _capacity,
+ int64_t _block_size);
+ virtual ~Allocator();
+
+ /*
+ * returns allocator type name as per names in config
+ */
+ virtual const char* get_type() const = 0;
+
+ /*
+ * Allocate required number of blocks in n number of extents.
+ * Min and Max number of extents are limited by:
+ * a. alloc unit
+ * b. max_alloc_size.
+ * as no extent can be lesser than block_size and greater than max_alloc size.
+ * Apart from that extents can vary between these lower and higher limits according
+ * to free block search algorithm and availability of contiguous space.
+ */
+ virtual int64_t allocate(uint64_t want_size, uint64_t block_size,
+ uint64_t max_alloc_size, int64_t hint,
+ PExtentVector *extents) = 0;
+
+ int64_t allocate(uint64_t want_size, uint64_t block_size,
+ int64_t hint, PExtentVector *extents) {
+ return allocate(want_size, block_size, want_size, hint, extents);
+ }
+
+ /* Bulk release. Implementations may override this method to handle the whole
+ * set at once. This could save e.g. unnecessary mutex dance. */
+ virtual void release(const release_set_t& release_set) = 0;
+ void release(const PExtentVector& release_set);
+
+ virtual void dump() = 0;
+ virtual void foreach(
+ std::function<void(uint64_t offset, uint64_t length)> notify) = 0;
+
+ virtual void init_add_free(uint64_t offset, uint64_t length) = 0;
+ virtual void init_rm_free(uint64_t offset, uint64_t length) = 0;
+
+ virtual uint64_t get_free() = 0;
+ virtual double get_fragmentation()
+ {
+ return 0.0;
+ }
+ virtual double get_fragmentation_score();
+ virtual void shutdown() = 0;
+
+ static Allocator *create(
+ CephContext* cct,
+ std::string_view type,
+ int64_t size,
+ int64_t block_size,
+ const std::string_view name = ""
+ );
+
+
+ const std::string& get_name() const;
+ int64_t get_capacity() const
+ {
+ return device_size;
+ }
+ int64_t get_block_size() const
+ {
+ return block_size;
+ }
+
+ // The following class implements Allocator's free extents histogram.
+ // Which is a set of N buckets to track extents layout.
+ // Extent matches a bucket depending on its length using the following
+ // length spans:
+ // [0..4K] (4K..16K] (16K..64K] .. (4M..16M] (16M..]
+ // Each bucket tracks:
+ // - total amount of extents of specific lengths
+ // - amount of extents aligned with allocation boundary
+ // - amount of allocation units in aligned extents
+ //
+ class FreeStateHistogram {
+ const LenPartitionedSetTraitsPow2 myTraits;
+ enum {
+ BASE_BITS = 12, // 4096 bytes
+ FACTOR = 2,
+ };
+ struct free_state_hist_bucket {
+ size_t total = 0;
+ size_t aligned = 0;
+ size_t alloc_units = 0;
+ };
+ std::vector<free_state_hist_bucket> buckets;
+ public:
+
+ FreeStateHistogram(size_t num_buckets)
+ : myTraits(num_buckets, BASE_BITS, FACTOR) {
+ buckets.resize(num_buckets);
+ }
+
+ void record_extent(uint64_t alloc_unit, uint64_t off, uint64_t len);
+ void foreach(
+ std::function<void(uint64_t, uint64_t, uint64_t, uint64_t)> cb);
+ };
+
+private:
+ class SocketHook;
+ SocketHook* asok_hook = nullptr;
+protected:
+ const int64_t device_size = 0;
+ const int64_t block_size = 0;
+};
+
+#endif
\ No newline at end of file
int64_t block_size,
uint64_t max_mem,
std::string_view name) :
- Allocator(name, device_size, block_size),
+ AllocatorBase(name, device_size, block_size),
cct(cct),
range_size_alloc_threshold(
cct->_conf.get_val<uint64_t>("bluestore_avl_alloc_bf_threshold")),
#include <boost/intrusive/avl_set.hpp>
#include "Allocator.h"
+#include "AllocatorBase.h"
#include "os/bluestore/bluestore_types.h"
#include "include/mempool.h"
boost::intrusive::avl_set_member_hook<> size_hook;
};
-class AvlAllocator : public Allocator {
+class AvlAllocator : public AllocatorBase {
struct dispose_rs {
void operator()(range_seg_t* p)
{
int64_t capacity,
int64_t alloc_unit,
std::string_view name) :
- Allocator(name, capacity, alloc_unit),
+ AllocatorBase(name, capacity, alloc_unit),
cct(_cct)
{
ldout(cct, 10) << __func__ << " 0x" << std::hex << capacity << "/"
#include <mutex>
#include "Allocator.h"
+#include "AllocatorBase.h"
#include "os/bluestore/bluestore_types.h"
#include "fastbmap_allocator_impl.h"
#include "include/mempool.h"
#include "common/debug.h"
-class BitmapAllocator : public Allocator,
+class BitmapAllocator : public AllocatorBase,
public AllocatorLevel02<AllocatorLevel01Loose> {
CephContext* cct;
public:
double _rweight_factor,
bool with_cache,
std::string_view name) :
- Allocator(name, device_size, block_size),
+ AllocatorBase(name, device_size, block_size),
myTraits(RANGE_SIZE_BUCKET_COUNT),
cct(_cct),
range_count_cap(max_mem / sizeof(range_seg_t))
#include "include/cpp-btree/btree_set.h"
#include "Allocator.h"
+#include "AllocatorBase.h"
#include "os/bluestore/bluestore_types.h"
#include "include/mempool.h"
*
*
*/
-class Btree2Allocator : public Allocator {
+class Btree2Allocator : public AllocatorBase {
enum {
RANGE_SIZE_BUCKET_COUNT = 14,
};
private:
CephContext* cct = nullptr;
- Allocator::OpportunisticExtentCache* cache = nullptr;
+ AllocatorBase::OpportunisticExtentCache* cache = nullptr;
std::mutex lock;
template<class T>
int64_t block_size,
uint64_t max_mem,
std::string_view name) :
- Allocator(name, device_size, block_size),
+ AllocatorBase(name, device_size, block_size),
range_size_alloc_threshold(
cct->_conf.get_val<uint64_t>("bluestore_avl_alloc_bf_threshold")),
range_size_alloc_free_pct(
#include "include/cpp-btree/btree_map.h"
#include "include/cpp-btree/btree_set.h"
#include "Allocator.h"
+#include "AllocatorBase.h"
#include "os/bluestore/bluestore_types.h"
#include "include/mempool.h"
-class BtreeAllocator : public Allocator {
+class BtreeAllocator : public AllocatorBase {
struct range_seg_t {
uint64_t start; ///< starting offset of this segment
uint64_t end; ///< ending offset (non-inclusive)
int64_t capacity,
int64_t _block_size,
std::string_view name)
- : Allocator(name, capacity, _block_size),
+ : AllocatorBase(name, capacity, _block_size),
cct(cct), num_free(0),
free(10)
{
#include <mutex>
#include "Allocator.h"
+#include "AllocatorBase.h"
#include "include/btree_map.h"
#include "include/interval_set.h"
#include "os/bluestore/bluestore_types.h"
#include "include/mempool.h"
#include "common/ceph_mutex.h"
-class StupidAllocator : public Allocator {
+class StupidAllocator : public AllocatorBase {
CephContext* cct;
ceph::mutex lock = ceph::make_mutex("StupidAllocator::lock");