Compare commits

..

4 Commits

Author SHA1 Message Date
2f3b1f37a2 WIP "Heap" metadata storage scheme
Some checks failed
Test / test_rebalance_verify_ec_imm (push) Successful in 1m39s
Test / test_write_no_same (push) Successful in 8s
Test / test_write (push) Successful in 32s
Test / test_rebalance_verify_imm (push) Failing after 2m15s
Test / test_write_xor (push) Successful in 35s
Test / test_heal_pg_size_2 (push) Successful in 2m18s
Test / test_heal_local_read (push) Successful in 2m17s
Test / test_heal_ec (push) Successful in 2m20s
Test / test_heal_antietcd (push) Successful in 2m20s
Test / test_heal_csum_32k_dmj (push) Successful in 2m20s
Test / test_heal_csum_32k (push) Successful in 2m19s
Test / test_heal_csum_4k_dmj (push) Successful in 2m18s
Test / test_heal_csum_32k_dj (push) Failing after 2m29s
Test / test_resize_auto (push) Successful in 8s
Test / test_snapshot_pool2 (push) Successful in 14s
Test / test_osd_tags (push) Successful in 7s
Test / test_enospc (push) Successful in 9s
Test / test_enospc_xor (push) Successful in 13s
Test / test_enospc_imm (push) Successful in 10s
Test / test_enospc_imm_xor (push) Successful in 13s
Test / test_scrub (push) Successful in 15s
Test / test_scrub_zero_osd_2 (push) Successful in 13s
Test / test_scrub_xor (push) Successful in 15s
Test / test_heal_csum_4k_dj (push) Successful in 2m18s
Test / test_heal_csum_4k (push) Successful in 2m18s
Test / test_scrub_pg_size_3 (push) Successful in 16s
Test / test_scrub_pg_size_6_pg_minsize_4_osd_count_6_ec (push) Successful in 18s
Test / test_scrub_ec (push) Successful in 15s
Test / test_nfs (push) Successful in 13s
Test / test_resize (push) Failing after 3m5s
2025-05-18 11:56:16 +03:00
abd5cbfbe4 Pass clean_bitmap explicitly 2025-05-18 11:55:04 +03:00
db915184c6 Allow to purge running OSDs again, as in 2.1.0 and earlier
All checks were successful
Test / test_rebalance_verify_ec_imm (push) Successful in 1m40s
Test / test_write_no_same (push) Successful in 8s
Test / test_switch_primary (push) Successful in 31s
Test / test_write (push) Successful in 32s
Test / test_write_xor (push) Successful in 36s
Test / test_heal_pg_size_2 (push) Successful in 2m14s
Test / test_heal_local_read (push) Successful in 2m18s
Test / test_heal_ec (push) Successful in 2m17s
Test / test_heal_antietcd (push) Successful in 2m19s
Test / test_heal_csum_32k_dmj (push) Successful in 2m21s
Test / test_heal_csum_32k_dj (push) Successful in 2m21s
Test / test_heal_csum_32k (push) Successful in 2m20s
Test / test_heal_csum_4k_dmj (push) Successful in 2m19s
Test / test_resize (push) Successful in 13s
Test / test_resize_auto (push) Successful in 9s
Test / test_osd_tags (push) Successful in 6s
Test / test_snapshot_pool2 (push) Successful in 13s
Test / test_enospc (push) Successful in 9s
Test / test_enospc_xor (push) Successful in 14s
Test / test_enospc_imm_xor (push) Successful in 14s
Test / test_scrub (push) Successful in 14s
Test / test_scrub_zero_osd_2 (push) Successful in 14s
Test / test_scrub_xor (push) Successful in 16s
Test / test_scrub_pg_size_3 (push) Successful in 12s
Test / test_heal_csum_4k_dj (push) Successful in 2m21s
Test / test_scrub_pg_size_6_pg_minsize_4_osd_count_6_ec (push) Successful in 17s
Test / test_scrub_ec (push) Successful in 14s
Test / test_heal_csum_4k (push) Successful in 2m18s
Test / test_nfs (push) Successful in 10s
Test / test_enospc_imm (push) Successful in 11s
2025-05-11 13:59:28 +03:00
5ae6fea49c Add a note about local reads 2025-05-11 01:23:48 +03:00
16 changed files with 414 additions and 82 deletions

View File

