Implement bitmap-granular (4k) metadata & data checksums

cached-reads
Vitaliy Filippov 2022-09-19 01:12:33 +03:00
parent a0aac7eb2a
commit cc1f03971d
26 changed files with 738 additions and 174 deletions

View File

@ -77,6 +77,7 @@ Output:
-EINVAL = invalid input parameters
-ENOENT = requested object/version does not exist for reads
-ENOSPC = no space left in the store for writes
-EDOM = checksum error.
- version = the version actually read or written
## BS_OP_DELETE

View File

@ -40,10 +40,23 @@ void blockstore_disk_t::parse_config(std::map<std::string, std::string> & config
data_block_size = parse_size(config["block_size"]);
journal_device = config["journal_device"];
journal_offset = parse_size(config["journal_offset"]);
disk_alignment = strtoull(config["disk_alignment"].c_str(), NULL, 10);
journal_block_size = strtoull(config["journal_block_size"].c_str(), NULL, 10);
meta_block_size = strtoull(config["meta_block_size"].c_str(), NULL, 10);
bitmap_granularity = strtoull(config["bitmap_granularity"].c_str(), NULL, 10);
disk_alignment = parse_size(config["disk_alignment"]);
journal_block_size = parse_size(config["journal_block_size"]);
meta_block_size = parse_size(config["meta_block_size"]);
bitmap_granularity = parse_size(config["bitmap_granularity"]);
if (config["data_csum_type"] == "crc32c")
{
data_csum_type = BLOCKSTORE_CSUM_CRC32C;
}
else if (config["data_csum_type"] == "" || config["data_csum_type"] == "none")
{
data_csum_type = BLOCKSTORE_CSUM_NONE;
}
else
{
throw std::runtime_error("data_csum_type="+config["data_csum_type"]+" is unsupported, only \"crc32c\" and \"none\" are supported");
}
csum_block_size = parse_size(config["csum_block_size"]);
// Validate
if (!data_block_size)
{
@ -91,7 +104,28 @@ void blockstore_disk_t::parse_config(std::map<std::string, std::string> & config
}
if (data_block_size % bitmap_granularity)
{
throw std::runtime_error("Block size must be a multiple of sparse write tracking granularity");
throw std::runtime_error("Data block size must be a multiple of sparse write tracking granularity");
}
if (!data_csum_type)
{
csum_block_size = 0;
}
else if (!csum_block_size)
{
csum_block_size = bitmap_granularity;
}
if (csum_block_size && (csum_block_size % bitmap_granularity))
{
throw std::runtime_error("Checksum block size must be a multiple of sparse write tracking granularity");
}
if (csum_block_size && (data_block_size % csum_block_size))
{
throw std::runtime_error("Checksum block size must be a divisor of data block size");
}
if (csum_block_size && csum_block_size != bitmap_granularity)
{
// FIXME: Support other checksum block sizes (with read-modify-write)
throw std::runtime_error("Checksum block sizes other than bitmap_granularity are not supported yet");
}
if (meta_device == "")
{
@ -110,7 +144,8 @@ void blockstore_disk_t::parse_config(std::map<std::string, std::string> & config
throw std::runtime_error("journal_offset must be a multiple of journal_block_size = "+std::to_string(journal_block_size));
}
clean_entry_bitmap_size = data_block_size / bitmap_granularity / 8;
clean_entry_size = sizeof(clean_disk_entry) + 2*clean_entry_bitmap_size;
clean_dyn_size = clean_entry_bitmap_size + dirty_dyn_size(data_block_size);
clean_entry_size = sizeof(clean_disk_entry) + clean_dyn_size + 4 /*entry_csum*/;
}
void blockstore_disk_t::calc_lengths(bool skip_meta_check)

View File

@ -8,6 +8,10 @@
#include <string>
#include <map>
#define BLOCKSTORE_CSUM_NONE 0
// Lower byte of checksum type is its length
#define BLOCKSTORE_CSUM_CRC32C 0x104
struct blockstore_disk_t
{
std::string data_device, meta_device, journal_device;
@ -21,6 +25,10 @@ struct blockstore_disk_t
uint64_t meta_block_size = 4096;
// Sparse write tracking granularity. 4 KB is a good choice. Must be a multiple of disk_alignment
uint64_t bitmap_granularity = 4096;
// Data checksum type, BLOCKSTORE_CSUM_NONE or BLOCKSTORE_CSUM_CRC32C
uint32_t data_csum_type = BLOCKSTORE_CSUM_NONE;
// Checksum block size, must be a multiple of bitmap_granularity
uint32_t csum_block_size = 4096;
// By default, Blockstore locks all opened devices exclusively. This option can be used to disable locking
bool disable_flock = false;
@ -31,7 +39,7 @@ struct blockstore_disk_t
uint32_t block_order;
uint64_t block_count;
uint32_t clean_entry_bitmap_size = 0, clean_entry_size = 0;
uint32_t clean_entry_bitmap_size = 0, clean_entry_size = 0, clean_dyn_size = 0;
void parse_config(std::map<std::string, std::string> & config);
void open_data();
@ -39,4 +47,9 @@ struct blockstore_disk_t
void open_journal();
void calc_lengths(bool skip_meta_check = false);
void close_all();
inline uint64_t dirty_dyn_size(uint64_t len)
{
return clean_entry_bitmap_size + (csum_block_size ? len/csum_block_size * (data_csum_type & 0xFF) : 0);
}
};

View File

@ -479,23 +479,43 @@ resume_1:
bs->ringloop->wakeup();
}
// Reads completed, submit writes and set bitmap bits
if (bs->dsk.clean_entry_bitmap_size)
if (bs->dsk.clean_entry_bitmap_size || bs->dsk.csum_block_size)
{
new_clean_bitmap = (bs->inmemory_meta
? (uint8_t*)meta_new.buf + meta_new.pos*bs->dsk.clean_entry_size + sizeof(clean_disk_entry)
: (uint8_t*)bs->clean_bitmap + (clean_loc >> bs->dsk.block_order)*(2*bs->dsk.clean_entry_bitmap_size));
if (clean_init_bitmap)
: (uint8_t*)bs->clean_dyn_data + (clean_loc >> bs->dsk.block_order)*bs->dsk.clean_dyn_size);
if (bs->dsk.clean_entry_bitmap_size && clean_init_bitmap)
{
// Initial internal bitmap bits from the big write
memset(new_clean_bitmap, 0, bs->dsk.clean_entry_bitmap_size);
bitmap_set(new_clean_bitmap, clean_bitmap_offset, clean_bitmap_len, bs->dsk.bitmap_granularity);
}
}
// Copy initial (big_write) data checksums
if (bs->dsk.csum_block_size && clean_init_bitmap)
{
uint8_t *new_clean_data_csum = (uint8_t*)new_clean_bitmap + 2*bs->dsk.clean_entry_bitmap_size +
clean_bitmap_offset / bs->dsk.csum_block_size * (bs->dsk.data_csum_type & 0xFF);
uint64_t dyn_size = bs->dsk.dirty_dyn_size(clean_bitmap_len);
uint8_t *dyn_ptr = dyn_size > sizeof(void*) ? clean_init_data_csum : (uint8_t*)&clean_init_data_csum;
uint64_t csum_len = clean_bitmap_len / bs->dsk.csum_block_size * (bs->dsk.data_csum_type & 0xFF);
memcpy(new_clean_data_csum, dyn_ptr + bs->dsk.clean_entry_bitmap_size, csum_len);
}
for (it = v.begin(); it != v.end(); it++)
{
// Set internal bitmap bits
if (new_clean_bitmap)
{
bitmap_set(new_clean_bitmap, it->offset, it->len, bs->dsk.bitmap_granularity);
}
// Copy small_write data checksums
if (bs->dsk.csum_block_size)
{
uint8_t *new_clean_data_csum = (uint8_t*)new_clean_bitmap + 2*bs->dsk.clean_entry_bitmap_size +
it->offset / bs->dsk.csum_block_size * (bs->dsk.data_csum_type & 0xFF);
uint64_t csum_len = it->len / bs->dsk.csum_block_size * (bs->dsk.data_csum_type & 0xFF);
memcpy(new_clean_data_csum, it->csum_buf, csum_len);
}
await_sqe(4);
data->iov = (struct iovec){ it->buf, (size_t)it->len };
data->callback = simple_callback_w;
@ -590,9 +610,14 @@ resume_1:
// copy latest external bitmap/attributes
if (bs->dsk.clean_entry_bitmap_size)
{
void *bmp_ptr = bs->dsk.clean_entry_bitmap_size > sizeof(void*) ? dirty_end->second.bitmap : &dirty_end->second.bitmap;
memcpy((uint8_t*)(new_entry+1) + bs->dsk.clean_entry_bitmap_size, bmp_ptr, bs->dsk.clean_entry_bitmap_size);
uint64_t dyn_size = bs->dsk.dirty_dyn_size(dirty_end->second.len);
void *dyn_ptr = dyn_size > sizeof(void*) ? dirty_end->second.dyn_data : &dirty_end->second.dyn_data;
// copy bitmap
memcpy((uint8_t*)(new_entry+1) + bs->dsk.clean_entry_bitmap_size, dyn_ptr, bs->dsk.clean_entry_bitmap_size);
}
// calculate metadata entry checksum
uint32_t *new_entry_csum = (uint32_t*)((uint8_t*)new_entry + bs->dsk.clean_entry_size - 4);
*new_entry_csum = crc32c(0, new_entry, bs->dsk.clean_entry_size - 4);
}
await_sqe(6);
data->iov = (struct iovec){ meta_new.buf, bs->dsk.meta_block_size };
@ -688,7 +713,9 @@ resume_1:
.size = sizeof(journal_entry_start),
.reserved = 0,
.journal_start = new_trim_pos,
.version = JOURNAL_VERSION,
.version = JOURNAL_VERSION_V2,
.data_csum_type = bs->dsk.data_csum_type,
.csum_block_size = bs->dsk.csum_block_size,
};
((journal_entry_start*)flusher->journal_superblock)->crc32 = je_crc32((journal_entry*)flusher->journal_superblock);
data->iov = (struct iovec){ flusher->journal_superblock, bs->dsk.journal_block_size };
@ -791,6 +818,16 @@ bool journal_flusher_co::scan_dirty(int wait_base)
submit_len = it == v.end() || it->offset >= end_offset ? end_offset-offset : it->offset-offset;
it = v.insert(it, (copy_buffer_t){ .offset = offset, .len = submit_len });
copy_count++;
if (bs->dsk.csum_block_size)
{
uint64_t dyn_size = bs->dsk.dirty_dyn_size(dirty_it->second.len);
// FIXME Remove this > sizeof(void*) inline perversion from everywhere.
// I think it doesn't matter but I couldn't stop myself from implementing it :)
uint8_t *dyn_from = (uint8_t*)(dyn_size > sizeof(void*) ? dirty_it->second.dyn_data : &dirty_it->second.dyn_data) +
bs->dsk.clean_entry_bitmap_size +
(offset - dirty_it->second.offset) / bs->dsk.csum_block_size * (bs->dsk.data_csum_type & 0xFF);
it->csum_buf = dyn_from;
}
if (bs->journal.inmemory)
{
// Take it from memory, don't copy it
@ -823,6 +860,7 @@ bool journal_flusher_co::scan_dirty(int wait_base)
clean_init_bitmap = true;
clean_bitmap_offset = dirty_it->second.offset;
clean_bitmap_len = dirty_it->second.len;
clean_init_data_csum = (uint8_t*)dirty_it->second.dyn_data;
skip_copy = true;
}
else if (IS_DELETE(dirty_it->second.state) && !skip_copy)

