Compare commits

...

2 Commits

Author SHA1 Message Date
Vitaliy Filippov 26ee963b3e WIP "Heap" metadata storage scheme 2025-01-12 02:28:29 +03:00
Vitaliy Filippov 13ed3b63e4 Pass clean_bitmap explicitly 2025-01-12 02:23:37 +03:00
11 changed files with 395 additions and 72 deletions

@ -1 +1 @@
Subproject commit 8de8b467acbca50cfd8835c20e0e379110f3b32b
Subproject commit a21350e484cefa5728f23c227323b4b0822e738f

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

@ -120,9 +120,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
@ -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
);
}

View File

@ -41,6 +41,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()
@ -424,13 +426,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
{
@ -270,6 +330,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;
@ -315,7 +377,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);
@ -343,9 +405,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,
@ -354,7 +416,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;
bs->dsk.meta_format = BLOCKSTORE_META_FORMAT_V2;
}
else if (hdr->version == BLOCKSTORE_META_FORMAT_V1)
@ -152,11 +178,11 @@ resume_1:
bs->dsk.meta_format = BLOCKSTORE_META_FORMAT_V1;
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);
}
@ -181,7 +207,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)
@ -225,9 +256,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)
@ -254,6 +286,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)
{
// we have to zero out additional entries
@ -320,8 +387,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)
@ -411,6 +479,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

@ -110,6 +110,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;