Fix build under clang and some warnings

Build problems fixed:
- void* pointer arithmetic which is a GNU extension (works as byte*)
- "variable size object may not be initialized" which is OK under GCC
- nullptr_t related error in json11 (it lacks 'operator <' in clang)

Warnings fixed:
- empty nested struct initializer { 0 } replaced by {}
- removed several unused lambda captures
non-odp-rdma
Vitaliy Filippov 2022-01-15 23:55:10 +03:00
parent 8f64fc61e7
commit 7bdd92ca4f
34 changed files with 209 additions and 200 deletions

@ -1 +1 @@
Subproject commit 5dc108754ad40d3b1d024f9bd7cca0595ef1a1db
Subproject commit 6e201464060ace53db809d65da7b0e2800673f8f

2
json11

@ -1 +1 @@
Subproject commit 55363fc2653b8802637a3d2e73a06839a72c585d
Subproject commit 52a3af664f40775426b189c85b6088d436d05df3

View File

@ -185,7 +185,7 @@ void journal_flusher_t::release_trim()
void journal_flusher_t::dump_diagnostics()
{
const char *unflushable_type = "";
obj_ver_id unflushable = { 0 };
obj_ver_id unflushable = {};
// Try to find out if there is a flushable object for information
for (object_id cur_oid: flush_queue)
{
@ -486,8 +486,8 @@ resume_1:
if (bs->clean_entry_bitmap_size)
{
new_clean_bitmap = (bs->inmemory_meta
? meta_new.buf + meta_new.pos*bs->clean_entry_size + sizeof(clean_disk_entry)
: bs->clean_bitmap + (clean_loc >> bs->block_order)*(2*bs->clean_entry_bitmap_size));
? (uint8_t*)meta_new.buf + meta_new.pos*bs->clean_entry_size + sizeof(clean_disk_entry)
: (uint8_t*)bs->clean_bitmap + (clean_loc >> bs->block_order)*(2*bs->clean_entry_bitmap_size));
if (clean_init_bitmap)
{
memset(new_clean_bitmap, 0, bs->clean_entry_bitmap_size);
@ -533,7 +533,7 @@ resume_1:
return false;
}
// zero out old metadata entry
memset(meta_old.buf + meta_old.pos*bs->clean_entry_size, 0, bs->clean_entry_size);
memset((uint8_t*)meta_old.buf + meta_old.pos*bs->clean_entry_size, 0, bs->clean_entry_size);
await_sqe(15);
data->iov = (struct iovec){ meta_old.buf, bs->meta_block_size };
data->callback = simple_callback_w;
@ -544,7 +544,7 @@ resume_1:
}
if (has_delete)
{
clean_disk_entry *new_entry = (clean_disk_entry*)(meta_new.buf + meta_new.pos*bs->clean_entry_size);
clean_disk_entry *new_entry = (clean_disk_entry*)((uint8_t*)meta_new.buf + meta_new.pos*bs->clean_entry_size);
if (new_entry->oid.inode != 0 && new_entry->oid != cur.oid)
{
printf("Fatal error (metadata corruption or bug): tried to delete metadata entry %lu (%lx:%lx v%lu) while deleting %lx:%lx\n",
@ -553,11 +553,11 @@ resume_1:
exit(1);
}
// zero out new metadata entry
memset(meta_new.buf + meta_new.pos*bs->clean_entry_size, 0, bs->clean_entry_size);
memset((uint8_t*)meta_new.buf + meta_new.pos*bs->clean_entry_size, 0, bs->clean_entry_size);
}
else
{
clean_disk_entry *new_entry = (clean_disk_entry*)(meta_new.buf + meta_new.pos*bs->clean_entry_size);
clean_disk_entry *new_entry = (clean_disk_entry*)((uint8_t*)meta_new.buf + meta_new.pos*bs->clean_entry_size);
if (new_entry->oid.inode != 0 && new_entry->oid != cur.oid)
{
printf("Fatal error (metadata corruption or bug): tried to overwrite non-zero metadata entry %lu (%lx:%lx v%lu) with %lx:%lx v%lu\n",
@ -575,7 +575,7 @@ resume_1:
if (bs->clean_entry_bitmap_size)
{
void *bmp_ptr = bs->clean_entry_bitmap_size > sizeof(void*) ? dirty_end->second.bitmap : &dirty_end->second.bitmap;
memcpy((void*)(new_entry+1) + bs->clean_entry_bitmap_size, bmp_ptr, bs->clean_entry_bitmap_size);
memcpy((uint8_t*)(new_entry+1) + bs->clean_entry_bitmap_size, bmp_ptr, bs->clean_entry_bitmap_size);
}
}
await_sqe(6);
@ -762,7 +762,7 @@ bool journal_flusher_co::scan_dirty(int wait_base)
if (bs->journal.inmemory)
{
// Take it from memory
memcpy(it->buf, bs->journal.buffer + submit_offset, submit_len);
memcpy(it->buf, (uint8_t*)bs->journal.buffer + submit_offset, submit_len);
}
else
{
@ -826,7 +826,7 @@ bool journal_flusher_co::modify_meta_read(uint64_t meta_loc, flusher_meta_write_
wr.pos = ((meta_loc >> bs->block_order) % (bs->meta_block_size / bs->clean_entry_size));
if (bs->inmemory_meta)
{
wr.buf = bs->metadata_buffer + wr.sector;
wr.buf = (uint8_t*)bs->metadata_buffer + wr.sector;
return true;
}
wr.it = flusher->meta_sectors.find(wr.sector);

View File

@ -372,7 +372,7 @@ void blockstore_impl_t::enqueue_op(blockstore_op_t *op)
};
}
unstable_writes.clear();
op->callback = [this, old_callback](blockstore_op_t *op)
op->callback = [old_callback](blockstore_op_t *op)
{
obj_ver_id *vers = (obj_ver_id*)op->buf;
delete[] vers;

View File

@ -280,7 +280,7 @@ class blockstore_impl_t
friend class blockstore_init_meta;
friend class blockstore_init_journal;
friend class blockstore_journal_check_t;
friend struct blockstore_journal_check_t;
friend class journal_flusher_t;
friend class journal_flusher_co;

View File

@ -148,7 +148,7 @@ resume_1:
{
GET_SQE();
data->iov = {
metadata_buffer + (bs->inmemory_meta
(uint8_t*)metadata_buffer + (bs->inmemory_meta
? metadata_read
: (prev == 1 ? bs->metadata_buf_size : 0)),
bs->meta_len - metadata_read > bs->metadata_buf_size ? bs->metadata_buf_size : bs->meta_len - metadata_read,
@ -169,13 +169,13 @@ resume_1:
if (prev_done)
{
void *done_buf = bs->inmemory_meta
? (metadata_buffer + done_pos)
: (metadata_buffer + (prev_done == 2 ? bs->metadata_buf_size : 0));
? ((uint8_t*)metadata_buffer + done_pos)
: ((uint8_t*)metadata_buffer + (prev_done == 2 ? bs->metadata_buf_size : 0));
unsigned count = bs->meta_block_size / bs->clean_entry_size;
for (int sector = 0; sector < done_len; sector += bs->meta_block_size)
{
// handle <count> entries
handle_entries(done_buf + sector, count, bs->block_order);
handle_entries((uint8_t*)done_buf + sector, count, bs->block_order);
done_cnt += count;
}
prev_done = 0;
@ -215,7 +215,7 @@ void blockstore_init_meta::handle_entries(void* entries, unsigned count, int blo
{
for (unsigned i = 0; i < count; i++)
{
clean_disk_entry *entry = (clean_disk_entry*)(entries + i*bs->clean_entry_size);
clean_disk_entry *entry = (clean_disk_entry*)((uint8_t*)entries + i*bs->clean_entry_size);
if (!bs->inmemory_meta && bs->clean_entry_bitmap_size)
{
memcpy(bs->clean_bitmap + (done_cnt+i)*2*bs->clean_entry_bitmap_size, &entry->bitmap, 2*bs->clean_entry_bitmap_size);
@ -440,7 +440,7 @@ resume_1:
if (!bs->journal.inmemory)
submitted_buf = memalign_or_die(MEM_ALIGNMENT, JOURNAL_BUFFER_SIZE);
else
submitted_buf = bs->journal.buffer + journal_pos;
submitted_buf = (uint8_t*)bs->journal.buffer + journal_pos;
data->iov = {
submitted_buf,
end - journal_pos < JOURNAL_BUFFER_SIZE ? end - journal_pos : JOURNAL_BUFFER_SIZE,
@ -570,7 +570,7 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
resume:
while (pos < bs->journal.block_size)
{
journal_entry *je = (journal_entry*)(buf + proc_pos - done_pos + pos);
journal_entry *je = (journal_entry*)((uint8_t*)buf + proc_pos - done_pos + pos);
if (je->magic != JOURNAL_MAGIC || je_crc32(je) != je->crc32 ||
je->type < JE_MIN || je->type > JE_MAX || started && je->crc32_prev != crc32_last)
{
@ -619,7 +619,7 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
if (location >= done_pos && location+je->small_write.len <= done_pos+len)
{
// data is within this buffer
data_crc32 = crc32c(0, buf + location - done_pos, je->small_write.len);
data_crc32 = crc32c(0, (uint8_t*)buf + location - done_pos, je->small_write.len);
}
else
{
@ -634,7 +634,7 @@ 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, done[i].buf + part_begin - done[i].pos, part_end - part_begin);
data_crc32 = crc32c(data_crc32, (uint8_t*)done[i].buf + part_begin - done[i].pos, part_end - part_begin);
}
}
if (covered < je->small_write.len)
@ -650,9 +650,9 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
// 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(buf + proc_pos - done_pos + pos, 0, bs->journal.block_size - pos);
memset((uint8_t*)buf + proc_pos - done_pos + pos, 0, bs->journal.block_size - pos);
bs->journal.next_free = prev_free;
init_write_buf = buf + proc_pos - done_pos;
init_write_buf = (uint8_t*)buf + proc_pos - done_pos;
init_write_sector = proc_pos;
return 0;
}
@ -665,7 +665,7 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
.version = je->small_write.version,
};
void *bmp = NULL;
void *bmp_from = (void*)je + sizeof(journal_entry_small_write);
void *bmp_from = (uint8_t*)je + sizeof(journal_entry_small_write);
if (bs->clean_entry_bitmap_size <= sizeof(void*))
{
memcpy(&bmp, bmp_from, bs->clean_entry_bitmap_size);
@ -745,7 +745,7 @@ int blockstore_init_journal::handle_journal_part(void *buf, uint64_t done_pos, u
.version = je->big_write.version,
};
void *bmp = NULL;
void *bmp_from = (void*)je + sizeof(journal_entry_big_write);
void *bmp_from = (uint8_t*)je + sizeof(journal_entry_big_write);
if (bs->clean_entry_bitmap_size <= sizeof(void*))
{
memcpy(&bmp, bmp_from, bs->clean_entry_bitmap_size);

View File

@ -6,7 +6,7 @@
class blockstore_init_meta
{
blockstore_impl_t *bs;
int wait_state = 0, wait_count = 0;
int wait_state = 0;
bool zero_on_init = false;
void *metadata_buffer = NULL;
uint64_t metadata_read = 0;

View File

@ -137,13 +137,13 @@ journal_entry* prefill_single_journal_entry(journal_t & journal, uint16_t type,
journal.in_sector_pos = 0;
journal.next_free = (journal.next_free+journal.block_size) < journal.len ? journal.next_free + journal.block_size : journal.block_size;
memset(journal.inmemory
? journal.buffer + journal.sector_info[journal.cur_sector].offset
: journal.sector_buf + journal.block_size*journal.cur_sector, 0, journal.block_size);
? (uint8_t*)journal.buffer + journal.sector_info[journal.cur_sector].offset
: (uint8_t*)journal.sector_buf + journal.block_size*journal.cur_sector, 0, journal.block_size);
}
journal_entry *je = (struct journal_entry*)(
(journal.inmemory
? journal.buffer + journal.sector_info[journal.cur_sector].offset
: journal.sector_buf + journal.block_size*journal.cur_sector) + journal.in_sector_pos
? (uint8_t*)journal.buffer + journal.sector_info[journal.cur_sector].offset
: (uint8_t*)journal.sector_buf + journal.block_size*journal.cur_sector) + journal.in_sector_pos
);
journal.in_sector_pos += size;
je->magic = JOURNAL_MAGIC;
@ -169,8 +169,8 @@ void blockstore_impl_t::prepare_journal_sector_write(int cur_sector, blockstore_
journal.sector_info[cur_sector].flush_count++;
data->iov = (struct iovec){
(journal.inmemory
? journal.buffer + journal.sector_info[cur_sector].offset
: journal.sector_buf + journal.block_size*cur_sector),
? (uint8_t*)journal.buffer + journal.sector_info[cur_sector].offset
: (uint8_t*)journal.sector_buf + journal.block_size*cur_sector),
journal.block_size
};
data->callback = [this, flush_id = journal.submit_id](ring_data_t *data) { handle_journal_write(data, flush_id); };

View File

@ -24,7 +24,7 @@ int blockstore_impl_t::fulfill_read_push(blockstore_op_t *op, void *buf, uint64_
}
if (journal.inmemory && IS_JOURNAL(item_state))
{
memcpy(buf, journal.buffer + offset, len);
memcpy(buf, (uint8_t*)journal.buffer + offset, len);
return 1;
}
BS_SUBMIT_GET_SQE(sqe, data);
@ -75,7 +75,7 @@ int blockstore_impl_t::fulfill_read(blockstore_op_t *read_op, uint64_t &fulfille
};
it = PRIV(read_op)->read_vec.insert(it, el);
if (!fulfill_read_push(read_op,
read_op->buf + el.offset - read_op->offset,
(uint8_t*)read_op->buf + el.offset - read_op->offset,
item_location + el.offset - item_start,
el.len, item_state, item_version))
{
@ -102,7 +102,7 @@ uint8_t* blockstore_impl_t::get_clean_entry_bitmap(uint64_t block_loc, int offse
{
uint64_t sector = (meta_loc / (meta_block_size / clean_entry_size)) * meta_block_size;
uint64_t pos = (meta_loc % (meta_block_size / clean_entry_size));
clean_entry_bitmap = (uint8_t*)(metadata_buffer + sector + pos*clean_entry_size + sizeof(clean_disk_entry) + offset);
clean_entry_bitmap = ((uint8_t*)metadata_buffer + sector + pos*clean_entry_size + sizeof(clean_disk_entry) + offset);
}
else
clean_entry_bitmap = (uint8_t*)(clean_bitmap + meta_loc*2*clean_entry_bitmap_size + offset);

View File

@ -102,7 +102,7 @@ bool blockstore_impl_t::enqueue_write(blockstore_op_t *op)
// Issue an additional sync so that the previous big write can reach the journal
blockstore_op_t *sync_op = new blockstore_op_t;
sync_op->opcode = BS_OP_SYNC;
sync_op->callback = [this, op](blockstore_op_t *sync_op)
sync_op->callback = [](blockstore_op_t *sync_op)
{
delete sync_op;
};
@ -380,7 +380,7 @@ int blockstore_impl_t::dequeue_write(blockstore_op_t *op)
if (journal.inmemory)
{
// Copy data
memcpy(journal.buffer + journal.next_free, op->buf, op->len);
memcpy((uint8_t*)journal.buffer + journal.next_free, op->buf, op->len);
}
BS_SUBMIT_GET_SQE(sqe2, data2);
data2->iov = (struct iovec){ op->buf, op->len };

View File

@ -412,7 +412,7 @@ struct snap_merger_t
uint64_t bitmap_size = target_block_size / gran;
while (rwo->end < bitmap_size)
{
auto bit = ((*(uint8_t*)(rwo->op.bitmap_buf + (rwo->end >> 3))) & (1 << (rwo->end & 0x7)));
auto bit = ((*((uint8_t*)rwo->op.bitmap_buf + (rwo->end >> 3))) & (1 << (rwo->end & 0x7)));
if (!bit)
{
if (rwo->end > rwo->start)
@ -459,7 +459,7 @@ struct snap_merger_t
subop->len = end-start;
subop->version = version;
subop->flags = OSD_OP_IGNORE_READONLY;
subop->iov.push_back(rwo->buf+start, end-start);
subop->iov.push_back((uint8_t*)rwo->buf+start, end-start);
subop->callback = [this, rwo](cluster_op_t *subop)
{
rwo->todo--;
@ -495,7 +495,7 @@ struct snap_merger_t
subop->offset = offset;
subop->len = 0;
subop->flags = OSD_OP_IGNORE_READONLY;
subop->callback = [this](cluster_op_t *subop)
subop->callback = [](cluster_op_t *subop)
{
if (subop->retval != 0)
{
@ -519,10 +519,10 @@ struct snap_merger_t
deleted_unsynced++;
if (deleted_unsynced >= fsync_interval)
{
uint64_t from = last_fsync_offset, to = last_written_offset;
uint64_t to = last_written_offset;
cluster_op_t *subop = new cluster_op_t;
subop->opcode = OSD_OP_SYNC;
subop->callback = [this, from, to](cluster_op_t *subop)
subop->callback = [this, to](cluster_op_t *subop)
{
delete subop;
// We can now delete source data between <from> and <to>

View File

@ -534,8 +534,8 @@ void cluster_client_t::copy_write(cluster_op_t *op, std::map<object_id, cluster_
unsigned iov_len = (op->iov.buf[iov_idx].iov_len - iov_pos);
if (iov_len <= cur_len)
{
memcpy(dirty_it->second.buf + pos - dirty_it->first.stripe,
op->iov.buf[iov_idx].iov_base + iov_pos, iov_len);
memcpy((uint8_t*)dirty_it->second.buf + pos - dirty_it->first.stripe,
(uint8_t*)op->iov.buf[iov_idx].iov_base + iov_pos, iov_len);
pos += iov_len;
len -= iov_len;
cur_len -= iov_len;
@ -544,8 +544,8 @@ void cluster_client_t::copy_write(cluster_op_t *op, std::map<object_id, cluster_
}
else
{
memcpy(dirty_it->second.buf + pos - dirty_it->first.stripe,
op->iov.buf[iov_idx].iov_base + iov_pos, cur_len);
memcpy((uint8_t*)dirty_it->second.buf + pos - dirty_it->first.stripe,
(uint8_t*)op->iov.buf[iov_idx].iov_base + iov_pos, cur_len);
pos += cur_len;
len -= cur_len;
iov_pos += cur_len;
@ -762,7 +762,7 @@ static void add_iov(int size, bool skip, cluster_op_t *op, int &iov_idx, size_t
{
if (!skip)
{
iov.push_back(op->iov.buf[iov_idx].iov_base + iov_pos, cur_left);
iov.push_back((uint8_t*)op->iov.buf[iov_idx].iov_base + iov_pos, cur_left);
}
left -= cur_left;
iov_pos = 0;
@ -772,7 +772,7 @@ static void add_iov(int size, bool skip, cluster_op_t *op, int &iov_idx, size_t
{
if (!skip)
{
iov.push_back(op->iov.buf[iov_idx].iov_base + iov_pos, left);
iov.push_back((uint8_t*)op->iov.buf[iov_idx].iov_base + iov_pos, left);
}
iov_pos += left;
left = 0;
@ -817,7 +817,7 @@ void cluster_client_t::slice_rw(cluster_op_t *op)
// First allocation
memset(op->bitmap_buf, 0, object_bitmap_size);
}
op->part_bitmaps = op->bitmap_buf + object_bitmap_size;
op->part_bitmaps = (uint8_t*)op->bitmap_buf + object_bitmap_size;
op->bitmap_buf_size = bitmap_mem;
}
}
@ -839,7 +839,7 @@ void cluster_client_t::slice_rw(cluster_op_t *op)
while (cur < end)
{
unsigned bmp_loc = (cur - op->offset)/bs_bitmap_granularity;
bool skip = (((*(uint8_t*)(op->bitmap_buf + bmp_loc/8)) >> (bmp_loc%8)) & 0x1);
bool skip = (((*((uint8_t*)op->bitmap_buf + bmp_loc/8)) >> (bmp_loc%8)) & 0x1);
if (skip_prev != skip)
{
if (cur > prev)
@ -944,7 +944,7 @@ bool cluster_client_t::try_send(cluster_op_t *op, int i)
.meta_revision = meta_rev,
.version = op->opcode == OSD_OP_WRITE || op->opcode == OSD_OP_DELETE ? op->version : 0,
} },
.bitmap = (op->opcode == OSD_OP_READ || op->opcode == OSD_OP_READ_BITMAP ? op->part_bitmaps + pg_bitmap_size*i : NULL),
.bitmap = (op->opcode == OSD_OP_READ || op->opcode == OSD_OP_READ_BITMAP ? (uint8_t*)op->part_bitmaps + pg_bitmap_size*i : NULL),
.bitmap_len = (unsigned)(op->opcode == OSD_OP_READ || op->opcode == OSD_OP_READ_BITMAP ? pg_bitmap_size : 0),
.callback = [this, part](osd_op_t *op_part)
{
@ -1155,7 +1155,7 @@ void cluster_client_t::copy_part_bitmap(cluster_op_t *op, cluster_op_part_t *par
if (!(object_offset & 0x7) && !(part_offset & 0x7) && (part_len >= 8))
{
// Copy bytes
mem_or(op->bitmap_buf + object_offset/8, part->op.bitmap + part_offset/8, part_len/8);
mem_or((uint8_t*)op->bitmap_buf + object_offset/8, (uint8_t*)part->op.bitmap + part_offset/8, part_len/8);
object_offset += (part_len & ~0x7);
part_offset += (part_len & ~0x7);
part_len = (part_len & 0x7);
@ -1163,8 +1163,8 @@ void cluster_client_t::copy_part_bitmap(cluster_op_t *op, cluster_op_part_t *par
while (part_len > 0)
{
// Copy bits
(*(uint8_t*)(op->bitmap_buf + (object_offset >> 3))) |= (
(((*(uint8_t*)(part->op.bitmap + (part_offset >> 3))) >> (part_offset & 0x7)) & 0x1) << (object_offset & 0x7)
(*((uint8_t*)op->bitmap_buf + (object_offset >> 3))) |= (
(((*((uint8_t*)part->op.bitmap + (part_offset >> 3))) >> (part_offset & 0x7)) & 0x1) << (object_offset & 0x7)
);
part_offset++;
object_offset++;

View File

@ -75,7 +75,7 @@ int main(int argc, char *argv[])
uint64_t s;
for (s = 0; s < self.journal_block; s += 8)
{
if (*((uint64_t*)(data+s)) != 0)
if (*((uint64_t*)((uint8_t*)data+s)) != 0)
break;
}
if (s == self.journal_block)
@ -139,7 +139,7 @@ int journal_dump_t::dump_block(void *buf)
bool wrapped = false;
while (pos < journal_block)
{
journal_entry *je = (journal_entry*)(buf + pos);
journal_entry *je = (journal_entry*)((uint8_t*)buf + pos);
if (je->magic != JOURNAL_MAGIC || je->type < JE_MIN || je->type > JE_MAX ||
!all && started && je->crc32_prev != crc32_last)
{

View File

@ -26,9 +26,8 @@
#include "blockstore.h"
#include "epoll_manager.h"
#include "fio_headers.h"
#include "json11/json11.hpp"
#include "fio_headers.h"
struct bs_data
{
@ -150,7 +149,6 @@ static int bs_init(struct thread_data *td)
static enum fio_q_status bs_queue(struct thread_data *td, struct io_u *io)
{
bs_data *bsd = (bs_data*)td->io_ops_data;
int n = bsd->op_n;
if (io->ddir == DDIR_SYNC && bsd->last_sync)
{
return FIO_Q_COMPLETED;
@ -178,7 +176,7 @@ static enum fio_q_status bs_queue(struct thread_data *td, struct io_u *io)
op->version = UINT64_MAX; // last unstable
op->offset = io->offset % bsd->bs->get_block_size();
op->len = io->xfer_buflen;
op->callback = [io, n](blockstore_op_t *op)
op->callback = [io](blockstore_op_t *op)
{
io->error = op->retval < 0 ? -op->retval : 0;
bs_data *bsd = (bs_data*)io->engine_data;
@ -200,7 +198,7 @@ static enum fio_q_status bs_queue(struct thread_data *td, struct io_u *io)
op->version = 0; // assign automatically
op->offset = io->offset % bsd->bs->get_block_size();
op->len = io->xfer_buflen;
op->callback = [io, n](blockstore_op_t *op)
op->callback = [io](blockstore_op_t *op)
{
io->error = op->retval < 0 ? -op->retval : 0;
bs_data *bsd = (bs_data*)io->engine_data;
@ -215,7 +213,7 @@ static enum fio_q_status bs_queue(struct thread_data *td, struct io_u *io)
break;
case DDIR_SYNC:
op->opcode = BS_OP_SYNC_STAB_ALL;
op->callback = [io, n](blockstore_op_t *op)
op->callback = [io](blockstore_op_t *op)
{
bs_data *bsd = (bs_data*)io->engine_data;
io->error = op->retval < 0 ? -op->retval : 0;

View File

@ -628,13 +628,13 @@ static std::string ws_format_frame(int type, uint64_t size)
res[p++] = size | /*mask*/0x80;
else if (size < 65536)
{
res[p++] = 126 | /*mask*/0x80;
res[p++] = (char)(126 | /*mask*/0x80);
res[p++] = (size >> 8) & 0xFF;
res[p++] = (size >> 0) & 0xFF;
}
else
{
res[p++] = 127 | /*mask*/0x80;
res[p++] = (char)(127 | /*mask*/0x80);
res[p++] = (size >> 56) & 0xFF;
res[p++] = (size >> 48) & 0xFF;
res[p++] = (size >> 40) & 0xFF;

View File

@ -544,7 +544,7 @@ json11::Json osd_messenger_t::read_config(const json11::Json & config)
int done = 0;
while (done < st.st_size)
{
int r = read(fd, (void*)buf.data()+done, st.st_size-done);
int r = read(fd, (uint8_t*)buf.data()+done, st.st_size-done);
if (r < 0)
{
fprintf(stderr, "Error reading %s: %s\n", config_path, strerror(errno));

View File

@ -141,7 +141,7 @@ struct osd_op_buf_list_t
else
{
iov.iov_len -= result;
iov.iov_base += result;
iov.iov_base = (uint8_t*)iov.iov_base + result;
break;
}
}

View File

@ -389,7 +389,7 @@ bool osd_messenger_t::try_send_rdma(osd_client_t *cl)
uint32_t len = (uint32_t)(op_size+iov.iov_len-rc->send_buf_pos < rc->max_msg
? iov.iov_len-rc->send_buf_pos : rc->max_msg-op_size);
sge[op_sge++] = {
.addr = (uintptr_t)(iov.iov_base+rc->send_buf_pos),
.addr = (uintptr_t)((uint8_t*)iov.iov_base+rc->send_buf_pos),
.length = len,
.lkey = rc->ctx->mr->lkey,
};
@ -519,7 +519,7 @@ void osd_messenger_t::handle_rdma_events()
}
if (cl->rdma_conn->send_buf_pos > 0)
{
cl->send_list[0].iov_base += cl->rdma_conn->send_buf_pos;
cl->send_list[0].iov_base = (uint8_t*)cl->send_list[0].iov_base + cl->rdma_conn->send_buf_pos;
cl->send_list[0].iov_len -= cl->rdma_conn->send_buf_pos;
cl->rdma_conn->send_buf_pos = 0;
}

View File

@ -142,13 +142,13 @@ bool osd_messenger_t::handle_read_buffer(osd_client_t *cl, void *curbuf, int rem
memcpy(cur->iov_base, curbuf, remain);
cl->read_remaining -= remain;
cur->iov_len -= remain;
cur->iov_base += remain;
cur->iov_base = (uint8_t*)cur->iov_base + remain;
remain = 0;
}
else
{
memcpy(cur->iov_base, curbuf, cur->iov_len);
curbuf += cur->iov_len;
curbuf = (uint8_t*)curbuf + cur->iov_len;
cl->read_remaining -= cur->iov_len;
remain -= cur->iov_len;
cur->iov_len = 0;
@ -390,7 +390,7 @@ void osd_messenger_t::handle_reply_ready(osd_op_t *op)
(tv_end.tv_sec - op->tv_begin.tv_sec)*1000000 +
(tv_end.tv_nsec - op->tv_begin.tv_nsec)/1000
);
set_immediate.push_back([this, op]()
set_immediate.push_back([op]()
{
// Copy lambda to be unaffected by `delete op`
std::function<void(osd_op_t*)>(op->callback)(op);

View File

@ -250,7 +250,7 @@ void osd_messenger_t::handle_send(int result, osd_client_t *cl)
else
{
iov.iov_len -= result;
iov.iov_base += result;
iov.iov_base = (uint8_t*)iov.iov_base + result;
break;
}
}

View File

@ -278,7 +278,7 @@ public:
stop = false;
cluster_op_t *close_sync = new cluster_op_t;
close_sync->opcode = OSD_OP_SYNC;
close_sync->callback = [this, &stop](cluster_op_t *op)
close_sync->callback = [&stop](cluster_op_t *op)
{
stop = true;
delete op;
@ -553,7 +553,7 @@ protected:
}
else
{
send_list[to_eat].iov_base += result;
send_list[to_eat].iov_base = (uint8_t*)send_list[to_eat].iov_base + result;
send_list[to_eat].iov_len -= result;
break;
}
@ -627,8 +627,8 @@ protected:
memcpy(cur_buf, b, inc);
cur_left -= inc;
result -= inc;
cur_buf += inc;
b += inc;
cur_buf = (uint8_t*)cur_buf + inc;
b = (uint8_t*)b + inc;
}
else
{
@ -667,7 +667,7 @@ protected:
op->offset = be64toh(cur_req.from);
op->len = be32toh(cur_req.len);
buf = malloc_or_die(sizeof(nbd_reply) + op->len);
op->iov.push_back(buf + sizeof(nbd_reply), op->len);
op->iov.push_back((uint8_t*)buf + sizeof(nbd_reply), op->len);
}
else if (req_type == NBD_CMD_FLUSH)
{
@ -695,7 +695,7 @@ protected:
if (req_type == NBD_CMD_WRITE)
{
cur_op = op;
cur_buf = buf + sizeof(nbd_reply);
cur_buf = (uint8_t*)buf + sizeof(nbd_reply);
cur_left = op->len;
read_state = CL_READ_DATA;
}

View File

@ -331,8 +331,8 @@ void osd_t::exec_op(osd_op_t *cur_op)
void osd_t::reset_stats()
{
msgr.stats = { 0 };
prev_stats = { 0 };
msgr.stats = {};
prev_stats = {};
memset(recovery_stat_count, 0, sizeof(recovery_stat_count));
memset(recovery_stat_bytes, 0, sizeof(recovery_stat_bytes));
}
@ -447,7 +447,7 @@ void osd_t::print_slow()
{
for (uint64_t i = 0; i < op->req.sec_stab.len; i += sizeof(obj_ver_id))
{
obj_ver_id *ov = (obj_ver_id*)(op->buf + i);
obj_ver_id *ov = (obj_ver_id*)((uint8_t*)op->buf + i);
bufprintf(i == 0 ? " %lx:%lx v%lu" : ", %lx:%lx v%lu", ov->oid.inode, ov->oid.stripe, ov->version);
}
}

View File

@ -166,8 +166,8 @@ class osd_t
osd_op_stats_t prev_stats;
std::map<uint64_t, inode_stats_t> inode_stats;
const char* recovery_stat_names[2] = { "degraded", "misplaced" };
uint64_t recovery_stat_count[2][2] = { 0 };
uint64_t recovery_stat_bytes[2][2] = { 0 };
uint64_t recovery_stat_count[2][2] = {};
uint64_t recovery_stat_bytes[2][2] = {};
// cluster connection
void parse_config(const json11::Json & config);

View File

@ -96,11 +96,11 @@ bool osd_t::prepare_primary_rw(osd_op_t *cur_op)
(pool_cfg.scheme == POOL_SCHEME_REPLICATED ? 0 : pg_it->second.pg_size)
)
);
void *data_buf = ((void*)op_data) + sizeof(osd_primary_op_data_t);
void *data_buf = (uint8_t*)op_data + sizeof(osd_primary_op_data_t);
op_data->pg_num = pg_num;
op_data->oid = oid;
op_data->stripes = (osd_rmw_stripe_t*)data_buf;
data_buf += sizeof(osd_rmw_stripe_t) * stripe_count;
data_buf = (uint8_t*)data_buf + sizeof(osd_rmw_stripe_t) * stripe_count;
op_data->scheme = pool_cfg.scheme;
op_data->pg_data_size = pg_data_size;
op_data->pg_size = pg_it->second.pg_size;
@ -110,17 +110,17 @@ bool osd_t::prepare_primary_rw(osd_op_t *cur_op)
for (int i = 0; i < stripe_count; i++)
{
op_data->stripes[i].bmp_buf = data_buf;
data_buf += clean_entry_bitmap_size;
data_buf = (uint8_t*)data_buf + clean_entry_bitmap_size;
}
op_data->chain_size = chain_size;
if (chain_size > 0)
{
op_data->read_chain = (inode_t*)data_buf;
data_buf += sizeof(inode_t) * chain_size;
data_buf = (uint8_t*)data_buf + sizeof(inode_t) * chain_size;
op_data->snapshot_bitmaps = data_buf;
data_buf += chain_size * stripe_count * clean_entry_bitmap_size;
data_buf = (uint8_t*)data_buf + chain_size * stripe_count * clean_entry_bitmap_size;
op_data->missing_flags = (uint8_t*)data_buf;
data_buf += chain_size * (pool_cfg.scheme == POOL_SCHEME_REPLICATED ? 0 : pg_it->second.pg_size);
data_buf = (uint8_t*)data_buf + chain_size * (pool_cfg.scheme == POOL_SCHEME_REPLICATED ? 0 : pg_it->second.pg_size);
// Copy chain
int chain_num = 0;
op_data->read_chain[chain_num++] = cur_op->req.rw.inode;
@ -248,7 +248,7 @@ resume_2:
{
// Send buffer in parts to avoid copying
cur_op->iov.push_back(
stripes[role].read_buf + (stripes[role].req_start - stripes[role].read_start),
(uint8_t*)stripes[role].read_buf + (stripes[role].req_start - stripes[role].read_start),
stripes[role].req_end - stripes[role].req_start
);
}

View File

@ -66,7 +66,7 @@ int osd_t::read_bitmaps(osd_op_t *cur_op, pg_t & pg, int base_state)
auto read_version = (vo_it != pg.ver_override.end() ? vo_it->second : UINT64_MAX);
// Read bitmap synchronously from the local database
bs->read_bitmap(
cur_oid, read_version, op_data->snapshot_bitmaps + chain_num*clean_entry_bitmap_size,
cur_oid, read_version, (uint8_t*)op_data->snapshot_bitmaps + chain_num*clean_entry_bitmap_size,
!chain_num ? &cur_op->reply.rw.version : NULL
);
}
@ -96,12 +96,15 @@ resume_1:
{
if (op_data->missing_flags[chain_num*pg.pg_size + i])
{
osd_rmw_stripe_t local_stripes[pg.pg_size] = { 0 };
osd_rmw_stripe_t local_stripes[pg.pg_size];
for (i = 0; i < pg.pg_size; i++)
{
local_stripes[i].missing = op_data->missing_flags[chain_num*pg.pg_size + i] && true;
local_stripes[i].bmp_buf = op_data->snapshot_bitmaps + (chain_num*pg.pg_size + i)*clean_entry_bitmap_size;
local_stripes[i].read_start = local_stripes[i].read_end = 1;
local_stripes[i] = {
.bmp_buf = (uint8_t*)op_data->snapshot_bitmaps + (chain_num*pg.pg_size + i)*clean_entry_bitmap_size,
.read_start = 1,
.read_end = 1,
.missing = op_data->missing_flags[chain_num*pg.pg_size + i] && true,
};
}
if (pg.scheme == POOL_SCHEME_XOR)
{
@ -146,7 +149,7 @@ int osd_t::collect_bitmap_requests(osd_op_t *cur_op, pg_t & pg, std::vector<bitm
.osd_num = read_target,
.oid = cur_oid,
.version = target_version,
.bmp_buf = op_data->snapshot_bitmaps + chain_num*clean_entry_bitmap_size,
.bmp_buf = (uint8_t*)op_data->snapshot_bitmaps + chain_num*clean_entry_bitmap_size,
});
}
else
@ -185,7 +188,7 @@ int osd_t::collect_bitmap_requests(osd_op_t *cur_op, pg_t & pg, std::vector<bitm
.stripe = cur_oid.stripe | i,
},
.version = target_version,
.bmp_buf = op_data->snapshot_bitmaps + (chain_num*pg.pg_size + i)*clean_entry_bitmap_size,
.bmp_buf = (uint8_t*)op_data->snapshot_bitmaps + (chain_num*pg.pg_size + i)*clean_entry_bitmap_size,
});
found++;
}
@ -266,15 +269,15 @@ int osd_t::submit_bitmap_subops(osd_op_t *cur_op, pg_t & pg)
int requested_count = subop->req.sec_read_bmp.len / sizeof(obj_ver_id);
if (subop->reply.hdr.retval == requested_count * (8 + clean_entry_bitmap_size))
{
void *cur_buf = subop->buf + 8;
void *cur_buf = (uint8_t*)subop->buf + 8;
for (int j = prev; j <= i; j++)
{
memcpy((*bitmap_requests)[j].bmp_buf, cur_buf, clean_entry_bitmap_size);
if ((*bitmap_requests)[j].oid.inode == cur_op->req.rw.inode)
{
memcpy(&cur_op->reply.rw.version, cur_buf-8, 8);
memcpy(&cur_op->reply.rw.version, (uint8_t*)cur_buf-8, 8);
}
cur_buf += 8 + clean_entry_bitmap_size;
cur_buf = (uint8_t*)cur_buf + 8 + clean_entry_bitmap_size;
}
}
if ((cur_op->op_data->errors + cur_op->op_data->done + 1) >= cur_op->op_data->n_subops)
@ -363,7 +366,7 @@ int osd_t::submit_chained_read_requests(pg_t & pg, osd_op_t *cur_op)
+ sizeof(osd_rmw_stripe_t) * stripe_count * op_data->chain_size
);
osd_rmw_stripe_t *chain_stripes = (osd_rmw_stripe_t*)(
((void*)op_data->chain_reads) + sizeof(osd_chain_read_t) * op_data->chain_read_count
(uint8_t*)op_data->chain_reads + sizeof(osd_chain_read_t) * op_data->chain_read_count
);
// Now process each subrequest as a separate read, including reconstruction if needed
// Prepare reads
@ -425,8 +428,8 @@ int osd_t::submit_chained_read_requests(pg_t & pg, osd_op_t *cur_op)
if (stripes[role].read_end > 0)
{
stripes[role].read_buf = cur_buf;
stripes[role].bmp_buf = op_data->snapshot_bitmaps + (chain_reads[cri].chain_pos*stripe_count + role)*clean_entry_bitmap_size;
cur_buf += stripes[role].read_end - stripes[role].read_start;
stripes[role].bmp_buf = (uint8_t*)op_data->snapshot_bitmaps + (chain_reads[cri].chain_pos*stripe_count + role)*clean_entry_bitmap_size;
cur_buf = (uint8_t*)cur_buf + stripes[role].read_end - stripes[role].read_start;
}
}
}
@ -474,7 +477,7 @@ void osd_t::send_chained_read_results(pg_t & pg, osd_op_t *cur_op)
osd_primary_op_data_t *op_data = cur_op->op_data;
int stripe_count = (pg.scheme == POOL_SCHEME_REPLICATED ? 1 : pg.pg_size);
osd_rmw_stripe_t *chain_stripes = (osd_rmw_stripe_t*)(
((void*)op_data->chain_reads) + sizeof(osd_chain_read_t) * op_data->chain_read_count
(uint8_t*)op_data->chain_reads + sizeof(osd_chain_read_t) * op_data->chain_read_count
);
// Reconstruct parts if needed
if (op_data->degraded)
@ -544,7 +547,7 @@ void osd_t::send_chained_read_results(pg_t & pg, osd_op_t *cur_op)
role_end = bs_block_size;
assert(stripes[role].read_buf);
cur_op->iov.push_back(
stripes[role].read_buf + (role_start - stripes[role].read_start),
(uint8_t*)stripes[role].read_buf + (role_start - stripes[role].read_start),
role_end - role_start
);
sent += role_end - role_start;

View File

@ -86,7 +86,7 @@ resume_2:
sizeof(obj_ver_osd_t)*this->copies_to_delete_after_sync_count
);
op_data->dirty_pgs = (pool_pg_num_t*)dirty_buf;
op_data->dirty_osds = (osd_num_t*)(dirty_buf + sizeof(pool_pg_num_t)*dirty_pgs.size());
op_data->dirty_osds = (osd_num_t*)((uint8_t*)dirty_buf + sizeof(pool_pg_num_t)*dirty_pgs.size());
op_data->dirty_pg_count = dirty_pgs.size();
op_data->dirty_osd_count = dirty_osds.size();
if (this->copies_to_delete_after_sync_count)

View File

@ -113,7 +113,7 @@ resume_3:
op_data->stripes[0].write_end != bs_block_size))
{
memcpy(
op_data->stripes[0].read_buf + op_data->stripes[0].req_start,
(uint8_t*)op_data->stripes[0].read_buf + op_data->stripes[0].req_start,
op_data->stripes[0].write_buf,
op_data->stripes[0].req_end - op_data->stripes[0].req_start
);

View File

@ -103,8 +103,8 @@ void reconstruct_stripes_xor(osd_rmw_stripe_t *stripes, int pg_size, uint32_t bi
assert(stripes[role].read_start >= stripes[prev].read_start &&
stripes[role].read_start >= stripes[other].read_start);
memxor(
stripes[prev].read_buf + (stripes[role].read_start - stripes[prev].read_start),
stripes[other].read_buf + (stripes[role].read_start - stripes[other].read_start),
(uint8_t*)stripes[prev].read_buf + (stripes[role].read_start - stripes[prev].read_start),
(uint8_t*)stripes[other].read_buf + (stripes[role].read_start - stripes[other].read_start),
stripes[role].read_buf, stripes[role].read_end - stripes[role].read_start
);
memxor(stripes[prev].bmp_buf, stripes[other].bmp_buf, stripes[role].bmp_buf, bitmap_size);
@ -115,7 +115,7 @@ void reconstruct_stripes_xor(osd_rmw_stripe_t *stripes, int pg_size, uint32_t bi
assert(stripes[role].read_start >= stripes[other].read_start);
memxor(
stripes[role].read_buf,
stripes[other].read_buf + (stripes[role].read_start - stripes[other].read_start),
(uint8_t*)stripes[other].read_buf + (stripes[role].read_start - stripes[other].read_start),
stripes[role].read_buf, stripes[role].read_end - stripes[role].read_start
);
memxor(stripes[role].bmp_buf, stripes[other].bmp_buf, stripes[role].bmp_buf, bitmap_size);
@ -202,10 +202,9 @@ reed_sol_matrix_t* get_jerasure_matrix(int pg_size, int pg_minsize)
int* get_jerasure_decoding_matrix(osd_rmw_stripe_t *stripes, int pg_size, int pg_minsize)
{
int edd = 0;
int erased[pg_size] = { 0 };
int erased[pg_size];
for (int i = 0; i < pg_size; i++)
if (stripes[i].read_end == 0 || stripes[i].missing)
erased[i] = 1;
erased[i] = (stripes[i].read_end == 0 || stripes[i].missing ? 1 : 0);
for (int i = 0; i < pg_minsize; i++)
if (stripes[i].read_end != 0 && stripes[i].missing)
edd++;
@ -241,7 +240,9 @@ void reconstruct_stripes_jerasure(osd_rmw_stripe_t *stripes, int pg_size, int pg
return;
}
int *decoding_matrix = dm_ids + pg_minsize;
char *data_ptrs[pg_size] = { 0 };
char *data_ptrs[pg_size];
for (int role = 0; role < pg_size; role++)
data_ptrs[role] = NULL;
for (int role = 0; role < pg_minsize; role++)
{
if (stripes[role].read_end != 0 && stripes[role].missing)
@ -254,7 +255,7 @@ void reconstruct_stripes_jerasure(osd_rmw_stripe_t *stripes, int pg_size, int pg
{
assert(stripes[other].read_start <= stripes[role].read_start);
assert(stripes[other].read_end >= stripes[role].read_end);
data_ptrs[other] = (char*)(stripes[other].read_buf + (stripes[role].read_start - stripes[other].read_start));
data_ptrs[other] = (char*)stripes[other].read_buf + (stripes[role].read_start - stripes[other].read_start);
}
}
data_ptrs[role] = (char*)stripes[role].read_buf;
@ -330,7 +331,7 @@ void* alloc_read_buffer(osd_rmw_stripe_t *stripes, int read_pg_size, uint64_t ad
{
if (stripes[role].read_end != 0)
{
stripes[role].read_buf = buf + buf_pos;
stripes[role].read_buf = (uint8_t*)buf + buf_pos;
buf_pos += stripes[role].read_end - stripes[role].read_start;
}
}
@ -446,12 +447,12 @@ void* calc_rmw(void *request_buf, osd_rmw_stripe_t *stripes, uint64_t *read_osd_
{
if (stripes[role].req_end != 0)
{
stripes[role].write_buf = request_buf + in_pos;
stripes[role].write_buf = (uint8_t*)request_buf + in_pos;
in_pos += stripes[role].req_end - stripes[role].req_start;
}
else if (role >= pg_minsize && write_osd_set[role] != 0 && end != 0)
{
stripes[role].write_buf = rmw_buf + buf_pos;
stripes[role].write_buf = (uint8_t*)rmw_buf + buf_pos;
buf_pos += end - start;
}
}
@ -476,13 +477,13 @@ static void get_old_new_buffers(osd_rmw_stripe_t & stripe, uint32_t wr_start, ui
if (ne && (!oe || ns <= os))
{
// NEW or NEW->OLD
bufs[nbufs++] = { .buf = stripe.write_buf + ns - stripe.req_start, .len = ne-ns };
bufs[nbufs++] = { .buf = (uint8_t*)stripe.write_buf + ns - stripe.req_start, .len = ne-ns };
if (os < ne)
os = ne;
if (oe > os)
{
// NEW->OLD
bufs[nbufs++] = { .buf = stripe.read_buf + os - stripe.read_start, .len = oe-os };
bufs[nbufs++] = { .buf = (uint8_t*)stripe.read_buf + os - stripe.read_start, .len = oe-os };
}
}
else if (oe)
@ -491,18 +492,18 @@ static void get_old_new_buffers(osd_rmw_stripe_t & stripe, uint32_t wr_start, ui
if (ne)
{
// OLD->NEW or OLD->NEW->OLD
bufs[nbufs++] = { .buf = stripe.read_buf + os - stripe.read_start, .len = ns-os };
bufs[nbufs++] = { .buf = stripe.write_buf + ns - stripe.req_start, .len = ne-ns };
bufs[nbufs++] = { .buf = (uint8_t*)stripe.read_buf + os - stripe.read_start, .len = ns-os };
bufs[nbufs++] = { .buf = (uint8_t*)stripe.write_buf + ns - stripe.req_start, .len = ne-ns };
if (oe > ne)
{
// OLD->NEW->OLD
bufs[nbufs++] = { .buf = stripe.read_buf + ne - stripe.read_start, .len = oe-ne };
bufs[nbufs++] = { .buf = (uint8_t*)stripe.read_buf + ne - stripe.read_start, .len = oe-ne };
}
}
else
{
// OLD
bufs[nbufs++] = { .buf = stripe.read_buf + os - stripe.read_start, .len = oe-os };
bufs[nbufs++] = { .buf = (uint8_t*)stripe.read_buf + os - stripe.read_start, .len = oe-os };
}
}
}
@ -517,7 +518,7 @@ static void xor_multiple_buffers(buf_len_t *xor1, int n1, buf_len_t *xor2, int n
{
// We know for sure that ranges overlap
uint32_t end = std::min(end1, end2);
memxor(xor1[i1].buf + pos-start1, xor2[i2].buf + pos-start2, dest+pos, end-pos);
memxor((uint8_t*)xor1[i1].buf + pos-start1, (uint8_t*)xor2[i2].buf + pos-start2, (uint8_t*)dest+pos, end-pos);
pos = end;
if (pos >= end1)
{
@ -586,7 +587,7 @@ static void calc_rmw_parity_copy_mod(osd_rmw_stripe_t *stripes, int pg_size, int
{
// Copy modified chunk into the read buffer to write it back
memcpy(
stripes[role].read_buf + stripes[role].req_start,
(uint8_t*)stripes[role].read_buf + stripes[role].req_start,
stripes[role].write_buf,
stripes[role].req_end - stripes[role].req_start
);
@ -609,7 +610,7 @@ static void calc_rmw_parity_copy_parity(osd_rmw_stripe_t *stripes, int pg_size,
{
// Copy new parity into the read buffer to write it back
memcpy(
stripes[role].read_buf + start,
(uint8_t*)stripes[role].read_buf + start,
stripes[role].write_buf,
end - start
);
@ -698,9 +699,15 @@ void calc_rmw_parity_jerasure(osd_rmw_stripe_t *stripes, int pg_size, int pg_min
{
// Calculate new coding chunks
buf_len_t bufs[pg_size][3];
int nbuf[pg_size] = { 0 }, curbuf[pg_size] = { 0 };
int nbuf[pg_size], curbuf[pg_size];
uint32_t positions[pg_size];
void *data_ptrs[pg_size] = { 0 };
void *data_ptrs[pg_size];
for (int i = 0; i < pg_size; i++)
{
data_ptrs[i] = NULL;
nbuf[i] = 0;
curbuf[i] = 0;
}
for (int i = 0; i < pg_minsize; i++)
{
get_old_new_buffers(stripes[i], start, end, bufs[i], nbuf[i]);
@ -719,7 +726,7 @@ void calc_rmw_parity_jerasure(osd_rmw_stripe_t *stripes, int pg_size, int pg_min
{
assert(curbuf[i] < nbuf[i]);
assert(bufs[i][curbuf[i]].buf);
data_ptrs[i] = bufs[i][curbuf[i]].buf + pos-positions[i];
data_ptrs[i] = (uint8_t*)bufs[i][curbuf[i]].buf + pos-positions[i];
uint32_t this_end = bufs[i][curbuf[i]].len + positions[i];
if (next_end > this_end)
next_end = this_end;

View File

@ -90,7 +90,7 @@ void dump_stripes(osd_rmw_stripe_t *stripes, int pg_size)
void test1()
{
osd_num_t osd_set[3] = { 1, 0, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 1.1
split_stripes(2, 128*1024, 128*1024-4096, 8192, stripes);
assert(stripes[0].req_start == 128*1024-4096 && stripes[0].req_end == 128*1024);
@ -129,7 +129,7 @@ void test4()
const uint32_t bmp = 4;
unsigned bitmaps[3] = { 0 };
osd_num_t osd_set[3] = { 1, 0, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 4.1
split_stripes(2, 128*1024, 128*1024-4096, 8192, stripes);
for (int i = 0; i < 3; i++)
@ -142,11 +142,11 @@ void test4()
assert(stripes[0].write_start == 128*1024-4096 && stripes[0].write_end == 128*1024);
assert(stripes[1].write_start == 0 && stripes[1].write_end == 4096);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].read_buf == rmw_buf+128*1024);
assert(stripes[1].read_buf == rmw_buf+128*1024*2);
assert(stripes[2].read_buf == rmw_buf+128*1024*3-4096);
assert(stripes[0].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+128*1024*2);
assert(stripes[2].read_buf == (uint8_t*)rmw_buf+128*1024*3-4096);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+4096);
assert(stripes[1].write_buf == (uint8_t*)write_buf+4096);
assert(stripes[2].write_buf == rmw_buf);
// Test 4.2
set_pattern(write_buf, 8192, PATTERN0);
@ -183,7 +183,7 @@ void test4()
void test5()
{
osd_num_t osd_set[3] = { 1, 0, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 5.1
split_stripes(2, 128*1024, 0, 64*1024*3, stripes);
assert(stripes[0].req_start == 0 && stripes[0].req_end == 128*1024);
@ -198,11 +198,11 @@ void test5()
assert(stripes[0].write_start == 0 && stripes[0].write_end == 128*1024);
assert(stripes[1].write_start == 0 && stripes[1].write_end == 64*1024);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].read_buf == rmw_buf+128*1024);
assert(stripes[1].read_buf == rmw_buf+64*3*1024);
assert(stripes[2].read_buf == rmw_buf+64*4*1024);
assert(stripes[0].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+64*3*1024);
assert(stripes[2].read_buf == (uint8_t*)rmw_buf+64*4*1024);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+128*1024);
assert(stripes[1].write_buf == (uint8_t*)write_buf+128*1024);
assert(stripes[2].write_buf == rmw_buf);
free(rmw_buf);
free(write_buf);
@ -224,7 +224,7 @@ void test5()
void test6()
{
osd_num_t osd_set[3] = { 1, 2, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 6.1
split_stripes(2, 128*1024, 0, 64*1024*3, stripes);
void *write_buf = malloc(64*1024*3);
@ -236,10 +236,10 @@ void test6()
assert(stripes[1].write_start == 0 && stripes[1].write_end == 64*1024);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].read_buf == 0);
assert(stripes[1].read_buf == rmw_buf+128*1024);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[2].read_buf == 0);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+128*1024);
assert(stripes[1].write_buf == (uint8_t*)write_buf+128*1024);
assert(stripes[2].write_buf == rmw_buf);
free(rmw_buf);
free(write_buf);
@ -267,7 +267,7 @@ void test7()
{
osd_num_t osd_set[3] = { 1, 0, 3 };
osd_num_t write_osd_set[3] = { 1, 2, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 7.1
split_stripes(2, 128*1024, 128*1024-4096, 8192, stripes);
void *write_buf = malloc(8192);
@ -278,11 +278,11 @@ void test7()
assert(stripes[0].write_start == 128*1024-4096 && stripes[0].write_end == 128*1024);
assert(stripes[1].write_start == 0 && stripes[1].write_end == 4096);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].read_buf == rmw_buf+128*1024);
assert(stripes[1].read_buf == rmw_buf+128*1024*2);
assert(stripes[2].read_buf == rmw_buf+128*1024*3);
assert(stripes[0].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+128*1024*2);
assert(stripes[2].read_buf == (uint8_t*)rmw_buf+128*1024*3);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+4096);
assert(stripes[1].write_buf == (uint8_t*)write_buf+4096);
assert(stripes[2].write_buf == rmw_buf);
// Test 7.2
set_pattern(write_buf, 8192, PATTERN0);
@ -320,7 +320,7 @@ void test8()
{
osd_num_t osd_set[3] = { 0, 2, 3 };
osd_num_t write_osd_set[3] = { 1, 2, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 8.1
split_stripes(2, 128*1024, 0, 128*1024+4096, stripes);
void *write_buf = malloc(128*1024+4096);
@ -332,10 +332,10 @@ void test8()
assert(stripes[1].write_start == 0 && stripes[1].write_end == 4096);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].read_buf == NULL);
assert(stripes[1].read_buf == rmw_buf+128*1024);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[2].read_buf == NULL);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+128*1024);
assert(stripes[1].write_buf == (uint8_t*)write_buf+128*1024);
assert(stripes[2].write_buf == rmw_buf);
// Test 8.2
set_pattern(write_buf, 128*1024+4096, PATTERN0);
@ -345,7 +345,7 @@ void test8()
assert(stripes[1].write_start == 0 && stripes[1].write_end == 4096); // recheck again
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024); // recheck again
assert(stripes[0].write_buf == write_buf); // recheck again
assert(stripes[1].write_buf == write_buf+128*1024); // recheck again
assert(stripes[1].write_buf == (uint8_t*)write_buf+128*1024); // recheck again
assert(stripes[2].write_buf == rmw_buf); // recheck again
check_pattern(stripes[2].write_buf, 4096, 0); // new parity
check_pattern(stripes[2].write_buf+4096, 128*1024-4096, PATTERN0^PATTERN1); // new parity
@ -375,7 +375,7 @@ void test9()
{
osd_num_t osd_set[3] = { 0, 2, 3 };
osd_num_t write_osd_set[3] = { 1, 2, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 9.0
split_stripes(2, 128*1024, 64*1024, 0, stripes);
assert(stripes[0].req_start == 0 && stripes[0].req_end == 0);
@ -391,8 +391,8 @@ void test9()
assert(stripes[1].write_start == 0 && stripes[1].write_end == 0);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 0);
assert(stripes[0].read_buf == rmw_buf);
assert(stripes[1].read_buf == rmw_buf+128*1024);
assert(stripes[2].read_buf == rmw_buf+128*1024*2);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[2].read_buf == (uint8_t*)rmw_buf+128*1024*2);
assert(stripes[0].write_buf == NULL);
assert(stripes[1].write_buf == NULL);
assert(stripes[2].write_buf == NULL);
@ -430,7 +430,7 @@ void test10()
{
osd_num_t osd_set[3] = { 1, 0, 0 };
osd_num_t write_osd_set[3] = { 1, 2, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 10.0
split_stripes(2, 128*1024, 0, 256*1024, stripes);
assert(stripes[0].req_start == 0 && stripes[0].req_end == 128*1024);
@ -450,7 +450,7 @@ void test10()
assert(stripes[1].read_buf == NULL);
assert(stripes[2].read_buf == NULL);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+128*1024);
assert(stripes[1].write_buf == (uint8_t*)write_buf+128*1024);
assert(stripes[2].write_buf == rmw_buf);
// Test 10.2
set_pattern(stripes[0].write_buf, 128*1024, PATTERN1);
@ -460,7 +460,7 @@ void test10()
assert(stripes[1].write_start == 0 && stripes[1].write_end == 128*1024);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+128*1024);
assert(stripes[1].write_buf == (uint8_t*)write_buf+128*1024);
assert(stripes[2].write_buf == rmw_buf);
check_pattern(stripes[2].write_buf, 128*1024, PATTERN1^PATTERN2);
free(rmw_buf);
@ -486,7 +486,7 @@ void test11()
{
osd_num_t osd_set[3] = { 1, 0, 0 };
osd_num_t write_osd_set[3] = { 1, 2, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 11.0
split_stripes(2, 128*1024, 128*1024, 256*1024, stripes);
assert(stripes[0].req_start == 0 && stripes[0].req_end == 0);
@ -502,7 +502,7 @@ void test11()
assert(stripes[0].write_start == 0 && stripes[0].write_end == 0);
assert(stripes[1].write_start == 0 && stripes[1].write_end == 128*1024);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].read_buf == rmw_buf+128*1024);
assert(stripes[0].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[1].read_buf == NULL);
assert(stripes[2].read_buf == NULL);
assert(stripes[0].write_buf == NULL);
@ -542,7 +542,7 @@ void test12()
{
osd_num_t osd_set[3] = { 1, 2, 0 };
osd_num_t write_osd_set[3] = { 1, 2, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
// Test 12.0
split_stripes(2, 128*1024, 0, 0, stripes);
assert(stripes[0].req_start == 0 && stripes[0].req_end == 0);
@ -557,8 +557,8 @@ void test12()
assert(stripes[0].write_start == 0 && stripes[0].write_end == 0);
assert(stripes[1].write_start == 0 && stripes[1].write_end == 0);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].read_buf == rmw_buf+128*1024);
assert(stripes[1].read_buf == rmw_buf+2*128*1024);
assert(stripes[0].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+2*128*1024);
assert(stripes[2].read_buf == NULL);
assert(stripes[0].write_buf == NULL);
assert(stripes[1].write_buf == NULL);
@ -597,7 +597,7 @@ void test13()
use_jerasure(4, 2, true);
osd_num_t osd_set[4] = { 1, 2, 0, 0 };
osd_num_t write_osd_set[4] = { 1, 2, 3, 4 };
osd_rmw_stripe_t stripes[4] = { 0 };
osd_rmw_stripe_t stripes[4] = {};
// Test 13.0
void *write_buf = malloc_or_die(8192);
split_stripes(2, 128*1024, 128*1024-4096, 8192, stripes);
@ -616,14 +616,14 @@ void test13()
assert(stripes[1].write_start == 0 && stripes[1].write_end == 4096);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[3].write_start == 0 && stripes[3].write_end == 128*1024);
assert(stripes[0].read_buf == rmw_buf+2*128*1024);
assert(stripes[1].read_buf == rmw_buf+3*128*1024-4096);
assert(stripes[0].read_buf == (uint8_t*)rmw_buf+2*128*1024);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+3*128*1024-4096);
assert(stripes[2].read_buf == NULL);
assert(stripes[3].read_buf == NULL);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+4096);
assert(stripes[1].write_buf == (uint8_t*)write_buf+4096);
assert(stripes[2].write_buf == rmw_buf);
assert(stripes[3].write_buf == rmw_buf+128*1024);
assert(stripes[3].write_buf == (uint8_t*)rmw_buf+128*1024);
// Test 13.2 - encode
set_pattern(write_buf, 8192, PATTERN3);
set_pattern(stripes[0].read_buf, 128*1024-4096, PATTERN1);
@ -634,9 +634,9 @@ void test13()
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[3].write_start == 0 && stripes[3].write_end == 128*1024);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+4096);
assert(stripes[1].write_buf == (uint8_t*)write_buf+4096);
assert(stripes[2].write_buf == rmw_buf);
assert(stripes[3].write_buf == rmw_buf+128*1024);
assert(stripes[3].write_buf == (uint8_t*)rmw_buf+128*1024);
// Test 13.3 - full decode and verify
osd_num_t read_osd_set[4] = { 0, 0, 3, 4 };
memset(stripes, 0, sizeof(stripes));
@ -658,11 +658,11 @@ void test13()
void *read_buf = alloc_read_buffer(stripes, 4, 0);
assert(read_buf);
assert(stripes[0].read_buf == read_buf);
assert(stripes[1].read_buf == read_buf+128*1024);
assert(stripes[2].read_buf == read_buf+2*128*1024);
assert(stripes[3].read_buf == read_buf+3*128*1024);
memcpy(read_buf+2*128*1024, rmw_buf, 128*1024);
memcpy(read_buf+3*128*1024, rmw_buf+128*1024, 128*1024);
assert(stripes[1].read_buf == (uint8_t*)read_buf+128*1024);
assert(stripes[2].read_buf == (uint8_t*)read_buf+2*128*1024);
assert(stripes[3].read_buf == (uint8_t*)read_buf+3*128*1024);
memcpy((uint8_t*)read_buf+2*128*1024, rmw_buf, 128*1024);
memcpy((uint8_t*)read_buf+3*128*1024, (uint8_t*)rmw_buf+128*1024, 128*1024);
reconstruct_stripes_jerasure(stripes, 4, 2, 0);
check_pattern(stripes[0].read_buf, 128*1024-4096, PATTERN1);
check_pattern(stripes[0].read_buf+128*1024-4096, 4096, PATTERN3);
@ -690,10 +690,10 @@ void test13()
assert(read_buf);
assert(stripes[0].read_buf == read_buf);
assert(stripes[1].read_buf == NULL);
assert(stripes[2].read_buf == read_buf+128*1024);
assert(stripes[3].read_buf == read_buf+2*128*1024);
memcpy(read_buf+128*1024, rmw_buf, 128*1024);
memcpy(read_buf+2*128*1024, rmw_buf+128*1024, 128*1024);
assert(stripes[2].read_buf == (uint8_t*)read_buf+128*1024);
assert(stripes[3].read_buf == (uint8_t*)read_buf+2*128*1024);
memcpy((uint8_t*)read_buf+128*1024, rmw_buf, 128*1024);
memcpy((uint8_t*)read_buf+2*128*1024, (uint8_t*)rmw_buf+128*1024, 128*1024);
reconstruct_stripes_jerasure(stripes, 4, 2, 0);
check_pattern(stripes[0].read_buf, 128*1024-4096, PATTERN1);
check_pattern(stripes[0].read_buf+128*1024-4096, 4096, PATTERN3);
@ -725,7 +725,7 @@ void test14()
use_jerasure(3, 2, true);
osd_num_t osd_set[3] = { 1, 2, 0 };
osd_num_t write_osd_set[3] = { 1, 2, 3 };
osd_rmw_stripe_t stripes[3] = { 0 };
osd_rmw_stripe_t stripes[3] = {};
unsigned bitmaps[3] = { 0 };
// Test 13.0
void *write_buf = malloc_or_die(8192);
@ -744,11 +744,11 @@ void test14()
assert(stripes[0].write_start == 128*1024-4096 && stripes[0].write_end == 128*1024);
assert(stripes[1].write_start == 0 && stripes[1].write_end == 4096);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].read_buf == rmw_buf+128*1024);
assert(stripes[1].read_buf == rmw_buf+2*128*1024-4096);
assert(stripes[0].read_buf == (uint8_t*)rmw_buf+128*1024);
assert(stripes[1].read_buf == (uint8_t*)rmw_buf+2*128*1024-4096);
assert(stripes[2].read_buf == NULL);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+4096);
assert(stripes[1].write_buf == (uint8_t*)write_buf+4096);
assert(stripes[2].write_buf == rmw_buf);
// Test 13.2 - encode
set_pattern(write_buf, 8192, PATTERN3);
@ -765,7 +765,7 @@ void test14()
assert(stripes[1].write_start == 0 && stripes[1].write_end == 4096);
assert(stripes[2].write_start == 0 && stripes[2].write_end == 128*1024);
assert(stripes[0].write_buf == write_buf);
assert(stripes[1].write_buf == write_buf+4096);
assert(stripes[1].write_buf == (uint8_t*)write_buf+4096);
assert(stripes[2].write_buf == rmw_buf);
// Test 13.3 - decode and verify
osd_num_t read_osd_set[4] = { 0, 2, 3 };
@ -788,8 +788,8 @@ void test14()
stripes[i].bmp_buf = bitmaps+i;
assert(read_buf);
assert(stripes[0].read_buf == read_buf);
assert(stripes[1].read_buf == read_buf+128*1024);
assert(stripes[2].read_buf == read_buf+2*128*1024);
assert(stripes[1].read_buf == (uint8_t*)read_buf+128*1024);
assert(stripes[2].read_buf == (uint8_t*)read_buf+2*128*1024);
set_pattern(stripes[1].read_buf, 4096, PATTERN3);
set_pattern(stripes[1].read_buf+4096, 128*1024-4096, PATTERN2);
memcpy(stripes[2].read_buf, rmw_buf, 128*1024);

View File

@ -54,8 +54,8 @@ void osd_t::exec_secondary(osd_op_t *cur_op)
void *cur_buf = reply_buf;
for (int i = 0; i < n; i++)
{
bs->read_bitmap(ov[i].oid, ov[i].version, cur_buf + sizeof(uint64_t), (uint64_t*)cur_buf);
cur_buf += (8 + clean_entry_bitmap_size);
bs->read_bitmap(ov[i].oid, ov[i].version, (uint8_t*)cur_buf + sizeof(uint64_t), (uint64_t*)cur_buf);
cur_buf = (uint8_t*)cur_buf + (8 + clean_entry_bitmap_size);
}
free(cur_op->buf);
cur_op->buf = reply_buf;

View File

@ -3,6 +3,7 @@
#include <errno.h>
#include <stdlib.h>
#include <stdint.h>
#include <stdio.h>
#include <sys/types.h>
#include <sys/socket.h>
@ -30,7 +31,7 @@ int read_blocking(int fd, void *read_buf, size_t remaining)
continue;
}
done += r;
read_buf += r;
read_buf = (uint8_t*)read_buf + r;
}
return done;
}
@ -51,7 +52,7 @@ int write_blocking(int fd, void *write_buf, size_t remaining)
continue;
}
done += r;
write_buf += r;
write_buf = (uint8_t*)write_buf + r;
}
return done;
}
@ -78,7 +79,7 @@ int readv_blocking(int fd, iovec *iov, int iovcnt)
if (iov[v].iov_len > r)
{
iov[v].iov_len -= r;
iov[v].iov_base += r;
iov[v].iov_base = (uint8_t*)iov[v].iov_base + r;
break;
}
else
@ -113,7 +114,7 @@ int writev_blocking(int fd, iovec *iov, int iovcnt)
if (iov[v].iov_len > r)
{
iov[v].iov_len -= r;
iov[v].iov_base += r;
iov[v].iov_base = (uint8_t*)iov[v].iov_base + r;
break;
}
else
@ -151,7 +152,7 @@ int sendv_blocking(int fd, iovec *iov, int iovcnt, int flags)
if (iov[v].iov_len > r)
{
iov[v].iov_len -= r;
iov[v].iov_base += r;
iov[v].iov_base = (uint8_t*)iov[v].iov_base + r;
break;
}
else

View File

@ -112,7 +112,7 @@ int bind_stub(std::string bind_address, int bind_port)
void run_stub(int peer_fd)
{
osd_any_op_t op;
osd_any_reply_t reply = { 0 };
osd_any_reply_t reply = {};
void *buf = NULL;
while (1)
{

View File

@ -11,5 +11,5 @@
#define PATTERN2 0xffe3bad5f578a78e
#define PATTERN3 0x426bd7854eb08509
#define set_pattern(buf, len, pattern) for (uint64_t i = 0; i < len; i += 8) { *(uint64_t*)((void*)buf + i) = pattern; }
#define check_pattern(buf, len, pattern) { uint64_t bad = UINT64_MAX; for (uint64_t i = 0; i < len; i += 8) { if ((*(uint64_t*)(buf + i)) != (pattern)) { bad = i; break; } } if (bad != UINT64_MAX) { printf("mismatch at %lx\n", bad); } assert(bad == UINT64_MAX); }
#define set_pattern(buf, len, pattern) for (uint64_t i = 0; i < len; i += 8) { *(uint64_t*)((uint8_t*)buf + i) = pattern; }
#define check_pattern(buf, len, pattern) { uint64_t bad = UINT64_MAX; for (uint64_t i = 0; i < len; i += 8) { if ((*(uint64_t*)((uint8_t*)buf + i)) != (pattern)) { bad = i; break; } } if (bad != UINT64_MAX) { printf("mismatch at %lx\n", bad); } assert(bad == UINT64_MAX); }

View File

@ -94,7 +94,7 @@ again:
if (!timers.size())
{
nearest = -1;
itimerspec exp = { 0 };
itimerspec exp = {};
if (timerfd_settime(timerfd, 0, &exp, NULL))
{
throw std::runtime_error(std::string("timerfd_settime: ") + strerror(errno));