#include "kv/KeyValueDB.h"
#include <ostream>
#include "include/assert.h"
+#include "os/bluestore/bluestore_types.h"
class FreelistManager;
uint64_t need_size, uint64_t alloc_unit, int64_t hint,
uint64_t *offset, uint32_t *length) = 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 alloc_unit 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 int alloc_extents(uint64_t want_size, uint64_t alloc_unit,
+ uint64_t max_alloc_size, int64_t hint,
+ std::vector<AllocExtent> *extents, int *count) = 0;
+
+ virtual int alloc_extents(uint64_t want_size, uint64_t alloc_unit,
+ int64_t hint, std::vector<AllocExtent> *extents,
+ int *count) {
+ return alloc_extents(want_size, alloc_unit, want_size, hint, extents, count);
+ }
+
virtual int release(
uint64_t offset, uint64_t length) = 0;
+ virtual int release_extents(std::vector<AllocExtent> *extents, int count) {
+ int res = 0;
+ for (int i = 0; i < count; i++) {
+ res = release((*extents)[i].offset, (*extents)[i].length);
+ if (res != 0) {
+ break;
+ }
+ }
+ return res;
+ }
+
virtual void commit_start() = 0;
virtual void commit_finish() = 0;
* Find N number of free bits in bitmap. Need not be contiguous.
*/
int BmapEntry::find_any_free_bits(int start_offset, int64_t num_blocks,
- int64_t *allocated_blocks, int64_t block_offset,
- int64_t *scanned)
+ ExtentList *allocated_blocks, int64_t block_offset, int64_t *scanned)
{
int allocated = 0;
int required = num_blocks;
for (i = start_offset; i < BmapEntry::size() &&
allocated < required; i++) {
if (check_n_set_bit(i)) {
- allocated_blocks[allocated] = i + block_offset;
+ allocated_blocks->add_extents(i + block_offset, 1);
allocated++;
}
}
/*
* Allocate N blocks, dis-contiguous are fine
*/
-int64_t BitMapZone::alloc_blocks_dis(int64_t num_blocks, int64_t zone_blk_off, int64_t *alloc_blocks)
+int64_t BitMapZone::alloc_blocks_dis(int64_t num_blocks,
+ int64_t zone_blk_off,
+ ExtentList *alloc_blocks)
{
int64_t bmap_idx = 0;
int bit = 0;
int64_t scanned = 0;
blk_off = (iter.index() - 1) * BmapEntry::size() + zone_blk_off;
allocated += bmap->find_any_free_bits(bit, num_blocks - allocated,
- &alloc_blocks[allocated], blk_off, &scanned);
+ alloc_blocks, blk_off, &scanned);
+ if (allocated == num_blocks) {
+ break;
+ }
}
add_used_blocks(allocated);
return true;
}
-bool BitMapAreaIN::is_allocated(int64_t *alloc_blocks, int64_t num_blocks, int64_t blk_off)
-{
- for (int64_t i = 0; i < num_blocks; i++) {
- if (!is_allocated(alloc_blocks[i] - blk_off, 1)) {
- return false;
- }
- }
-
- return true;
-}
-
int64_t BitMapAreaIN::alloc_blocks_int(bool wait, bool wrap,
int64_t num_blocks, int64_t *start_block)
{
}
int64_t BitMapAreaIN::alloc_blocks_dis_int(bool wait, int64_t num_blocks,
- int64_t area_blk_off, int64_t *block_list)
+ int64_t area_blk_off, ExtentList *block_list)
{
BitMapArea *child = NULL;
int64_t allocated = 0;
blk_off = child->get_index() * m_child_size_blocks + area_blk_off;
allocated += child->alloc_blocks_dis(wait, num_blocks - allocated,
- blk_off, &block_list[allocated]);
+ blk_off, block_list);
child_unlock(child);
if (allocated == num_blocks) {
break;
}
int64_t BitMapAreaIN::alloc_blocks_dis(bool wait, int64_t num_blocks,
- int64_t blk_off, int64_t *block_list)
+ int64_t blk_off, ExtentList *block_list)
{
int64_t allocated = 0;
lock_shared();
- allocated += alloc_blocks_dis_int(wait, num_blocks, blk_off, &block_list[allocated]);
+ allocated += alloc_blocks_dis_int(wait, num_blocks, blk_off, block_list);
add_used_blocks(allocated);
- debug_assert(is_allocated(block_list, allocated, blk_off));
unlock();
return allocated;
}
int64_t BitMapAreaLeaf::alloc_blocks_dis_int(bool wait, int64_t num_blocks,
- int64_t area_blk_off, int64_t *block_list)
+ int64_t area_blk_off, ExtentList *block_list)
{
BitMapArea *child = NULL;
int64_t allocated = 0;
}
blk_off = child->get_index() * m_child_size_blocks + area_blk_off;
- allocated += child->alloc_blocks_dis(num_blocks - allocated,
- blk_off, &block_list[allocated]);
+ allocated += child->alloc_blocks_dis(num_blocks - allocated, blk_off, block_list);
child_unlock(child);
if (allocated == num_blocks) {
break;
BmapEntry::size();
unaligned_blocks = total_blocks % zone_size_block;
+ m_extra_blocks = unaligned_blocks? zone_size_block - unaligned_blocks: 0;
total_blocks = ROUND_UP_TO(total_blocks, zone_size_block);
m_alloc_mode = mode;
/*
* Mark extra padded blocks used from begning.
*/
- set_blocks_used(total_blocks - (zone_size_block - unaligned_blocks),
- (zone_size_block - unaligned_blocks));
+ set_blocks_used(total_blocks - m_extra_blocks, m_extra_blocks);
}
}
if (!reserve_blocks(num_blocks)) {
goto exit;
}
+
if (is_stats_on()) {
m_stats->add_alloc_calls(1);
m_stats->add_allocated(num_blocks);
/*
* Allocate N dis-contiguous blocks.
*/
-int64_t BitAllocator::alloc_blocks_dis(int64_t num_blocks, int64_t *block_list)
+int64_t BitAllocator::alloc_blocks_dis(int64_t num_blocks, ExtentList *block_list)
+{
+ return alloc_blocks_dis_work(num_blocks, block_list, false);
+}
+
+int64_t BitAllocator::alloc_blocks_dis_res(int64_t num_blocks, ExtentList *block_list)
+{
+ return alloc_blocks_dis_work(num_blocks, block_list, true);
+}
+
+int64_t BitAllocator::alloc_blocks_dis_work(int64_t num_blocks, ExtentList *block_list, bool reserved)
{
int scans = 1;
int64_t allocated = 0;
lock_shared();
serial_lock();
- if (!reserve_blocks(num_blocks)) {
+ if (!reserved && !reserve_blocks(num_blocks)) {
goto exit;
}
}
while (scans && allocated < num_blocks) {
- allocated += alloc_blocks_dis_int(false, num_blocks - allocated,
- blk_off, &block_list[allocated]);
+ allocated += alloc_blocks_dis_int(false, num_blocks - allocated, blk_off, block_list);
scans--;
}
unlock();
lock_excl();
serial_lock();
- allocated += alloc_blocks_dis_int(false, num_blocks - allocated,
- blk_off, &block_list[allocated]);
+ allocated += alloc_blocks_dis_int(false, num_blocks - allocated, blk_off, block_list);
if (is_stats_on()) {
m_stats->add_serial_scans(1);
}
}
unreserve(num_blocks, allocated);
- debug_assert(is_allocated(block_list, allocated, 0));
+ debug_assert(is_allocated_dis(block_list, allocated));
exit:
serial_unlock();
return allocated;
}
-void BitAllocator::free_blocks_dis(int64_t num_blocks, int64_t *block_list)
+bool BitAllocator::is_allocated_dis(ExtentList *blocks, int64_t num_blocks)
+{
+ int64_t count = 0;
+ for (int64_t j = 0; j < blocks->get_extent_count(); j++) {
+ auto p = blocks->get_nth_extent(j);
+ count += p.second;
+ if (!is_allocated(p.first, p.second)) {
+ return false;
+ }
+ }
+
+ debug_assert(count == num_blocks);
+ return true;
+}
+
+void BitAllocator::free_blocks_dis(int64_t num_blocks, ExtentList *block_list)
{
+ int64_t freed = 0;
lock_shared();
if (is_stats_on()) {
m_stats->add_free_calls(1);
m_stats->add_freed(num_blocks);
}
- for (int64_t i = 0; i < num_blocks; i++) {
- free_blocks_int(block_list[i], 1);
+ for (int64_t i = 0; i < block_list->get_extent_count(); i++) {
+ free_blocks_int(block_list->get_nth_extent(i).first,
+ block_list->get_nth_extent(i).second);
+ freed += block_list->get_nth_extent(i).second;
}
+ debug_assert(num_blocks == freed);
sub_used_blocks(num_blocks);
debug_assert(get_used_blocks() >= 0);
unlock();
#include <atomic>
#include <vector>
#include "include/intarith.h"
+#include "os/bluestore/bluestore_types.h"
+
class BitAllocatorStats {
public:
int *start_offset, int64_t *scanned);
int find_any_free_bits(int start_offset, int64_t num_blocks,
- int64_t *alloc_list, int64_t block_offset,
+ ExtentList *alloc_list, int64_t block_offset,
int64_t *scanned);
~BmapEntry();
static int get_level(int64_t total_blocks);
static int64_t get_level_factor(int level);
virtual bool is_allocated(int64_t start_block, int64_t num_blocks) = 0;
+ virtual bool is_allocated(ExtentList *blocks, int64_t num_blocks, int blk_off) {
+ debug_assert(0);
+ return true;
+ }
virtual bool is_exhausted() = 0;
virtual bool child_check_n_lock(BitMapArea *child, int64_t required) {
debug_assert(0);
}
virtual int64_t alloc_blocks_dis(bool wait, int64_t num_blocks,
- int64_t blk_off, int64_t *block_list) {
+ int64_t blk_off, ExtentList *block_list) {
debug_assert(0);
return 0;
}
virtual int64_t alloc_blocks_dis(int64_t num_blocks,
- int64_t blk_offset, int64_t *block_list) {
+ int64_t blk_off, ExtentList *block_list) {
debug_assert(0);
return 0;
}
}
int64_t alloc_blocks(int64_t num_blocks, int64_t *start_block);
- int64_t alloc_blocks_dis(int64_t num_blocks, int64_t blk_off, int64_t *block_list);
+ int64_t alloc_blocks_dis(int64_t num_blocks,
+ int64_t blk_off, ExtentList *block_list);
void set_blocks_used(int64_t start_block, int64_t num_blocks);
void free_blocks(int64_t start_block, int64_t num_blocks);
std::mutex m_blocks_lock;
BitMapAreaList *m_child_list;
- bool is_allocated(int64_t start_block, int64_t num_blocks);
- virtual bool is_allocated(int64_t *blocks, int64_t num_blocks, int64_t blk_off);
+ virtual bool is_allocated(int64_t start_block, int64_t num_blocks);
virtual bool is_exhausted();
bool child_check_n_lock(BitMapArea *child, int64_t required, bool lock) {
using BitMapArea::alloc_blocks_dis; //non-wait version
virtual int64_t alloc_blocks(bool wait, int64_t num_blocks, int64_t *start_block);
virtual int64_t alloc_blocks_dis_int(bool wait, int64_t num_blocks,
- int64_t blk_off, int64_t *block_list);
+ int64_t blk_off, ExtentList *block_list);
virtual int64_t alloc_blocks_dis(bool wait, int64_t num_blocks,
- int64_t blk_off, int64_t *block_list);
+ int64_t blk_off, ExtentList *block_list);
virtual void set_blocks_used_int(int64_t start_block, int64_t num_blocks);
virtual void set_blocks_used(int64_t start_block, int64_t num_blocks);
int64_t alloc_blocks_int(bool wait, bool wrap,
int64_t num_blocks, int64_t *start_block);
int64_t alloc_blocks_dis_int(bool wait, int64_t num_blocks,
- int64_t blk_off, int64_t *block_list);
+ int64_t blk_off, ExtentList *block_list);
void free_blocks_int(int64_t start_block, int64_t num_blocks);
virtual ~BitMapAreaLeaf();
pthread_rwlock_t m_rw_lock;
BitAllocatorStats *m_stats;
bool m_is_stats_on;
-
- int64_t truncated_blocks; //see init_check
+ int64_t m_extra_blocks;
bool is_stats_on() {
return m_is_stats_on;
bool check_input_dis(int64_t num_blocks);
void init_check(int64_t total_blocks, int64_t zone_size_block,
bmap_alloc_mode_t mode, bool def, bool stats_on);
+ int64_t alloc_blocks_dis_work(int64_t num_blocks, ExtentList *block_list, bool reserved);
public:
void set_blocks_used(int64_t start_block, int64_t num_blocks);
void unreserve_blocks(int64_t blocks);
- int64_t alloc_blocks_dis(int64_t num_blocks, int64_t *block_list);
- void free_blocks_dis(int64_t num_blocks, int64_t *block_list);
+ int64_t alloc_blocks_dis(int64_t num_blocks, ExtentList *block_list);
+ int64_t alloc_blocks_dis_res(int64_t num_blocks, ExtentList *block_list);
+
+ void free_blocks_dis(int64_t num_blocks, ExtentList *block_list);
+ bool is_allocated_dis(ExtentList *blocks, int64_t num_blocks);
+
+ int64_t size() {
+ return m_total_blocks - m_extra_blocks;
+ }
+ int64_t get_used_blocks() {
+ return BitMapAreaIN::get_used_blocks() - (m_extra_blocks + m_reserved_blocks);
+ }
- int64_t get_truncated_blocks() { return truncated_blocks; }
BitAllocatorStats *get_stats() {
return m_stats;
}
return 0;
}
+int BitMapAllocator::alloc_extents(
+ uint64_t want_size, uint64_t alloc_unit, uint64_t max_alloc_size,
+ int64_t hint, std::vector<AllocExtent> *extents, int *count)
+{
+ assert(!(alloc_unit % m_block_size));
+ assert(alloc_unit);
+
+ assert(!max_alloc_size || max_alloc_size >= alloc_unit);
+
+ dout(10) << __func__ <<" instance "<< (uint64_t) this
+ << " want_size " << want_size
+ << " alloc_unit " << alloc_unit
+ << " hint " << hint
+ << dendl;
+
+ if (alloc_unit > (uint64_t) m_block_size) {
+ return alloc_extents_cont(want_size, alloc_unit, max_alloc_size, hint, extents, count);
+ } else {
+ return alloc_extents_dis(want_size, alloc_unit, max_alloc_size, hint, extents, count);
+ }
+}
+
+/*
+ * Allocator extents with min alloc unit > bitmap block size.
+ */
+int BitMapAllocator::alloc_extents_cont(
+ uint64_t want_size, uint64_t alloc_unit, uint64_t max_alloc_size, int64_t hint,
+ std::vector<AllocExtent> *extents, int *count)
+{
+ *count = 0;
+ assert(alloc_unit);
+ assert(!(alloc_unit % m_block_size));
+ assert(!(max_alloc_size % m_block_size));
+
+ int64_t nblks = (want_size + m_block_size - 1) / m_block_size;
+ int64_t start_blk = 0;
+ int64_t need_blks = nblks;
+ int64_t max_blks = max_alloc_size / m_block_size;
+
+ ExtentList block_list = ExtentList(extents, m_block_size, max_alloc_size);
+
+ while (need_blks > 0) {
+ int64_t count = 0;
+ count = m_bit_alloc->alloc_blocks_res(need_blks > max_blks? max_blks: need_blks,
+ &start_blk);
+ if (count == 0) {
+ break;
+ }
+ dout(30) << __func__ <<" instance "<< (uint64_t) this
+ << " offset " << start_blk << " length " << count << dendl;
+ need_blks -= count;
+ block_list.add_extents(start_blk, count);
+ }
+
+ if (need_blks > 0) {
+ m_bit_alloc->free_blocks_dis(nblks - need_blks, &block_list);
+ return -ENOSPC;
+ }
+ *count = block_list.get_extent_count();
+
+ return 0;
+}
+
+int BitMapAllocator::alloc_extents_dis(
+ uint64_t want_size, uint64_t alloc_unit, uint64_t max_alloc_size,
+ int64_t hint, std::vector<AllocExtent> *extents, int *count)
+{
+ ExtentList block_list = ExtentList(extents, m_block_size, max_alloc_size);
+ int64_t nblks = (want_size + m_block_size - 1) / m_block_size;
+ int64_t num = 0;
+ *count = 0;
+
+ num = m_bit_alloc->alloc_blocks_dis_res(nblks, &block_list);
+ if (num < nblks) {
+ m_bit_alloc->free_blocks_dis(num, &block_list);
+ return -ENOSPC;
+ }
+ *count = block_list.get_extent_count();
+
+ return 0;
+}
+
int BitMapAllocator::release(
uint64_t offset, uint64_t length)
{
void insert_free(uint64_t offset, uint64_t len);
+ int alloc_extents_cont(uint64_t want_size, uint64_t alloc_unit, uint64_t max_alloc_size,
+ int64_t hint, std::vector<AllocExtent> *extents, int *count);
+
+ int alloc_extents_dis(uint64_t want_size, uint64_t alloc_unit, uint64_t max_alloc_size,
+ int64_t hint, std::vector<AllocExtent> *extents, int *count);
+
public:
BitMapAllocator();
BitMapAllocator(int64_t device_size, int64_t block_size);
uint64_t want_size, uint64_t alloc_unit, int64_t hint,
uint64_t *offset, uint32_t *length);
+ int alloc_extents(
+ uint64_t want_size, uint64_t alloc_unit, uint64_t max_alloc_size,
+ int64_t hint, std::vector<AllocExtent> *extents, int *count);
+
int release(
uint64_t offset, uint64_t length);
dout(10) << __func__ << " len 0x" << std::hex << len << std::dec
<< " from " << id << dendl;
assert(id < alloc.size());
+ uint64_t min_alloc_size = g_conf->bluefs_alloc_size;
- uint64_t left = ROUND_UP_TO(len, g_conf->bluefs_alloc_size);
+ uint64_t left = ROUND_UP_TO(len, min_alloc_size);
int r = -ENOSPC;
if (alloc[id]) {
r = alloc[id]->reserve(left);
if (!ev->empty()) {
hint = ev->back().end();
}
- while (left > 0) {
- bluefs_extent_t e;
- e.bdev = id;
- int r = alloc[id]->allocate(left, g_conf->bluefs_alloc_size, hint,
- &e.offset, &e.length);
+
+ int count = 0;
+ std::vector<AllocExtent> extents =
+ std::vector<AllocExtent>(left / min_alloc_size);
+
+ r = alloc[id]->alloc_extents(left, min_alloc_size,
+ hint, &extents, &count);
+ assert(r == 0);
+ for (int i = 0; i < count; i++) {
+ bluefs_extent_t e = bluefs_extent_t(id, extents[i].offset, extents[i].length);
if (r < 0) {
assert(0 == "allocate failed... wtf");
return r;
}
- if (!ev->empty() && ev->back().end() == e.offset)
+ if (!ev->empty() && ev->back().end() == (uint64_t) e.offset) {
ev->back().length += e.length;
- else
+ } else {
ev->push_back(e);
- if (e.length >= left)
- break;
- left -= e.length;
- hint = e.end();
+ }
}
+
return 0;
}
csum_order = std::min(wctx->csum_order, ctz(l->length()));
}
}
- while (final_length > 0) {
- bluestore_pextent_t e;
- uint32_t l;
- uint64_t want = max_alloc_size ? MIN(final_length, max_alloc_size) : final_length;
- int r = alloc->allocate(want, min_alloc_size, hint,
- &e.offset, &l);
- assert(r == 0);
- need -= l;
- e.length = l;
+
+ int count = 0;
+ std::vector<AllocExtent> extents =
+ std::vector<AllocExtent>(final_length / min_alloc_size);
+
+ int r = alloc->alloc_extents(final_length, min_alloc_size, max_alloc_size,
+ hint, &extents, &count);
+
+ need -= final_length;
+ assert(r == 0);
+ for (int i = 0; i < count; i++) {
+ bluestore_pextent_t e = bluestore_pextent_t(extents[i]);
txc->allocated.insert(e.offset, e.length);
txc->statfs_delta.allocated() += e.length;
b->blob.extents.push_back(e);
- final_length -= e.length;
hint = e.end();
}
+
dout(20) << __func__ << " blob " << *b
<< " csum_order " << csum_order
<< " csum_length 0x" << std::hex << csum_length << std::dec
}
}
- assert(0 == "caller didn't reserve?");
return -ENOSPC;
found:
return 0;
}
+int StupidAllocator::alloc_extents(
+ uint64_t want_size, uint64_t alloc_unit, uint64_t max_alloc_size,
+ int64_t hint, std::vector<AllocExtent> *extents, int *count)
+{
+ uint64_t allocated_size = 0;
+ uint64_t offset = 0;
+ uint32_t length = 0;
+ int res = 0;
+
+ if (max_alloc_size == 0) {
+ max_alloc_size = want_size;
+ }
+
+ ExtentList block_list = ExtentList(extents, 1, max_alloc_size);
+
+ while (allocated_size < want_size) {
+ res = allocate(MIN(max_alloc_size, (want_size - allocated_size)),
+ alloc_unit, hint, &offset, &length);
+ if (res != 0) {
+ /*
+ * Allocation failed.
+ */
+ break;
+ }
+ block_list.add_extents(offset, length);
+ allocated_size += length;
+ hint = offset + length;
+ }
+
+ *count = block_list.get_extent_count();
+ if (want_size - allocated_size > 0) {
+ release_extents(extents, *count);
+ return -ENOSPC;
+ }
+
+ return 0;
+}
+
int StupidAllocator::release(
uint64_t offset, uint64_t length)
{
#include "Allocator.h"
#include "include/btree_interval_set.h"
+#include "os/bluestore/bluestore_types.h"
class StupidAllocator : public Allocator {
std::mutex lock;
int reserve(uint64_t need);
void unreserve(uint64_t unused);
+ int alloc_extents(
+ uint64_t want_size, uint64_t alloc_unit, uint64_t max_alloc_size,
+ int64_t hint, std::vector<AllocExtent> *extents, int *count);
+
int allocate(
uint64_t want_size, uint64_t alloc_unit, int64_t hint,
uint64_t *offset, uint32_t *length);
{
vector<bluefs_extent_t>::iterator p = extents.begin();
while (p != extents.end()) {
- if (offset >= p->length) {
+ if ((int64_t) offset >= p->length) {
offset -= p->length;
++p;
} else {
#include "include/utime.h"
#include "include/encoding.h"
-struct bluefs_extent_t {
- uint64_t offset;
- uint32_t length;
+class bluefs_extent_t : public AllocExtent{
+public:
uint16_t bdev;
bluefs_extent_t(uint16_t b = 0, uint64_t o = 0, uint32_t l = 0)
- : offset(o), length(l), bdev(b) {}
-
- uint64_t end() const {
- return offset + length;
- }
+ : AllocExtent(o, l), bdev(b) {}
void encode(bufferlist&) const;
void decode(bufferlist::iterator&);
#include "common/Checksummer.h"
#include "include/stringify.h"
+void ExtentList::add_extents(int64_t start, int64_t count) {
+ AllocExtent *last_extent = NULL;
+ bool can_merge = false;
+
+ if (m_num_extents > 0) {
+ last_extent = &((*m_extents)[m_num_extents - 1]);
+ uint64_t last_offset = (last_extent->offset + last_extent->length) /
+ m_block_size;
+ uint32_t last_length = last_extent->length / m_block_size;
+ int64_t max_blocks = m_max_alloc_size / m_block_size;
+ if ((last_offset == (uint64_t) start) &&
+ (!max_blocks || (last_length + count) <= max_blocks)) {
+ can_merge = true;
+ }
+ }
+
+ if (can_merge) {
+ last_extent->length += (count * m_block_size);
+ } else {
+ (*m_extents)[m_num_extents].offset = start * m_block_size;
+ (*m_extents)[m_num_extents].length = count * m_block_size;
+ m_num_extents++;
+ }
+ assert((int64_t) m_extents->size() >= m_num_extents);
+}
+
// bluestore_bdev_label_t
void bluestore_bdev_label_t::encode(bufferlist& bl) const
};
WRITE_CLASS_ENCODER(bluestore_cnode_t)
-/// pextent: physical extent
-struct bluestore_pextent_t {
- const static uint64_t INVALID_OFFSET = ~0ull;
+class AllocExtent {
+public:
+ uint64_t offset;
+ uint32_t length;
- uint64_t offset, length; ///< location on device
-
- bluestore_pextent_t() : offset(0), length(0) {}
- bluestore_pextent_t(uint64_t o, uint64_t l) : offset(o), length(l) {}
+ AllocExtent() {
+ offset = 0;
+ length = 0;
+ }
+ AllocExtent(int64_t off, int32_t len) : offset(off), length(len) { }
uint64_t end() const {
return offset + length;
}
+};
+
+class ExtentList {
+ std::vector<AllocExtent> *m_extents;
+ int64_t m_num_extents;
+ int64_t m_block_size;
+ uint64_t m_max_alloc_size;
+
+public:
+ void init(std::vector<AllocExtent> *extents, int64_t block_size, uint64_t max_alloc_size) {
+ m_extents = extents;
+ m_num_extents = 0;
+ m_block_size = block_size;
+ m_max_alloc_size = max_alloc_size;
+ }
+
+ ExtentList(std::vector<AllocExtent> *extents, int64_t block_size) {
+ init(extents, block_size, 0);
+ }
+
+ ExtentList(std::vector<AllocExtent> *extents, int64_t block_size, uint64_t max_alloc_size) {
+ init(extents, block_size, max_alloc_size);
+ }
+
+ void reset() {
+ m_num_extents = 0;
+ }
+
+ void add_extents(int64_t start, int64_t count);
+
+ std::vector<AllocExtent> *get_extents() {
+ return m_extents;
+ }
+
+ std::pair<int64_t, int64_t> get_nth_extent(int index) {
+ return std::make_pair
+ ((*m_extents)[index].offset / m_block_size,
+ (*m_extents)[index].length / m_block_size);
+ }
+
+ int64_t get_extent_count() {
+ return m_num_extents;
+ }
+};
+
+
+/// pextent: physical extent
+struct bluestore_pextent_t : public AllocExtent{
+ const static uint64_t INVALID_OFFSET = ~0ull;
+
+ bluestore_pextent_t() : AllocExtent() {}
+ bluestore_pextent_t(uint64_t o, uint64_t l) : AllocExtent(o, l) {}
+ bluestore_pextent_t(AllocExtent &ext) : AllocExtent(ext.offset, ext.length) { }
bool is_valid() const {
return offset != INVALID_OFFSET;
unittest_bit_alloc_CXXFLAGS = $(UNITTEST_CXXFLAGS)
check_TESTPROGRAMS += unittest_bit_alloc
+unittest_alloc_SOURCES = test/objectstore/Allocator_test.cc
+unittest_alloc_LDADD = $(LIBOS) $(UNITTEST_LDADD) $(CEPH_GLOBAL)
+unittest_alloc_CXXFLAGS = $(UNITTEST_CXXFLAGS)
+check_TESTPROGRAMS += unittest_alloc
+
unittest_bluestore_types_SOURCES = test/objectstore/test_bluestore_types.cc
unittest_bluestore_types_LDADD = $(LIBOS) $(UNITTEST_LDADD) $(CEPH_GLOBAL)
unittest_bluestore_types_CXXFLAGS = $(UNITTEST_CXXFLAGS)
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * In memory space allocator test cases.
+ * Author: Ramesh Chander, Ramesh.Chander@sandisk.com
+ */
+#include "os/bluestore/Allocator.h"
+#include "global/global_init.h"
+#include <iostream>
+#include "include/Context.h"
+#include "common/ceph_argparse.h"
+#include "global/global_init.h"
+#include "common/Mutex.h"
+#include "common/Cond.h"
+#include "common/errno.h"
+#include "include/stringify.h"
+#include <gtest/gtest.h>
+#include <os/bluestore/BitAllocator.h>
+
+#if GTEST_HAS_PARAM_TEST
+
+class AllocTest : public ::testing::TestWithParam<const char*> {
+public:
+ boost::scoped_ptr<Allocator> alloc;
+ AllocTest(): alloc(0) { }
+ void init_alloc(int64_t size, uint64_t min_alloc_size) {
+ std::cout << "Creating alloc type " << string(GetParam()) << " \n";
+ alloc.reset(Allocator::create(string(GetParam()), size, min_alloc_size));
+ }
+
+ void init_close() {
+ alloc.reset(0);
+ }
+};
+
+TEST_P(AllocTest, test_alloc_init)
+{
+ int64_t blocks = BmapEntry::size();
+ init_alloc(blocks, 1);
+ ASSERT_EQ(alloc->get_free(), 0);
+ alloc->shutdown();
+ blocks = BitMapZone::get_total_blocks() * 2 + 16;
+ init_alloc(blocks, 1);
+ ASSERT_EQ(alloc->get_free(), 0);
+ alloc->shutdown();
+ blocks = BitMapZone::get_total_blocks() * 2;
+ init_alloc(blocks, 1);
+ ASSERT_EQ(alloc->get_free(), 0);
+}
+
+TEST_P(AllocTest, test_alloc_min_alloc)
+{
+ int64_t block_size = 1024;
+ int64_t blocks = BitMapZone::get_total_blocks() * 2 * block_size;
+ uint64_t offset = 0;
+ uint32_t length = 0;
+ int count = 0;
+
+ init_alloc(blocks, block_size);
+ alloc->init_add_free(block_size, block_size);
+ EXPECT_EQ(alloc->reserve(block_size), 0);
+ EXPECT_EQ(alloc->allocate(block_size, block_size, 0, &offset, &length), 0);
+
+ /*
+ * Allocate extent and make sure all comes in single extent.
+ */
+ {
+ alloc->init_add_free(0, block_size * 4);
+ EXPECT_EQ(alloc->reserve(block_size * 4), 0);
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (4, AllocExtent(0, 0));
+
+ EXPECT_EQ(alloc->alloc_extents(4 * (uint64_t)block_size, (uint64_t) block_size,
+ 0, (int64_t) 0, &extents, &count), 0);
+ EXPECT_EQ(extents[0].length, 4 * block_size);
+ EXPECT_EQ(extents[1].length, 0);
+ EXPECT_EQ(count, 1);
+ }
+
+ /*
+ * Allocate extent and make sure we get two different extents.
+ */
+ {
+ alloc->init_add_free(0, block_size * 2);
+ alloc->init_add_free(3 * block_size, block_size * 2);
+ EXPECT_EQ(alloc->reserve(block_size * 4), 0);
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (4, AllocExtent(0, 0));
+
+ EXPECT_EQ(alloc->alloc_extents(4 * (uint64_t)block_size, (uint64_t) block_size,
+ 0, (int64_t) 0, &extents, &count), 0);
+ EXPECT_EQ(extents[0].length, 2 * block_size);
+ EXPECT_EQ(extents[1].length, 2 * block_size);
+ EXPECT_EQ(extents[2].length, 0);
+ EXPECT_EQ(count, 2);
+ }
+ alloc->shutdown();
+}
+
+TEST_P(AllocTest, test_alloc_min_max_alloc)
+{
+ int64_t block_size = 1024;
+ int64_t blocks = BitMapZone::get_total_blocks() * 2 * block_size;
+ int count = 0;
+
+ init_alloc(blocks, block_size);
+
+ /*
+ * Make sure we get all extents different when
+ * min_alloc_size == max_alloc_size
+ */
+ {
+ alloc->init_add_free(0, block_size * 4);
+ EXPECT_EQ(alloc->reserve(block_size * 4), 0);
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (4, AllocExtent(0, 0));
+
+ EXPECT_EQ(alloc->alloc_extents(4 * (uint64_t)block_size, (uint64_t) block_size,
+ block_size, (int64_t) 0, &extents, &count), 0);
+ for (int i = 0; i < 4; i++) {
+ EXPECT_EQ(extents[i].length, block_size);
+ }
+ EXPECT_EQ(count, 4);
+ }
+
+
+ /*
+ * Make sure we get extents of length max_alloc size
+ * when max alloc size > min_alloc size
+ */
+ {
+ alloc->init_add_free(0, block_size * 4);
+ EXPECT_EQ(alloc->reserve(block_size * 4), 0);
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (2, AllocExtent(0, 0));
+
+ EXPECT_EQ(alloc->alloc_extents(4 * (uint64_t)block_size, (uint64_t) block_size,
+ 2 * block_size, (int64_t) 0, &extents, &count), 0);
+ for (int i = 0; i < 2; i++) {
+ EXPECT_EQ(extents[i].length, block_size * 2);
+ }
+ EXPECT_EQ(count, 2);
+ }
+
+ /*
+ * Allocate and free.
+ */
+ {
+ alloc->init_add_free(0, block_size * 16);
+ EXPECT_EQ(alloc->reserve(block_size * 16), 0);
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (8, AllocExtent(0, 0));
+
+ EXPECT_EQ(alloc->alloc_extents(16 * (uint64_t)block_size, (uint64_t) block_size,
+ 2 * block_size, (int64_t) 0, &extents, &count), 0);
+
+ EXPECT_EQ(count, 8);
+ for (int i = 0; i < 8; i++) {
+ EXPECT_EQ(extents[i].length, 2 * block_size);
+ }
+ EXPECT_EQ(alloc->release_extents(&extents, count), 0);
+ }
+}
+
+TEST_P(AllocTest, test_alloc_failure)
+{
+ int64_t block_size = 1024;
+ int64_t blocks = BitMapZone::get_total_blocks() * block_size;
+ int count = 0;
+
+ init_alloc(blocks, block_size);
+ {
+ alloc->init_add_free(0, block_size * 256);
+ alloc->init_add_free(block_size * 512, block_size * 256);
+
+ EXPECT_EQ(alloc->reserve(block_size * 512), 0);
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (4, AllocExtent(0, 0));
+
+ EXPECT_EQ(alloc->alloc_extents(512 * (uint64_t)block_size, (uint64_t) block_size * 256,
+ block_size * 256, (int64_t) 0, &extents, &count), 0);
+ alloc->init_add_free(0, block_size * 256);
+ alloc->init_add_free(block_size * 512, block_size * 256);
+ EXPECT_EQ(alloc->reserve(block_size * 512), 0);
+ EXPECT_EQ(alloc->alloc_extents(512 * (uint64_t)block_size, (uint64_t) block_size * 512,
+ block_size * 512, (int64_t) 0, &extents, &count), -ENOSPC);
+ }
+}
+
+INSTANTIATE_TEST_CASE_P(
+ Allocator,
+ AllocTest,
+ ::testing::Values("stupid", "bitmap"));
+
+#else
+
+TEST(DummyTest, ValueParameterizedTestsAreNotSupportedOnThisPlatform) {}
+#endif
+
+int main(int argc, char **argv)
+{
+ vector<const char*> args;
+ argv_to_vec(argc, (const char **)argv, args);
+ env_to_vec(args);
+
+ global_init(NULL, args, CEPH_ENTITY_TYPE_CLIENT, CODE_ENVIRONMENT_UTILITY, 0);
+
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
#include <sstream>
#include <gtest/gtest.h>
-#define bmap_test_assert(x) EXPECT_EQ(true, (x))
+#define bmap_test_assert(x) ASSERT_EQ(true, (x))
#define NUM_THREADS 16
#define MAX_BLOCKS (1024 * 1024 * 1)
for (i = 0; i < num_items; i++) {
(*arr)[i].init(i);
}
- //BitMapList<BmapEntityTmp> *list = new BitMapList<BmapEntityTmp>(arr, num_items, 0);
BitMapEntityIter<BmapEntityTmp> iter = BitMapEntityIter<BmapEntityTmp>(arr, off, false);
i = off;
zone = new BitMapZone(total_blocks, 0);
lock = zone->lock_excl_try();
bmap_test_assert(lock);
- int64_t blocks[1024] = {0};
for (int i = 0; i < zone->size(); i++) {
allocated = zone->alloc_blocks(1, &start_block);
bmap_test_assert(allocated == 1);
zone->free_blocks(i, 1);
}
- allocated = zone->alloc_blocks_dis(zone->size() / 2, 0, blocks);
+ int64_t blk_size = 1024;
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (zone->size() / 2, AllocExtent(-1, -1));
+
+ ExtentList *block_list = new ExtentList(&extents, blk_size);
+ allocated = zone->alloc_blocks_dis(zone->size() / 2, 0, block_list);
bmap_test_assert(allocated == zone->size() / 2);
}
alloc->free_blocks(i, 1);
}
- int64_t blocks[alloc->size() / 2];
- memset(blocks, 0, sizeof(blocks));
- allocated = alloc->alloc_blocks_dis(alloc->size()/2, blocks);
- bmap_test_assert(allocated == alloc->size() / 2);
+ int64_t blk_size = 1024;
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (alloc->size(), AllocExtent(-1, -1));
- allocated = alloc->alloc_blocks_dis(1, blocks);
- bmap_test_assert(allocated == 0);
+ ExtentList *block_list = new ExtentList(&extents, blk_size);
- alloc->free_blocks(alloc->size()/2, 1);
- allocated = alloc->alloc_blocks_dis(1, blocks);
+ allocated = alloc->alloc_blocks_dis(alloc->size()/2, block_list);
+ bmap_test_assert(allocated == alloc->size() / 2);
- bmap_test_assert(allocated == 1);
- bmap_test_assert(blocks[0] == alloc->size()/2);
+ block_list->reset();
+ allocated = alloc->alloc_blocks_dis(1, block_list);
+ bmap_test_assert(allocated == 0);
- alloc->free_blocks(0, alloc->size());
- delete alloc;
+ alloc->free_blocks(alloc->size()/2, 1);
+
+ block_list->reset();
+ allocated = alloc->alloc_blocks_dis(1, block_list);
+ bmap_test_assert(allocated == 1);
+
+ bmap_test_assert((int64_t) extents[0].offset == alloc->size()/2 * blk_size);
+
+ delete alloc;
// unaligned zones
total_blocks = zone_size * 2 + 11;
g_ceph_context->_conf->apply_changes(NULL);
}
+bool alloc_extents_max_block(BitAllocator *alloc,
+ int64_t max_alloc,
+ int64_t total_alloc)
+{
+ int64_t blk_size = 1;
+ int64_t allocated = 0;
+ int64_t verified = 0;
+ int64_t count = 0;
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (total_alloc, AllocExtent(-1, -1));
+
+ ExtentList *block_list = new ExtentList(&extents, blk_size, max_alloc);
+
+ allocated = alloc->alloc_blocks_dis(total_alloc, block_list);
+ EXPECT_EQ(allocated, total_alloc);
+
+ max_alloc = total_alloc > max_alloc? max_alloc: total_alloc;
+
+ for (auto &p: extents) {
+ count++;
+ EXPECT_EQ(p.length, max_alloc);
+ verified += p.length;
+ if (verified >= total_alloc) {
+ break;
+ }
+ }
+
+ EXPECT_EQ(total_alloc / max_alloc, count);
+ return true;
+}
+
+TEST(BitAllocator2, test_bmap_alloc)
+{
+ int64_t total_blocks = 1024 * 4;
+ int64_t zone_size = 1024;
+ BitAllocator *alloc = new BitAllocator(total_blocks, zone_size, CONCURRENT);
+
+ alloc_extents_max_block(alloc, 1, 16);
+ alloc_extents_max_block(alloc, 4, 16);
+ alloc_extents_max_block(alloc, 16, 16);
+ alloc_extents_max_block(alloc, 32, 16);
+}
+
void
verify_blocks(int64_t num_blocks, int64_t *blocks)
{
while (num_iters--) {
printf("Allocating in tid %d.\n", my_tid);
+ debug_assert(alloc->reserve_blocks(num_blocks));
for (int i = 0; i < num_blocks; i++) {
- alloced = alloc->alloc_blocks(1, &start_block);
+ alloced = alloc->alloc_blocks_res(1, &start_block);
bmap_test_assert(alloced == 1);
total_alloced++;
allocated_blocks[i] = start_block;
}
}
+void
+do_work_dis(BitAllocator *alloc)
+{
+ int num_iters = 10;
+ int64_t alloced = 0;
+ int64_t num_blocks = alloc->size() / NUM_THREADS;
+
+ std::vector<AllocExtent> extents = std::vector<AllocExtent>
+ (num_blocks, AllocExtent(-1, -1));
+ ExtentList *block_list = new ExtentList(&extents, 4096);
+
+ while (num_iters--) {
+ debug_assert(alloc->reserve_blocks(num_blocks));
+ alloced = alloc->alloc_blocks_dis_res(num_blocks, block_list);
+ debug_assert(alloced == num_blocks);
+
+ debug_assert(alloc->is_allocated_dis(block_list, num_blocks));
+ alloc->free_blocks_dis(num_blocks, block_list);
+ block_list->reset();
+ }
+}
+
int tid = 0;
+static bool cont = true;
+
void *
worker(void *args)
{
my_tid = __sync_fetch_and_add(&tid, 1);
BitAllocator *alloc = (BitAllocator *) args;
printf("Starting thread %d", my_tid);
- do_work(alloc);
+ if (cont) {
+ do_work(alloc);
+ } else {
+ do_work_dis(alloc);
+ }
+
return NULL;
}
bmap_test_assert(total_blocks <= MAX_BLOCKS);
BitAllocator *alloc = new BitAllocator(total_blocks, zone_size, CONCURRENT);
- printf("Spawning %d threads for parallel test.....\n", NUM_THREADS);
- for (int j = 0; j < NUM_THREADS; j++) {
- if (pthread_create(&pthreads[j], NULL, worker, alloc)) {
- printf("Unable to create worker thread.\n");
- exit(0);
+ for (int k = 0; k < 2; k++) {
+ cont = k;
+ printf("Spawning %d threads for parallel test. Mode Cont = %d.....\n", NUM_THREADS, cont);
+ for (int j = 0; j < NUM_THREADS; j++) {
+ if (pthread_create(&pthreads[j], NULL, worker, alloc)) {
+ printf("Unable to create worker thread.\n");
+ exit(0);
+ }
}
- }
- for (int j = 0; j < NUM_THREADS; j++) {
- pthread_join(pthreads[j], NULL);
+ for (int j = 0; j < NUM_THREADS; j++) {
+ pthread_join(pthreads[j], NULL);
+ }
}
- // max_blks / num threads and free those. Make sure threads
- // always gets blocks
- // Do this with dis-contiguous and contiguous allocations
- // do multithreaded allocation and check allocations are unique
}
int main(int argc, char **argv)
add_ceph_unittest(unittest_bit_alloc ${CMAKE_RUNTIME_OUTPUT_DIRECTORY}/unittest_bit_alloc)
target_link_libraries(unittest_bit_alloc os global)
+add_executable(unittest_alloc
+ Allocator_test.cc
+ )
+add_ceph_unittest(unittest_alloc ${CMAKE_RUNTIME_OUTPUT_DIRECTORY}/unittest_alloc)
+target_link_libraries(unittest_alloc os global)
+
# unittest_bluefs
add_executable(unittest_bluefs
test_bluefs.cc