View File

@ -5,6 +5,7 @@ struct copy_buffer_t
{
uint64_t offset, len;
void *buf;
uint8_t *csum_buf;
};
struct meta_sector_t
@ -56,6 +57,7 @@ class journal_flusher_co
flusher_meta_write_t meta_old, meta_new;
bool clean_init_bitmap;
uint64_t clean_bitmap_offset, clean_bitmap_len;
uint8_t *clean_init_data_csum;
void *new_clean_bitmap;
uint64_t new_trim_pos;

View File

@ -38,8 +38,8 @@ blockstore_impl_t::~blockstore_impl_t()
dsk.close_all();
if (metadata_buffer)
free(metadata_buffer);
if (clean_bitmap)
free(clean_bitmap);
if (clean_dyn_data)
free(clean_dyn_data);
}
bool blockstore_impl_t::is_started()

View File

@ -94,10 +94,9 @@
// "VITAstor"
#define BLOCKSTORE_META_MAGIC_V1 0x726F747341544956l
#define BLOCKSTORE_META_VERSION_V1 1
#define BLOCKSTORE_META_VERSION_V2 2
// metadata header (superblock)
// FIXME: After adding the OSD superblock, add a key to metadata
// and journal headers to check if they belong to the same OSD
struct __attribute__((__packed__)) blockstore_meta_header_v1_t
{
uint64_t zero;
@ -108,14 +107,29 @@ struct __attribute__((__packed__)) blockstore_meta_header_v1_t
uint32_t bitmap_granularity;
};
struct __attribute__((__packed__)) blockstore_meta_header_v2_t
{
uint64_t zero;
uint64_t magic;
uint64_t version;
uint32_t meta_block_size;
uint32_t data_block_size;
uint32_t bitmap_granularity;
uint32_t data_csum_type;
uint32_t csum_block_size;
uint32_t header_csum;
};
// 32 bytes = 24 bytes + block bitmap (4 bytes by default) + external attributes (also bitmap, 4 bytes by default)
// per "clean" entry on disk with fixed metadata tables
// FIXME: maybe add crc32's to metadata
struct __attribute__((__packed__)) clean_disk_entry
{
object_id oid;
uint64_t version;
uint8_t bitmap[];
// Two more fields come after bitmap in metadata version 2:
// uint32_t data_csum[];
// uint32_t entry_csum;
};
// 32 = 16 + 16 bytes per "clean" entry in memory (object_id => clean_entry)
@ -125,7 +139,7 @@ struct __attribute__((__packed__)) clean_entry
uint64_t location;
};
// 64 = 24 + 40 bytes per dirty entry in memory (obj_ver_id => dirty_entry)
// 64 = 24 + 40 bytes per dirty entry in memory (obj_ver_id => dirty_entry). Plus checksums
struct __attribute__((__packed__)) dirty_entry
{
uint32_t state;
@ -134,7 +148,7 @@ struct __attribute__((__packed__)) dirty_entry
uint32_t offset; // data offset within object (stripe)
uint32_t len; // data length
uint64_t journal_sector; // journal sector used for this entry
void* bitmap; // either external bitmap itself when it fits, or a pointer to it when it doesn't
void* dyn_data; // dynamic data: external bitmap and data block checksums. may be a pointer to the in-memory journal
};
// - Sync must be submitted after previous writes/deletes (not before!)
@ -167,6 +181,9 @@ struct fulfill_read_t
{
uint64_t offset, len;
uint64_t journal_sector; // sector+1 if used and !journal.inmemory, otherwise 0
uint32_t item_state;
uint64_t disk_offset;
void *csum;
};
#define PRIV(op) ((blockstore_op_private_t*)(op)->private_data)
@ -253,7 +270,7 @@ class blockstore_impl_t
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;
uint8_t *clean_bitmap = NULL;
uint8_t *clean_dyn_data = NULL;
blockstore_dirty_db_t dirty_db;
std::vector<blockstore_op_t*> submit_queue;
std::vector<obj_ver_id> unsynced_big_writes, unsynced_small_writes;
@ -311,7 +328,7 @@ class blockstore_impl_t
// Read
int dequeue_read(blockstore_op_t *read_op);
int fulfill_read(blockstore_op_t *read_op, 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);
uint32_t item_state, uint64_t item_version, uint64_t item_location, uint64_t journal_sector, uint8_t *csum);
int fulfill_read_push(blockstore_op_t *op, void *buf, uint64_t offset, uint64_t len,
uint32_t item_state, uint64_t item_version);
void handle_read_event(ring_data_t *data, blockstore_op_t *op);
@ -342,6 +359,7 @@ class blockstore_impl_t
int continue_rollback(blockstore_op_t *op);
void mark_rolled_back(const obj_ver_id & ov);
void erase_dirty(blockstore_dirty_db_t::iterator dirty_start, blockstore_dirty_db_t::iterator dirty_end, uint64_t clean_loc);
void free_dirty_dyn_data(dirty_entry & e);
// List
void process_list(blockstore_op_t *op);

View File