@@ -310,6 +310,10 @@ If you set this parameter to "random", clients will try to distribute read reque
all available secondary OSDs. This mode is mainly useful for tests, but, probably, not
really required in production setups.
[PG locks](osd.en.md#enable_pg_locks) are required for local reads to function. However,
PG locks are enabled automatically by default for pools with enabled local reads, so you
don't have to enable them explicitly.
## max_osd_combinations
- Type: integer

View File

@@ -309,6 +309,9 @@ meow недоступен".
запросы чтения по всем доступным вторичным OSD. Этот режим в основном полезен для тестов,
но, скорее всего, редко нужен в реальных инсталляциях.
Для работы локальных чтений требуются [блокировки PG](osd.ru.md#enable_pg_locks). Включать
их явно не нужно - они включаются автоматически для пулов с включёнными локальными чтениями.
## max_osd_combinations
- Тип: целое число

View File

@@ -31,6 +31,7 @@
#define DEFAULT_DATA_BLOCK_ORDER 17
#define MIN_DATA_BLOCK_SIZE 4*1024
#define MAX_DATA_BLOCK_SIZE 128*1024*1024
#define MAX_META_BLOCK_SIZE 64*1024
#define DEFAULT_BITMAP_GRANULARITY 4096
#define BS_OP_MIN 1

View File

@@ -127,9 +127,9 @@ void blockstore_disk_t::parse_config(std::map<std::string, std::string> & config
{
throw std::runtime_error("meta_block_size must be a multiple of "+std::to_string(DIRECT_IO_ALIGNMENT));
}
else if (meta_block_size > MAX_DATA_BLOCK_SIZE)
else if (meta_block_size > MAX_META_BLOCK_SIZE)
{
throw std::runtime_error("meta_block_size must not exceed "+std::to_string(MAX_DATA_BLOCK_SIZE));
throw std::runtime_error("meta_block_size must not exceed "+std::to_string(MAX_META_BLOCK_SIZE));
}
if (data_offset % disk_alignment)
{

View File

@@ -427,13 +427,6 @@ stop_flusher:
printf("Flushing %jx:%jx v%ju\n", cur.oid.inode, cur.oid.stripe, cur.version);
#endif
flusher->active_flushers++;
// Find it in clean_db
{
auto & clean_db = bs->clean_db_shard(cur.oid);
auto clean_it = clean_db.find(cur.oid);
old_clean_ver = (clean_it != clean_db.end() ? clean_it->second.version : 0);
old_clean_loc = (clean_it != clean_db.end() ? clean_it->second.location : UINT64_MAX);
}
// Scan dirty versions of the object to determine what we need to read
scan_dirty();
// Writes and deletes shouldn't happen at the same time
@@ -538,7 +531,7 @@ resume_2:
{
// zero out old metadata entry
{
clean_disk_entry *old_entry = (clean_disk_entry*)((uint8_t*)meta_old.buf + meta_old.pos*bs->dsk.clean_entry_size);
clean_disk_entry *old_entry = (clean_disk_entry*)((uint8_t*)meta_old.buf + meta_old.pos);
if (old_entry->oid.inode != 0 && old_entry->oid != cur.oid)
{
printf("Fatal error (metadata corruption or bug): tried to wipe metadata entry %ju (%jx:%jx v%ju) as old location of %jx:%jx\n",
@@ -547,7 +540,7 @@ resume_2:
exit(1);
}
}
memset((uint8_t*)meta_old.buf + meta_old.pos*bs->dsk.clean_entry_size, 0, bs->dsk.clean_entry_size);
memset((uint8_t*)meta_old.buf + meta_old.pos, 0, bs->dsk.clean_entry_size);
resume_20:
if (meta_old.sector != meta_new.sector && !write_meta_block(meta_old, 20))
return false;
@@ -608,7 +601,7 @@ resume_2:
void journal_flusher_co::update_metadata_entry()
{
clean_disk_entry *new_entry = (clean_disk_entry*)((uint8_t*)meta_new.buf + meta_new.pos*bs->dsk.clean_entry_size);
clean_disk_entry *new_entry = (clean_disk_entry*)((uint8_t*)meta_new.buf + meta_new.pos);
if (new_entry->oid.inode != 0 && new_entry->oid != cur.oid)
{
printf(
@@ -623,7 +616,7 @@ void journal_flusher_co::update_metadata_entry()
if (has_delete)
{
// Zero out the new metadata entry
memset((uint8_t*)meta_new.buf + meta_new.pos*bs->dsk.clean_entry_size, 0, bs->dsk.clean_entry_size);
memset((uint8_t*)meta_new.buf + meta_new.pos, 0, bs->dsk.clean_entry_size);
}
else
{
@@ -805,7 +798,7 @@ bool journal_flusher_co::clear_incomplete_csum_block_bits(int wait_base)
}
}
{
clean_disk_entry *new_entry = (clean_disk_entry*)((uint8_t*)meta_new.buf + meta_new.pos*bs->dsk.clean_entry_size);
clean_disk_entry *new_entry = (clean_disk_entry*)((uint8_t*)meta_new.buf + meta_new.pos);
if (new_entry->oid != cur.oid)
{
printf(
@@ -912,6 +905,12 @@ void journal_flusher_co::calc_block_checksums(uint32_t *new_data_csums, bool ski
void journal_flusher_co::scan_dirty()
{
// Find it in clean_db
auto & clean_db = bs->clean_db_shard(cur.oid);
auto clean_it = clean_db.find(cur.oid);
old_clean_ver = (clean_it != clean_db.end() ? clean_it->second.version : 0);
old_clean_loc = (clean_it != clean_db.end() ? clean_it->second.location : UINT64_MAX);
auto old_clean_bitmap = (clean_it != clean_db.end() ? bs->get_clean_entry_bitmap(clean_it, 0) : NULL);
dirty_it = dirty_start = dirty_end;
v.clear();
copy_count = 0;
@@ -1037,13 +1036,12 @@ void journal_flusher_co::scan_dirty()
read_to_fill_incomplete = 0;
return;
}
uint8_t *bmp_ptr = bs->get_clean_entry_bitmap(old_clean_loc, 0);
uint64_t fulfilled = 0;
int last = v.size()-1;
while (last >= 0 && (v[last].copy_flags & COPY_BUF_CSUM_FILL))
last--;
read_to_fill_incomplete = bs->fill_partial_checksum_blocks(
v, fulfilled, bmp_ptr, NULL, false, NULL, v[0].offset/bs->dsk.csum_block_size * bs->dsk.csum_block_size,
v, fulfilled, old_clean_bitmap, NULL, false, NULL, v[0].offset/bs->dsk.csum_block_size * bs->dsk.csum_block_size,
((v[last].offset+v[last].len-1) / bs->dsk.csum_block_size + 1) * bs->dsk.csum_block_size
);
}
@@ -1139,7 +1137,7 @@ bool journal_flusher_co::modify_meta_do_reads(int wait_base)
resume_0:
if (!modify_meta_read(clean_loc, meta_new, wait_base+0))
return false;
new_clean_bitmap = (uint8_t*)meta_new.buf + meta_new.pos*bs->dsk.clean_entry_size + sizeof(clean_disk_entry);
new_clean_bitmap = (uint8_t*)meta_new.buf + meta_new.pos + sizeof(clean_disk_entry);
if (old_clean_loc != UINT64_MAX && old_clean_loc != clean_loc)
{
resume_1:
@@ -1193,7 +1191,7 @@ bool journal_flusher_co::modify_meta_read(uint64_t meta_loc, flusher_meta_write_
// so I'll avoid it as long as I can.
wr.submitted = false;
wr.sector = ((meta_loc >> bs->dsk.block_order) / (bs->dsk.meta_block_size / bs->dsk.clean_entry_size)) * bs->dsk.meta_block_size;
wr.pos = ((meta_loc >> bs->dsk.block_order) % (bs->dsk.meta_block_size / bs->dsk.clean_entry_size));
wr.pos = ((meta_loc >> bs->dsk.block_order) % (bs->dsk.meta_block_size / bs->dsk.clean_entry_size)) * bs->dsk.clean_entry_size;
if (bs->inmemory_meta)
{
wr.buf = (uint8_t*)bs->metadata_buffer + wr.sector;

View File

@@ -42,6 +42,8 @@ blockstore_impl_t::~blockstore_impl_t()
free(metadata_buffer);
if (clean_bitmaps)
free(clean_bitmaps);
if (heap_meta.blocks)
delete[] heap_meta.blocks;
}
bool blockstore_impl_t::is_started()
@@ -431,13 +433,29 @@ blockstore_clean_db_t& blockstore_impl_t::clean_db_shard(object_id oid)
{
uint64_t pg_num = 0;
uint64_t pool_id = (oid.inode >> (64-POOL_ID_BITS));
auto sh_it = clean_db_settings.find(pool_id);
if (sh_it != clean_db_settings.end())
auto sett_it = clean_db_settings.find(pool_id);
if (sett_it != clean_db_settings.end())
{
// like map_to_pg()
pg_num = (oid.stripe / sh_it->second.pg_stripe_size) % sh_it->second.pg_count + 1;
pg_num = (oid.stripe / sett_it->second.pg_stripe_size) % sett_it->second.pg_count + 1;
}
return clean_db_shards[(pool_id << (64-POOL_ID_BITS)) | pg_num];
auto shard_id = (pool_id << (64-POOL_ID_BITS)) | pg_num;
if (dsk.meta_format == BLOCKSTORE_META_FORMAT_HEAP)
{
auto sh_it = clean_db_shards.find(shard_id);
if (sh_it == clean_db_shards.end())
{
// clean_db_t stores larger entries with heap_meta, but we disguise it as smaller clean_entry :)
// patched cpp-btree with extra_data
clean_db_shards[shard_id] = blockstore_clean_db_t(
sizeof(clean_entry_heap_t) - sizeof(clean_entry)
+ (inmemory_meta ? dsk.clean_dyn_size : 2*dsk.clean_entry_bitmap_size)
);
return clean_db_shards[shard_id];
}
return sh_it->second;
}
return clean_db_shards[shard_id];
}
void blockstore_impl_t::reshard_clean_db(pool_id_t pool, uint32_t pg_count, uint32_t pg_stripe_size)

View File

@@ -96,6 +96,9 @@
#define BLOCKSTORE_META_MAGIC_V1 0x726F747341544956l
#define BLOCKSTORE_META_FORMAT_V1 1
#define BLOCKSTORE_META_FORMAT_V2 2
#define BLOCKSTORE_META_FORMAT_HEAP 3
#define BLOCKSTORE_META_HEADER_V1_SIZE 36
#define BLOCKSTORE_META_HEADER_V2_SIZE 48
// metadata header (superblock)
struct __attribute__((__packed__)) blockstore_meta_header_v1_t
@@ -119,6 +122,7 @@ struct __attribute__((__packed__)) blockstore_meta_header_v2_t
uint32_t data_csum_type;
uint32_t csum_block_size;
uint32_t header_csum;
uint32_t block_id_bits; // 32 by default in heap meta
};
// 32 bytes = 24 bytes + block bitmap (4 bytes by default) + external attributes (also bitmap, 4 bytes by default)
@@ -140,6 +144,62 @@ struct __attribute__((__packed__)) clean_entry
uint64_t location;
};
typedef uint32_t heap_block_num_t;
// 50 = 16 (key=object_id) + 26 (value) + 8 (bitmap) + N (checksum) bytes per "clean" entry in memory
struct __attribute__((__packed__)) clean_entry_heap_t
{
uint64_t version;
uint64_t location; // UINT64_MAX = deleted
// previous versions invalidated by this version
heap_block_num_t prev_versions;
// metadata block number
heap_block_num_t meta_block;
// offset within block
uint16_t block_offset;
uint8_t bitmap[];
};
struct __attribute__((__packed__)) heap_meta_block_header_t
{
uint64_t magic;
uint64_t seq_num;
uint32_t invalidates_blocks;
};
// 48+checksums = (40+bitmap)+checksums bytes per on-disk "heap" entry
// for 128 KB block without checksums, it's 48 bytes - 84 entries per 4 kb metadata block
// for 128 KB block with 4k checksums, it's 176 bytes - 22 entries per 4 kb metadata block
// for 1 MB block without checksums, it's 80 bytes - 50 entries per 4 kb metadata block
// for 1 MB block with 4k checksums, it's 1104 bytes O_o - only 3 entries per 4 kb metadata block
// for 1 MB block with 32k checksums, it's 176 bytes again
struct __attribute__((__packed__)) heap_meta_entry_t
{
object_id oid;
uint64_t version;
uint64_t location; // UINT64_MAX = deleted
uint64_t reserved;
uint8_t bitmap[];
};
struct heap_meta_block_t
{
heap_block_num_t offset = 0;
uint64_t seq_num = 0;
uint32_t used_space = 0;
std::vector<uint64_t> invalidates_blocks;
};
struct heap_meta_t
{
heap_block_num_t block_count = 0;
heap_meta_block_t *blocks = NULL;
// used space => block number
std::multimap<uint32_t, heap_block_num_t> used_space_map;
heap_block_num_t cur_written_block = 0;
uint8_t *written_block_buf = NULL;
};
// 64 = 24 + 40 bytes per dirty entry in memory (obj_ver_id => dirty_entry). Plus checksums
struct __attribute__((__packed__)) dirty_entry
{
@@ -272,6 +332,8 @@ class blockstore_impl_t
struct ring_consumer_t ring_consumer;
heap_meta_t heap_meta;
std::map<pool_id_t, pool_shard_settings_t> clean_db_settings;
std::map<pool_pg_id_t, blockstore_clean_db_t> clean_db_shards;
std::map<uint64_t, int> no_inode_stats;
@@ -317,7 +379,7 @@ class blockstore_impl_t
void open_data();
void open_meta();
void open_journal();
uint8_t* get_clean_entry_bitmap(uint64_t block_loc, int offset);
uint8_t* get_clean_entry_bitmap(blockstore_clean_db_t::iterator clean_it, int offset);
blockstore_clean_db_t& clean_db_shard(object_id oid);
void reshard_clean_db(pool_id_t pool_id, uint32_t pg_count, uint32_t pg_stripe_size);
@@ -345,9 +407,9 @@ class blockstore_impl_t
uint64_t &fulfilled, uint32_t item_start, uint32_t item_end,
uint32_t item_state, uint64_t item_version, uint64_t item_location,
uint64_t journal_sector, uint8_t *csum, int *dyn_data);
bool fulfill_clean_read(blockstore_op_t *read_op, uint64_t & fulfilled,
uint8_t *clean_entry_bitmap, int *dyn_data,
uint32_t item_start, uint32_t item_end, uint64_t clean_loc, uint64_t clean_ver);
bool fulfill_clean_read_journal(blockstore_op_t *read_op, uint64_t & fulfilled,
uint8_t *clean_entry_bitmap, int *dyn_data, uint32_t item_start, uint32_t item_end, uint64_t clean_loc, uint64_t clean_ver);
bool fulfill_clean_read_meta(blockstore_op_t *read_op, uint64_t & fulfilled, blockstore_clean_db_t::iterator clean_it);
int fill_partial_checksum_blocks(std::vector<copy_buffer_t> & rv, uint64_t & fulfilled,
uint8_t *clean_entry_bitmap, int *dyn_data, bool from_journal, uint8_t *read_buf, uint64_t read_offset, uint64_t read_end);
int pad_journal_read(std::vector<copy_buffer_t> & rv, copy_buffer_t & cp,
@@ -356,7 +418,7 @@ class blockstore_impl_t
bool read_range_fulfilled(std::vector<copy_buffer_t> & rv, uint64_t & fulfilled, uint8_t *read_buf,
uint8_t *clean_entry_bitmap, uint32_t item_start, uint32_t item_end);
bool read_checksum_block(blockstore_op_t *op, int rv_pos, uint64_t &fulfilled, uint64_t clean_loc);
uint8_t* read_clean_meta_block(blockstore_op_t *read_op, uint64_t clean_loc, int rv_pos);
uint8_t* read_clean_meta_block(blockstore_op_t *op, blockstore_clean_db_t::iterator clean_it, int rv_pos);
bool verify_padded_checksums(uint8_t *clean_entry_bitmap, uint8_t *csum_buf, uint32_t offset,
iovec *iov, int n_iov, std::function<void(uint32_t, uint32_t, uint32_t)> bad_block_cb);
bool verify_journal_checksums(uint8_t *csums, uint32_t offset,

View File

@@ -54,6 +54,7 @@ int blockstore_init_meta::loop()
else if (wait_state == 4) goto resume_4;
else if (wait_state == 5) goto resume_5;
else if (wait_state == 6) goto resume_6;
else if (wait_state == 7) goto resume_7;
printf("Reading blockstore metadata\n");
if (bs->inmemory_meta)
metadata_buffer = bs->metadata_buffer;
@@ -78,6 +79,7 @@ resume_1:
if (iszero((uint64_t*)metadata_buffer, bs->dsk.meta_block_size / sizeof(uint64_t)))
{
{
memset(metadata_buffer, 0, bs->dsk.meta_block_size);
blockstore_meta_header_v2_t *hdr = (blockstore_meta_header_v2_t *)metadata_buffer;
hdr->zero = 0;
hdr->magic = BLOCKSTORE_META_MAGIC_V1;
@@ -85,12 +87,19 @@ resume_1:
hdr->meta_block_size = bs->dsk.meta_block_size;
hdr->data_block_size = bs->dsk.data_block_size;
hdr->bitmap_granularity = bs->dsk.bitmap_granularity;
if (bs->dsk.meta_format >= BLOCKSTORE_META_FORMAT_HEAP)
{
hdr->block_id_bits = sizeof(heap_block_num_t);
}
if (bs->dsk.meta_format >= BLOCKSTORE_META_FORMAT_V2)
{
hdr->data_csum_type = bs->dsk.data_csum_type;
hdr->csum_block_size = bs->dsk.csum_block_size;
hdr->header_csum = 0;
hdr->header_csum = crc32c(0, hdr, sizeof(*hdr));
hdr->header_csum = crc32c(0, hdr,
bs->dsk.meta_format == BLOCKSTORE_META_FORMAT_V2
? BLOCKSTORE_META_HEADER_V2_SIZE
: sizeof(*hdr));
}
}
if (bs->readonly)
@@ -128,7 +137,7 @@ resume_1:
);
exit(1);
}
if (hdr->version == BLOCKSTORE_META_FORMAT_V2)
if (hdr->version == BLOCKSTORE_META_FORMAT_HEAP)
{
uint32_t csum = hdr->header_csum;
hdr->header_csum = 0;
@@ -138,6 +147,23 @@ resume_1:
exit(1);
}
hdr->header_csum = csum;
bs->dsk.meta_format = BLOCKSTORE_META_FORMAT_HEAP;
if (hdr->block_id_bits != sizeof(heap_block_num_t))
{
printf("Heap metadata block ID size (%u) is not supported by this build\n", hdr->block_id_bits);
exit(1);
}
}
else if (hdr->version == BLOCKSTORE_META_FORMAT_V2)
{
uint32_t csum = hdr->header_csum;
hdr->header_csum = 0;
if (crc32c(0, hdr, BLOCKSTORE_META_HEADER_V2_SIZE) != csum)
{
printf("Metadata header is corrupt (checksum mismatch).\n");
exit(1);
}
hdr->header_csum = csum;
if (bs->dsk.meta_format != BLOCKSTORE_META_FORMAT_V2)
{
bs->dsk.meta_format = BLOCKSTORE_META_FORMAT_V2;
@@ -160,11 +186,11 @@ resume_1:
printf("Warning: Starting with metadata in the old format without checksums, as stored on disk\n");
}
}
else if (hdr->version > BLOCKSTORE_META_FORMAT_V2)
else
{
printf(
"Metadata format is too new for me (stored version is %ju, max supported %u).\n",
hdr->version, BLOCKSTORE_META_FORMAT_V2
hdr->version, BLOCKSTORE_META_FORMAT_HEAP
);
exit(1);
}
@@ -189,7 +215,12 @@ resume_1:
// Skip superblock
md_offset = bs->dsk.meta_block_size;
next_offset = md_offset;
entries_per_block = bs->dsk.meta_block_size / bs->dsk.clean_entry_size;
entries_per_block = bs->dsk.meta_block_size / bs->dsk.clean_entry_size; // FIXME only array
if (bs->dsk.meta_format == BLOCKSTORE_META_FORMAT_HEAP)
{
bs->heap_meta.blocks = new heap_meta_block_t[bs->dsk.meta_len / bs->dsk.meta_block_size];
bs->heap_meta.block_count = bs->dsk.meta_len / bs->dsk.meta_block_size;
}
// Read the rest of the metadata
resume_2:
if (next_offset < bs->dsk.meta_len && submitted == 0)
@@ -233,9 +264,10 @@ resume_2:
bool changed = false;
for (uint64_t sector = 0; sector < bufs[i].size; sector += bs->dsk.meta_block_size)
{
// handle <count> entries
if (handle_meta_block(bufs[i].buf + sector, entries_per_block,
((bufs[i].offset + sector - md_offset) / bs->dsk.meta_block_size) * entries_per_block))
auto this_changed = bs->dsk.meta_format == BLOCKSTORE_META_FORMAT_HEAP
? handle_heap_meta_block(bufs[i].buf + sector, bufs[i].offset + sector - md_offset)
: handle_array_meta_block(bufs[i].buf + sector, bufs[i].offset + sector - md_offset);
if (this_changed)
changed = true;
}
if (changed && !bs->inmemory_meta && !bs->readonly)
@@ -262,6 +294,41 @@ resume_2:
wait_state = 2;
return 1;
}
if (bs->dsk.meta_format == BLOCKSTORE_META_FORMAT_HEAP)
{
// build used_space index
for (heap_block_num_t i = 0; i < bs->heap_meta.block_count; i++)
{
bs->heap_meta.used_space_map.emplace(std::pair<uint32_t, heap_block_num_t>(bs->heap_meta.blocks[i].used_space, i));
}
}
if (heap_invalidated_block_seq.size() && !bs->readonly)
{
// zero out invalidated blocks not zeroed during the previous OSD execution
for (auto inv_seq: heap_invalidated_block_seq)
{
auto num_it = heap_block_by_seq.find(inv_seq);
if (num_it != heap_block_by_seq.end())
heap_invalidated_block_nums.push_back(num_it->second);
}
memset(metadata_buffer, 0, bs->dsk.meta_block_size);
for (i = 0; i < heap_invalidated_block_nums.size(); i++)
{
GET_SQE();
last_read_offset = heap_invalidated_block_nums[i]*bs->dsk.meta_block_size;
data->iov = { metadata_buffer, (size_t)bs->dsk.meta_block_size };
data->callback = [this](ring_data_t *data) { handle_event(data, -1); };
my_uring_prep_writev(sqe, bs->dsk.meta_fd, &data->iov, 1, bs->dsk.meta_offset + last_read_offset);
bs->ringloop->submit();
submitted++;
resume_7:
if (submitted > 0)
{
wait_state = 7;
return 1;
}
}
}
if (entries_to_zero.size() && !bs->inmemory_meta && !bs->readonly)
{
std::sort(entries_to_zero.begin(), entries_to_zero.end());
@@ -329,8 +396,9 @@ resume_6:
return 0;
}
bool blockstore_init_meta::handle_meta_block(uint8_t *buf, uint64_t entries_per_block, uint64_t done_cnt)
bool blockstore_init_meta::handle_array_meta_block(uint8_t *buf, uint64_t block_offset)
{
uint64_t done_cnt = (block_offset / bs->dsk.meta_block_size) * entries_per_block;
bool updated = false;
uint64_t max_i = entries_per_block;
if (max_i > bs->dsk.block_count-done_cnt)
@@ -429,6 +497,132 @@ bool blockstore_init_meta::handle_meta_block(uint8_t *buf, uint64_t entries_per_
return updated;
}
static int bitmap_count_ones(uint8_t *bitmap, int size)
{
int n = 0, i = 0;
for (; i <= size-sizeof(unsigned); i += sizeof(unsigned))
{
n += __builtin_popcount(*(unsigned*)(bitmap+i));
}
for (; i < size; i++)
{
n += __builtin_popcount(*(unsigned char*)(bitmap+i));
}
return n;
}
// v3 / heap / "cow" metadata block
bool blockstore_init_meta::handle_heap_meta_block(uint8_t *buf, uint64_t block_offset)
{
if ((block_offset / bs->dsk.meta_block_size) > (heap_block_num_t)-1)
{
fprintf(stderr, "Metadata area too large\n");
exit(1);
}
// Validate block CRC
uint32_t block_crc = *(uint32_t*)(buf + bs->dsk.meta_block_size - 4);
if (crc32c(0, buf, bs->dsk.meta_block_size-4) != block_crc)
{
return false;
}
// Validate header
heap_meta_block_header_t *hdr = (heap_meta_block_header_t*)buf;
if (hdr->magic != BLOCKSTORE_META_MAGIC_V1)
{
return false;
}
if (hdr->invalidates_blocks > (bs->dsk.meta_block_size-4-sizeof(heap_meta_block_header_t))/sizeof(uint64_t))
{
fprintf(stderr, "Metadata block at %jx contains too large invalidates_blocks count: %x\n", block_offset, hdr->invalidates_blocks);
exit(1);
}
if (heap_invalidated_block_seq.find(hdr->seq_num) != heap_invalidated_block_seq.end())
{
// Check if the block is invalidated and handled after the block that invalidates it
return false;
}
uint64_t hdr_size = sizeof(heap_meta_block_header_t) + hdr->invalidates_blocks*8;
heap_meta_block_t & blk = bs->heap_meta.blocks[block_offset/bs->dsk.meta_block_size];
blk.offset = block_offset;
blk.seq_num = hdr->seq_num;
blk.used_space = hdr_size + 4;
uint64_t *hdr_inv = (uint64_t*)(hdr + 1);
for (int i = 0; i < hdr->invalidates_blocks; i++)
{
blk.invalidates_blocks.push_back(hdr_inv[i]);
heap_invalidated_block_seq.insert(hdr_inv[i]);
}
heap_block_by_seq[hdr->seq_num] = block_offset;
// Process sub-blocks
uint64_t heap_entry_size = sizeof(heap_meta_entry_t) + bs->dsk.clean_dyn_size;
for (uint64_t pos = sizeof(heap_meta_block_header_t); pos < bs->dsk.meta_block_size-4; pos += heap_entry_size)
{
heap_meta_entry_t *diskentry = (heap_meta_entry_t*)(buf + pos);
if (!diskentry->oid.inode || !diskentry->version)
{
continue;
}
auto & clean_db = bs->clean_db_shard(diskentry->oid);
auto mementry = (clean_entry_heap_t*)(&clean_db[diskentry->oid]);
bool exists = mementry->version != 0;
if (exists && mementry->version >= diskentry->version)
{
if (mementry->version == diskentry->version)
{
// Voluntarily allow duplicates of in-memory entries with different
// bitmaps to support checksum updates with hole-punching
int old_count = bitmap_count_ones(mementry->bitmap, bs->dsk.clean_entry_bitmap_size);
int new_count = bitmap_count_ones(diskentry->bitmap, bs->dsk.clean_entry_bitmap_size);
if (old_count < new_count)
{
continue;
}
}
else
{
continue;
}
}
blk.used_space += heap_entry_size;
if (exists && mementry->location != UINT64_MAX)
{
// free the previous block
uint64_t old_clean_loc = mementry->location >> bs->dsk.block_order;
#ifdef BLOCKSTORE_DEBUG
printf("Free block %ju from %jx:%jx v%ju\n", 1+old_clean_loc,
diskentry->oid.inode, diskentry->oid.stripe, mementry->version);
#endif
bs->data_alloc->set(old_clean_loc, false);
bs->inode_space_stats[diskentry->oid.inode] -= bs->dsk.data_block_size;
bs->used_blocks--;
bs->heap_meta.blocks[mementry->meta_block].used_space -= heap_entry_size;
}
if (diskentry->location != UINT64_MAX)
{
bs->data_alloc->set(diskentry->location >> bs->dsk.block_order, true);
bs->inode_space_stats[diskentry->oid.inode] += bs->dsk.data_block_size;
bs->used_blocks++;
#ifdef BLOCKSTORE_DEBUG
printf("Allocate block (heap entry) %ju: %jx:%jx v%ju\n", 1 + (diskentry->location >> bs->dsk.block_order),
diskentry->oid.inode, diskentry->oid.stripe, diskentry->version);
#endif
}
mementry->version = diskentry->version;
mementry->location = diskentry->location;
mementry->meta_block = block_offset / bs->dsk.meta_block_size;
mementry->block_offset = block_offset % bs->dsk.meta_block_size;
if (exists)
{
mementry->prev_versions++;
}
// Extra data: 2 bitmaps + checksums or just 2 bitmaps if inmemory_meta is disabled
memcpy(&mementry->bitmap, &diskentry->bitmap, bs->inmemory_meta ? bs->dsk.clean_dyn_size : 2*bs->dsk.clean_entry_bitmap_size);
entries_loaded++;
}
// We have to zero out headers of invalidated blocks, but we'll do it later
return false;
}
blockstore_init_journal::blockstore_init_journal(blockstore_impl_t *bs)
{
this->bs = bs;

View File

@@ -28,7 +28,13 @@ class blockstore_init_meta
unsigned entries_per_block = 0;
int i = 0, j = 0;
std::vector<uint64_t> entries_to_zero;
bool handle_meta_block(uint8_t *buf, uint64_t count, uint64_t done_cnt);
std::map<uint64_t, heap_block_num_t> heap_block_by_seq;
std::set<uint64_t> heap_invalidated_block_seq;
std::vector<heap_block_num_t> heap_invalidated_block_nums;
bool handle_array_meta_block(uint8_t *buf, uint64_t block_offset);
bool handle_heap_meta_block(uint8_t *buf, uint64_t block_offset);
void handle_event(ring_data_t *data, int buf_num);
public:
blockstore_init_meta(blockstore_impl_t *bs);

View File

@@ -111,6 +111,10 @@ void blockstore_impl_t::parse_config(blockstore_config_t & config, bool init)
{
metadata_buf_size = 4*1024*1024;
}
if (metadata_buf_size % dsk.meta_block_size)
{
metadata_buf_size = ((metadata_buf_size+dsk.meta_block_size-1) / dsk.meta_block_size) * dsk.meta_block_size;
}
if (dsk.meta_device == dsk.data_device)
{
disable_meta_fsync = disable_data_fsync;

View File

@@ -148,10 +148,14 @@ int blockstore_impl_t::fulfill_read(blockstore_op_t *read_op,
return r;
}
uint8_t* blockstore_impl_t::get_clean_entry_bitmap(uint64_t block_loc, int offset)
uint8_t* blockstore_impl_t::get_clean_entry_bitmap(blockstore_clean_db_t::iterator clean_it, int offset)
{
if (dsk.meta_format == BLOCKSTORE_META_FORMAT_HEAP)
{
return ((uint8_t*)&clean_it->second) + sizeof(clean_entry_heap_t) + offset;
}
uint8_t *clean_entry_bitmap;
uint64_t meta_loc = block_loc >> dsk.block_order;
uint64_t meta_loc = clean_it->second.location >> dsk.block_order;
if (inmemory_meta)
{
uint64_t sector = (meta_loc / (dsk.meta_block_size / dsk.clean_entry_size)) * dsk.meta_block_size;
@@ -159,7 +163,9 @@ uint8_t* blockstore_impl_t::get_clean_entry_bitmap(uint64_t block_loc, int offse
clean_entry_bitmap = ((uint8_t*)metadata_buffer + sector + pos*dsk.clean_entry_size + sizeof(clean_disk_entry) + offset);
}
else
{
clean_entry_bitmap = (uint8_t*)(clean_bitmaps + meta_loc*2*dsk.clean_entry_bitmap_size + offset);
}
return clean_entry_bitmap;
}
@@ -433,7 +439,7 @@ int blockstore_impl_t::dequeue_read(blockstore_op_t *read_op)
if (!IS_JOURNAL(dirty.state))
{
// Read from data disk, possibly checking checksums
if (!fulfill_clean_read(read_op, fulfilled, bmp_ptr, dyn_data,
if (!fulfill_clean_read_journal(read_op, fulfilled, bmp_ptr, dyn_data,
dirty.offset, dirty.offset+dirty.len, dirty.location, dirty_it->first.version))
{
goto undo_read;
@@ -464,14 +470,13 @@ int blockstore_impl_t::dequeue_read(blockstore_op_t *read_op)
result_version = clean_it->second.version;
if (read_op->bitmap)
{
void *bmp_ptr = get_clean_entry_bitmap(clean_it->second.location, dsk.clean_entry_bitmap_size);
void *bmp_ptr = get_clean_entry_bitmap(clean_it, dsk.clean_entry_bitmap_size);
memcpy(read_op->bitmap, bmp_ptr, dsk.clean_entry_bitmap_size);
}
}
if (fulfilled < read_op->len)
{
if (!fulfill_clean_read(read_op, fulfilled, NULL, NULL, 0, dsk.data_block_size,
clean_it->second.location, clean_it->second.version))
if (!fulfill_clean_read_meta(read_op, fulfilled, clean_it))
{
goto undo_read;
}
@@ -581,40 +586,22 @@ int blockstore_impl_t::pad_journal_read(std::vector<copy_buffer_t> & rv, copy_bu
return 0;
}
bool blockstore_impl_t::fulfill_clean_read(blockstore_op_t *read_op, uint64_t & fulfilled,
bool blockstore_impl_t::fulfill_clean_read_journal(blockstore_op_t *read_op, uint64_t & fulfilled,
uint8_t *clean_entry_bitmap, int *dyn_data, uint32_t item_start, uint32_t item_end, uint64_t clean_loc, uint64_t clean_ver)
{
bool from_journal = clean_entry_bitmap != NULL;
if (!clean_entry_bitmap)
{
// NULL clean_entry_bitmap means we're reading from data, not from the journal,
// and the bitmap location is obvious
clean_entry_bitmap = get_clean_entry_bitmap(clean_loc, 0);
}
if (dsk.csum_block_size > dsk.bitmap_granularity)
{
auto & rv = PRIV(read_op)->read_vec;
int req = fill_partial_checksum_blocks(rv, fulfilled, clean_entry_bitmap, dyn_data, from_journal,
int req = fill_partial_checksum_blocks(rv, fulfilled, clean_entry_bitmap, dyn_data, true,
(uint8_t*)read_op->buf, read_op->offset, read_op->offset+read_op->len);
if (!inmemory_meta && !from_journal && req > 0)
{
// Read checksums from disk
uint8_t *csum_buf = read_clean_meta_block(read_op, clean_loc, rv.size()-req);
for (int i = req; i > 0; i--)
{
rv[rv.size()-i].csum_buf = csum_buf;
}
}
for (int i = req; i > 0; i--)
{
if (!read_checksum_block(read_op, i, fulfilled, clean_loc))
{
return false;
}
}
PRIV(read_op)->clean_block_used = req > 0;
}
else if (from_journal)
else
{
// Don't scan bitmap - journal writes don't have holes (internal bitmap)!
uint8_t *csum = !dsk.csum_block_size ? 0 : (clean_entry_bitmap + dsk.clean_entry_bitmap_size +
@@ -635,6 +622,43 @@ bool blockstore_impl_t::fulfill_clean_read(blockstore_op_t *read_op, uint64_t &
assert(fulfill_read(read_op, fulfilled, item_end, dsk.data_block_size, (BS_ST_DELETE | BS_ST_STABLE), 0, 0, 0, NULL, NULL));
}
}
// Increment reference counter if clean data is being read from the disk
if (PRIV(read_op)->clean_block_used)
{
auto & uo = used_clean_objects[clean_loc];
uo.refs++;
if (dsk.csum_block_size && flusher->is_mutated(clean_loc))
uo.was_changed = true;
PRIV(read_op)->clean_block_used = clean_loc;
}
return true;
}
bool blockstore_impl_t::fulfill_clean_read_meta(blockstore_op_t *read_op, uint64_t & fulfilled, blockstore_clean_db_t::iterator clean_it)
{
uint8_t *clean_entry_bitmap = get_clean_entry_bitmap(clean_it, 0);
uint64_t clean_loc = clean_it->second.location;
if (dsk.csum_block_size > dsk.bitmap_granularity)
{
auto & rv = PRIV(read_op)->read_vec;
int req = fill_partial_checksum_blocks(rv, fulfilled, clean_entry_bitmap, NULL, false,
(uint8_t*)read_op->buf, read_op->offset, read_op->offset+read_op->len);
if (!inmemory_meta && req > 0)
{
// Read checksums from disk
uint8_t *csum_buf = read_clean_meta_block(read_op, clean_it, rv.size()-req);
for (int i = req; i > 0; i--)
{
rv[rv.size()-i].csum_buf = csum_buf;
}
}
for (int i = req; i > 0; i--)
{
if (!read_checksum_block(read_op, i, fulfilled, clean_loc))
return false;
}
PRIV(read_op)->clean_block_used = req > 0;
}
else
{
bool csum_done = !dsk.csum_block_size || inmemory_meta;
@@ -662,13 +686,13 @@ bool blockstore_impl_t::fulfill_clean_read(blockstore_op_t *read_op, uint64_t &
if (!csum_done)
{
// Read checksums from disk
csum_buf = read_clean_meta_block(read_op, clean_loc, PRIV(read_op)->read_vec.size());
csum_buf = read_clean_meta_block(read_op, clean_it, PRIV(read_op)->read_vec.size());
csum_done = true;
}
uint8_t *csum = !dsk.csum_block_size ? 0 : (csum_buf + 2*dsk.clean_entry_bitmap_size + bmp_start*(dsk.data_csum_type & 0xFF));
if (!fulfill_read(read_op, fulfilled, bmp_start * dsk.bitmap_granularity,
bmp_end * dsk.bitmap_granularity, (BS_ST_BIG_WRITE | BS_ST_STABLE), 0,
clean_loc + bmp_start * dsk.bitmap_granularity, 0, csum, dyn_data))
clean_loc + bmp_start * dsk.bitmap_granularity, 0, csum, NULL))
{
return false;
}
@@ -688,11 +712,22 @@ bool blockstore_impl_t::fulfill_clean_read(blockstore_op_t *read_op, uint64_t &
return true;
}
uint8_t* blockstore_impl_t::read_clean_meta_block(blockstore_op_t *op, uint64_t clean_loc, int rv_pos)
uint8_t* blockstore_impl_t::read_clean_meta_block(blockstore_op_t *op, blockstore_clean_db_t::iterator clean_it, int rv_pos)
{
uint64_t sector, pos;
auto & rv = PRIV(op)->read_vec;
auto sector = ((clean_loc >> dsk.block_order) / (dsk.meta_block_size / dsk.clean_entry_size)) * dsk.meta_block_size;
auto pos = ((clean_loc >> dsk.block_order) % (dsk.meta_block_size / dsk.clean_entry_size)) * dsk.clean_entry_size;
if (dsk.meta_format == BLOCKSTORE_META_FORMAT_HEAP)
{
auto clean_heap_entry = (clean_entry_heap_t*)(&clean_it->second);
sector = clean_heap_entry->meta_block * dsk.meta_block_size;
pos = clean_heap_entry->block_offset;
}
else
{
auto clean_loc = clean_it->second.location;
sector = ((clean_loc >> dsk.block_order) / (dsk.meta_block_size / dsk.clean_entry_size)) * dsk.meta_block_size;
pos = ((clean_loc >> dsk.block_order) % (dsk.meta_block_size / dsk.clean_entry_size)) * dsk.clean_entry_size;
}
uint8_t *buf = (uint8_t*)memalign_or_die(MEM_ALIGNMENT, dsk.meta_block_size);
rv.insert(rv.begin()+rv_pos, (copy_buffer_t){
.copy_flags = COPY_BUF_META_BLOCK|COPY_BUF_CSUM_FILL,
@@ -807,11 +842,6 @@ bool blockstore_impl_t::verify_clean_padded_checksums(blockstore_op_t *op, uint6
if (from_journal)
return verify_padded_checksums(dyn_data, dyn_data + dsk.clean_entry_bitmap_size, offset, iov, n_iov, bad_block_cb);
clean_loc = (clean_loc >> dsk.block_order) << dsk.block_order;
if (!dyn_data)
{
assert(inmemory_meta);
dyn_data = get_clean_entry_bitmap(clean_loc, 0);
}
return verify_padded_checksums(dyn_data, dyn_data + 2*dsk.clean_entry_bitmap_size, offset, iov, n_iov, bad_block_cb);
}
@@ -869,8 +899,18 @@ void blockstore_impl_t::handle_read_event(ring_data_t *data, blockstore_op_t *op
auto & uo = used_clean_objects.at((rv[i].disk_offset >> dsk.block_order) << dsk.block_order);
if (!uo.was_changed)
{
bool from_journal = (rv[i].copy_flags & COPY_BUF_JOURNALED_BIG);
auto csum_buf = rv[i].csum_buf;
if (!from_journal && !csum_buf)
{
assert(inmemory_meta);
auto & clean_db = clean_db_shard(op->oid);
auto clean_it = clean_db.find(op->oid);
assert(clean_it != clean_db.end());
csum_buf = get_clean_entry_bitmap(clean_it, 0);
}
verify_clean_padded_checksums(
op, rv[i].disk_offset, rv[i].csum_buf, (rv[i].copy_flags & COPY_BUF_JOURNALED_BIG), iov, n_iov,
op, rv[i].disk_offset, csum_buf, from_journal, iov, n_iov,
[&](uint32_t bad_block, uint32_t calc_csum, uint32_t stored_csum)
{
ok = false;
@@ -1019,7 +1059,7 @@ int blockstore_impl_t::read_bitmap(object_id oid, uint64_t target_version, void
*result_version = clean_it->second.version;
if (bitmap)
{
void *bmp_ptr = get_clean_entry_bitmap(clean_it->second.location, dsk.clean_entry_bitmap_size);
void *bmp_ptr = get_clean_entry_bitmap(clean_it, dsk.clean_entry_bitmap_size);
memcpy(bitmap, bmp_ptr, dsk.clean_entry_bitmap_size);
}
return 0;

View File

@@ -57,7 +57,7 @@ bool blockstore_impl_t::enqueue_write(blockstore_op_t *op)
version = clean_it->second.version + 1;
if (!is_del)
{
void *bmp_ptr = get_clean_entry_bitmap(clean_it->second.location, dsk.clean_entry_bitmap_size);
void *bmp_ptr = get_clean_entry_bitmap(clean_it, dsk.clean_entry_bitmap_size);
memcpy(dyn_ptr, bmp_ptr, dsk.clean_entry_bitmap_size);
}
}
@@ -341,7 +341,7 @@ int blockstore_impl_t::dequeue_write(blockstore_op_t *op)
cancel_all_writes(op, dirty_it, -ENOSPC);
return 2;
}
if (inmemory_meta)
if (inmemory_meta && dsk.meta_format != BLOCKSTORE_META_FORMAT_HEAP)
{
// Check once more that metadata entry is zeroed (the reverse means a bug or corruption)
uint64_t sector = (loc / (dsk.meta_block_size / dsk.clean_entry_size)) * dsk.meta_block_size;

View File

@@ -283,6 +283,7 @@ static json11::Json::object parse_args(int narg, const char *args[])
!strcmp(opt, "readonly") || !strcmp(opt, "readwrite") ||
!strcmp(opt, "force") || !strcmp(opt, "reverse") ||
!strcmp(opt, "allow-data-loss") || !strcmp(opt, "allow_data_loss") ||
!strcmp(opt, "allow-up") || !strcmp(opt, "allow_up") ||
!strcmp(opt, "down-ok") || !strcmp(opt, "down_ok") ||
!strcmp(opt, "dry-run") || !strcmp(opt, "dry_run") ||
!strcmp(opt, "help") || !strcmp(opt, "all") ||

View File

@@ -15,7 +15,7 @@ struct rm_osd_t
{
cli_tool_t *parent;
bool dry_run, force_warning, force_dataloss;
bool dry_run, force_warning, force_dataloss, allow_up;
uint64_t etcd_tx_retry_ms = 500;
uint64_t etcd_tx_retries = 10000;
std::vector<uint64_t> osd_ids;
@@ -168,7 +168,7 @@ struct rm_osd_t
: strtoupper(e["effect"].string_value())+" PGs"))
)+" after deleting OSD(s).\n";
}
if (still_up.size())
if (still_up.size() && !allow_up)
error += (still_up.size() == 1 ? "OSD " : "OSDs ") + implode(", ", still_up) +
(still_up.size() == 1 ? "is" : "are") + " still up. Use `vitastor-disk purge` to delete them.\n";
if (is_dataloss && !force_dataloss && !dry_run)
@@ -476,6 +476,7 @@ std::function<bool(cli_result_t &)> cli_tool_t::start_rm_osd(json11::Json cfg)
auto rm_osd = new rm_osd_t();
rm_osd->parent = this;
rm_osd->dry_run = cfg["dry_run"].bool_value();
rm_osd->allow_up = cfg["allow_up"].bool_value();
rm_osd->force_dataloss = cfg["allow_data_loss"].bool_value();
rm_osd->force_warning = rm_osd->force_dataloss || cfg["force"].bool_value();
if (!cfg["etcd_tx_retries"].is_null())

View File

@@ -435,7 +435,7 @@ int disk_tool_t::purge_devices(const std::vector<std::string> & devices)
printf("%s\n", json11::Json(result).dump().c_str());
return 0;
}
std::vector<std::string> rm_osd_cli = { "vitastor-cli", "rm-osd" };
std::vector<std::string> rm_osd_cli = { "vitastor-cli", "rm-osd", "--allow-up" };
for (auto osd_num: osd_numbers)
{
rm_osd_cli.push_back(std::to_string(osd_num));