]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
os/bluestore: add BtreeAllocator
authorKefu Chai <kchai@redhat.com>
Sat, 12 Jun 2021 12:58:36 +0000 (20:58 +0800)
committerKefu Chai <kchai@redhat.com>
Fri, 18 Jun 2021 01:13:12 +0000 (09:13 +0800)
because each AVL tree node needs to keep track of its children, the
overhead of each extent in AvlAllocator is relatively higher than that
of bitmap allocator, per node sizes 80 bytes, per per mempool stats provided
by unittest_alloc_bench. while btree has lower overhead, as it keeps multiple
entries in each node / block, the node size of abseil's btree implementation
defaults to 256, so each node is able to contain up to 256 values. this means
we have lower overhead than the binary-trees like AVL and red-black tree.

but because the overhead of rebalance the btree is higher than that of
binary tree, its takes more CPU cycles when performing alloc-release
than its AVL version. but its upside is that its memory layout is
more compact, under some use case, it might be a better alternative
than AvlAllocator.

Signed-off-by: Kefu Chai <kchai@redhat.com>
src/crimson/os/alienstore/CMakeLists.txt
src/os/CMakeLists.txt
src/os/bluestore/Allocator.cc
src/os/bluestore/BtreeAllocator.cc [new file with mode: 0644]
src/os/bluestore/BtreeAllocator.h [new file with mode: 0644]
src/test/objectstore/Allocator_aging_fragmentation.cc
src/test/objectstore/Allocator_bench.cc

index 882fe9439ec357d8467016daab186a02b958ce27..6c4182d20fff21e09945ff30176ad216d7e9f230 100644 (file)
@@ -47,6 +47,7 @@ set(alien_store_srcs
   ${PROJECT_SOURCE_DIR}/src/os/ObjectStore.cc
   ${PROJECT_SOURCE_DIR}/src/os/bluestore/Allocator.cc
   ${PROJECT_SOURCE_DIR}/src/os/bluestore/AvlAllocator.cc
+  ${PROJECT_SOURCE_DIR}/src/os/bluestore/BtreeAllocator.cc
   ${PROJECT_SOURCE_DIR}/src/os/bluestore/BitmapFreelistManager.cc
   ${PROJECT_SOURCE_DIR}/src/os/bluestore/BlueFS.cc
   ${PROJECT_SOURCE_DIR}/src/os/bluestore/bluefs_types.cc
index 9008f2ed8c04ed56dcb8967d20141c52c4810c9a..d31b4c70cc596f879f050220781a8ed1b6f064e0 100644 (file)
@@ -33,6 +33,7 @@ if(WITH_BLUESTORE)
     bluestore/StupidAllocator.cc
     bluestore/BitmapAllocator.cc
     bluestore/AvlAllocator.cc
+    bluestore/BtreeAllocator.cc
     bluestore/HybridAllocator.cc
   )
 endif(WITH_BLUESTORE)
index 50ca3fa91632170c723717e33b9f5425672ad352..8e4a08b2587abeb0b148bb652e754ac186f5a882 100644 (file)
@@ -5,6 +5,7 @@
 #include "StupidAllocator.h"
 #include "BitmapAllocator.h"
 #include "AvlAllocator.h"
+#include "BtreeAllocator.h"
 #include "HybridAllocator.h"
 #ifdef HAVE_LIBZBD
 #include "ZonedAllocator.h"