@ -77,13 +77,17 @@ resume_1:
if (iszero((uint64_t*)metadata_buffer, bs->dsk.meta_block_size / sizeof(uint64_t)))
{
{
blockstore_meta_header_v1_t *hdr = (blockstore_meta_header_v1_t *)metadata_buffer;
blockstore_meta_header_v2_t *hdr = (blockstore_meta_header_v2_t *)metadata_buffer;
hdr->zero = 0;
hdr->magic = BLOCKSTORE_META_MAGIC_V1;
hdr->version = BLOCKSTORE_META_VERSION_V1;
hdr->version = BLOCKSTORE_META_VERSION_V2;
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;
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));
}
if (bs->readonly)
{
@ -109,21 +113,46 @@ resume_1:
}
else
{
blockstore_meta_header_v1_t *hdr = (blockstore_meta_header_v1_t *)metadata_buffer;
if (hdr->zero != 0 ||
hdr->magic != BLOCKSTORE_META_MAGIC_V1 ||
hdr->version != BLOCKSTORE_META_VERSION_V1)
blockstore_meta_header_v2_t *hdr = (blockstore_meta_header_v2_t *)metadata_buffer;
if (hdr->zero != 0 || hdr->magic != BLOCKSTORE_META_MAGIC_V1 || hdr->version < BLOCKSTORE_META_VERSION_V1)
{
printf(
"Metadata is corrupt or old version.\n"
" If this is a new OSD please zero out the metadata area before starting it.\n"
" If you need to upgrade from 0.5.x please request it via the issue tracker.\n"
"Metadata is corrupt or too old (pre-0.6.x).\n"
" If this is a new OSD, please zero out the metadata area before starting it.\n"
" If you need to upgrade from 0.5.x, convert metadata with vitastor-disk.\n"
);
exit(1);
}
if (hdr->version == BLOCKSTORE_META_VERSION_V2)
{
uint32_t csum = hdr->header_csum;
hdr->header_csum = 0;
if (crc32c(0, hdr, sizeof(*hdr)) != csum)
{
printf("Metadata header is corrupt (CRC mismatch).\n");
exit(1);
}
hdr->header_csum = csum;
}
else if (hdr->version == BLOCKSTORE_META_VERSION_V1)
{
hdr->data_csum_type = 0;
hdr->csum_block_size = 0;
hdr->header_csum = 0;
}
else if (hdr->version > BLOCKSTORE_META_VERSION_V2)
{
printf(
"Metadata format is too new for me (stored version is %lu, max supported %u).\n",
hdr->version, BLOCKSTORE_META_VERSION_V2
);
exit(1);
}
if (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)
hdr->bitmap_granularity != bs->dsk.bitmap_granularity ||
hdr->data_csum_type != bs->dsk.data_csum_type ||
hdr->csum_block_size != bs->dsk.csum_block_size)
{
printf(
"Configuration stored in metadata superblock"
@ -279,12 +308,19 @@ bool blockstore_init_meta::handle_meta_block(uint8_t *buf, uint64_t entries_per_
for (uint64_t i = 0; i < max_i; i++)
{
clean_disk_entry *entry = (clean_disk_entry*)(buf + i*bs->dsk.clean_entry_size);
if (!bs->inmemory_meta && bs->dsk.clean_entry_bitmap_size)
{
memcpy(bs->clean_bitmap + (done_cnt+i)*2*bs->dsk.clean_entry_bitmap_size, &entry->bitmap, 2*bs->dsk.clean_entry_bitmap_size);
}
if (entry->oid.inode > 0)
{
// Check entry crc32
uint32_t *entry_csum = (uint32_t*)((uint8_t*)entry + bs->dsk.clean_entry_size - 4);
if (*entry_csum != crc32c(0, entry, bs->dsk.clean_entry_size - 4))
{
printf("Metadata entry %lu is corrupt (checksum mismatch), skipping\n", done_cnt+i);
continue;
}
if (!bs->inmemory_meta && bs->dsk.clean_entry_bitmap_size)
{
memcpy(bs->clean_dyn_data + (done_cnt+i)*bs->dsk.clean_dyn_size, &entry->bitmap, bs->dsk.clean_dyn_size);
}
auto & clean_db = bs->clean_db_shard(entry->oid);
auto clean_it = clean_db.find(entry->oid);
if (clean_it == clean_db.end() || clean_it->second.version < entry->version)
@ -440,7 +476,9 @@ resume_1:
.size = sizeof(journal_entry_start),
.reserved = 0,
.journal_start = bs->journal.block_size,
.version = JOURNAL_VERSION,
.version = JOURNAL_VERSION_V2,
.data_csum_type = bs->dsk.data_csum_type,
.csum_block_size = bs->dsk.csum_block_size,
};
((journal_entry_start*)submitted_buf)->crc32 = je_crc32((journal_entry*)submitted_buf);
if (bs->readonly)
@ -492,18 +530,19 @@ resume_1:
if (je_start->magic != JOURNAL_MAGIC ||
je_start->type != JE_START ||
je_crc32((journal_entry*)je_start) != je_start->crc32 ||
je_start->size != sizeof(journal_entry_start) && je_start->size != JE_START_LEGACY_SIZE)
je_start->size != JE_START_V0_SIZE && je_start->size != JE_START_V1_SIZE && je_start->size != JE_START_V2_SIZE)
{
// Entry is corrupt
fprintf(stderr, "First entry of the journal is corrupt\n");
fprintf(stderr, "First entry of the journal is corrupt or unsupported\n");
exit(1);
}
if (je_start->size == JE_START_LEGACY_SIZE || je_start->version != JOURNAL_VERSION)
if (je_start->size == JE_START_V0_SIZE || je_start->version != JOURNAL_VERSION_V2)
{
// FIXME: Support v1 too
fprintf(
stderr, "The code only supports journal version %d, but it is %lu on disk."
" Please use the previous version to flush the journal before upgrading OSD\n",
JOURNAL_VERSION, je_start->size == JE_START_LEGACY_SIZE ? 0 : je_start->version
" Please use vitastor-disk to rewrite the journal\n",
JOURNAL_VERSION_V2, je_start->size == JE_START_V0_SIZE ? 0 : je_start->version
);
exit(1);
}
@ -705,11 +744,14 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
snprintf(err, 1024, "BUG: calculated journal data offset (%08lx) != stored journal data offset (%08lx)", location, je->small_write.data_offset);
throw std::runtime_error(err);
}
uint32_t data_crc32 = 0;
small_write_data.clear();
if (location >= done_pos && location+je->small_write.len <= done_pos+len)
{
// data is within this buffer
data_crc32 = crc32c(0, (uint8_t*)buf + location - done_pos, je->small_write.len);
small_write_data.push_back((iovec){
.iov_base = (uint8_t*)buf + location - done_pos,
.iov_len = je->small_write.len,
});
}
else
{
@ -724,7 +766,10 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
? location+je->small_write.len : done[i].pos+done[i].len);
uint64_t part_begin = (location < done[i].pos ? done[i].pos : location);
covered += part_end - part_begin;
data_crc32 = crc32c(data_crc32, (uint8_t*)done[i].buf + part_begin - done[i].pos, part_end - part_begin);
small_write_data.push_back((iovec){
.iov_base = (uint8_t*)done[i].buf + part_begin - done[i].pos,
.iov_len = part_end - part_begin,
});
}
}
if (covered < je->small_write.len)
@ -734,12 +779,60 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
return 2;
}
}
if (data_crc32 != je->small_write.crc32_data)
bool data_csum_valid = true;
if (!bs->dsk.csum_block_size)
{
uint32_t data_crc32 = 0;
for (auto & sd: small_write_data)
{
data_crc32 = crc32c(data_crc32, sd.iov_base, sd.iov_len);
}
data_csum_valid = data_crc32 == je->small_write.crc32_data;
if (!data_csum_valid)
{
printf("Journal entry data is corrupt (data crc32 %x != %x)\n", data_crc32, je->small_write.crc32_data);
}
}
else
{
uint32_t *block_csums = (uint32_t*)((uint8_t*)je + sizeof(journal_entry_small_write) + bs->dsk.clean_entry_bitmap_size);
uint32_t block_crc32 = 0;
int sd_num = 0;
size_t sd_pos = 0;
for (uint64_t pos = 0; pos < je->small_write.len; pos += bs->dsk.csum_block_size, block_csums++)
{
size_t block_left = bs->dsk.csum_block_size;
block_crc32 = 0;
while (block_left > 0)
{
if (small_write_data[sd_num].iov_len >= sd_pos+block_left)
{
block_crc32 = crc32c(block_crc32, small_write_data[sd_num].iov_base+sd_pos, block_left);
sd_pos += block_left;
break;
}
else
{
block_crc32 = crc32c(block_crc32, small_write_data[sd_num].iov_base+sd_pos, small_write_data[sd_num].iov_len-sd_pos);
block_left -= (small_write_data[sd_num].iov_len-sd_pos);
sd_pos = 0;
sd_num++;
}
}
if (block_crc32 != *block_csums)
{
printf("Journal entry data is corrupt (block %lu crc32 %x != %x)\n",
pos / bs->dsk.csum_block_size, block_crc32, *block_csums);
data_csum_valid = false;
break;
}
}
}
if (!data_csum_valid)
{
// journal entry is corrupt, stop here
// interesting thing is that we must clear the corrupt entry if we're not readonly,
// because we don't write next entries in the same journal block
printf("Journal entry data is corrupt (data crc32 %x != %x)\n", data_crc32, je->small_write.crc32_data);
memset((uint8_t*)buf + proc_pos - done_pos + pos, 0, bs->journal.block_size - pos);
bs->journal.next_free = prev_free;
init_write_buf = (uint8_t*)buf + proc_pos - done_pos;
@ -755,11 +848,19 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
.oid = je->small_write.oid,
.version = je->small_write.version,
};
void *bmp = NULL;
void *bmp_from = (uint8_t*)je + sizeof(journal_entry_small_write);
if (bs->dsk.clean_entry_bitmap_size <= sizeof(void*))
uint64_t dyn_size = bs->dsk.dirty_dyn_size(je->small_write.len);
void *dyn = NULL;
void *dyn_from = (uint8_t*)je + sizeof(journal_entry_small_write);
if (dyn_size <= sizeof(void*))
{
memcpy(&bmp, bmp_from, bs->dsk.clean_entry_bitmap_size);
// Bitmap without checksum is only 4 bytes for 128k objects, save it inline
// It can even contain 4 byte bitmap + 4 byte CRC32 for 4 kb writes :)
memcpy(&dyn, dyn_from, dyn_size);
}
else if (bs->journal.inmemory)
{
// Journal is kept in memory, refer to it instead of allocating a buffer
dyn = dyn_from;
}
else
{
@ -767,8 +868,8 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
// allocations for entry bitmaps. This can only be fixed by using
// a patched map with dynamic entry size, but not the btree_map,
// because it doesn't keep iterators valid all the time.
bmp = malloc_or_die(bs->dsk.clean_entry_bitmap_size);
memcpy(bmp, bmp_from, bs->dsk.clean_entry_bitmap_size);
dyn = malloc_or_die(dyn_size);
memcpy(dyn, dyn_from, dyn_size);
}
bs->dirty_db.emplace(ov, (dirty_entry){
.state = (BS_ST_SMALL_WRITE | BS_ST_SYNCED),
@ -777,7 +878,7 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
.offset = je->small_write.offset,
.len = je->small_write.len,
.journal_sector = proc_pos,
.bitmap = bmp,
.dyn_data = dyn,
});
bs->journal.used_sectors[proc_pos]++;
#ifdef BLOCKSTORE_DEBUG
@ -836,11 +937,18 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
.oid = je->big_write.oid,
.version = je->big_write.version,
};
void *bmp = NULL;
void *bmp_from = (uint8_t*)je + sizeof(journal_entry_big_write);
if (bs->dsk.clean_entry_bitmap_size <= sizeof(void*))
uint64_t dyn_size = bs->dsk.dirty_dyn_size(je->big_write.len);
void *dyn = NULL;
void *dyn_from = (uint8_t*)je + sizeof(journal_entry_big_write);
if (dyn_size <= sizeof(void*))
{
memcpy(&bmp, bmp_from, bs->dsk.clean_entry_bitmap_size);
// Bitmap without checksum is only 4 bytes for 128k objects, save it inline
memcpy(&dyn, dyn_from, dyn_size);
}
else if (bs->journal.inmemory)
{
// Journal is kept in memory, refer to it instead of allocating a buffer
dyn = dyn_from;
}
else
{
@ -848,8 +956,8 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
// allocations for entry bitmaps. This can only be fixed by using
// a patched map with dynamic entry size, but not the btree_map,
// because it doesn't keep iterators valid all the time.
bmp = malloc_or_die(bs->dsk.clean_entry_bitmap_size);
memcpy(bmp, bmp_from, bs->dsk.clean_entry_bitmap_size);
dyn = malloc_or_die(dyn_size);
memcpy(dyn, dyn_from, dyn_size);
}
auto dirty_it = bs->dirty_db.emplace(ov, (dirty_entry){
.state = (BS_ST_BIG_WRITE | BS_ST_SYNCED),
@ -858,7 +966,7 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
.offset = je->big_write.offset,
.len = je->big_write.len,
.journal_sector = proc_pos,
.bitmap = bmp,
.dyn_data = dyn,
}).first;
if (bs->data_alloc->get(je->big_write.location >> bs->dsk.block_order))
{

View File

@ -50,6 +50,7 @@ class blockstore_init_journal
uint64_t next_free;
std::vector<bs_init_journal_done> done;
std::vector<obj_ver_id> double_allocs;
std::vector<iovec> small_write_data;
uint64_t journal_pos = 0;
uint64_t continue_pos = 0;
void *init_write_buf = NULL;

View File

@ -8,7 +8,8 @@
#define MIN_JOURNAL_SIZE 4*1024*1024
#define JOURNAL_MAGIC 0x4A33
#define JOURNAL_VERSION 1
#define JOURNAL_VERSION_V1 1
#define JOURNAL_VERSION_V2 2
#define JOURNAL_BUFFER_SIZE 4*1024*1024
#define JOURNAL_ENTRY_HEADER_SIZE 16
@ -32,7 +33,7 @@
#define JE_BIG_WRITE_INSTANT 0x08
#define JE_MAX 0x08
// crc32c comes first to ease calculation and is equal to crc32()
// crc32c comes first to ease calculation
struct __attribute__((__packed__)) journal_entry_start
{
uint32_t crc32;
@ -42,8 +43,12 @@ struct __attribute__((__packed__)) journal_entry_start
uint32_t reserved;
uint64_t journal_start;
uint64_t version;
uint32_t data_csum_type;
uint32_t csum_block_size;
};
#define JE_START_LEGACY_SIZE 24
#define JE_START_V0_SIZE 24
#define JE_START_V1_SIZE 32
#define JE_START_V2_SIZE 40
struct __attribute__((__packed__)) journal_entry_small_write
{
@ -59,10 +64,12 @@ struct __attribute__((__packed__)) journal_entry_small_write
// small_write entries contain <len> bytes of data which is stored in next sectors
// data_offset is its offset within journal
uint64_t data_offset;
uint32_t crc32_data;
uint32_t crc32_data; // zero when data_csum_type != 0
// small_write and big_write entries are followed by the "external" bitmap
// its size is dynamic and included in journal entry's <size> field
uint8_t bitmap[];
// and then data checksums if data_csum_type != 0
// uint32_t data_crc32c[];
};
struct __attribute__((__packed__)) journal_entry_big_write
@ -80,6 +87,8 @@ struct __attribute__((__packed__)) journal_entry_big_write
// small_write and big_write entries are followed by the "external" bitmap
// its size is dynamic and included in journal entry's <size> field
uint8_t bitmap[];
// and then data checksums if data_csum_type != 0
// uint32_t data_crc32c[];
};
struct __attribute__((__packed__)) journal_entry_stable

View File

@ -133,19 +133,25 @@ void blockstore_impl_t::calc_lengths()
{
metadata_buffer = memalign(MEM_ALIGNMENT, dsk.meta_len);
if (!metadata_buffer)
throw std::runtime_error("Failed to allocate memory for the metadata");
throw std::runtime_error("Failed to allocate memory for the metadata ("+std::to_string(dsk.meta_len/1024/1024)+" MB)");
}
else if (dsk.clean_entry_bitmap_size)
else if (dsk.clean_entry_bitmap_size || dsk.data_csum_type)
{
clean_bitmap = (uint8_t*)malloc(dsk.block_count * 2*dsk.clean_entry_bitmap_size);
if (!clean_bitmap)
throw std::runtime_error("Failed to allocate memory for the metadata sparse write bitmap");
// FIXME: allow to store bitmap, but read checksums from the disk
clean_dyn_data = (uint8_t*)malloc(dsk.block_count * dsk.clean_dyn_size);
if (!clean_dyn_data)
{
throw std::runtime_error(
"Failed to allocate memory for the metadata sparse write bitmap ("+
std::to_string(dsk.block_count * dsk.clean_dyn_size / 1024 / 1024)+" MB)"
);
}
}
if (journal.inmemory)
{
journal.buffer = memalign(MEM_ALIGNMENT, journal.len);
if (!journal.buffer)
throw std::runtime_error("Failed to allocate memory for journal");
throw std::runtime_error("Failed to allocate memory for journal ("+std::to_string(journal.len/1024/1024)+" MB)");
}
else
{

View File

@ -41,8 +41,10 @@ int blockstore_impl_t::fulfill_read_push(blockstore_op_t *op, void *buf, uint64_
}
// FIXME I've seen a bug here so I want some tests
int blockstore_impl_t::fulfill_read(blockstore_op_t *read_op, 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)
int blockstore_impl_t::fulfill_read(blockstore_op_t *read_op,
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)
{
uint32_t cur_start = item_start;
if (cur_start < read_op->offset + read_op->len && item_end > read_op->offset)
@ -73,6 +75,9 @@ int blockstore_impl_t::fulfill_read(blockstore_op_t *read_op, uint64_t &fulfille
.offset = cur_start,
.len = it == PRIV(read_op)->read_vec.end() || it->offset >= item_end ? item_end-cur_start : it->offset-cur_start,
.journal_sector = journal_sector,
.item_state = item_state,
.disk_offset = item_location + el.offset - item_start,
.csum = !csum ? NULL : (csum + (cur_start - item_start) / dsk.csum_block_size * (dsk.data_csum_type & 0xFF)),
};
it = PRIV(read_op)->read_vec.insert(it, el);
if (!fulfill_read_push(read_op,
@ -106,7 +111,7 @@ 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_bitmap + meta_loc*2*dsk.clean_entry_bitmap_size + offset);
clean_entry_bitmap = (uint8_t*)(clean_dyn_data + meta_loc*dsk.clean_dyn_size + offset);
return clean_entry_bitmap;
}
@ -148,19 +153,21 @@ int blockstore_impl_t::dequeue_read(blockstore_op_t *read_op)
FINISH_OP(read_op);
return 2;
}
size_t dyn_size = dsk.dirty_dyn_size(dirty.len);
uint8_t *bmp_ptr = (uint8_t*)(dyn_size > sizeof(void*) ? dirty.dyn_data : &dirty.dyn_data);
if (!result_version)
{
result_version = dirty_it->first.version;
if (read_op->bitmap)
{
void *bmp_ptr = (dsk.clean_entry_bitmap_size > sizeof(void*) ? dirty_it->second.bitmap : &dirty_it->second.bitmap);
memcpy(read_op->bitmap, bmp_ptr, dsk.clean_entry_bitmap_size);
}
}
// If inmemory_journal is false, journal trim will have to wait until the read is completed
if (!fulfill_read(read_op, fulfilled, dirty.offset, dirty.offset + dirty.len,
dirty.state, dirty_it->first.version, dirty.location + (IS_JOURNAL(dirty.state) ? 0 : dirty.offset),
(IS_JOURNAL(dirty.state) ? dirty.journal_sector+1 : 0)))
(IS_JOURNAL(dirty.state) ? dirty.journal_sector+1 : 0),
journal.inmemory ? NULL : bmp_ptr+dsk.clean_entry_bitmap_size))
{
// need to wait. undo added requests, don't dequeue op
PRIV(read_op)->read_vec.clear();
@ -187,10 +194,12 @@ int blockstore_impl_t::dequeue_read(blockstore_op_t *read_op)
}
if (fulfilled < read_op->len)
{
uint8_t *clean_entry_bitmap = get_clean_entry_bitmap(clean_it->second.location, 0);
if (!dsk.clean_entry_bitmap_size)
{
if (!fulfill_read(read_op, fulfilled, 0, dsk.data_block_size,
(BS_ST_BIG_WRITE | BS_ST_STABLE), 0, clean_it->second.location, 0))
(BS_ST_BIG_WRITE | BS_ST_STABLE), 0, clean_it->second.location, 0,
clean_entry_bitmap + 2*dsk.clean_entry_bitmap_size))
{
// need to wait. undo added requests, don't dequeue op
PRIV(read_op)->read_vec.clear();
@ -199,7 +208,6 @@ int blockstore_impl_t::dequeue_read(blockstore_op_t *read_op)
}
else
{
uint8_t *clean_entry_bitmap = get_clean_entry_bitmap(clean_it->second.location, 0);
uint64_t bmp_start = 0, bmp_end = 0, bmp_size = dsk.data_block_size/dsk.bitmap_granularity;
while (bmp_start < bmp_size)
{
@ -211,7 +219,7 @@ int blockstore_impl_t::dequeue_read(blockstore_op_t *read_op)
{
// fill with zeroes
assert(fulfill_read(read_op, fulfilled, bmp_start * dsk.bitmap_granularity,
bmp_end * dsk.bitmap_granularity, (BS_ST_DELETE | BS_ST_STABLE), 0, 0, 0));
bmp_end * dsk.bitmap_granularity, (BS_ST_DELETE | BS_ST_STABLE), 0, 0, 0, NULL));
}
bmp_start = bmp_end;
while (clean_entry_bitmap[bmp_end >> 3] & (1 << (bmp_end & 0x7)) && bmp_end < bmp_size)
@ -220,9 +228,12 @@ int blockstore_impl_t::dequeue_read(blockstore_op_t *read_op)
}
if (bmp_end > bmp_start)
{
uint8_t *csum = !dsk.csum_block_size ? 0 : (clean_entry_bitmap +
2*dsk.clean_entry_bitmap_size +
bmp_start*dsk.bitmap_granularity/dsk.csum_block_size*(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_it->second.location + bmp_start * dsk.bitmap_granularity, 0))
clean_it->second.location + bmp_start * dsk.bitmap_granularity, 0, csum))
{
// need to wait. undo added requests, don't dequeue op
PRIV(read_op)->read_vec.clear();
@ -244,7 +255,7 @@ int blockstore_impl_t::dequeue_read(blockstore_op_t *read_op)
}
if (fulfilled < read_op->len)
{
assert(fulfill_read(read_op, fulfilled, 0, dsk.data_block_size, (BS_ST_DELETE | BS_ST_STABLE), 0, 0, 0));
assert(fulfill_read(read_op, fulfilled, 0, dsk.data_block_size, (BS_ST_DELETE | BS_ST_STABLE), 0, 0, 0, NULL));
assert(fulfilled == read_op->len);
}
read_op->version = result_version;
@ -282,6 +293,41 @@ void blockstore_impl_t::handle_read_event(ring_data_t *data, blockstore_op_t *op
// read error
op->retval = data->res;
}
else if (dsk.csum_block_size)
{
// verify checksum if required
uint64_t el_offset = (uint8_t*)data->iov.iov_base - ((uint8_t*)op->buf - op->offset);
auto & vecs = PRIV(op)->read_vec;
int el_min = 0, el_max = vecs.size();
while (el_max > el_min+1)
{
int mid = (el_min+el_max)/2;
if (el_offset < vecs[mid].offset)
el_max = mid;
else
el_min = mid;
}
auto & vec = vecs[el_min];
assert(vec.offset == el_offset);
if (vec.csum)
{
uint32_t *csum = (uint32_t*)vec.csum;
for (size_t p = 0; p < data->iov.iov_len; p += dsk.csum_block_size, csum++)
{
if (crc32c(0, data->iov.iov_base + p, dsk.csum_block_size) != *csum)
{
// checksum error
printf(
"Checksum mismatch in %s area at offset 0x%lx: %08lx vs %08lx\n",
IS_JOURNAL(vec.item_state) ? "journal" : "data",
crc32c(0, data->iov.iov_base + p, dsk.csum_block_size), *csum
);
op->retval = -EDOM;
break;
}
}
}
}
if (PRIV(op)->pending_ops == 0)
{
if (!journal.inmemory)
@ -324,8 +370,9 @@ int blockstore_impl_t::read_bitmap(object_id oid, uint64_t target_version, void
*result_version = dirty_it->first.version;
if (bitmap)
{
void *bmp_ptr = (dsk.clean_entry_bitmap_size > sizeof(void*) ? dirty_it->second.bitmap : &dirty_it->second.bitmap);
memcpy(bitmap, bmp_ptr, dsk.clean_entry_bitmap_size);
size_t dyn_size = dsk.dirty_dyn_size(dirty_it->second.len);
void *dyn_ptr = (dyn_size > sizeof(void*) ? dirty_it->second.dyn_data : &dirty_it->second.dyn_data);
memcpy(bitmap, dyn_ptr, dsk.clean_entry_bitmap_size);
}
return 0;
}

View File

@ -227,11 +227,7 @@ void blockstore_impl_t::erase_dirty(blockstore_dirty_db_t::iterator dirty_start,
journal.used_sectors.erase(dirty_it->second.journal_sector);
flusher->mark_trim_possible();
}
if (dsk.clean_entry_bitmap_size > sizeof(void*))
{
free(dirty_it->second.bitmap);
dirty_it->second.bitmap = NULL;
}
free_dirty_dyn_data(dirty_it->second);
if (dirty_it == dirty_start)
{
break;
@ -240,3 +236,20 @@ void blockstore_impl_t::erase_dirty(blockstore_dirty_db_t::iterator dirty_start,
}
dirty_db.erase(dirty_start, dirty_end);
}
void blockstore_impl_t::free_dirty_dyn_data(dirty_entry & e)
{
if (e.dyn_data)
{
size_t dyn_size = dsk.dirty_dyn_size(e.len);
if (dyn_size > sizeof(void*) &&
(!journal.inmemory || e.dyn_data < journal.buffer ||
e.dyn_data >= journal.buffer + journal.len))
{
// dyn_data contains the bitmap and checksums
// free it if it doesn't refer to the in-memory journal
free(e.dyn_data);
}
e.dyn_data = NULL;
}
}

View File

@ -115,8 +115,8 @@ int blockstore_impl_t::continue_sync(blockstore_op_t *op)
je->offset = dirty_entry.offset;
je->len = dirty_entry.len;
je->location = dirty_entry.location;
memcpy((void*)(je+1), (dsk.clean_entry_bitmap_size > sizeof(void*)
? dirty_entry.bitmap : &dirty_entry.bitmap), dsk.clean_entry_bitmap_size);
uint64_t dyn_size = dsk.dirty_dyn_size(dirty_entry.len);
memcpy((void*)(je+1), (dyn_size > sizeof(void*) ? dirty_entry.dyn_data : &dirty_entry.dyn_data), dyn_size);
je->crc32 = je_crc32((journal_entry*)je);
journal.crc32_last = je->crc32;
it++;

View File

@ -8,12 +8,18 @@ bool blockstore_impl_t::enqueue_write(blockstore_op_t *op)
// Check or assign version number
bool found = false, deleted = false, unsynced = false, is_del = (op->opcode == BS_OP_DELETE);
bool wait_big = false, wait_del = false;
void *bmp = NULL;
uint64_t version = 1;
if (!is_del && dsk.clean_entry_bitmap_size > sizeof(void*))
void *dyn = NULL;
if (is_del)
{
bmp = calloc_or_die(1, dsk.clean_entry_bitmap_size);
op->len = 0;
}
size_t dyn_size = dsk.dirty_dyn_size(op->len);
if (!is_del && dyn_size > sizeof(void*))
{
dyn = calloc_or_die(1, dyn_size);
}
uint8_t *dyn_ptr = (uint8_t*)(dyn_size > sizeof(void*) ? dyn : &dyn);
uint64_t version = 1;
if (dirty_db.size() > 0)
{
auto dirty_it = dirty_db.upper_bound((obj_ver_id){
@ -33,10 +39,9 @@ bool blockstore_impl_t::enqueue_write(blockstore_op_t *op)
: ((dirty_it->second.state & BS_ST_WORKFLOW_MASK) == BS_ST_WAIT_BIG);
if (!is_del && !deleted)
{
if (dsk.clean_entry_bitmap_size > sizeof(void*))
memcpy(bmp, dirty_it->second.bitmap, dsk.clean_entry_bitmap_size);
else
bmp = dirty_it->second.bitmap;
void *dyn_from = dsk.dirty_dyn_size(dirty_it->second.len) > sizeof(void*)
? dirty_it->second.dyn_data : &dirty_it->second.dyn_data;
memcpy(dyn_ptr, dyn_from, dsk.clean_entry_bitmap_size);
}
}
}
@ -50,7 +55,7 @@ bool blockstore_impl_t::enqueue_write(blockstore_op_t *op)
if (!is_del)
{
void *bmp_ptr = get_clean_entry_bitmap(clean_it->second.location, dsk.clean_entry_bitmap_size);
memcpy((dsk.clean_entry_bitmap_size > sizeof(void*) ? bmp : &bmp), bmp_ptr, dsk.clean_entry_bitmap_size);
memcpy((dyn_size > sizeof(void*) ? dyn : &dyn), bmp_ptr, dsk.clean_entry_bitmap_size);
}
}
else
@ -112,9 +117,9 @@ bool blockstore_impl_t::enqueue_write(blockstore_op_t *op)
printf("Write %lx:%lx v%lu requested, but we already have v%lu\n", op->oid.inode, op->oid.stripe, op->version, version);
#endif
op->retval = -EEXIST;
if (!is_del && dsk.clean_entry_bitmap_size > sizeof(void*))
if (!is_del && dyn_size > sizeof(void*))
{
free(bmp);
free(dyn);
}
return false;
}
@ -158,26 +163,36 @@ bool blockstore_impl_t::enqueue_write(blockstore_op_t *op)
if (op->bitmap)
{
// Only allow to overwrite part of the object bitmap respective to the write's offset/len
uint8_t *bmp_ptr = (uint8_t*)(dsk.clean_entry_bitmap_size > sizeof(void*) ? bmp : &bmp);
// FIXME Don't merge bitmaps here - parallel writes into one object may break the bitmap
uint32_t bit = op->offset/dsk.bitmap_granularity;
uint32_t bits_left = op->len/dsk.bitmap_granularity;
while (!(bit % 8) && bits_left >= 8)
{
// Copy bytes
bmp_ptr[bit/8] = ((uint8_t*)op->bitmap)[bit/8];
dyn_ptr[bit/8] = ((uint8_t*)op->bitmap)[bit/8];
bit += 8;
bits_left -= 8;
}
while (bits_left > 0)
{
// Copy bits
bmp_ptr[bit/8] = (bmp_ptr[bit/8] & ~(1 << (bit%8)))
dyn_ptr[bit/8] = (dyn_ptr[bit/8] & ~(1 << (bit%8)))
| (((uint8_t*)op->bitmap)[bit/8] & (1 << bit%8));
bit++;
bits_left--;
}
}
}
// Calculate checksums
// FIXME: Allow to receive them from outside
if (dsk.data_csum_type)
{
uint32_t *data_csum = (uint32_t*)(dyn_ptr + dsk.clean_entry_bitmap_size);
for (uint32_t i = 0; i < op->len / dsk.csum_block_size; i++)
{
data_csum[i] = crc32c(0, op->buf + i*dsk.csum_block_size, dsk.csum_block_size);
}
}
dirty_db.emplace((obj_ver_id){
.oid = op->oid,
.version = op->version,
@ -188,7 +203,7 @@ bool blockstore_impl_t::enqueue_write(blockstore_op_t *op)
.offset = is_del ? 0 : op->offset,
.len = is_del ? 0 : op->len,
.journal_sector = 0,
.bitmap = bmp,
.dyn_data = dyn,
});
return true;
}
@ -197,8 +212,7 @@ void blockstore_impl_t::cancel_all_writes(blockstore_op_t *op, blockstore_dirty_
{
while (dirty_it != dirty_db.end() && dirty_it->first.oid == op->oid)
{
if (dsk.clean_entry_bitmap_size > sizeof(void*))
free(dirty_it->second.bitmap);
free_dirty_dyn_data(dirty_it->second);
dirty_db.erase(dirty_it++);
}
bool found = false;
@ -395,9 +409,10 @@ int blockstore_impl_t::dequeue_write(blockstore_op_t *op)
}
}
// Then pre-fill journal entry
uint64_t dyn_size = dsk.dirty_dyn_size(op->len);
journal_entry_small_write *je = (journal_entry_small_write*)prefill_single_journal_entry(
journal, op->opcode == BS_OP_WRITE_STABLE ? JE_SMALL_WRITE_INSTANT : JE_SMALL_WRITE,
sizeof(journal_entry_small_write) + dsk.clean_entry_bitmap_size
sizeof(journal_entry_small_write) + dyn_size
);
dirty_it->second.journal_sector = journal.sector_info[journal.cur_sector].offset;
journal.used_sectors[journal.sector_info[journal.cur_sector].offset]++;
@ -431,8 +446,8 @@ int blockstore_impl_t::dequeue_write(blockstore_op_t *op)
je->offset = op->offset;
je->len = op->len;
je->data_offset = journal.next_free;
je->crc32_data = crc32c(0, op->buf, op->len);
memcpy((void*)(je+1), (dsk.clean_entry_bitmap_size > sizeof(void*) ? dirty_it->second.bitmap : &dirty_it->second.bitmap), dsk.clean_entry_bitmap_size);
je->crc32_data = dsk.csum_block_size ? 0 : crc32c(0, op->buf, op->len);
memcpy((void*)(je+1), (dyn_size > sizeof(void*) ? dirty_it->second.dyn_data : &dirty_it->second.dyn_data), dyn_size);
je->crc32 = je_crc32((journal_entry*)je);
journal.crc32_last = je->crc32;
if (immediate_commit != IMMEDIATE_NONE)
@ -509,9 +524,15 @@ resume_2:
return 0;
}
BS_SUBMIT_CHECK_SQES(1);
auto dirty_it = dirty_db.find((obj_ver_id){
.oid = op->oid,
.version = op->version,
});
assert(dirty_it != dirty_db.end());
uint64_t dyn_size = dsk.dirty_dyn_size(op->len);
journal_entry_big_write *je = (journal_entry_big_write*)prefill_single_journal_entry(
journal, op->opcode == BS_OP_WRITE_STABLE ? JE_BIG_WRITE_INSTANT : JE_BIG_WRITE,
sizeof(journal_entry_big_write) + dsk.clean_entry_bitmap_size
sizeof(journal_entry_big_write) + dyn_size
);
dirty_it->second.journal_sector = journal.sector_info[journal.cur_sector].offset;
journal.used_sectors[journal.sector_info[journal.cur_sector].offset]++;
@ -527,7 +548,7 @@ resume_2:
je->offset = op->offset;
je->len = op->len;
je->location = dirty_it->second.location;
memcpy((void*)(je+1), (dsk.clean_entry_bitmap_size > sizeof(void*) ? dirty_it->second.bitmap : &dirty_it->second.bitmap), dsk.clean_entry_bitmap_size);
memcpy((void*)(je+1), (dyn_size > sizeof(void*) ? dirty_it->second.dyn_data : &dirty_it->second.dyn_data), dyn_size);
je->crc32 = je_crc32((journal_entry*)je);
journal.crc32_last = je->crc32;
prepare_journal_sector_write(journal.cur_sector, op);