@@ -133,6 +134,8 @@ Allocator *Allocator::create(CephContext* cct, std::string_view type,
     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 HybridAllocator(cct, size, block_size,
       cct->_conf.get_val<uint64_t>("bluestore_hybrid_alloc_mem_cap"),
diff --git a/src/os/bluestore/BtreeAllocator.cc b/src/os/bluestore/BtreeAllocator.cc
new file mode 100644 (file)
index 0000000..6184375
--- /dev/null
@@ -0,0 +1,469 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "BtreeAllocator.h"
+
+#include <limits>
+
+#include "common/config_proxy.h"
+#include "common/debug.h"
+
+#define dout_context cct
+#define dout_subsys ceph_subsys_bluestore
+#undef  dout_prefix
+#define dout_prefix *_dout << "BtreeAllocator "
+
+/*
+ * This is a helper function that can be used by the allocator to find
+ * a suitable block to allocate. This will search the specified B-tree
+ * looking for a block that matches the specified criteria.
+ */
+uint64_t BtreeAllocator::_pick_block_after(uint64_t *cursor,
+                                          uint64_t size,
+                                          uint64_t align)
+{
+  auto rs_start = range_tree.lower_bound(*cursor);
+  for (auto rs = rs_start; rs != range_tree.end(); ++rs) {
+    uint64_t offset = p2roundup(rs->first, align);
+    if (offset + size <= rs->second) {
+      *cursor = offset + size;
+      return offset;
+    }
+  }
+  if (*cursor == 0) {
+    // If we already started from beginning, don't bother with searching from beginning
+    return -1ULL;
+  }
+  // If we reached end, start from beginning till cursor.
+  for (auto rs = range_tree.begin(); rs != rs_start; ++rs) {
+    uint64_t offset = p2roundup(rs->first, align);
+    if (offset + size <= rs->second) {
+      *cursor = offset + size;
+      return offset;
+    }
+  }
+  return -1ULL;
+}
+
+uint64_t BtreeAllocator::_pick_block_fits(uint64_t size,
+                                        uint64_t align)
+{
+  // instead of searching from cursor, just pick the smallest range which fits
+  // the needs
+  auto rs_start = range_size_tree.lower_bound(range_value_t{0,size});
+  for (auto rs = rs_start; rs != range_size_tree.end(); ++rs) {
+    uint64_t offset = p2roundup(rs->start, align);
+    if (offset + size <= rs->start + rs->size) {
+      return offset;
+    }
+  }
+  return -1ULL;
+}
+
+void BtreeAllocator::_add_to_tree(uint64_t start, uint64_t size)
+{
+  ceph_assert(size != 0);
+
+  uint64_t end = start + size;
+
+  auto rs_after = range_tree.upper_bound(start);
+
+  /* Make sure we don't overlap with either of our neighbors */
+  auto rs_before = range_tree.end();
+  if (rs_after != range_tree.begin()) {
+    rs_before = std::prev(rs_after);
+  }
+
+  bool merge_before = (rs_before != range_tree.end() && rs_before->second == start);
+  bool merge_after = (rs_after != range_tree.end() && rs_after->first == end);
+
+  if (merge_before && merge_after) {
+    // | before   |//////| after |
+    // | before >>>>>>>>>>>>>>>  |
+    range_seg_t seg_before{rs_before->first, rs_before->second};
+    range_seg_t seg_after{rs_after->first, rs_after->second};
+    // expand the head seg before rs_{before,after} are invalidated
+    rs_before->second = seg_after.end;
+    // remove the tail seg from offset tree
+    range_tree.erase(rs_after);
+    // remove the head and tail seg from size tree
+    range_size_tree.erase(seg_before);
+    range_size_tree.erase(seg_after);
+    // insert the merged seg into size tree
+    range_size_tree.emplace(seg_before.start, seg_after.end);
+  } else if (merge_before) {
+    // | before   |//////|
+    // | before >>>>>>>> |
+    // remove the head seg from the size tree
+    range_seg_t seg_before{rs_before->first, rs_before->second};
+    range_size_tree.erase(seg_before);
+    // expand the head seg in the offset tree
+    rs_before->second = end;
+    // insert the merged seg into size tree
+    range_size_tree.emplace(seg_before.start, end);
+  } else if (merge_after) {
+    // |//////| after |
+    // | merge after  |
+    // remove the tail seg from size tree
+    range_seg_t seg_after{rs_after->first, rs_after->second};
+    range_size_tree.erase(seg_after);
+    // remove the tail seg from offset tree
+    range_tree.erase(rs_after);
+    // insert the merged seg
+    range_tree.emplace(start, seg_after.end);
+    range_size_tree.emplace(start, seg_after.end);
+  } else {
+    // no neighbours
+    range_tree.emplace_hint(rs_after, start, end);
+    range_size_tree.emplace(start, end);
+  }
+  num_free += size;
+}
+
+void BtreeAllocator::_process_range_removal(uint64_t start, uint64_t end,
+  BtreeAllocator::range_tree_t::iterator& rs)
+{
+  bool left_over = (rs->first != start);
+  bool right_over = (rs->second != end);
+
+  range_seg_t seg_whole{rs->first, rs->second};
+  range_size_tree.erase(seg_whole);
+
+  // | left <|////|  right |
+  if (left_over && right_over) {
+    // add the spin-off right seg
+    range_seg_t seg_after{end, seg_whole.end};
+    range_tree.emplace_hint(rs, seg_after.start, seg_after.end);
+    range_size_tree.emplace(seg_after);
+    // shink the left seg in offset tree
+    rs->second = start;
+    // insert the shrinked left seg back into size tree
+    range_size_tree.emplace(seg_whole.start, start);
+  } else if (left_over) {
+    // | left <|///////////|
+    // shrink the left seg in the offset tree
+    rs->second = start;
+    // insert the shrinked left seg back into size tree
+    range_size_tree.emplace(seg_whole.start, start);
+  } else if (right_over) {
+    // |//////////| right |
+    // remove the whole seg from offset tree
+    range_tree.erase(rs);
+    // add the spin-off right seg
+    range_seg_t seg_after{end, seg_whole.end};
+    range_tree.emplace(seg_after.start, seg_after.end);
+    range_size_tree.emplace(seg_after);
+  } else {
+    range_tree.erase(rs);
+  }
+  num_free -= (end - start);
+}
+
+void BtreeAllocator::_remove_from_tree(uint64_t start, uint64_t size)
+{
+  uint64_t end = start + size;
+
+  ceph_assert(size != 0);
+  ceph_assert(size <= num_free);
+
+  auto rs = range_tree.find(start);
+  /* Make sure we completely overlap with someone */
+  ceph_assert(rs != range_tree.end());
+  ceph_assert(rs->first <= start);
+  ceph_assert(rs->second >= end);
+
+  _process_range_removal(start, end, rs);
+}
+
+void BtreeAllocator::_try_remove_from_tree(uint64_t start, uint64_t size,
+  std::function<void(uint64_t, uint64_t, bool)> cb)
+{
+  uint64_t end = start + size;
+
+  ceph_assert(size != 0);
+
+  auto rs = range_tree.find(start);
+
+  if (rs == range_tree.end() || rs->first >= end) {
+    cb(start, size, false);
+    return;
+  }
+
+  do {
+
+    auto next_rs = rs;
+    ++next_rs;
+
+    if (start < rs->first) {
+      cb(start, rs->first - start, false);
+      start = rs->first;
+    }
+    auto range_end = std::min(rs->second, end);
+    _process_range_removal(start, range_end, rs);
+    cb(start, range_end - start, true);
+    start = range_end;
+
+    rs = next_rs;
+  } while (rs != range_tree.end() && rs->first < end && start < end);
+  if (start < end) {
+    cb(start, end - start, false);
+  }
+}
+
+int64_t BtreeAllocator::_allocate(
+  uint64_t want,
+  uint64_t unit,
+  uint64_t max_alloc_size,
+  int64_t  hint, // unused, for now!
+  PExtentVector* extents)
+{
+  uint64_t allocated = 0;
+  while (allocated < want) {
+    uint64_t offset, length;
+    int r = _allocate(std::min(max_alloc_size, want - allocated),
+                      unit, &offset, &length);
+    if (r < 0) {
+      // Allocation failed.
+      break;
+    }
+    extents->emplace_back(offset, length);
+    allocated += length;
+  }
+  assert(range_size_tree.size() == range_tree.size());
+  return allocated ? allocated : -ENOSPC;
+}
+
+int BtreeAllocator::_allocate(
+  uint64_t size,
+  uint64_t unit,
+  uint64_t *offset,
+  uint64_t *length)
+{
+  uint64_t max_size = 0;
+  if (auto p = range_size_tree.rbegin(); p != range_size_tree.rend()) {
+    max_size = p->size;
+  }
+
+  bool force_range_size_alloc = false;
+  if (max_size < size) {
+    if (max_size < unit) {
+      return -ENOSPC;
+    }
+    size = p2align(max_size, unit);
+    ceph_assert(size > 0);
+    force_range_size_alloc = true;
+  }
+
+  const int free_pct = num_free * 100 / device_size;
+  uint64_t start = 0;
+  /*
+   * If we're running low on space switch to using the size
+   * sorted B-tree (best-fit).
+   */
+  if (force_range_size_alloc ||
+      max_size < range_size_alloc_threshold ||
+      free_pct < range_size_alloc_free_pct) {
+    do {
+      start = _pick_block_fits(size, unit);
+      dout(20) << __func__ << " best fit=" << start << " size=" << size << dendl;
+      if (start != uint64_t(-1ULL)) {
+        break;
+      }
+      // try to collect smaller extents as we could fail to retrieve
+      // that large block due to misaligned extents
+      size = p2align(size >> 1, unit);
+    } while (size >= unit);
+  } else {
+    do {
+      /*
+       * Find the largest power of 2 block size that evenly divides the
+       * requested size. This is used to try to allocate blocks with similar
+       * alignment from the same area (i.e. same cursor bucket) but it does
+       * not guarantee that other allocations sizes may exist in the same
+       * region.
+       */
+      uint64_t* cursor = &lbas[cbits(size) - 1];
+      start = _pick_block_after(cursor, size, unit);
+      dout(20) << __func__ << " first fit=" << start << " size=" << size << dendl;
+      if (start != uint64_t(-1ULL)) {
+        break;
+      }
+      // try to collect smaller extents as we could fail to retrieve
+      // that large block due to misaligned extents
+      size = p2align(size >> 1, unit);
+    } while (size >= unit);
+  }
+  if (start == -1ULL) {
+    return -ENOSPC;
+  }
+
+  _remove_from_tree(start, size);
+
+  *offset = start;
+  *length = size;
+  return 0;
+}
+
+void BtreeAllocator::_release(const interval_set<uint64_t>& release_set)
+{
+  for (auto p = release_set.begin(); p != release_set.end(); ++p) {
+    const auto offset = p.get_start();
+    const auto length = p.get_len();
+    ceph_assert(offset + length <= uint64_t(device_size));
+    ldout(cct, 10) << __func__ << std::hex
+      << " offset 0x" << offset
+      << " length 0x" << length
+      << std::dec << dendl;
+    _add_to_tree(offset, length);
+  }
+}
+
+void BtreeAllocator::_release(const PExtentVector& release_set) {
+  for (auto& e : release_set) {
+    ldout(cct, 10) << __func__ << std::hex
+      << " offset 0x" << e.offset
+      << " length 0x" << e.length
+      << std::dec << dendl;
+    _add_to_tree(e.offset, e.length);
+  }
+}
+
+void BtreeAllocator::_shutdown()
+{
+  range_size_tree.clear();
+  range_tree.clear();
+}
+
+BtreeAllocator::BtreeAllocator(CephContext* cct,
+                              int64_t device_size,
+                              int64_t block_size,
+                              uint64_t max_mem,
+                              std::string_view name) :
+  Allocator(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(
+    cct->_conf.get_val<uint64_t>("bluestore_avl_alloc_bf_free_pct")),
+  range_count_cap(max_mem / sizeof(range_seg_t)),
+  cct(cct)
+{}
+
+BtreeAllocator::BtreeAllocator(CephContext* cct,
+                              int64_t device_size,
+                              int64_t block_size,
+                              std::string_view name) :
+  BtreeAllocator(cct, device_size, block_size, 0 /* max_mem */, name)
+{}
+
+BtreeAllocator::~BtreeAllocator()
+{
+  shutdown();
+}
+
+int64_t BtreeAllocator::allocate(
+  uint64_t want,
+  uint64_t unit,
+  uint64_t max_alloc_size,
+  int64_t  hint, // unused, for now!
+  PExtentVector* extents)
+{
+  ldout(cct, 10) << __func__ << std::hex
+                 << " want 0x" << want
+                 << " unit 0x" << unit
+                 << " max_alloc_size 0x" << max_alloc_size
+                 << " hint 0x" << hint
+                 << std::dec << dendl;
+  ceph_assert(isp2(unit));
+  ceph_assert(want % unit == 0);
+
+  if (max_alloc_size == 0) {
+    max_alloc_size = want;
+  }
+  if (constexpr auto cap = std::numeric_limits<decltype(bluestore_pextent_t::length)>::max();
+      max_alloc_size >= cap) {
+    max_alloc_size = p2align(uint64_t(cap), (uint64_t)block_size);
+  }
+  std::lock_guard l(lock);
+  return _allocate(want, unit, max_alloc_size, hint, extents);
+}
+
+void BtreeAllocator::release(const interval_set<uint64_t>& release_set) {
+  std::lock_guard l(lock);
+  _release(release_set);
+}
+
+uint64_t BtreeAllocator::get_free()
+{
+  std::lock_guard l(lock);
+  return num_free;
+}
+
+double BtreeAllocator::get_fragmentation()
+{
+  std::lock_guard l(lock);
+  return _get_fragmentation();
+}
+
+void BtreeAllocator::dump()
+{
+  std::lock_guard l(lock);
+  _dump();
+}
+
+void BtreeAllocator::_dump() const
+{
+  ldout(cct, 0) << __func__ << " range_tree: " << dendl;
+  for (auto& rs : range_tree) {
+    ldout(cct, 0) << std::hex
+      << "0x" << rs.first << "~" << rs.second
+      << std::dec
+      << dendl;
+  }
+
+  ldout(cct, 0) << __func__ << " range_size_tree: " << dendl;
+  for (auto& rs : range_size_tree) {
+    ldout(cct, 0) << std::hex
+      << "0x" << rs.size << "@" << rs.start
+      << std::dec
+      << dendl;
+  }
+}
+
+void BtreeAllocator::dump(std::function<void(uint64_t offset, uint64_t length)> notify)
+{
+  for (auto& rs : range_tree) {
+    notify(rs.first, rs.second - rs.first);
+  }
+}
+
+void BtreeAllocator::init_add_free(uint64_t offset, uint64_t length)
+{
+  if (!length)
+    return;
+  std::lock_guard l(lock);
+  ceph_assert(offset + length <= uint64_t(device_size));
+  ldout(cct, 10) << __func__ << std::hex
+                 << " offset 0x" << offset
+                 << " length 0x" << length
+                 << std::dec << dendl;
+  _add_to_tree(offset, length);
+}
+
+void BtreeAllocator::init_rm_free(uint64_t offset, uint64_t length)
+{
+  if (!length)
+    return;
+  std::lock_guard l(lock);
+  ceph_assert(offset + length <= uint64_t(device_size));
+  ldout(cct, 10) << __func__ << std::hex
+                 << " offset 0x" << offset
+                 << " length 0x" << length
+                 << std::dec << dendl;
+  _remove_from_tree(offset, length);
+}
+
+void BtreeAllocator::shutdown()
+{
+  std::lock_guard l(lock);
+  _shutdown();
+}
diff --git a/src/os/bluestore/BtreeAllocator.h b/src/os/bluestore/BtreeAllocator.h
new file mode 100644 (file)
index 0000000..0a68015
--- /dev/null
@@ -0,0 +1,199 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*-
+// vim: ts=8 sw=2 smarttab
+
+#pragma once
+
+#include <mutex>
+#include "include/cpp-btree/btree_map.h"
+#include "include/cpp-btree/btree_set.h"
+#include "Allocator.h"
+#include "os/bluestore/bluestore_types.h"
+#include "include/mempool.h"
+
+class BtreeAllocator : public Allocator {
+  struct range_seg_t {
+    uint64_t start;   ///< starting offset of this segment
+    uint64_t end;     ///< ending offset (non-inclusive)
+
+    range_seg_t(uint64_t start, uint64_t end)
+      : start{start},
+        end{end}
+    {}
+    inline uint64_t length() const {
+      return end - start;
+    }
+  };
+
+  struct range_value_t {
+    uint64_t size;
+    uint64_t start;
+    range_value_t(uint64_t start, uint64_t end)
+      : size{end - start},
+        start{start}
+    {}
+    range_value_t(const range_seg_t& rs)
+      : size{rs.length()},
+        start{rs.start}
+    {}
+  };
+  // do the radix sort
+  struct compare_range_value_t {
+    int operator()(const range_value_t& lhs,
+                   const range_value_t& rhs) const noexcept {
+      if (lhs.size < rhs.size) {
+        return -1;
+      } else if (lhs.size > rhs.size) {
+        return 1;
+      }
+      if (lhs.start < rhs.start) {
+        return -1;
+      } else if (lhs.start > rhs.start) {
+        return 1;
+      } else {
+        return 0;
+      }
+    }
+  };
+protected:
+  /*
+  * ctor intended for the usage from descendant class(es) which
+  * provides handling for spilled over entries
+  * (when entry count >= max_entries)
+  */
+  BtreeAllocator(CephContext* cct, int64_t device_size, int64_t block_size,
+    uint64_t max_mem,
+    std::string_view name);
+
+public:
+  BtreeAllocator(CephContext* cct, int64_t device_size, int64_t block_size,
+                 std::string_view name);
+  ~BtreeAllocator();
+  const char* get_type() const override
+  {
+    return "btree";
+  }
+  int64_t allocate(
+    uint64_t want,
+    uint64_t unit,
+    uint64_t max_alloc_size,
+    int64_t  hint,
+    PExtentVector *extents) override;
+  void release(const interval_set<uint64_t>& release_set) override;
+  uint64_t get_free() override;
+  double get_fragmentation() override;
+
+  void dump() override;
+  void dump(std::function<void(uint64_t offset, uint64_t length)> notify) override;
+  void init_add_free(uint64_t offset, uint64_t length) override;
+  void init_rm_free(uint64_t offset, uint64_t length) override;
+  void shutdown() override;
+
+private:
+  // pick a range by search from cursor forward
+  uint64_t _pick_block_after(
+    uint64_t *cursor,
+    uint64_t size,
+    uint64_t align);
+  // pick a range with exactly the same size or larger
+  uint64_t _pick_block_fits(
+    uint64_t size,
+    uint64_t align);
+  int _allocate(
+    uint64_t size,
+    uint64_t unit,
+    uint64_t *offset,
+    uint64_t *length);
+
+  template<class T>
+  using pool_allocator = mempool::bluestore_alloc::pool_allocator<T>;
+  using range_tree_t =
+    btree::btree_map<
+      uint64_t /* start */,
+      uint64_t /* end */,
+      std::less<uint64_t>,
+      pool_allocator<std::pair<uint64_t, uint64_t>>>;
+  range_tree_t range_tree;    ///< main range tree
+  /*
+   * The range_size_tree should always contain the
+   * same number of segments as the range_tree.
+   * The only difference is that the range_size_tree
+   * is ordered by segment sizes.
+   */
+  using range_size_tree_t =
+    btree::btree_set<
+      range_value_t /* size, start */,
+      compare_range_value_t,
+      pool_allocator<range_value_t>>;
+  range_size_tree_t range_size_tree;
+
+  uint64_t num_free = 0;     ///< total bytes in freelist
+
+  /*
+   * This value defines the number of elements in the ms_lbas array.
+   * The value of 64 was chosen as it covers all power of 2 buckets
+   * up to UINT64_MAX.
+   * This is the equivalent of highest-bit of UINT64_MAX.
+   */
+  static constexpr unsigned MAX_LBAS = 64;
+  uint64_t lbas[MAX_LBAS] = {0};
+
+  /*
+   * Minimum size which forces the dynamic allocator to change
+   * it's allocation strategy.  Once the allocator cannot satisfy
+   * an allocation of this size then it switches to using more
+   * aggressive strategy (i.e search by size rather than offset).
+   */
+  uint64_t range_size_alloc_threshold = 0;
+  /*
+   * The minimum free space, in percent, which must be available
+   * in allocator to continue allocations in a first-fit fashion.
+   * Once the allocator's free space drops below this level we dynamically
+   * switch to using best-fit allocations.
+   */
+  int range_size_alloc_free_pct = 0;
+
+  /*
+  * Max amount of range entries allowed. 0 - unlimited
+  */
+  int64_t range_count_cap = 0;
+
+private:
+  CephContext* cct;
+  std::mutex lock;
+
+  double _get_fragmentation() const {
+    auto free_blocks = p2align(num_free, (uint64_t)block_size) / block_size;
+    if (free_blocks <= 1) {
+      return .0;
+    }
+    return (static_cast<double>(range_tree.size() - 1) / (free_blocks - 1));
+  }
+  void _dump() const;
+
+  uint64_t _lowest_size_available() const {
+    auto rs = range_size_tree.begin();
+    return rs != range_size_tree.end() ? rs->size : 0;
+  }
+
+  int64_t _allocate(
+    uint64_t want,
+    uint64_t unit,
+    uint64_t max_alloc_size,
+    int64_t  hint,
+    PExtentVector *extents);
+
+  void _release(const interval_set<uint64_t>& release_set);
+  void _release(const PExtentVector&  release_set);
+  void _shutdown();
+
+  // called when extent to be released/marked free
+  void _add_to_tree(uint64_t start, uint64_t size);
+  void _process_range_removal(uint64_t start, uint64_t end, range_tree_t::iterator& rs);
+  void _remove_from_tree(uint64_t start, uint64_t size);
+  void _try_remove_from_tree(uint64_t start, uint64_t size,
+    std::function<void(uint64_t offset, uint64_t length, bool found)> cb);
+
+  uint64_t _get_free() const {
+    return num_free;
+  }
+};
index 8641b010748f917b8b11e45ad258cac3a5d2b9b6..57ad8bd902ee4b0b2532b40309b4f964b191ea75 100755 (executable)
@@ -459,5 +459,4 @@ TEST_P(AllocTest, test_bonus_empty_fragmented)
 INSTANTIATE_TEST_CASE_P(
   Allocator,
   AllocTest,
-  ::testing::Values("stupid", "bitmap", "avl"));
-
+  ::testing::Values("stupid", "bitmap", "avl", "btree"));
index 4afb2df4b4e7c0f3fdbb8155157757a59a1e4548..526562138a969c09190f580a5fe14e0a252839fd 100644 (file)
@@ -361,4 +361,4 @@ TEST_P(AllocTest, mempoolAccounting)
 INSTANTIATE_TEST_SUITE_P(
   Allocator,
   AllocTest,
-  ::testing::Values("stupid", "bitmap", "avl", "hybrid"));
+  ::testing::Values("stupid", "bitmap", "avl", "btree", "hybrid"));