View File

@ -10,6 +10,7 @@
#include "json11/json11.hpp"
#include "str_util.h"
#include "blockstore.h"
#include "blockstore_disk.h"
// Calculate offsets for a block device and print OSD command line parameters
void disk_tool_simple_offsets(json11::Json cfg, bool json_output)
@ -20,23 +21,39 @@ void disk_tool_simple_offsets(json11::Json cfg, bool json_output)
fprintf(stderr, "Device path is missing\n");
exit(1);
}
uint64_t object_size = parse_size(cfg["object_size"].string_value());
uint64_t data_block_size = parse_size(cfg["object_size"].string_value());
uint64_t bitmap_granularity = parse_size(cfg["bitmap_granularity"].string_value());
uint64_t journal_size = parse_size(cfg["journal_size"].string_value());
uint64_t device_block_size = parse_size(cfg["device_block_size"].string_value());
uint64_t journal_offset = parse_size(cfg["journal_offset"].string_value());
uint64_t device_size = parse_size(cfg["device_size"].string_value());
uint32_t csum_block_size = parse_size(cfg["csum_block_size"].string_value());
uint32_t data_csum_type = BLOCKSTORE_CSUM_NONE;
if (cfg["data_csum_type"] == "crc32c")
data_csum_type = BLOCKSTORE_CSUM_CRC32C;
else if (cfg["data_csum_type"].string_value() != "" && cfg["data_csum_type"].string_value() != "none")
{
fprintf(
stderr, "data_csum_type=%s is unsupported, only \"crc32c\" and \"none\" are supported",
cfg["data_csum_type"].string_value().c_str()
);
exit(1);
}
std::string format = cfg["format"].string_value();
if (json_output)
format = "json";
if (!object_size)
object_size = 1 << DEFAULT_DATA_BLOCK_ORDER;
if (!data_block_size)
data_block_size = 1 << DEFAULT_DATA_BLOCK_ORDER;
if (!bitmap_granularity)
bitmap_granularity = DEFAULT_BITMAP_GRANULARITY;
if (!journal_size)
journal_size = 16*1024*1024;
if (!device_block_size)
device_block_size = 4096;
if (!data_csum_type)
csum_block_size = 0;
else if (!csum_block_size)
csum_block_size = bitmap_granularity;
uint64_t orig_device_size = device_size;
if (!device_size)
{
@ -85,22 +102,30 @@ void disk_tool_simple_offsets(json11::Json cfg, bool json_output)
fprintf(stderr, "Invalid device block size specified: %lu\n", device_block_size);
exit(1);
}
if (object_size < device_block_size || object_size > MAX_DATA_BLOCK_SIZE ||
object_size & (object_size-1) != 0)
if (data_block_size < device_block_size || data_block_size > MAX_DATA_BLOCK_SIZE ||
data_block_size & (data_block_size-1) != 0)
{
fprintf(stderr, "Invalid object size specified: %lu\n", object_size);
fprintf(stderr, "Invalid object size specified: %lu\n", data_block_size);
exit(1);
}
if (bitmap_granularity < device_block_size || bitmap_granularity > object_size ||
if (bitmap_granularity < device_block_size || bitmap_granularity > data_block_size ||
bitmap_granularity & (bitmap_granularity-1) != 0)
{
fprintf(stderr, "Invalid bitmap granularity specified: %lu\n", bitmap_granularity);
exit(1);
}
if (csum_block_size && (data_block_size % csum_block_size))
{
fprintf(stderr, "csum_block_size must be a divisor of data_block_size\n");
exit(1);
}
journal_offset = ((journal_offset+device_block_size-1)/device_block_size)*device_block_size;
uint64_t meta_offset = journal_offset + ((journal_size+device_block_size-1)/device_block_size)*device_block_size;
uint64_t entries_per_block = (device_block_size / (24 + 2*object_size/bitmap_granularity/8));
uint64_t object_count = ((device_size-meta_offset)/object_size);
uint64_t data_csum_size = (data_csum_type ? data_block_size/csum_block_size*(data_csum_type & 0xFF) : 0);
uint64_t clean_entry_bitmap_size = data_block_size/bitmap_granularity/8;
uint64_t clean_entry_size = 24 /*sizeof(clean_disk_entry)*/ + 2*clean_entry_bitmap_size + data_csum_size + 4 /*entry_csum*/;
uint64_t entries_per_block = device_block_size / clean_entry_size;
uint64_t object_count = ((device_size-meta_offset)/data_block_size);
uint64_t meta_size = (1 + (object_count+entries_per_block-1)/entries_per_block) * device_block_size;
uint64_t data_offset = meta_offset + meta_size;
if (format == "json")

View File

@ -270,6 +270,19 @@ int main(int argc, char *argv[])
fprintf(stderr, "Invalid JSON: %s\n", json_err.c_str());
return 1;
}
if (entries[0]["type"] == "start")
{
self.dsk.data_csum_type = csum_type_from_str(entries[0]["data_csum_type"].string_value());
self.dsk.csum_block_size = entries[0]["csum_block_size"].uint64_value();
}
if (self.options["data_csum_type"] != "")
{
self.dsk.data_csum_type = csum_type_from_str(self.options["data_csum_type"]);
}
if (self.options["csum_block_size"] != "")
{
self.dsk.csum_block_size = stoull_full(self.options["csum_block_size"], 0);
}
return self.write_json_journal(entries);
}
else if (!strcmp(cmd[0], "dump-meta"))

View File

@ -64,17 +64,19 @@ struct disk_tool_t
ring_loop_t *ringloop;
ring_consumer_t ring_consumer;
int remap_active;
journal_entry_start je_start;
uint8_t *new_journal_buf, *new_meta_buf, *new_journal_ptr, *new_journal_data;
uint64_t new_journal_in_pos;
int64_t data_idx_diff;
uint64_t total_blocks, free_first, free_last;
uint64_t new_clean_entry_bitmap_size, new_clean_entry_size, new_entries_per_block;
uint64_t new_clean_entry_bitmap_size, new_data_csum_size, new_clean_entry_size, new_entries_per_block;
int new_journal_fd, new_meta_fd;
resizer_data_moving_t *moving_blocks;
bool started;
void *small_write_data;
uint32_t data_crc32;
bool data_csum_valid;
uint32_t crc32_last;
uint32_t new_crc32_prev;
@ -84,11 +86,11 @@ struct disk_tool_t
void dump_journal_entry(int num, journal_entry *je, bool json);
int process_journal(std::function<int(void*)> block_fn);
int process_journal_block(void *buf, std::function<void(int, journal_entry*)> iter_fn);
int process_meta(std::function<void(blockstore_meta_header_v1_t *)> hdr_fn,
int process_meta(std::function<void(blockstore_meta_header_v2_t *)> hdr_fn,
std::function<void(uint64_t, clean_disk_entry*, uint8_t*)> record_fn);
int dump_meta();
void dump_meta_header(blockstore_meta_header_v1_t *hdr);
void dump_meta_header(blockstore_meta_header_v2_t *hdr);
void dump_meta_entry(uint64_t block_num, clean_disk_entry *entry, uint8_t *bitmap);
int write_json_journal(json11::Json entries);
@ -96,7 +98,7 @@ struct disk_tool_t
int resize_data();
int resize_parse_params();
void resize_init(blockstore_meta_header_v1_t *hdr);
void resize_init(blockstore_meta_header_v2_t *hdr);
int resize_remap_blocks();
int resize_copy_data();
int resize_rewrite_journal();
@ -141,3 +143,5 @@ json11::Json read_parttable(std::string dev);
uint64_t dev_size_from_parttable(json11::Json pt);
uint64_t free_from_parttable(json11::Json pt);
int fix_partition_type(std::string dev_by_uuid);
std::string csum_type_str(uint32_t data_csum_type);
uint32_t csum_type_from_str(std::string data_csum_type);

View File

@ -120,8 +120,22 @@ int disk_tool_t::process_journal(std::function<int(void*)> block_fn)
fprintf(stderr, "offset %08lx: journal superblock is invalid\n", journal_pos);
r = 1;
}
else if (je->start.size != JE_START_V0_SIZE && je->start.version != JOURNAL_VERSION_V1 && je->start.version != JOURNAL_VERSION_V2)
{
fprintf(stderr, "offset %08lx: journal superblock contains version %lu, but I only understand 0, 1 and 2\n",
journal_pos, je->start.size == JE_START_V0_SIZE ? 0 : je->start.version);
r = 1;
}
else
{
memcpy(&je_start, je, sizeof(je_start));
if (je_start.size == JE_START_V0_SIZE)
je_start.version = 0;
if (je_start.version < JOURNAL_VERSION_V2)
{
je_start.data_csum_type = 0;
je_start.csum_block_size = 0;
}
started = false;
crc32_last = 0;
block_fn(data);
@ -183,7 +197,21 @@ int disk_tool_t::process_journal_block(void *buf, std::function<void(int, journa
}
small_write_data = memalign_or_die(MEM_ALIGNMENT, je->small_write.len);
assert(pread(dsk.journal_fd, small_write_data, je->small_write.len, dsk.journal_offset+je->small_write.data_offset) == je->small_write.len);
data_crc32 = crc32c(0, small_write_data, je->small_write.len);
data_crc32 = je_start.csum_block_size ? 0 : crc32c(0, small_write_data, je->small_write.len);
data_csum_valid = (data_crc32 == je->small_write.crc32_data);
if (je_start.csum_block_size)
{
uint32_t data_csum_size = je->small_write.len/je_start.csum_block_size*(je_start.data_csum_type & 0xFF);
uint32_t *block_csums = (uint32_t*)((uint8_t*)je + je->size - data_csum_size);
for (uint32_t pos = 0; pos < je->small_write.len; pos += je_start.csum_block_size, block_csums++)
{
if (crc32c(0, small_write_data + pos, je_start.csum_block_size) != *block_csums)
{
data_csum_valid = false;
break;
}
}
}
}
iter_fn(entry, je);
if (je->type == JE_SMALL_WRITE || je->type == JE_SMALL_WRITE_INSTANT)
@ -223,9 +251,17 @@ void disk_tool_t::dump_journal_entry(int num, journal_entry *je, bool json)
if (je->type == JE_START)
{
printf(
json ? ",\"type\":\"start\",\"start\":\"0x%lx\"}" : "je_start start=%08lx\n",
json ? ",\"type\":\"start\",\"start\":\"0x%lx\"" : "je_start start=%08lx",
je->start.journal_start
);
if (je->start.data_csum_type)
{
printf(
json ? ",\"data_csum_type\":\"%s\",\"csum_block_size\":%u" : " data_csum_type=%s csum_block_size=%u",
csum_type_str(je->start.data_csum_type).c_str(), je->start.csum_block_size
);
}
printf(json ? "}" : "\n");
}
else if (je->type == JE_SMALL_WRITE || je->type == JE_SMALL_WRITE_INSTANT)
{
@ -242,10 +278,11 @@ void disk_tool_t::dump_journal_entry(int num, journal_entry *je, bool json)
printf(json ? ",\"bad_loc\":true,\"calc_loc\":\"0x%lx\""
: " (mismatched, calculated = %lu)", journal_pos);
}
if (je->small_write.size > sizeof(journal_entry_small_write))
uint32_t data_csum_size = (!je_start.csum_block_size ? 0 : je->small_write.len/je_start.csum_block_size*(je_start.data_csum_type & 0xFF));
if (je->size > sizeof(journal_entry_small_write) + data_csum_size)
{
printf(json ? ",\"bitmap\":\"" : " (bitmap: ");
for (int i = sizeof(journal_entry_small_write); i < je->small_write.size; i++)
for (int i = sizeof(journal_entry_small_write); i < je->size - data_csum_size; i++)
{
printf("%02x", ((uint8_t*)je)[i]);
}
@ -260,10 +297,21 @@ void disk_tool_t::dump_journal_entry(int num, journal_entry *je, bool json)
}
printf(json ? "\"" : ")");
}
if (data_csum_size > 0)
{
printf(json ? ",\"block_csums\":\"" : " block_csums=");
uint8_t *block_csums = (uint8_t*)je + je->size - data_csum_size;
for (int i = 0; i < data_csum_size; i++)
printf("%02x", block_csums[i]);
printf(json ? "\"" : "");
}
else
{
printf(json ? ",\"data_crc32\":\"%08x\"" : " data_crc32=%08x", je->small_write.crc32_data);
}
printf(
json ? ",\"data_crc32\":\"%08x\",\"data_valid\":%s}" : " data_crc32=%08x%s\n",
je->small_write.crc32_data,
(data_crc32 != je->small_write.crc32_data
json ? ",\"data_valid\":%s}" : "%s\n",
(data_csum_valid
? (json ? "false" : " (invalid)")
: (json ? "true" : " (valid)"))
);
@ -278,10 +326,19 @@ void disk_tool_t::dump_journal_entry(int num, journal_entry *je, bool json)
je->big_write.version, je->big_write.offset, je->big_write.len,
je->big_write.location
);
if (je->big_write.size > sizeof(journal_entry_big_write))
uint32_t data_csum_size = (!je_start.csum_block_size ? 0 : je->big_write.len/je_start.csum_block_size*(je_start.data_csum_type & 0xFF));
if (data_csum_size > 0)
{
printf(json ? ",\"block_csums\":\"" : " block_csums=");
uint8_t *block_csums = (uint8_t*)je + je->size - data_csum_size;
for (int i = 0; i < data_csum_size; i++)
printf("%02x", block_csums[i]);
printf(json ? "\"" : "");
}
if (je->big_write.size > sizeof(journal_entry_big_write) + data_csum_size)
{
printf(json ? ",\"bitmap\":\"" : " (bitmap: ");
for (int i = sizeof(journal_entry_big_write); i < je->big_write.size; i++)
for (int i = sizeof(journal_entry_big_write); i < je->big_write.size - data_csum_size; i++)
{
printf("%02x", ((uint8_t*)je)[i]);
}
@ -338,7 +395,9 @@ int disk_tool_t::write_json_journal(json11::Json entries)
.type = JE_START,
.size = sizeof(journal_entry_start),
.journal_start = dsk.journal_block_size,
.version = JOURNAL_VERSION,
.version = JOURNAL_VERSION_V2,
.data_csum_type = dsk.data_csum_type,
.csum_block_size = dsk.csum_block_size,
};
((journal_entry*)new_journal_buf)->crc32 = je_crc32((journal_entry*)new_journal_buf);
new_journal_ptr += dsk.journal_block_size;
@ -358,9 +417,11 @@ int disk_tool_t::write_json_journal(json11::Json entries)
uint32_t entry_size = (type == JE_START
? sizeof(journal_entry_start)
: (type == JE_SMALL_WRITE || type == JE_SMALL_WRITE_INSTANT
? sizeof(journal_entry_small_write) + dsk.clean_entry_bitmap_size
? sizeof(journal_entry_small_write) + dsk.clean_entry_bitmap_size +
(dsk.data_csum_type ? rec["len"].uint64_value()/dsk.csum_block_size*(dsk.data_csum_type & 0xFF) : 0)
: (type == JE_BIG_WRITE || type == JE_BIG_WRITE_INSTANT
? sizeof(journal_entry_big_write) + dsk.clean_entry_bitmap_size
? sizeof(journal_entry_big_write) + dsk.clean_entry_bitmap_size +
(dsk.data_csum_type ? rec["len"].uint64_value()/dsk.csum_block_size*(dsk.data_csum_type & 0xFF) : 0)
: sizeof(journal_entry_del))));
if (dsk.journal_block_size < new_journal_in_pos + entry_size)
{
@ -402,12 +463,24 @@ int disk_tool_t::write_json_journal(json11::Json entries)
.offset = (uint32_t)rec["offset"].uint64_value(),
.len = (uint32_t)rec["len"].uint64_value(),
.data_offset = (uint64_t)(new_journal_data-new_journal_buf),
.crc32_data = (uint32_t)sscanf_json("%x", rec["data_crc32"]),
.crc32_data = !dsk.data_csum_type ? 0 : (uint32_t)sscanf_json("%x", rec["data_crc32"]),
};
fromhexstr(rec["bitmap"].string_value(), dsk.clean_entry_bitmap_size, ((uint8_t*)ne) + sizeof(journal_entry_small_write));
uint32_t data_csum_size = !dsk.data_csum_type ? 0 : ne->small_write.len/dsk.csum_block_size*(dsk.data_csum_type & 0xFF);
fromhexstr(rec["bitmap"].string_value(), dsk.clean_entry_bitmap_size, ((uint8_t*)ne) + sizeof(journal_entry_small_write) + data_csum_size);
fromhexstr(rec["data"].string_value(), ne->small_write.len, new_journal_data);
if (dsk.data_csum_type)
fromhexstr(rec["block_csums"].string_value(), data_csum_size, ((uint8_t*)ne) + sizeof(journal_entry_small_write));
if (rec["data"].is_string())
ne->small_write.crc32_data = crc32c(0, new_journal_data, ne->small_write.len);
{
if (!dsk.data_csum_type)
ne->small_write.crc32_data = crc32c(0, new_journal_data, ne->small_write.len);
else if (dsk.data_csum_type == BLOCKSTORE_CSUM_CRC32C)
{
uint32_t *block_csums = (uint32_t*)(((uint8_t*)ne) + sizeof(journal_entry_small_write));
for (uint32_t i = 0; i < ne->small_write.len; i += dsk.csum_block_size, block_csums++)
*block_csums = crc32c(0, new_journal_data+i, dsk.csum_block_size);
}
}
new_journal_data += ne->small_write.len;
}
else if (type == JE_BIG_WRITE || type == JE_BIG_WRITE_INSTANT)
@ -426,7 +499,10 @@ int disk_tool_t::write_json_journal(json11::Json entries)
.len = (uint32_t)rec["len"].uint64_value(),
.location = sscanf_json(NULL, rec["loc"]),
};
fromhexstr(rec["bitmap"].string_value(), dsk.clean_entry_bitmap_size, ((uint8_t*)ne) + sizeof(journal_entry_big_write));
uint32_t data_csum_size = !dsk.data_csum_type ? 0 : ne->big_write.len/dsk.csum_block_size*(dsk.data_csum_type & 0xFF);
fromhexstr(rec["bitmap"].string_value(), dsk.clean_entry_bitmap_size, ((uint8_t*)ne) + sizeof(journal_entry_big_write) + data_csum_size);
if (dsk.data_csum_type)
fromhexstr(rec["block_csums"].string_value(), data_csum_size, ((uint8_t*)ne) + sizeof(journal_entry_big_write));
}
else if (type == JE_STABLE || type == JE_ROLLBACK || type == JE_DELETE)
{

View File

@ -5,7 +5,7 @@
#include "rw_blocking.h"
#include "osd_id.h"
int disk_tool_t::process_meta(std::function<void(blockstore_meta_header_v1_t *)> hdr_fn,
int disk_tool_t::process_meta(std::function<void(blockstore_meta_header_v2_t *)> hdr_fn,
std::function<void(uint64_t, clean_disk_entry*, uint8_t*)> record_fn)
{
if (dsk.meta_block_size % DIRECT_IO_ALIGNMENT)
@ -28,12 +28,38 @@ int disk_tool_t::process_meta(std::function<void(blockstore_meta_header_v1_t *)>
lseek64(dsk.meta_fd, dsk.meta_offset, 0);
read_blocking(dsk.meta_fd, data, dsk.meta_block_size);
// Check superblock
blockstore_meta_header_v1_t *hdr = (blockstore_meta_header_v1_t *)data;
if (hdr->zero == 0 &&
hdr->magic == BLOCKSTORE_META_MAGIC_V1 &&
hdr->version == BLOCKSTORE_META_VERSION_V1)
blockstore_meta_header_v2_t *hdr = (blockstore_meta_header_v2_t *)data;
if (hdr->zero == 0 && hdr->magic == BLOCKSTORE_META_MAGIC_V1)
{
// Vitastor 0.6-0.7 - static array of clean_disk_entry with bitmaps
if (hdr->version == BLOCKSTORE_META_VERSION_V1)
{
// Vitastor 0.6-0.7 - static array of clean_disk_entry with bitmaps
hdr->data_csum_type = 0;
hdr->csum_block_size = 0;
hdr->header_csum = 0;
}
else if (hdr->version == BLOCKSTORE_META_VERSION_V2)
{
// Vitastor 0.8 - static array of clean_disk_entry with bitmaps and checksums
if (hdr->data_csum_type != 0 &&
hdr->data_csum_type != BLOCKSTORE_CSUM_CRC32C)
{
fprintf(stderr, "I don't know checksum format %u, the only supported format is crc32c = %u.\n", hdr->data_csum_type, BLOCKSTORE_CSUM_CRC32C);
free(data);
close(dsk.meta_fd);
dsk.meta_fd = -1;
return 1;
}
}
else
{
// Unsupported version
fprintf(stderr, "Metadata format is too new for me (stored version is %lu, max supported %u).\n", hdr->version, BLOCKSTORE_META_VERSION_V2);
free(data);
close(dsk.meta_fd);
dsk.meta_fd = -1;
return 1;
}
if (hdr->meta_block_size != dsk.meta_block_size)
{
fprintf(stderr, "Using block size of %u bytes based on information from the superblock\n", hdr->meta_block_size);
@ -45,12 +71,19 @@ int disk_tool_t::process_meta(std::function<void(blockstore_meta_header_v1_t *)>
memcpy(new_data, data, dsk.meta_block_size);
free(data);
data = new_data;
hdr = (blockstore_meta_header_v1_t *)data;
hdr = (blockstore_meta_header_v2_t *)data;
}
}
dsk.data_block_size = hdr->data_block_size;
dsk.csum_block_size = hdr->csum_block_size;
dsk.data_csum_type = hdr->data_csum_type;
dsk.bitmap_granularity = hdr->bitmap_granularity;
dsk.clean_entry_bitmap_size = hdr->data_block_size / hdr->bitmap_granularity / 8;
dsk.clean_entry_size = sizeof(clean_disk_entry) + 2*dsk.clean_entry_bitmap_size;
dsk.clean_entry_bitmap_size = (hdr->data_block_size / hdr->bitmap_granularity + 7) / 8;
dsk.clean_entry_size = sizeof(clean_disk_entry) + 2*dsk.clean_entry_bitmap_size
+ (hdr->data_csum_type
? ((hdr->data_block_size+hdr->csum_block_size-1)/hdr->csum_block_size
*(hdr->data_csum_type & 0xff))
: 0) + 4 /*entry_csum*/;
uint64_t block_num = 0;
hdr_fn(hdr);
meta_pos = dsk.meta_block_size;
@ -67,6 +100,12 @@ int disk_tool_t::process_meta(std::function<void(blockstore_meta_header_v1_t *)>
clean_disk_entry *entry = (clean_disk_entry*)((uint8_t*)data + blk + ioff);
if (entry->oid.inode)
{
uint32_t *entry_csum = (uint32_t*)((uint8_t*)entry + dsk.clean_entry_size - 4);
if (*entry_csum != crc32c(0, entry, dsk.clean_entry_size - 4))
{
fprintf(stderr, "Metadata entry %lu is corrupt (checksum mismatch), skipping\n", block_num);
continue;
}
record_fn(block_num, entry, entry->bitmap);
}
}
@ -107,21 +146,35 @@ int disk_tool_t::process_meta(std::function<void(blockstore_meta_header_v1_t *)>
int disk_tool_t::dump_meta()
{
int r = process_meta(
[this](blockstore_meta_header_v1_t *hdr) { dump_meta_header(hdr); },
[this](blockstore_meta_header_v2_t *hdr) { dump_meta_header(hdr); },
[this](uint64_t block_num, clean_disk_entry *entry, uint8_t *bitmap) { dump_meta_entry(block_num, entry, bitmap); }
);
printf("\n]}\n");
if (r == 0)
printf("\n]}\n");
return r;
}
void disk_tool_t::dump_meta_header(blockstore_meta_header_v1_t *hdr)
void disk_tool_t::dump_meta_header(blockstore_meta_header_v2_t *hdr)
{
if (hdr)
{
printf(
"{\"version\":\"0.6\",\"meta_block_size\":%u,\"data_block_size\":%u,\"bitmap_granularity\":%u,\"entries\":[\n",
hdr->meta_block_size, hdr->data_block_size, hdr->bitmap_granularity
);
if (hdr->version == BLOCKSTORE_META_VERSION_V1)
{
printf(
"{\"version\":\"0.6\",\"meta_block_size\":%u,\"data_block_size\":%u,\"bitmap_granularity\":%u,"
"\"entries\":[\n",
hdr->meta_block_size, hdr->data_block_size, hdr->bitmap_granularity
);
}
else if (hdr->version == BLOCKSTORE_META_VERSION_V2)
{
printf(
"{\"version\":\"0.8\",\"meta_block_size\":%u,\"data_block_size\":%u,\"bitmap_granularity\":%u,"
"\"data_csum_type\":%s,\"csum_block_size\":%u,\"entries\":[\n",
hdr->meta_block_size, hdr->data_block_size, hdr->bitmap_granularity,
csum_type_str(hdr->data_csum_type).c_str(), hdr->csum_block_size
);
}
}
else
{
@ -151,6 +204,15 @@ void disk_tool_t::dump_meta_entry(uint64_t block_num, clean_disk_entry *entry, u
{
printf("%02x", bitmap[dsk.clean_entry_bitmap_size + i]);
}
if (dsk.csum_block_size && dsk.data_csum_type)
{
uint8_t *csums = bitmap + dsk.clean_entry_bitmap_size*2;
printf("\",\"block_csums\":\"");
for (uint64_t i = 0; i < (dsk.data_block_size+dsk.csum_block_size-1)/dsk.csum_block_size*(dsk.data_csum_type & 0xFF); i++)
{
printf("%02x", csums[i]);
}
}
printf("\"}");
}
else
@ -164,18 +226,30 @@ int disk_tool_t::write_json_meta(json11::Json meta)
{
new_meta_buf = (uint8_t*)memalign_or_die(MEM_ALIGNMENT, new_meta_len);
memset(new_meta_buf, 0, new_meta_len);
blockstore_meta_header_v1_t *new_hdr = (blockstore_meta_header_v1_t *)new_meta_buf;
blockstore_meta_header_v2_t *new_hdr = (blockstore_meta_header_v2_t *)new_meta_buf;
new_hdr->zero = 0;
new_hdr->magic = BLOCKSTORE_META_MAGIC_V1;
new_hdr->version = BLOCKSTORE_META_VERSION_V1;
new_hdr->version = meta["version"].uint64_value() == BLOCKSTORE_META_VERSION_V1
? BLOCKSTORE_META_VERSION_V1 : BLOCKSTORE_META_VERSION_V2;
new_hdr->meta_block_size = meta["meta_block_size"].uint64_value()
? meta["meta_block_size"].uint64_value() : 4096;
new_hdr->data_block_size = meta["data_block_size"].uint64_value()
? meta["data_block_size"].uint64_value() : 131072;
new_hdr->bitmap_granularity = meta["bitmap_granularity"].uint64_value()
? meta["bitmap_granularity"].uint64_value() : 4096;
new_clean_entry_bitmap_size = new_hdr->data_block_size / new_hdr->bitmap_granularity / 8;
new_clean_entry_size = sizeof(clean_disk_entry) + 2*new_clean_entry_bitmap_size;
new_hdr->data_csum_type = meta["data_csum_type"].is_number()
? meta["data_csum_type"].uint64_value()
: (meta["data_csum_type"].string_value() == "crc32c"
? BLOCKSTORE_CSUM_CRC32C
: BLOCKSTORE_CSUM_NONE);
new_hdr->csum_block_size = meta["csum_block_size"].uint64_value();
uint32_t new_clean_entry_header_size = (new_hdr->version == BLOCKSTORE_META_VERSION_V1
? sizeof(clean_disk_entry) : sizeof(clean_disk_entry) + 4 /*entry_csum*/);
new_clean_entry_bitmap_size = (new_hdr->data_block_size / new_hdr->bitmap_granularity + 7) / 8;
new_data_csum_size = (new_hdr->data_csum_type
? ((new_hdr->data_block_size+new_hdr->csum_block_size-1)/new_hdr->csum_block_size*(new_hdr->data_csum_type & 0xFF))
: 0);
new_clean_entry_size = new_clean_entry_header_size + 2*new_clean_entry_bitmap_size + new_data_csum_size;
new_entries_per_block = new_hdr->meta_block_size / new_clean_entry_size;
for (const auto & e: meta["entries"].array_items())
{
@ -194,8 +268,21 @@ int disk_tool_t::write_json_meta(json11::Json meta)
new_entry->oid.inode = (sscanf_json(NULL, e["pool"]) << (64-POOL_ID_BITS)) | sscanf_json(NULL, e["inode"]);
new_entry->oid.stripe = sscanf_json(NULL, e["stripe"]);
new_entry->version = sscanf_json(NULL, e["version"]);
fromhexstr(e["bitmap"].string_value(), new_clean_entry_bitmap_size, ((uint8_t*)new_entry) + sizeof(clean_disk_entry));
fromhexstr(e["ext_bitmap"].string_value(), new_clean_entry_bitmap_size, ((uint8_t*)new_entry) + sizeof(clean_disk_entry) + new_clean_entry_bitmap_size);
fromhexstr(e["bitmap"].string_value(), new_clean_entry_bitmap_size,
((uint8_t*)new_entry) + sizeof(clean_disk_entry));
fromhexstr(e["ext_bitmap"].string_value(), new_clean_entry_bitmap_size,
((uint8_t*)new_entry) + sizeof(clean_disk_entry) + new_clean_entry_bitmap_size);
if (new_hdr->version == BLOCKSTORE_META_VERSION_V2)
{
if (new_hdr->data_csum_type != 0)
{
fromhexstr(e["data_csum"].string_value(), new_data_csum_size,
((uint8_t*)new_entry) + sizeof(clean_disk_entry) + 2*new_clean_entry_bitmap_size);
}
uint32_t *new_entry_csum = (uint32_t*)(((uint8_t*)new_entry) + sizeof(clean_disk_entry) +
2*new_clean_entry_bitmap_size + new_data_csum_size);
*new_entry_csum = crc32c(0, new_entry, new_clean_entry_size - 4);
}
}
int r = resize_write_new_meta();
free(new_meta_buf);

View File

@ -29,7 +29,7 @@ int disk_tool_t::resize_data()
fprintf(stderr, "Reading metadata\n");
data_alloc = new allocator((new_data_len < dsk.data_len ? dsk.data_len : new_data_len) / dsk.data_block_size);
r = process_meta(
[this](blockstore_meta_header_v1_t *hdr)
[this](blockstore_meta_header_v2_t *hdr)
{
resize_init(hdr);
},
@ -139,7 +139,7 @@ int disk_tool_t::resize_parse_params()
return 0;
}
void disk_tool_t::resize_init(blockstore_meta_header_v1_t *hdr)
void disk_tool_t::resize_init(blockstore_meta_header_v2_t *hdr)
{
if (hdr && dsk.data_block_size != hdr->data_block_size)
{
@ -149,6 +149,15 @@ void disk_tool_t::resize_init(blockstore_meta_header_v1_t *hdr)
}
dsk.data_block_size = hdr->data_block_size;
}
if (hdr && (dsk.data_csum_type != hdr->data_csum_type || dsk.csum_block_size != hdr->csum_block_size))
{
if (dsk.data_csum_type)
{
fprintf(stderr, "Using data checksum type %s from metadata superblock\n", csum_type_str(hdr->data_csum_type).c_str());
}
dsk.data_csum_type = hdr->data_csum_type;
dsk.csum_block_size = hdr->csum_block_size;
}
if (((new_data_len-dsk.data_len) % dsk.data_block_size) ||
((new_data_offset-dsk.data_offset) % dsk.data_block_size))
{
@ -160,8 +169,12 @@ void disk_tool_t::resize_init(blockstore_meta_header_v1_t *hdr)
free_last = (new_data_offset+new_data_len < dsk.data_offset+dsk.data_len)
? (dsk.data_offset+dsk.data_len-new_data_offset-new_data_len) / dsk.data_block_size
: 0;
uint32_t new_clean_entry_header_size = sizeof(clean_disk_entry) + 4 /*entry_csum*/;
new_clean_entry_bitmap_size = dsk.data_block_size / (hdr ? hdr->bitmap_granularity : 4096) / 8;
new_clean_entry_size = sizeof(clean_disk_entry) + 2 * new_clean_entry_bitmap_size;
new_data_csum_size = (dsk.data_csum_type
? ((dsk.data_block_size+dsk.csum_block_size-1)/dsk.csum_block_size*(dsk.data_csum_type & 0xFF))
: 0);
new_clean_entry_size = new_clean_entry_header_size + 2*new_clean_entry_bitmap_size + new_data_csum_size;
new_entries_per_block = dsk.meta_block_size/new_clean_entry_size;
uint64_t new_meta_blocks = 1 + (new_data_len/dsk.data_block_size + new_entries_per_block-1) / new_entries_per_block;
if (!new_meta_len)
@ -349,13 +362,25 @@ int disk_tool_t::resize_rewrite_journal()
{
if (je->type == JE_START)
{
if (je_start.data_csum_type != dsk.data_csum_type ||
je_start.csum_block_size != dsk.csum_block_size)
{
fprintf(
stderr, "Error: journal header has different checksum parameters: %s/%u vs %s/%u\n",
csum_type_str(je_start.data_csum_type).c_str(), je_start.csum_block_size,
csum_type_str(dsk.data_csum_type).c_str(), dsk.csum_block_size
);
exit(1);
}
journal_entry *ne = (journal_entry*)(new_journal_ptr + new_journal_in_pos);
*((journal_entry_start*)ne) = (journal_entry_start){
.magic = JOURNAL_MAGIC,
.type = JE_START,
.size = sizeof(journal_entry_start),
.journal_start = dsk.journal_block_size,
.version = JOURNAL_VERSION,
.version = JOURNAL_VERSION_V2,
.data_csum_type = dsk.data_csum_type,
.csum_block_size = dsk.csum_block_size,
};
ne->crc32 = je_crc32(ne);
new_journal_ptr += dsk.journal_block_size;
@ -436,15 +461,17 @@ int disk_tool_t::resize_rewrite_meta()
new_meta_buf = (uint8_t*)memalign_or_die(MEM_ALIGNMENT, new_meta_len);
memset(new_meta_buf, 0, new_meta_len);
int r = process_meta(
[this](blockstore_meta_header_v1_t *hdr)
[this](blockstore_meta_header_v2_t *hdr)
{
blockstore_meta_header_v1_t *new_hdr = (blockstore_meta_header_v1_t *)new_meta_buf;
blockstore_meta_header_v2_t *new_hdr = (blockstore_meta_header_v2_t *)new_meta_buf;
new_hdr->zero = 0;
new_hdr->magic = BLOCKSTORE_META_MAGIC_V1;
new_hdr->version = BLOCKSTORE_META_VERSION_V1;
new_hdr->meta_block_size = dsk.meta_block_size;
new_hdr->data_block_size = dsk.data_block_size;
new_hdr->bitmap_granularity = dsk.bitmap_granularity ? dsk.bitmap_granularity : 4096;
new_hdr->data_csum_type = dsk.data_csum_type;
new_hdr->csum_block_size = dsk.csum_block_size;
},
[this](uint64_t block_num, clean_disk_entry *entry, uint8_t *bitmap)
{
@ -463,7 +490,7 @@ int disk_tool_t::resize_rewrite_meta()
new_entry->oid = entry->oid;
new_entry->version = entry->version;
if (bitmap)
memcpy(new_entry->bitmap, bitmap, 2*new_clean_entry_bitmap_size);
memcpy(new_entry->bitmap, bitmap, 2*new_clean_entry_bitmap_size + new_data_csum_size);
else
memset(new_entry->bitmap, 0xff, 2*new_clean_entry_bitmap_size);
}

View File

@ -373,3 +373,22 @@ int fix_partition_type(std::string dev_by_uuid)
std::string out;
return shell_exec({ "sfdisk", "--no-reread", "--force", "/dev/"+parent_dev }, script, &out, NULL);
}
std::string csum_type_str(uint32_t data_csum_type)
{
std::string csum_type;
if (data_csum_type == BLOCKSTORE_CSUM_NONE)
csum_type = "none";
else if (data_csum_type == BLOCKSTORE_CSUM_CRC32C)
csum_type = "crc32c";
else
csum_type = std::to_string(data_csum_type);
return csum_type;
}
uint32_t csum_type_from_str(std::string data_csum_type)
{
if (data_csum_type == "crc32c")
return BLOCKSTORE_CSUM_CRC32C;
return stoull_full(data_csum_type, 0);
}

View File

@ -27,7 +27,7 @@ fi
start_osd()
{
local i=$1
build/src/vitastor-osd --osd_num $i --bind_address 127.0.0.1 $NO_SAME $OSD_ARGS --etcd_address $ETCD_URL $(build/src/vitastor-disk simple-offsets --format options ./testdata/test_osd$i.bin 2>/dev/null) >>./testdata/osd$i.log 2>&1 &
build/src/vitastor-osd --osd_num $i --bind_address 127.0.0.1 $NO_SAME $OSD_ARGS --etcd_address $ETCD_URL $(build/src/vitastor-disk simple-offsets --format options ./testdata/test_osd$i.bin $OFFSET_ARGS 2>/dev/null) >>./testdata/osd$i.log 2>&1 &
eval OSD${i}_PID=$!
}

View File

@ -4,7 +4,7 @@
OSD_SIZE=1024
OSD_COUNT=5
OSD_ARGS=
OSD_ARGS="$OSD_ARGS"
for i in $(seq 1 $OSD_COUNT); do
dd if=/dev/zero of=./testdata/test_osd$i.bin bs=1024 count=1 seek=$((OSD_SIZE*1024-1))
build/src/vitastor-osd --osd_num $i --bind_address 127.0.0.1 $OSD_ARGS --etcd_address $ETCD_URL $(build/src/vitastor-disk simple-offsets --format options ./testdata/test_osd$i.bin 2>/dev/null) >>./testdata/osd$i.log 2>&1 &

View File

@ -8,7 +8,8 @@ etcdctl --endpoints=http://127.0.0.1:12379/v3 del --prefix /vitastor/pg/state
etcdctl --endpoints=http://127.0.0.1:12379/v3 del --prefix /vitastor/osd/state
OSD_COUNT=3
OSD_ARGS=
OSD_ARGS="$OSD_ARGS"
OFFSET_ARGS="$OFFSET_ARGS"
for i in $(seq 1 $OSD_COUNT); do
build/src/vitastor-osd --osd_num $i --bind_address 127.0.0.1 $OSD_ARGS --etcd_address $ETCD_URL $(build/src/vitastor-disk simple-offsets --format options ./testdata/test_osd$i.bin 2>/dev/null) >>./testdata/osd$i.log 2>&1 &
eval OSD${i}_PID=$!

View File

@ -1,7 +1,7 @@
#!/bin/bash -ex
# Test the `no_same_sector_overwrites` mode
OSD_ARGS="--journal_no_same_sector_overwrites true --journal_sector_buffer_count 1024 --disable_data_fsync 1 --immediate_commit all"
OSD_ARGS="--journal_no_same_sector_overwrites true --journal_sector_buffer_count 1024 --disable_data_fsync 1 --immediate_commit all $OSD_ARGS"
GLOBAL_CONF='{"immediate_commit":"all"}'
. `dirname $0`/run_3osds.sh