1
0
mirror of git://sourceware.org/git/lvm2.git synced 2025-09-28 09:44:18 +03:00

Compare commits

...

4 Commits

Author SHA1 Message Date
Joe Thornber
510a6e86a3 Merge branch '2019-09-05-add-io-manager' of git+ssh://sourceware.org/git/lvm2 into 2019-09-05-add-io-manager 2019-09-09 18:21:20 +01:00
Joe Thornber
e4693f7f8d [io-manager] Reopen without O_DIRECT if we need to write a partial block.
Still need to do a bit more testing.

Also need to think more about how we guarantee that we've reopened
the same device.  Pass in a device_id_extractor object to push the
issue up a layer?
2019-09-09 18:18:46 +01:00
David Teigland
13c702f862 io-manager: clean up block sizes 2019-09-06 14:40:39 -05:00
Joe Thornber
4b25dd7bc2 [io-manager] Drop in new io-manager files
io-manager is an updated version of the bcache.

This commit contains:

- io-manager

- io-processor: a utility that runs an action across regions of disk.
  using the io-manager to prefetch in correctly sized batches.

- unit tests for the above (~100 total)

- remove unit tests for bcache.

- There's a name clash between bcache/io_engine and io_manager/io_engine.
  So I've renamed the bcache one to io_engine_ for now.  bcache will disappear
  shortly.
2019-09-06 10:22:02 +01:00
16 changed files with 5620 additions and 1214 deletions

View File

@@ -36,6 +36,9 @@ SOURCES =\
device/dev-luks.c \
device/dev-dasd.c \
device/dev-lvm1-pool.c \
device/io-manager.c \
device/io-manager-utils.c \
device/io-processor.c \
display/display.c \
error/errseg.c \
unknown/unknown.c \

View File

@@ -129,18 +129,18 @@ static struct control_block *_iocb_to_cb(struct iocb *icb)
//----------------------------------------------------------------
struct async_engine {
struct io_engine e;
struct io_engine_ e;
io_context_t aio_context;
struct cb_set *cbs;
unsigned page_mask;
};
static struct async_engine *_to_async(struct io_engine *e)
static struct async_engine *_to_async(struct io_engine_ *e)
{
return container_of(e, struct async_engine, e);
}
static void _async_destroy(struct io_engine *ioe)
static void _async_destroy(struct io_engine_ *ioe)
{
int r;
struct async_engine *e = _to_async(ioe);
@@ -159,7 +159,7 @@ static int _last_byte_fd;
static uint64_t _last_byte_offset;
static int _last_byte_sector_size;
static bool _async_issue(struct io_engine *ioe, enum dir d, int fd,
static bool _async_issue(struct io_engine_ *ioe, enum dir d, int fd,
sector_t sb, sector_t se, void *data, void *context)
{
int r;
@@ -310,7 +310,7 @@ static bool _async_issue(struct io_engine *ioe, enum dir d, int fd,
#define MAX_IO 256
#define MAX_EVENT 64
static bool _async_wait(struct io_engine *ioe, io_complete_fn fn)
static bool _async_wait(struct io_engine_ *ioe, io_complete_fn fn)
{
int i, r;
struct io_event event[MAX_EVENT];
@@ -353,12 +353,12 @@ static bool _async_wait(struct io_engine *ioe, io_complete_fn fn)
return true;
}
static unsigned _async_max_io(struct io_engine *e)
static unsigned _async_max_io(struct io_engine_ *e)
{
return MAX_IO;
}
struct io_engine *create_async_io_engine(void)
struct io_engine_ *create_async_io_engine_(void)
{
int r;
struct async_engine *e = malloc(sizeof(*e));
@@ -399,22 +399,22 @@ struct sync_io {
};
struct sync_engine {
struct io_engine e;
struct io_engine_ e;
struct dm_list complete;
};
static struct sync_engine *_to_sync(struct io_engine *e)
static struct sync_engine *_to_sync(struct io_engine_ *e)
{
return container_of(e, struct sync_engine, e);
}
static void _sync_destroy(struct io_engine *ioe)
static void _sync_destroy(struct io_engine_ *ioe)
{
struct sync_engine *e = _to_sync(ioe);
free(e);
}
static bool _sync_issue(struct io_engine *ioe, enum dir d, int fd,
static bool _sync_issue(struct io_engine_ *ioe, enum dir d, int fd,
sector_t sb, sector_t se, void *data, void *context)
{
int rv;
@@ -570,7 +570,7 @@ static bool _sync_issue(struct io_engine *ioe, enum dir d, int fd,
return true;
}
static bool _sync_wait(struct io_engine *ioe, io_complete_fn fn)
static bool _sync_wait(struct io_engine_ *ioe, io_complete_fn fn)
{
struct sync_io *io, *tmp;
struct sync_engine *e = _to_sync(ioe);
@@ -584,12 +584,12 @@ static bool _sync_wait(struct io_engine *ioe, io_complete_fn fn)
return true;
}
static unsigned _sync_max_io(struct io_engine *e)
static unsigned _sync_max_io(struct io_engine_ *e)
{
return 1;
}
struct io_engine *create_sync_io_engine(void)
struct io_engine_ *create_sync_io_engine_(void)
{
struct sync_engine *e = malloc(sizeof(*e));
@@ -653,7 +653,7 @@ struct bcache {
uint64_t nr_cache_blocks;
unsigned max_io;
struct io_engine *engine;
struct io_engine_ *engine;
void *raw_data;
struct block *raw_blocks;
@@ -1081,7 +1081,7 @@ static void _preemptive_writeback(struct bcache *cache)
* Public interface
*--------------------------------------------------------------*/
struct bcache *bcache_create(sector_t block_sectors, unsigned nr_cache_blocks,
struct io_engine *engine)
struct io_engine_ *engine)
{
struct bcache *cache;
unsigned max_io = engine->max_io(engine);

View File

@@ -39,16 +39,16 @@ typedef uint64_t sector_t;
typedef void io_complete_fn(void *context, int io_error);
struct io_engine {
void (*destroy)(struct io_engine *e);
bool (*issue)(struct io_engine *e, enum dir d, int fd,
struct io_engine_ {
void (*destroy)(struct io_engine_ *e);
bool (*issue)(struct io_engine_ *e, enum dir d, int fd,
sector_t sb, sector_t se, void *data, void *context);
bool (*wait)(struct io_engine *e, io_complete_fn fn);
unsigned (*max_io)(struct io_engine *e);
bool (*wait)(struct io_engine_ *e, io_complete_fn fn);
unsigned (*max_io)(struct io_engine_ *e);
};
struct io_engine *create_async_io_engine(void);
struct io_engine *create_sync_io_engine(void);
struct io_engine_ *create_async_io_engine_(void);
struct io_engine_ *create_sync_io_engine_(void);
/*----------------------------------------------------------------*/
@@ -72,7 +72,7 @@ struct block {
* Ownership of engine passes. Engine will be destroyed even if this fails.
*/
struct bcache *bcache_create(sector_t block_size, unsigned nr_cache_blocks,
struct io_engine *engine);
struct io_engine_ *engine);
void bcache_destroy(struct bcache *cache);
enum bcache_get_flags {

View File

@@ -0,0 +1,290 @@
/*
* Copyright (C) 2018 Red Hat, Inc. All rights reserved.
*
* This file is part of LVM2.
*
* This copyrighted material is made available to anyone wishing to use,
* modify, copy, or redistribute it subject to the terms and conditions
* of the GNU Lesser General Public License v.2.1.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
#include "lib/device/io-manager.h"
// FIXME: need to define this in a common place (that doesn't pull in deps)
#ifndef SECTOR_SHIFT
#define SECTOR_SHIFT 9
#endif
//----------------------------------------------------------------
static void byte_range_to_block_range(struct io_manager *iom, uint64_t start, size_t len,
block_address *bb, block_address *be)
{
block_address block_size = io_block_sectors(iom) << SECTOR_SHIFT;
*bb = start / block_size;
*be = (start + len + block_size - 1) / block_size;
}
static uint64_t _min(uint64_t lhs, uint64_t rhs)
{
if (rhs < lhs)
return rhs;
return lhs;
}
//----------------------------------------------------------------
void io_prefetch_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len)
{
block_address bb, be;
byte_range_to_block_range(iom, start, len, &bb, &be);
while (bb < be) {
io_prefetch_block(iom, dev, bb);
bb++;
}
}
//----------------------------------------------------------------
bool io_read_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len, void *data)
{
struct block *b;
block_address bb, be;
uint64_t block_size = io_block_sectors(iom) << SECTOR_SHIFT;
uint64_t block_offset = start % block_size;
io_prefetch_bytes(iom, dev, start, len);
byte_range_to_block_range(iom, start, len, &bb, &be);
for (; bb != be; bb++) {
if (!io_get_block(iom, dev, bb, 0, &b))
return false;
size_t blen = _min(block_size - block_offset, len);
memcpy(data, ((unsigned char *) b->data) + block_offset, blen);
io_put_block(b);
block_offset = 0;
len -= blen;
data = ((unsigned char *) data) + blen;
}
return true;
}
//----------------------------------------------------------------
// Writing bytes and zeroing bytes are very similar, so we factor out
// this common code.
struct updater;
typedef bool (*partial_update_fn)(struct updater *u, struct io_dev *dev, block_address bb, uint64_t offset, size_t len);
typedef bool (*whole_update_fn)(struct updater *u, struct io_dev *dev, block_address bb, block_address be);
struct updater {
struct io_manager *iom;
partial_update_fn partial_fn;
whole_update_fn whole_fn;
void *data;
};
static bool _update_bytes(struct updater *u, struct io_dev *dev, uint64_t start, size_t len)
{
struct io_manager *iom = u->iom;
block_address bb, be;
uint64_t block_size = io_block_sectors(iom) << SECTOR_SHIFT;
uint64_t block_offset = start % block_size;
uint64_t nr_whole;
byte_range_to_block_range(iom, start, len, &bb, &be);
// If the last block is partial, we will require a read, so let's
// prefetch it.
if ((start + len) % block_size)
io_prefetch_block(iom, dev, (start + len) / block_size);
// First block may be partial
if (block_offset) {
size_t blen = _min(block_size - block_offset, len);
if (!u->partial_fn(u, dev, bb, block_offset, blen))
return false;
len -= blen;
if (!len)
return true;
bb++;
}
// Now we write out a set of whole blocks
nr_whole = len / block_size;
if (!u->whole_fn(u, dev, bb, bb + nr_whole))
return false;
bb += nr_whole;
len -= nr_whole * block_size;
if (!len)
return true;
// Finally we write a partial end block
return u->partial_fn(u, dev, bb, 0, len);
}
// Return a mask with a bit set for each sector touched by the region.
// To be used with io_get_block_mask().
static uint64_t _region_to_mask(uint64_t offset, size_t len)
{
unsigned i;
uint64_t r = 0;
unsigned sb = offset >> SECTOR_SHIFT;
unsigned se = (offset + len + ((1 << SECTOR_SHIFT) - 1)) >> SECTOR_SHIFT;
for (i = sb; i < se; i++)
r |= 1ull << i;
return r;
}
//----------------------------------------------------------------
static bool _write_partial(struct updater *u, struct io_dev *dev, block_address bb,
uint64_t offset, size_t len)
{
struct block *b;
if (!io_get_block_mask(u->iom, dev, bb, GF_DIRTY,
_region_to_mask(offset, len), &b))
return false;
memcpy(((unsigned char *) b->data) + offset, u->data, len);
u->data = ((unsigned char *) u->data) + len;
io_put_block(b);
return true;
}
static bool _write_whole(struct updater *u, struct io_dev *dev, block_address bb, block_address be)
{
struct block *b;
uint64_t block_size = io_block_sectors(u->iom) << SECTOR_SHIFT;
for (; bb != be; bb++) {
// We don't need to read the block since we are overwriting
// it completely.
if (!io_get_block(u->iom, dev, bb, GF_ZERO, &b))
return false;
memcpy(b->data, u->data, block_size);
u->data = ((unsigned char *) u->data) + block_size;
io_put_block(b);
}
return true;
}
bool io_write_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len, void *data)
{
struct updater u;
u.iom = iom;
u.partial_fn = _write_partial;
u.whole_fn = _write_whole;
u.data = data;
return _update_bytes(&u, dev, start, len);
}
//----------------------------------------------------------------
static bool _zero_partial(struct updater *u, struct io_dev *dev, block_address bb, uint64_t offset, size_t len)
{
struct block *b;
if (!io_get_block_mask(u->iom, dev, bb, GF_DIRTY,
_region_to_mask(offset, len), &b))
return false;
memset(((unsigned char *) b->data) + offset, 0, len);
io_put_block(b);
return true;
}
static bool _zero_whole(struct updater *u, struct io_dev *dev, block_address bb, block_address be)
{
struct block *b;
for (; bb != be; bb++) {
if (!io_get_block(u->iom, dev, bb, GF_ZERO, &b))
return false;
io_put_block(b);
}
return true;
}
bool io_zero_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len)
{
struct updater u;
u.iom = iom;
u.partial_fn = _zero_partial;
u.whole_fn = _zero_whole;
u.data = NULL;
return _update_bytes(&u, dev, start, len);
}
//----------------------------------------------------------------
static bool _set_partial(struct updater *u, struct io_dev *dev, block_address bb, uint64_t offset, size_t len)
{
struct block *b;
uint8_t val = *((uint8_t *) u->data);
if (!io_get_block_mask(u->iom, dev, bb, GF_DIRTY,
_region_to_mask(offset, len), &b))
return false;
memset(((unsigned char *) b->data) + offset, val, len);
io_put_block(b);
return true;
}
static bool _set_whole(struct updater *u, struct io_dev *dev, block_address bb, block_address be)
{
struct block *b;
uint8_t val = *((uint8_t *) u->data);
uint64_t len = io_block_sectors(u->iom) * 512;
for (; bb != be; bb++) {
if (!io_get_block(u->iom, dev, bb, GF_ZERO, &b))
return false;
memset((unsigned char *) b->data, val, len);
io_put_block(b);
}
return true;
}
bool io_set_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len, uint8_t val)
{
struct updater u;
u.iom = iom;
u.partial_fn = _set_partial;
u.whole_fn = _set_whole;
u.data = &val;
return _update_bytes(&u, dev, start, len);
}

2065
lib/device/io-manager.c Normal file

File diff suppressed because it is too large Load Diff

226
lib/device/io-manager.h Normal file
View File

@@ -0,0 +1,226 @@
/*
* Copyright (C) 2018 Red Hat, Inc. All rights reserved.
*
* This file is part of LVM2.
*
* This copyrighted material is made available to anyone wishing to use,
* modify, copy, or redistribute it subject to the terms and conditions
* of the GNU Lesser General Public License v.2.1.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
#ifndef IO_MANAGER_H
#define IO_MANAGER_H
#include "device_mapper/all.h"
#include <linux/fs.h>
#include <stdbool.h>
#include <stdint.h>
/*----------------------------------------------------------------*/
// FIXME: move somewhere more sensible
#define container_of(v, t, head) \
((t *)((const char *)(v) - (const char *)&((t *) 0)->head))
/*----------------------------------------------------------------*/
enum dir {
DIR_READ = 1,
DIR_WRITE = 2
};
typedef uint64_t block_address;
typedef uint64_t sector_t;
typedef void io_complete_fn(void *context, int io_error);
enum {
EF_READ_ONLY = 1,
EF_EXCL = 2
};
// The io engine must support io with any sector alignment.
// For instance aio will need to fall back to sync io if the
// io is not page aligned.
struct io_engine {
void (*destroy)(struct io_engine *e);
int (*open)(struct io_engine *e, const char *path, unsigned flags, bool o_direct);
void (*close)(struct io_engine *e, int fd);
unsigned (*max_io)(struct io_engine *e);
bool (*issue)(struct io_engine *e, enum dir d, int fd,
sector_t sb, sector_t se, void *data, void *context);
bool (*wait)(struct io_engine *e, io_complete_fn fn);
// The path is there purely for logging.
bool (*get_size)(struct io_engine *e, const char *path, int fd, sector_t *size);
bool (*get_block_sizes)(struct io_engine *e, const char *path, int fd,
unsigned *physical_block_size, unsigned *logical_block_size);
};
struct io_engine *create_async_io_engine(void);
struct io_engine *create_sync_io_engine(void);
// Same as create_async_io_engine(), except writes are not acted upon.
// Used when running with --test.
struct io_engine *create_test_io_engine(void);
/*----------------------------------------------------------------*/
struct io_manager;
struct io_dev;
struct io_dev_internal;
struct block {
/* clients may only access these two fields */
uint64_t index;
void *data;
struct io_manager *iom;
struct io_dev_internal *dev;
struct dm_list list;
unsigned flags;
unsigned ref_count;
int error;
enum dir io_dir;
unsigned io_count;
// Bits mark which sectors of the block should be written.
uint64_t dirty_bits;
};
/*
* Ownership of engine passes. Engine will be destroyed even if this fails.
*
* 'max_cache_devs' limits the number of devices that are held open because we
* are caching data from them. If to many devices are used the least recently used
* dev will be closed, and all its data invalidated.
*/
struct io_manager *io_manager_create(sector_t block_size, unsigned nr_cache_blocks,
unsigned max_cache_devs, struct io_engine *engine,
bool use_o_direct);
void io_manager_destroy(struct io_manager *iom);
// IMPORTANT: It is up to the caller to normalise the device path. io does
// not detect if two relative path refer to the same file, or if 2 device nodes
// refer to the same underlying dev.
// There may be more than one holder of a device at a time. But since we cannot
// promote a dev from being opened non-exclusive to exclusive, there are some
// restrictions:
//
// - You may have concurrent non-exclusive holders.
// - You may have concurrent exclusive holders.
// - You may not have mixed holders.
// - If blocks are in the cache that were acquired by a non exclusive holder,
// they will all be invalidated if a device is opened exclusively.
struct io_dev *io_get_dev(struct io_manager *iom, const char *path, unsigned flags);
void io_put_dev(struct io_dev *dev);
enum io_get_flags {
/*
* The block will be zeroed before get_block returns it. This
* potentially avoids a read if the block is not already in the cache.
* GF_DIRTY is implicit.
*/
GF_ZERO = (1 << 0),
/*
* Indicates the caller is intending to change the data in the block, a
* writeback will occur after the block is released.
*/
GF_DIRTY = (1 << 1)
};
sector_t io_block_sectors(struct io_manager *iom);
unsigned io_nr_cache_blocks(struct io_manager *iom);
unsigned io_max_prefetches(struct io_manager *iom);
unsigned io_max_cache_devs(struct io_manager *iom);
/*
* Use the prefetch method to take advantage of asynchronous IO. For example,
* if you wanted to read a block from many devices concurrently you'd do
* something like this:
*
* dm_list_iterate_items (dev, &devices)
* io_prefetch_block(cache, dev, block);
*
* dm_list_iterate_items (dev, &devices) {
* if (!io_get_block(cache, dev, block, &b))
* fail();
*
* process_block(b);
* }
*
* It's slightly sub optimal, since you may not run the gets in the order that
* they complete. But we're talking a very small difference, and it's worth it
* to keep callbacks out of this interface.
*/
void io_prefetch_block(struct io_manager *iom, struct io_dev *dev, block_address index);
/*
* Returns true on success.
*/
bool io_get_block(struct io_manager *iom, struct io_dev *dev, block_address index,
unsigned flags, struct block **result);
// The mask is used to specify which sectors should be written.
// 'mask' is ignored unless the get flags are GF_ZERO or GF_DIRTY.
bool io_get_block_mask(struct io_manager *iom, struct io_dev *dev, block_address index,
unsigned flags, uint64_t mask, struct block **result);
void io_put_block(struct block *b);
/*
* flush() does not attempt to writeback locked blocks. flush will fail
* (return false), if any unlocked dirty data cannot be written back.
*/
bool io_flush(struct io_manager *iom);
bool io_flush_dev(struct io_manager *iom, struct io_dev *dev);
/*
* Remove blocks from the cache.
*
* If the block is dirty it will be written back first. If the writeback fails
* false will be returned.
*
* If any of the blocks are currently held, false will be returned.
*/
bool io_invalidate_block(struct io_manager *iom, struct io_dev *dev, block_address index);
bool io_invalidate_dev(struct io_manager *iom, struct io_dev *dev);
bool io_invalidate_all(struct io_manager *iom);
bool io_dev_size(struct io_dev *dev, uint64_t *sectors);
bool io_dev_block_sizes(struct io_dev *dev, unsigned *physical_block_size, unsigned *logical_block_size);
// For testing and debug only
void *io_get_dev_context(struct io_dev *dev);
int io_get_fd(void *dev_context);
bool io_is_well_formed(struct io_manager *iom);
//----------------------------------------------------------------
// The next four functions are utilities written in terms of the above
// api. This is simpler to use than the block based api, and I would
// expect almost all clients to use this interface in spite of the extra
// memory copying involved.
// Prefetches the blocks neccessary to satisfy a byte range.
void io_prefetch_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len);
// Reads, writes and zeroes bytes. Returns false if errors occur.
bool io_read_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len, void *data);
bool io_write_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len, void *data);
bool io_zero_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len);
bool io_set_bytes(struct io_manager *iom, struct io_dev *dev, uint64_t start, size_t len, uint8_t val);
//----------------------------------------------------------------
#endif

279
lib/device/io-processor.c Normal file
View File

@@ -0,0 +1,279 @@
/*
* Copyright (C) 2018 Red Hat, Inc. All rights reserved.
*
* This file is part of LVM2.
*
* This copyrighted material is made available to anyone wishing to use,
* modify, copy, or redistribute it subject to the terms and conditions
* of the GNU General Public License v.2.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
#include "lib/device/io-processor.h"
#include "base/data-struct/list.h"
#include "base/memory/zalloc.h"
#include "lib/log/lvm-logging.h"
#include "lib/log/log.h"
//----------------------------------------------------------------
struct job {
struct dm_list list;
char *path;
uint64_t start;
uint64_t len;
void *context;
};
struct io_processor {
struct processor_ops *ops;
io_task_fn task;
io_error_fn err;
struct dm_list jobs;
size_t buffer_size;
void *buffer;
};
static void _free_job(struct job *j)
{
free(j->path);
free(j);
}
struct io_processor *io_processor_create_internal(struct processor_ops *ops,
io_task_fn t, io_error_fn err)
{
struct io_processor *iop = zalloc(sizeof(*iop));
if (iop) {
iop->ops= ops;
iop->task = t;
iop->err = err;
dm_list_init(&iop->jobs);
}
return iop;
}
void io_processor_destroy(struct io_processor *iop)
{
struct job *j, *tmp;
iop->ops->destroy(iop->ops);
dm_list_iterate_items_safe(j, tmp, &iop->jobs)
_free_job(j);
free(iop->buffer);
free(iop);
}
static bool _ensure_buffer(struct io_processor *iop, uint64_t sectors)
{
uint64_t size = sectors * 512;
void *buffer;
// is the existing buffer big enough?
if (size <= iop->buffer_size)
return true;
if (posix_memalign(&buffer, 8, size)) {
log_error("unable to allocate job buffer");
return false;
}
if (iop->buffer)
free(iop->buffer);
iop->buffer = buffer;
iop->buffer_size = size;
return true;
}
bool io_processor_add(struct io_processor *iop, const char *path,
uint64_t start, uint64_t len,
void *context)
{
struct job *j = zalloc(sizeof(*j));
if (!j)
return false;
j->path = strdup(path);
if (!j->path) {
free(j);
return false;
}
j->start = start;
j->len = len;
j->context = context;
if (!_ensure_buffer(iop, len)) {
free(j->path);
free(j);
return false;
}
dm_list_add(&iop->jobs, &j->list);
return true;
}
static void _fail_job(struct io_processor *iop, struct job *j)
{
iop->err(j->context);
dm_list_del(&j->list);
_free_job(j);
}
static uint64_t min(uint64_t lhs, uint64_t rhs)
{
if (lhs < rhs)
return lhs;
if (rhs < lhs)
return rhs;
return lhs;
}
static void _batch(struct io_processor *iop, unsigned count)
{
unsigned blocks_covered;
struct job *j, *tmp;
struct dm_list batch;
void *dev;
dm_list_init(&batch);
// prefetch
dm_list_iterate_items_safe(j, tmp, &iop->jobs) {
if (!count)
break;
dev = iop->ops->get_dev(iop->ops, j->path, EF_READ_ONLY);
if (!dev) {
_fail_job(iop, j);
continue;
}
blocks_covered = iop->ops->prefetch_bytes(iop->ops, dev, j->start, j->len);
iop->ops->put_dev(iop->ops, dev);
count -= min(count, blocks_covered);
dm_list_move(&batch, &j->list);
}
// read
dm_list_iterate_items_safe(j, tmp, &batch) {
dev = iop->ops->get_dev(iop->ops, j->path, EF_READ_ONLY);
if (!dev) {
_fail_job(iop, j);
continue;
}
if (!iop->ops->read_bytes(iop->ops, dev, j->start, j->len, iop->buffer)) {
iop->ops->put_dev(iop->ops, dev);
_fail_job(iop, j);
continue;
}
iop->ops->put_dev(iop->ops, dev);
iop->task(j->context, iop->buffer, j->len);
dm_list_del(&j->list);
_free_job(j);
}
}
void io_processor_exec(struct io_processor *iop)
{
unsigned batch_size = iop->ops->batch_size(iop->ops);
while (!dm_list_empty(&iop->jobs))
_batch(iop, batch_size);
}
//----------------------------------------------------------------
struct iom_ops {
struct processor_ops ops;
struct io_manager *iom;
};
// How many blocks does a byte range cover?
static unsigned _blocks_covered(struct io_manager *iom, uint64_t start, uint64_t len)
{
uint64_t bs = io_block_sectors(iom) * 512;
uint64_t b = start / bs;
uint64_t e = (start + len + bs - 1) / bs;
return e - b;
}
static void _iom_destroy(struct processor_ops *ops)
{
struct iom_ops *p = container_of(ops, struct iom_ops, ops);
free(p);
}
static unsigned _iom_batch_size(struct processor_ops *ops)
{
struct io_manager *iom = container_of(ops, struct iom_ops, ops)->iom;
return io_max_prefetches(iom);
}
static void *_iom_get_dev(struct processor_ops *ops, const char *path, unsigned flags)
{
struct io_manager *iom = container_of(ops, struct iom_ops, ops)->iom;
return io_get_dev(iom, path, flags);
}
static void _iom_put_dev(struct processor_ops *ops, void *dev)
{
return io_put_dev(dev);
}
static unsigned _iom_prefetch_bytes(struct processor_ops *ops, void *dev, uint64_t start, size_t len)
{
struct io_manager *iom = container_of(ops, struct iom_ops, ops)->iom;
io_prefetch_bytes(iom, dev, start, len);
return _blocks_covered(iom, start, len);
}
static bool _iom_read_bytes(struct processor_ops *ops, void *dev, uint64_t start, size_t len, void *data)
{
struct io_manager *iom = container_of(ops, struct iom_ops, ops)->iom;
return io_read_bytes(iom, dev, start, len, data);
}
struct io_processor *io_processor_create(struct io_manager *iom,
io_task_fn t, io_error_fn err)
{
struct io_processor *iop;
struct iom_ops *p = zalloc(sizeof(*p));
p->ops.destroy = _iom_destroy;
p->ops.batch_size = _iom_batch_size;
p->ops.get_dev = _iom_get_dev;
p->ops.put_dev = _iom_put_dev;
p->ops.prefetch_bytes = _iom_prefetch_bytes;
p->ops.read_bytes = _iom_read_bytes;
p->iom = iom;
iop = io_processor_create_internal(&p->ops, t, err);
if (!iop)
free(p);
return iop;
}
//----------------------------------------------------------------

60
lib/device/io-processor.h Normal file
View File

@@ -0,0 +1,60 @@
/*
* Copyright (C) 2018 Red Hat, Inc. All rights reserved.
*
* This file is part of LVM2.
*
* This copyrighted material is made available to anyone wishing to use,
* modify, copy, or redistribute it subject to the terms and conditions
* of the GNU General Public License v.2.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
#ifndef LIB_DEVICE_IO_PARALLEL_H
#define LIB_DEVICE_IO_PARALLEL_H
#include "lib/device/io-manager.h"
//----------------------------------------------------------------
// io-manager utility that let you run a common task on many
// io-manager blocks in parallel. This doesn't use multiple threads
// but it does take care to prefetch data in parallel, so you will
// get a big speed up over a simple get/process/put loop.
struct io_processor;
typedef void (*io_task_fn)(void *context, void *data, uint64_t len);
typedef void (*io_error_fn)(void *context);
struct io_processor *io_processor_create(struct io_manager *iom,
io_task_fn t, io_error_fn err);
void io_processor_destroy(struct io_processor *iop);
// path is copied. start and len are in bytes.
bool io_processor_add(struct io_processor *iop, const char *path, uint64_t start,
uint64_t len, void *context);
void io_processor_exec(struct io_processor *iop);
//----------------------------------------------------------------
// For unit testing
struct processor_ops {
void (*destroy)(struct processor_ops *ops);
unsigned (*batch_size)(struct processor_ops *ops);
void *(*get_dev)(struct processor_ops *ops, const char *path, unsigned flags);
void (*put_dev)(struct processor_ops *ops, void *dev);
// returns the number of blocks covered
unsigned (*prefetch_bytes)(struct processor_ops *ops, void *dev, uint64_t start, size_t len);
bool (*read_bytes)(struct processor_ops *ops, void *dev, uint64_t start, size_t len, void *data);
};
struct io_processor *io_processor_create_internal(struct processor_ops *ops,
io_task_fn t, io_error_fn err);
//----------------------------------------------------------------
#endif

View File

@@ -813,7 +813,7 @@ out:
static int _setup_bcache(void)
{
struct io_engine *ioe = NULL;
struct io_engine_ *ioe = NULL;
int iomem_kb = io_memory_size();
int block_size_kb = (BCACHE_BLOCK_SIZE_IN_SECTORS * 512) / 1024;
int cache_blocks;
@@ -829,14 +829,14 @@ static int _setup_bcache(void)
_current_bcache_size_bytes = cache_blocks * BCACHE_BLOCK_SIZE_IN_SECTORS * 512;
if (use_aio()) {
if (!(ioe = create_async_io_engine())) {
if (!(ioe = create_async_io_engine_())) {
log_warn("Failed to set up async io, using sync io.");
init_use_aio(0);
}
}
if (!ioe) {
if (!(ioe = create_sync_io_engine())) {
if (!(ioe = create_sync_io_engine_())) {
log_error("Failed to set up sync io.");
return 0;
}

View File

@@ -17,14 +17,14 @@ UNIT_SOURCE=\
device_mapper/vdo/status.c \
\
test/unit/activation-generator_t.c \
test/unit/bcache_t.c \
test/unit/bcache_utils_t.c \
test/unit/bitset_t.c \
test/unit/config_t.c \
test/unit/dmlist_t.c \
test/unit/dmstatus_t.c \
test/unit/framework.c \
test/unit/io_engine_t.c \
test/unit/io-manager_t.c \
test/unit/io-manager-utils_t.c \
test/unit/io-processor_t.c \
test/unit/matcher_t.c \
test/unit/percent_t.c \
test/unit/radix_tree_t.c \

View File

@@ -1,924 +0,0 @@
/*
* Copyright (C) 2018 Red Hat, Inc. All rights reserved.
*
* This file is part of LVM2.
*
* This copyrighted material is made available to anyone wishing to use,
* modify, copy, or redistribute it subject to the terms and conditions
* of the GNU General Public License v.2.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
#include "units.h"
#include "lib/device/bcache.h"
#include <errno.h>
#include <stdio.h>
#include <stdlib.h>
#include <unistd.h>
#define SHOW_MOCK_CALLS 0
/*----------------------------------------------------------------
* Mock engine
*--------------------------------------------------------------*/
struct mock_engine {
struct io_engine e;
struct dm_list expected_calls;
struct dm_list issued_io;
unsigned max_io;
sector_t block_size;
};
enum method {
E_DESTROY,
E_ISSUE,
E_WAIT,
E_MAX_IO
};
struct mock_call {
struct dm_list list;
enum method m;
bool match_args;
enum dir d;
int fd;
block_address b;
bool issue_r;
bool wait_r;
};
struct mock_io {
struct dm_list list;
int fd;
sector_t sb;
sector_t se;
void *data;
void *context;
bool r;
};
static const char *_show_method(enum method m)
{
switch (m) {
case E_DESTROY:
return "destroy()";
case E_ISSUE:
return "issue()";
case E_WAIT:
return "wait()";
case E_MAX_IO:
return "max_io()";
}
return "<unknown>";
}
static void _expect(struct mock_engine *e, enum method m)
{
struct mock_call *mc = malloc(sizeof(*mc));
mc->m = m;
mc->match_args = false;
dm_list_add(&e->expected_calls, &mc->list);
}
static void _expect_read(struct mock_engine *e, int fd, block_address b)
{
struct mock_call *mc = malloc(sizeof(*mc));
mc->m = E_ISSUE;
mc->match_args = true;
mc->d = DIR_READ;
mc->fd = fd;
mc->b = b;
mc->issue_r = true;
mc->wait_r = true;
dm_list_add(&e->expected_calls, &mc->list);
}
static void _expect_read_any(struct mock_engine *e)
{
struct mock_call *mc = malloc(sizeof(*mc));
mc->m = E_ISSUE;
mc->match_args = false;
mc->issue_r = true;
mc->wait_r = true;
dm_list_add(&e->expected_calls, &mc->list);
}
static void _expect_write(struct mock_engine *e, int fd, block_address b)
{
struct mock_call *mc = malloc(sizeof(*mc));
mc->m = E_ISSUE;
mc->match_args = true;
mc->d = DIR_WRITE;
mc->fd = fd;
mc->b = b;
mc->issue_r = true;
mc->wait_r = true;
dm_list_add(&e->expected_calls, &mc->list);
}
static void _expect_read_bad_issue(struct mock_engine *e, int fd, block_address b)
{
struct mock_call *mc = malloc(sizeof(*mc));
mc->m = E_ISSUE;
mc->match_args = true;
mc->d = DIR_READ;
mc->fd = fd;
mc->b = b;
mc->issue_r = false;
mc->wait_r = true;
dm_list_add(&e->expected_calls, &mc->list);
}
static void _expect_write_bad_issue(struct mock_engine *e, int fd, block_address b)
{
struct mock_call *mc = malloc(sizeof(*mc));
mc->m = E_ISSUE;
mc->match_args = true;
mc->d = DIR_WRITE;
mc->fd = fd;
mc->b = b;
mc->issue_r = false;
mc->wait_r = true;
dm_list_add(&e->expected_calls, &mc->list);
}
static void _expect_read_bad_wait(struct mock_engine *e, int fd, block_address b)
{
struct mock_call *mc = malloc(sizeof(*mc));
mc->m = E_ISSUE;
mc->match_args = true;
mc->d = DIR_READ;
mc->fd = fd;
mc->b = b;
mc->issue_r = true;
mc->wait_r = false;
dm_list_add(&e->expected_calls, &mc->list);
}
static void _expect_write_bad_wait(struct mock_engine *e, int fd, block_address b)
{
struct mock_call *mc = malloc(sizeof(*mc));
mc->m = E_ISSUE;
mc->match_args = true;
mc->d = DIR_WRITE;
mc->fd = fd;
mc->b = b;
mc->issue_r = true;
mc->wait_r = false;
dm_list_add(&e->expected_calls, &mc->list);
}
static struct mock_call *_match_pop(struct mock_engine *e, enum method m)
{
struct mock_call *mc;
if (dm_list_empty(&e->expected_calls))
test_fail("unexpected call to method %s\n", _show_method(m));
mc = dm_list_item(e->expected_calls.n, struct mock_call);
dm_list_del(&mc->list);
if (mc->m != m)
test_fail("expected %s, but got %s\n", _show_method(mc->m), _show_method(m));
#if SHOW_MOCK_CALLS
else
fprintf(stderr, "%s called (expected)\n", _show_method(m));
#endif
return mc;
}
static void _match(struct mock_engine *e, enum method m)
{
free(_match_pop(e, m));
}
static void _no_outstanding_expectations(struct mock_engine *e)
{
struct mock_call *mc;
if (!dm_list_empty(&e->expected_calls)) {
fprintf(stderr, "unsatisfied expectations:\n");
dm_list_iterate_items (mc, &e->expected_calls)
fprintf(stderr, " %s\n", _show_method(mc->m));
}
T_ASSERT(dm_list_empty(&e->expected_calls));
}
static struct mock_engine *_to_mock(struct io_engine *e)
{
return container_of(e, struct mock_engine, e);
}
static void _mock_destroy(struct io_engine *e)
{
struct mock_engine *me = _to_mock(e);
_match(me, E_DESTROY);
T_ASSERT(dm_list_empty(&me->issued_io));
T_ASSERT(dm_list_empty(&me->expected_calls));
free(_to_mock(e));
}
static bool _mock_issue(struct io_engine *e, enum dir d, int fd,
sector_t sb, sector_t se, void *data, void *context)
{
bool r, wait_r;
struct mock_io *io;
struct mock_call *mc;
struct mock_engine *me = _to_mock(e);
mc = _match_pop(me, E_ISSUE);
if (mc->match_args) {
T_ASSERT(d == mc->d);
T_ASSERT(fd == mc->fd);
T_ASSERT(sb == mc->b * me->block_size);
T_ASSERT(se == (mc->b + 1) * me->block_size);
}
r = mc->issue_r;
wait_r = mc->wait_r;
free(mc);
if (r) {
io = malloc(sizeof(*io));
if (!io)
abort();
io->fd = fd;
io->sb = sb;
io->se = se;
io->data = data;
io->context = context;
io->r = wait_r;
dm_list_add(&me->issued_io, &io->list);
}
return r;
}
static bool _mock_wait(struct io_engine *e, io_complete_fn fn)
{
struct mock_io *io;
struct mock_engine *me = _to_mock(e);
_match(me, E_WAIT);
// FIXME: provide a way to control how many are completed and whether
// they error.
T_ASSERT(!dm_list_empty(&me->issued_io));
io = dm_list_item(me->issued_io.n, struct mock_io);
dm_list_del(&io->list);
fn(io->context, io->r ? 0 : -EIO);
free(io);
return true;
}
static unsigned _mock_max_io(struct io_engine *e)
{
struct mock_engine *me = _to_mock(e);
_match(me, E_MAX_IO);
return me->max_io;
}
static struct mock_engine *_mock_create(unsigned max_io, sector_t block_size)
{
struct mock_engine *m = malloc(sizeof(*m));
m->e.destroy = _mock_destroy;
m->e.issue = _mock_issue;
m->e.wait = _mock_wait;
m->e.max_io = _mock_max_io;
m->max_io = max_io;
m->block_size = block_size;
dm_list_init(&m->expected_calls);
dm_list_init(&m->issued_io);
return m;
}
/*----------------------------------------------------------------
* Fixtures
*--------------------------------------------------------------*/
struct fixture {
struct mock_engine *me;
struct bcache *cache;
};
static struct fixture *_fixture_init(sector_t block_size, unsigned nr_cache_blocks)
{
struct fixture *f = malloc(sizeof(*f));
f->me = _mock_create(16, block_size);
T_ASSERT(f->me);
_expect(f->me, E_MAX_IO);
f->cache = bcache_create(block_size, nr_cache_blocks, &f->me->e);
T_ASSERT(f->cache);
return f;
}
static void _fixture_exit(struct fixture *f)
{
_expect(f->me, E_DESTROY);
bcache_destroy(f->cache);
free(f);
}
static void *_small_fixture_init(void)
{
return _fixture_init(128, 16);
}
static void _small_fixture_exit(void *context)
{
_fixture_exit(context);
}
static void *_large_fixture_init(void)
{
return _fixture_init(128, 1024);
}
static void _large_fixture_exit(void *context)
{
_fixture_exit(context);
}
/*----------------------------------------------------------------
* Tests
*--------------------------------------------------------------*/
#define MEG 2048
#define SECTOR_SHIFT 9
static void good_create(sector_t block_size, unsigned nr_cache_blocks)
{
struct bcache *cache;
struct mock_engine *me = _mock_create(16, 128);
_expect(me, E_MAX_IO);
cache = bcache_create(block_size, nr_cache_blocks, &me->e);
T_ASSERT(cache);
_expect(me, E_DESTROY);
bcache_destroy(cache);
}
static void bad_create(sector_t block_size, unsigned nr_cache_blocks)
{
struct bcache *cache;
struct mock_engine *me = _mock_create(16, 128);
_expect(me, E_MAX_IO);
cache = bcache_create(block_size, nr_cache_blocks, &me->e);
T_ASSERT(!cache);
_expect(me, E_DESTROY);
me->e.destroy(&me->e);
}
static void test_create(void *fixture)
{
good_create(8, 16);
}
static void test_nr_cache_blocks_must_be_positive(void *fixture)
{
bad_create(8, 0);
}
static void test_block_size_must_be_positive(void *fixture)
{
bad_create(0, 16);
}
static void test_block_size_must_be_multiple_of_page_size(void *fixture)
{
static unsigned _bad_examples[] = {3, 9, 13, 1025};
unsigned i;
for (i = 0; i < DM_ARRAY_SIZE(_bad_examples); i++)
bad_create(_bad_examples[i], 16);
for (i = 1; i < 100; i++)
good_create(i * 8, 16);
}
static void test_get_triggers_read(void *context)
{
struct fixture *f = context;
int fd = 17; // arbitrary key
struct block *b;
_expect_read(f->me, fd, 0);
_expect(f->me, E_WAIT);
T_ASSERT(bcache_get(f->cache, fd, 0, 0, &b));
bcache_put(b);
_expect_read(f->me, fd, 1);
_expect(f->me, E_WAIT);
T_ASSERT(bcache_get(f->cache, fd, 1, GF_DIRTY, &b));
_expect_write(f->me, fd, 1);
_expect(f->me, E_WAIT);
bcache_put(b);
}
static void test_repeated_reads_are_cached(void *context)
{
struct fixture *f = context;
int fd = 17; // arbitrary key
unsigned i;
struct block *b;
_expect_read(f->me, fd, 0);
_expect(f->me, E_WAIT);
for (i = 0; i < 100; i++) {
T_ASSERT(bcache_get(f->cache, fd, 0, 0, &b));
bcache_put(b);
}
}
static void test_block_gets_evicted_with_many_reads(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
const unsigned nr_cache_blocks = 16;
int fd = 17; // arbitrary key
unsigned i;
struct block *b;
for (i = 0; i < nr_cache_blocks; i++) {
_expect_read(me, fd, i);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, i, 0, &b));
bcache_put(b);
}
// Not enough cache blocks to hold this one
_expect_read(me, fd, nr_cache_blocks);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, nr_cache_blocks, 0, &b));
bcache_put(b);
// Now if we run through we should find one block has been
// evicted. We go backwards because the oldest is normally
// evicted first.
_expect_read_any(me);
_expect(me, E_WAIT);
for (i = nr_cache_blocks; i; i--) {
T_ASSERT(bcache_get(cache, fd, i - 1, 0, &b));
bcache_put(b);
}
}
static void test_prefetch_issues_a_read(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
const unsigned nr_cache_blocks = 16;
int fd = 17; // arbitrary key
unsigned i;
struct block *b;
for (i = 0; i < nr_cache_blocks; i++) {
// prefetch should not wait
_expect_read(me, fd, i);
bcache_prefetch(cache, fd, i);
}
_no_outstanding_expectations(me);
for (i = 0; i < nr_cache_blocks; i++) {
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, i, 0, &b));
bcache_put(b);
}
}
static void test_too_many_prefetches_does_not_trigger_a_wait(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
const unsigned nr_cache_blocks = 16;
int fd = 17; // arbitrary key
unsigned i;
for (i = 0; i < 10 * nr_cache_blocks; i++) {
// prefetch should not wait
if (i < nr_cache_blocks)
_expect_read(me, fd, i);
bcache_prefetch(cache, fd, i);
}
// Destroy will wait for any in flight IO triggered by prefetches.
for (i = 0; i < nr_cache_blocks; i++)
_expect(me, E_WAIT);
}
static void test_dirty_data_gets_written_back(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
int fd = 17; // arbitrary key
struct block *b;
// Expect the read
_expect_read(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, 0, GF_DIRTY, &b));
bcache_put(b);
// Expect the write
_expect_write(me, fd, 0);
_expect(me, E_WAIT);
}
static void test_zeroed_data_counts_as_dirty(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
int fd = 17; // arbitrary key
struct block *b;
// No read
T_ASSERT(bcache_get(cache, fd, 0, GF_ZERO, &b));
bcache_put(b);
// Expect the write
_expect_write(me, fd, 0);
_expect(me, E_WAIT);
}
static void test_flush_waits_for_all_dirty(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
const unsigned count = 16;
int fd = 17; // arbitrary key
unsigned i;
struct block *b;
for (i = 0; i < count; i++) {
if (i % 2) {
T_ASSERT(bcache_get(cache, fd, i, GF_ZERO, &b));
} else {
_expect_read(me, fd, i);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, i, 0, &b));
}
bcache_put(b);
}
for (i = 0; i < count; i++) {
if (i % 2)
_expect_write(me, fd, i);
}
for (i = 0; i < count; i++) {
if (i % 2)
_expect(me, E_WAIT);
}
bcache_flush(cache);
_no_outstanding_expectations(me);
}
static void test_multiple_files(void *context)
{
static int _fds[] = {1, 128, 345, 678, 890};
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
unsigned i;
for (i = 0; i < DM_ARRAY_SIZE(_fds); i++) {
_expect_read(me, _fds[i], 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, _fds[i], 0, 0, &b));
bcache_put(b);
}
}
static void test_read_bad_issue(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
_expect_read_bad_issue(me, 17, 0);
T_ASSERT(!bcache_get(cache, 17, 0, 0, &b));
}
static void test_read_bad_issue_intermittent(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
_expect_read_bad_issue(me, fd, 0);
T_ASSERT(!bcache_get(cache, fd, 0, 0, &b));
_expect_read(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, 0, 0, &b));
bcache_put(b);
}
static void test_read_bad_wait(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
_expect_read_bad_wait(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(!bcache_get(cache, fd, 0, 0, &b));
}
static void test_read_bad_wait_intermittent(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
_expect_read_bad_wait(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(!bcache_get(cache, fd, 0, 0, &b));
_expect_read(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, 0, 0, &b));
bcache_put(b);
}
static void test_write_bad_issue_stops_flush(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
T_ASSERT(bcache_get(cache, fd, 0, GF_ZERO, &b));
_expect_write_bad_issue(me, fd, 0);
bcache_put(b);
T_ASSERT(!bcache_flush(cache));
// we'll let it succeed the second time
_expect_write(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_flush(cache));
}
static void test_write_bad_io_stops_flush(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
T_ASSERT(bcache_get(cache, fd, 0, GF_ZERO, &b));
_expect_write_bad_wait(me, fd, 0);
_expect(me, E_WAIT);
bcache_put(b);
T_ASSERT(!bcache_flush(cache));
// we'll let it succeed the second time
_expect_write(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_flush(cache));
}
static void test_invalidate_not_present(void *context)
{
struct fixture *f = context;
struct bcache *cache = f->cache;
int fd = 17;
T_ASSERT(bcache_invalidate(cache, fd, 0));
}
static void test_invalidate_present(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
_expect_read(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, 0, 0, &b));
bcache_put(b);
T_ASSERT(bcache_invalidate(cache, fd, 0));
}
static void test_invalidate_after_read_error(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
_expect_read_bad_issue(me, fd, 0);
T_ASSERT(!bcache_get(cache, fd, 0, 0, &b));
T_ASSERT(bcache_invalidate(cache, fd, 0));
}
static void test_invalidate_after_write_error(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
T_ASSERT(bcache_get(cache, fd, 0, GF_ZERO, &b));
bcache_put(b);
// invalidate should fail if the write fails
_expect_write_bad_wait(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(!bcache_invalidate(cache, fd, 0));
// and should succeed if the write does
_expect_write(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_invalidate(cache, fd, 0));
// a read is not required to get the block
_expect_read(me, fd, 0);
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, fd, 0, 0, &b));
bcache_put(b);
}
static void test_invalidate_held_block(void *context)
{
struct fixture *f = context;
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
struct block *b;
int fd = 17;
T_ASSERT(bcache_get(cache, fd, 0, GF_ZERO, &b));
T_ASSERT(!bcache_invalidate(cache, fd, 0));
_expect_write(me, fd, 0);
_expect(me, E_WAIT);
bcache_put(b);
}
//----------------------------------------------------------------
// Chasing a bug reported by dct
static void _cycle(struct fixture *f, unsigned nr_cache_blocks)
{
struct mock_engine *me = f->me;
struct bcache *cache = f->cache;
unsigned i;
struct block *b;
for (i = 0; i < nr_cache_blocks; i++) {
// prefetch should not wait
_expect_read(me, i, 0);
bcache_prefetch(cache, i, 0);
}
// This double checks the reads occur in response to the prefetch
_no_outstanding_expectations(me);
for (i = 0; i < nr_cache_blocks; i++) {
_expect(me, E_WAIT);
T_ASSERT(bcache_get(cache, i, 0, 0, &b));
bcache_put(b);
}
_no_outstanding_expectations(me);
}
static void test_concurrent_reads_after_invalidate(void *context)
{
struct fixture *f = context;
unsigned i, nr_cache_blocks = 16;
_cycle(f, nr_cache_blocks);
for (i = 0; i < nr_cache_blocks; i++)
bcache_invalidate_fd(f->cache, i);
_cycle(f, nr_cache_blocks);
}
/*----------------------------------------------------------------
* Top level
*--------------------------------------------------------------*/
#define T(path, desc, fn) register_test(ts, "/base/device/bcache/" path, desc, fn)
static struct test_suite *_tiny_tests(void)
{
struct test_suite *ts = test_suite_create(NULL, NULL);
if (!ts) {
fprintf(stderr, "out of memory\n");
exit(1);
}
T("create-destroy", "simple create/destroy", test_create);
T("cache-blocks-positive", "nr cache blocks must be positive", test_nr_cache_blocks_must_be_positive);
T("block-size-positive", "block size must be positive", test_block_size_must_be_positive);
T("block-size-multiple-page", "block size must be a multiple of page size", test_block_size_must_be_multiple_of_page_size);
return ts;
}
static struct test_suite *_small_tests(void)
{
struct test_suite *ts = test_suite_create(_small_fixture_init, _small_fixture_exit);
if (!ts) {
fprintf(stderr, "out of memory\n");
exit(1);
}
T("get-reads", "bcache_get() triggers read", test_get_triggers_read);
T("reads-cached", "repeated reads are cached", test_repeated_reads_are_cached);
T("blocks-get-evicted", "block get evicted with many reads", test_block_gets_evicted_with_many_reads);
T("prefetch-reads", "prefetch issues a read", test_prefetch_issues_a_read);
T("prefetch-never-waits", "too many prefetches does not trigger a wait", test_too_many_prefetches_does_not_trigger_a_wait);
T("writeback-occurs", "dirty data gets written back", test_dirty_data_gets_written_back);
T("zero-flag-dirties", "zeroed data counts as dirty", test_zeroed_data_counts_as_dirty);
T("read-multiple-files", "read from multiple files", test_multiple_files);
T("read-bad-issue", "read fails if io engine unable to issue", test_read_bad_issue);
T("read-bad-issue-intermittent", "failed issue, followed by succes", test_read_bad_issue_intermittent);
T("read-bad-io", "read issued ok, but io fails", test_read_bad_wait);
T("read-bad-io-intermittent", "failed io, followed by success", test_read_bad_wait_intermittent);
T("write-bad-issue-stops-flush", "flush fails temporarily if any block fails to write", test_write_bad_issue_stops_flush);
T("write-bad-io-stops-flush", "flush fails temporarily if any block fails to write", test_write_bad_io_stops_flush);
T("invalidate-not-present", "invalidate a block that isn't in the cache", test_invalidate_not_present);
T("invalidate-present", "invalidate a block that is in the cache", test_invalidate_present);
T("invalidate-read-error", "invalidate a block that errored", test_invalidate_after_read_error);
T("invalidate-write-error", "invalidate a block that errored", test_invalidate_after_write_error);
T("invalidate-fails-in-held", "invalidating a held block fails", test_invalidate_held_block);
T("concurrent-reads-after-invalidate", "prefetch should still issue concurrent reads after invalidate",
test_concurrent_reads_after_invalidate);
return ts;
}
static struct test_suite *_large_tests(void)
{
struct test_suite *ts = test_suite_create(_large_fixture_init, _large_fixture_exit);
if (!ts) {
fprintf(stderr, "out of memory\n");
exit(1);
}
T("flush-waits", "flush waits for all dirty", test_flush_waits_for_all_dirty);
return ts;
}
void bcache_tests(struct dm_list *all_tests)
{
dm_list_add(all_tests, &_tiny_tests()->list);
dm_list_add(all_tests, &_small_tests()->list);
dm_list_add(all_tests, &_large_tests()->list);
}

View File

@@ -12,8 +12,9 @@
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
#include "lib/device/io-manager.h"
#include "framework.h"
#include "units.h"
#include "lib/device/bcache.h"
#include <errno.h>
#include <stdio.h>
@@ -31,9 +32,9 @@
#define INIT_PATTERN 123
struct fixture {
int fd;
char fname[32];
struct bcache *cache;
struct io_manager *iom;
struct io_dev *dev;
};
static inline uint8_t _pattern_at(uint8_t pat, uint8_t byte)
@@ -46,43 +47,59 @@ static uint64_t byte(block_address b, uint64_t offset)
return b * T_BLOCK_SIZE + offset;
}
// With testing in tmpfs directory O_DIRECT cannot be used
// tmpfs has f_fsid == 0 (unsure if this is best guess)
static bool _use_o_direct_internal(void)
{
struct statvfs fsdata;
if (statvfs(".", &fsdata))
// assume we can
return true;
return fsdata.f_fsid;
}
static bool _use_o_direct(void)
{
static bool latch = false;
static bool result;
if (!latch) {
latch = true;
result = _use_o_direct_internal();
if (!result)
printf(" Running test in tmpfs, *NOT* using O_DIRECT\n");
}
return result;
}
static void *_fix_init(struct io_engine *engine)
{
int fd;
uint8_t buffer[T_BLOCK_SIZE];
struct fixture *f = malloc(sizeof(*f));
unsigned b, i;
struct statvfs fsdata;
static int _runs_is_tmpfs = -1;
if (_runs_is_tmpfs == -1) {
// With testing in tmpfs directory O_DIRECT cannot be used
// tmpfs has f_fsid == 0 (unsure if this is best guess)
_runs_is_tmpfs = (statvfs(".", &fsdata) == 0 && !fsdata.f_fsid) ? 1 : 0;
if (_runs_is_tmpfs)
printf(" Running test in tmpfs, *NOT* using O_DIRECT\n");
}
T_ASSERT(f);
snprintf(f->fname, sizeof(f->fname), "unit-test-XXXXXX");
f->fd = mkstemp(f->fname);
T_ASSERT(f->fd >= 0);
fd = mkstemp(f->fname);
T_ASSERT(fd >= 0);
for (b = 0; b < NR_BLOCKS; b++) {
for (i = 0; i < sizeof(buffer); i++)
buffer[i] = _pattern_at(INIT_PATTERN, byte(b, i));
T_ASSERT(write(f->fd, buffer, T_BLOCK_SIZE) > 0);
T_ASSERT(write(fd, buffer, T_BLOCK_SIZE) > 0);
}
close(fd);
if (!_runs_is_tmpfs) {
close(f->fd);
// reopen with O_DIRECT
f->fd = open(f->fname, O_RDWR | O_DIRECT);
T_ASSERT(f->fd >= 0);
}
f->iom = io_manager_create(T_BLOCK_SIZE / 512, NR_BLOCKS, 256, engine, true);
T_ASSERT(f->iom);
f->cache = bcache_create(T_BLOCK_SIZE / 512, NR_BLOCKS, engine);
T_ASSERT(f->cache);
f->dev = io_get_dev(f->iom, f->fname, 0);
T_ASSERT(f->dev);
return f;
}
@@ -105,8 +122,8 @@ static void _fix_exit(void *fixture)
{
struct fixture *f = fixture;
bcache_destroy(f->cache);
close(f->fd);
io_put_dev(f->dev);
io_manager_destroy(f->iom);
unlink(f->fname);
free(f);
}
@@ -135,20 +152,20 @@ static void _verify(struct fixture *f, uint64_t byte_b, uint64_t byte_e, uint8_t
uint64_t offset = byte_b % T_BLOCK_SIZE;
uint64_t blen, len = byte_e - byte_b;
// Verify via bcache_read_bytes
// Verify via io_read_bytes
{
unsigned i;
size_t len2 = byte_e - byte_b;
uint8_t *buffer = malloc(len2);
T_ASSERT(bcache_read_bytes(f->cache, f->fd, byte_b, len2, buffer));
T_ASSERT(io_read_bytes(f->iom, f->dev, byte_b, len2, buffer));
for (i = 0; i < len; i++)
T_ASSERT_EQUAL(buffer[i], _pattern_at(pat, byte_b + i));
free(buffer);
}
// Verify again, driving bcache directly
// Verify again, driving io directly
for (; bb != be; bb++) {
T_ASSERT(bcache_get(f->cache, f->fd, bb, 0, &b));
T_ASSERT(io_get_block(f->iom, f->dev, bb, 0, &b));
blen = _min(T_BLOCK_SIZE - offset, len);
_verify_bytes(b, bb * T_BLOCK_SIZE, offset, blen, pat);
@@ -156,7 +173,7 @@ static void _verify(struct fixture *f, uint64_t byte_b, uint64_t byte_e, uint8_t
offset = 0;
len -= blen;
bcache_put(b);
io_put_block(b);
}
}
@@ -170,7 +187,7 @@ static void _verify_set(struct fixture *f, uint64_t byte_b, uint64_t byte_e, uin
uint64_t blen, len = byte_e - byte_b;
for (; bb != be; bb++) {
T_ASSERT(bcache_get(f->cache, f->fd, bb, 0, &b));
T_ASSERT(io_get_block(f->iom, f->dev, bb, 0, &b));
blen = _min(T_BLOCK_SIZE - offset, len);
for (i = 0; i < blen; i++)
@@ -179,7 +196,7 @@ static void _verify_set(struct fixture *f, uint64_t byte_b, uint64_t byte_e, uin
offset = 0;
len -= blen;
bcache_put(b);
io_put_block(b);
}
}
@@ -198,30 +215,35 @@ static void _do_write(struct fixture *f, uint64_t byte_b, uint64_t byte_e, uint8
for (i = 0; i < len; i++)
buffer[i] = _pattern_at(pat, byte_b + i);
T_ASSERT(bcache_write_bytes(f->cache, f->fd, byte_b, byte_e - byte_b, buffer));
T_ASSERT(io_write_bytes(f->iom, f->dev, byte_b, byte_e - byte_b, buffer));
free(buffer);
}
static void _do_zero(struct fixture *f, uint64_t byte_b, uint64_t byte_e)
{
T_ASSERT(bcache_zero_bytes(f->cache, f->fd, byte_b, byte_e - byte_b));
T_ASSERT(io_zero_bytes(f->iom, f->dev, byte_b, byte_e - byte_b));
}
static void _do_set(struct fixture *f, uint64_t byte_b, uint64_t byte_e, uint8_t val)
{
T_ASSERT(bcache_set_bytes(f->cache, f->fd, byte_b, byte_e - byte_b, val));
T_ASSERT(io_set_bytes(f->iom, f->dev, byte_b, byte_e - byte_b, val));
}
static void _reopen(struct fixture *f)
{
struct io_engine *engine;
bcache_destroy(f->cache);
io_put_dev(f->dev);
io_manager_destroy(f->iom);
engine = create_async_io_engine();
T_ASSERT(engine);
f->cache = bcache_create(T_BLOCK_SIZE / 512, NR_BLOCKS, engine);
T_ASSERT(f->cache);
f->iom = io_manager_create(T_BLOCK_SIZE / 512, NR_BLOCKS, 256, engine, _use_o_direct());
T_ASSERT(f->iom);
f->dev = io_get_dev(f->iom, f->fname, 0);
T_ASSERT(f->iom);
}
//----------------------------------------------------------------
@@ -370,8 +392,6 @@ static void _test_set_many_boundaries(void *fixture)
//----------------------------------------------------------------
#define T(path, desc, fn) register_test(ts, "/base/device/bcache/utils/async/" path, desc, fn)
static struct test_suite *_async_tests(void)
{
struct test_suite *ts = test_suite_create(_async_init, _fix_exit);
@@ -380,7 +400,7 @@ static struct test_suite *_async_tests(void)
exit(1);
}
#define T(path, desc, fn) register_test(ts, "/base/device/bcache/utils/async/" path, desc, fn)
#define T(path, desc, fn) register_test(ts, "/base/device/io-manager/utils/async/" path, desc, fn)
T("rw-first-block", "read/write/verify the first block", _test_rw_first_block);
T("rw-last-block", "read/write/verify the last block", _test_rw_last_block);
T("rw-several-blocks", "read/write/verify several whole blocks", _test_rw_several_whole_blocks);
@@ -415,7 +435,7 @@ static struct test_suite *_sync_tests(void)
exit(1);
}
#define T(path, desc, fn) register_test(ts, "/base/device/bcache/utils/sync/" path, desc, fn)
#define T(path, desc, fn) register_test(ts, "/base/device/io-manager/utils/sync/" path, desc, fn)
T("rw-first-block", "read/write/verify the first block", _test_rw_first_block);
T("rw-last-block", "read/write/verify the last block", _test_rw_last_block);
T("rw-several-blocks", "read/write/verify several whole blocks", _test_rw_several_whole_blocks);
@@ -441,7 +461,7 @@ static struct test_suite *_sync_tests(void)
return ts;
}
void bcache_utils_tests(struct dm_list *all_tests)
void io_manager_utils_tests(struct dm_list *all_tests)
{
dm_list_add(all_tests, &_async_tests()->list);
dm_list_add(all_tests, &_sync_tests()->list);

2128
test/unit/io-manager_t.c Normal file

File diff suppressed because it is too large Load Diff

469
test/unit/io-processor_t.c Normal file
View File

@@ -0,0 +1,469 @@
/*
* Copyright (C) 2018 Red Hat, Inc. All rights reserved.
*
* This file is part of LVM2.
*
* This copyrighted material is made available to anyone wishing to use,
* modify, copy, or redistribute it subject to the terms and conditions
* of the GNU General Public License v.2.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
#include "lib/device/io-processor.h"
#include "framework.h"
#include "units.h"
#include "base/memory/zalloc.h"
#include <stdio.h>
#include <stdlib.h>
#include <unistd.h>
#define SHOW_MOCK_CALLS 0
//----------------------------------------------------------------
static unsigned _rand(unsigned max)
{
return rand() % max;
}
//----------------------------------------------------------------
enum method {
M_DESTROY,
M_BATCH_SIZE,
M_GET_DEV,
M_PUT_DEV,
M_PREFETCH,
M_READ,
M_TASK,
M_ERROR
};
struct expectation {
struct dm_list list;
enum method m;
bool succeed;
};
struct dev {
struct dm_list list;
unsigned id;
};
struct mock_ops {
struct processor_ops ops;
struct dm_list expectations;
unsigned batch_size;
};
static const char *_show_method(enum method m)
{
switch (m) {
case M_DESTROY: return "destroy";
case M_BATCH_SIZE: return "batch_size";
case M_GET_DEV: return "get_dev";
case M_PUT_DEV: return "put_dev";
case M_PREFETCH: return "prefetch";
case M_READ: return "read";
case M_TASK: return "task";
case M_ERROR: return "error";
}
return "<unknown>";
}
static struct expectation *_match_pop(struct mock_ops *mops, enum method m)
{
struct expectation *e;
if (dm_list_empty(&mops->expectations))
test_fail("unexpected call to method %s\n", _show_method(m));
e = dm_list_item(mops->expectations.n, struct expectation);
dm_list_del(&e->list);
if (e->m != m)
test_fail("expected %s, but got %s\n", _show_method(e->m), _show_method(m));
#if SHOW_MOCK_CALLS
else
fprintf(stderr, "%s called (expected)\n", _show_method(m));
#endif
return e;
}
static bool _match(struct mock_ops *mops, enum method m)
{
struct expectation *e = _match_pop(mops, m);
bool r = e->succeed;
free(e);
return r;
}
static void _mock_destroy(struct processor_ops *ops)
{
struct mock_ops *mops = container_of(ops, struct mock_ops, ops);
struct expectation *e, *tmp;
_match(mops, M_DESTROY);
if (!dm_list_empty(&mops->expectations)) {
dm_list_iterate_items_safe(e, tmp, &mops->expectations)
fprintf(stderr, " %s", _show_method(e->m));
test_fail("unsatisfied expectations");
}
free(mops);
}
static unsigned _mock_batch_size(struct processor_ops *ops)
{
struct mock_ops *mops = container_of(ops, struct mock_ops, ops);
_match(mops, M_BATCH_SIZE);
return mops->batch_size;
}
static void *_mock_get_dev(struct processor_ops *ops, const char *path, unsigned flags)
{
struct dev *d;
struct mock_ops *mops = container_of(ops, struct mock_ops, ops);
struct expectation *e = _match_pop(mops, M_GET_DEV);
if (!e->succeed) {
free(e);
return NULL;
}
free(e);
d = zalloc(sizeof(*d));
dm_list_init(&d->list);
d->id = 0;
return d;
}
static void _mock_put_dev(struct processor_ops *ops, void *dev)
{
struct dev *d = dev;
struct mock_ops *mops = container_of(ops, struct mock_ops, ops);
_match(mops, M_PUT_DEV);
free(d);
}
static unsigned _mock_prefetch_bytes(struct processor_ops *ops, void *dev, uint64_t start, size_t len)
{
struct mock_ops *mops = container_of(ops, struct mock_ops, ops);
_match(mops, M_PREFETCH);
return 1;
}
static bool _mock_read_bytes(struct processor_ops *ops, void *dev, uint64_t start, size_t len, void *data)
{
struct mock_ops *mops = container_of(ops, struct mock_ops, ops);
return _match(mops, M_READ);
}
static void _mock_task(void *context, void *data, uint64_t len)
{
struct mock_ops *mops = context;
_match(mops, M_TASK);
}
static void _mock_error(void *context)
{
struct mock_ops *mops = context;
_match(mops, M_ERROR);
}
static void _expect(struct mock_ops *mops, enum method m)
{
struct expectation *e = zalloc(sizeof(*e));
e->m = m;
e->succeed = true;
dm_list_add(&mops->expectations, &e->list);
}
static void _expect_fail(struct mock_ops *mops, enum method m)
{
struct expectation *e = zalloc(sizeof(*e));
e->m = m;
e->succeed = false;
dm_list_add(&mops->expectations, &e->list);
}
static struct mock_ops *_mock_ops_create(void)
{
struct mock_ops *mops = zalloc(sizeof(*mops));
mops->ops.destroy = _mock_destroy;
mops->ops.batch_size = _mock_batch_size;
mops->ops.get_dev = _mock_get_dev;
mops->ops.put_dev = _mock_put_dev;
mops->ops.prefetch_bytes = _mock_prefetch_bytes;
mops->ops.read_bytes = _mock_read_bytes;
dm_list_init(&mops->expectations);
mops->batch_size = 1;
return mops;
}
//----------------------------------------------------------------
struct fixture {
struct mock_ops *mops;
struct io_processor *iop;
};
static void *_fix_init(void)
{
struct fixture *f;
struct mock_ops *mops;
struct io_processor *iop;
f = zalloc(sizeof(*f));
T_ASSERT(f);
mops = _mock_ops_create();
T_ASSERT(mops);
iop = io_processor_create_internal(&mops->ops, _mock_task, _mock_error);
T_ASSERT(iop);
f->mops = mops;
f->iop = iop;
return f;
}
static void _fix_exit(void *context)
{
struct fixture *f = context;
_expect(f->mops, M_DESTROY);
io_processor_destroy(f->iop);
free(f);
}
//----------------------------------------------------------------
// Tests
//----------------------------------------------------------------
static void _test_create_destroy(void *context)
{
// empty
}
static void _test_add_but_no_run(void *context)
{
struct fixture *f = context;
unsigned i;
char buffer[128];
for (i = 0; i < 100; i++) {
snprintf(buffer, sizeof(buffer), "/dev/imaginary-%u", i);
io_processor_add(f->iop, buffer, _rand(10000), _rand(100), NULL);
}
}
static unsigned min(unsigned lhs, unsigned rhs)
{
if (lhs < rhs)
return lhs;
if (rhs < lhs)
return rhs;
return lhs;
}
static void check_batches(struct fixture *f, unsigned nr_areas, unsigned batch_size)
{
unsigned i, b, nr_batches;
const char *path = "/dev/foo-1";
f->mops->batch_size = batch_size;
_expect(f->mops, M_BATCH_SIZE);
for (i = 0; i < nr_areas; i++)
io_processor_add(f->iop, path, 0, 128, f->mops);
nr_batches = (nr_areas + (batch_size - 1)) / batch_size;
for (b = 0; b < nr_batches; b++) {
unsigned count = min(nr_areas - (b * batch_size), batch_size);
for (i = 0; i < count; i++) {
_expect(f->mops, M_GET_DEV);
_expect(f->mops, M_PREFETCH);
_expect(f->mops, M_PUT_DEV);
}
for (i = 0; i < count; i++) {
_expect(f->mops, M_GET_DEV);
_expect(f->mops, M_READ);
_expect(f->mops, M_PUT_DEV);
_expect(f->mops, M_TASK);
}
}
io_processor_exec(f->iop);
}
static void _test_area_vs_batch_size(void *context)
{
struct fixture *f = context;
check_batches(f, 2, 1);
check_batches(f, 2, 2);
check_batches(f, 128, 4);
check_batches(f, 512, 1024);
}
static void _test_get_fails(void *context)
{
struct fixture *f = context;
const char *path = "/dev/foo-1";
io_processor_add(f->iop, path, 0, 128, f->mops);
_expect(f->mops, M_BATCH_SIZE);
_expect_fail(f->mops, M_GET_DEV);
_expect(f->mops, M_ERROR);
io_processor_exec(f->iop);
}
static void _test_second_get_dev_fails(void *context)
{
struct fixture *f = context;
const char *path = "/dev/foo-1";
io_processor_add(f->iop, path, 0, 128, f->mops);
_expect(f->mops, M_BATCH_SIZE);
_expect(f->mops, M_GET_DEV);
_expect(f->mops, M_PREFETCH);
_expect(f->mops, M_PUT_DEV);
_expect_fail(f->mops, M_GET_DEV);
_expect(f->mops, M_ERROR);
io_processor_exec(f->iop);
}
static void _test_read_fails(void *context)
{
struct fixture *f = context;
const char *path = "/dev/foo-1";
io_processor_add(f->iop, path, 0, 128, f->mops);
_expect(f->mops, M_BATCH_SIZE);
_expect(f->mops, M_GET_DEV);
_expect(f->mops, M_PREFETCH);
_expect(f->mops, M_PUT_DEV);
_expect(f->mops, M_GET_DEV);
_expect_fail(f->mops, M_READ);
_expect(f->mops, M_PUT_DEV);
_expect(f->mops, M_ERROR);
io_processor_exec(f->iop);
}
static void _test_one_bad_one_good(void *context)
{
struct fixture *f = context;
const char *path1 = "/dev/foo-1";
const char *path2 = "/dev/foo-2";
io_processor_add(f->iop, path1, 0, 128, f->mops);
io_processor_add(f->iop, path2, 0, 128, f->mops);
f->mops->batch_size = 2;
_expect(f->mops, M_BATCH_SIZE);
_expect_fail(f->mops, M_GET_DEV);
_expect(f->mops, M_ERROR);
_expect(f->mops, M_GET_DEV);
_expect(f->mops, M_PREFETCH);
_expect(f->mops, M_PUT_DEV);
_expect(f->mops, M_GET_DEV);
_expect(f->mops, M_READ);
_expect(f->mops, M_PUT_DEV);
_expect(f->mops, M_TASK);
io_processor_exec(f->iop);
}
static void _test_one_good_one_bad(void *context)
{
struct fixture *f = context;
const char *path1 = "/dev/foo-1";
const char *path2 = "/dev/foo-2";
io_processor_add(f->iop, path1, 0, 128, f->mops);
io_processor_add(f->iop, path2, 0, 128, f->mops);
f->mops->batch_size = 2;
_expect(f->mops, M_BATCH_SIZE);
_expect(f->mops, M_GET_DEV);
_expect(f->mops, M_PREFETCH);
_expect(f->mops, M_PUT_DEV);
_expect_fail(f->mops, M_GET_DEV);
_expect(f->mops, M_ERROR);
_expect(f->mops, M_GET_DEV);
_expect(f->mops, M_READ);
_expect(f->mops, M_PUT_DEV);
_expect(f->mops, M_TASK);
io_processor_exec(f->iop);
}
//----------------------------------------------------------------
static struct test_suite *_tests(void)
{
struct test_suite *ts = test_suite_create(_fix_init, _fix_exit);
if (!ts) {
fprintf(stderr, "out of memory\n");
exit(1);
}
#define T(path, desc, fn) register_test(ts, "/base/device/io-processor/" path, desc, fn)
T("create-destroy", "empty test", _test_create_destroy);
T("create-add-destroy", "add jobs, but don't run them", _test_add_but_no_run);
T("areas-vs-batch-size", "process different nrs of areas vs batch size", _test_area_vs_batch_size);
T("get-fails", "get failure is propogated", _test_get_fails);
T("get-fails-second", "second get failure is propogated", _test_second_get_dev_fails);
T("read-fails", "read failure is propogated", _test_read_fails);
T("one-bad-one-good", "one bad, one good", _test_one_bad_one_good);
T("one-good-one-bad", "one good, one bad", _test_one_good_one_bad);
#undef T
return ts;
}
void io_processor_tests(struct dm_list *all_tests)
{
dm_list_add(all_tests, &_tests()->list);
}
//----------------------------------------------------------------

View File

@@ -1,210 +0,0 @@
/*
* Copyright (C) 2018 Red Hat, Inc. All rights reserved.
*
* This file is part of LVM2.
*
* This copyrighted material is made available to anyone wishing to use,
* modify, copy, or redistribute it subject to the terms and conditions
* of the GNU General Public License v.2.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
#include "units.h"
#include "lib/device/bcache.h"
#include <errno.h>
#include <stdio.h>
#include <stdlib.h>
#include <sys/types.h>
#include <sys/stat.h>
#include <fcntl.h>
#include <unistd.h>
//----------------------------------------------------------------
#define SECTOR_SIZE 512
#define BLOCK_SIZE_SECTORS 8
#define NR_BLOCKS 64
struct fixture {
struct io_engine *e;
uint8_t *data;
char fname[64];
int fd;
};
static void _fill_buffer(uint8_t *buffer, uint8_t seed, size_t count)
{
unsigned i;
uint8_t b = seed;
for (i = 0; i < count; i++) {
buffer[i] = b;
b = ((b << 5) + b) + i;
}
}
static void _check_buffer(uint8_t *buffer, uint8_t seed, size_t count)
{
unsigned i;
uint8_t b = seed;
for (i = 0; i < count; i++) {
T_ASSERT_EQUAL(buffer[i], b);
b = ((b << 5) + b) + i;
}
}
static void _print_buffer(const char *name, uint8_t *buffer, size_t count)
{
unsigned col;
fprintf(stderr, "%s:\n", name);
while (count) {
for (col = 0; count && col < 20; col++) {
fprintf(stderr, "%x, ", (unsigned) *buffer);
col++;
buffer++;
count--;
}
fprintf(stderr, "\n");
}
}
static void *_fix_init(void)
{
struct fixture *f = malloc(sizeof(*f));
T_ASSERT(f);
f->e = create_async_io_engine();
T_ASSERT(f->e);
if (posix_memalign((void **) &f->data, 4096, SECTOR_SIZE * BLOCK_SIZE_SECTORS))
test_fail("posix_memalign failed");
snprintf(f->fname, sizeof(f->fname), "unit-test-XXXXXX");
f->fd = mkstemp(f->fname);
T_ASSERT(f->fd >= 0);
_fill_buffer(f->data, 123, SECTOR_SIZE * BLOCK_SIZE_SECTORS);
T_ASSERT(write(f->fd, f->data, SECTOR_SIZE * BLOCK_SIZE_SECTORS) > 0);
T_ASSERT(lseek(f->fd, 0, SEEK_SET) != -1);
return f;
}
static void _fix_exit(void *fixture)
{
struct fixture *f = fixture;
close(f->fd);
unlink(f->fname);
free(f->data);
if (f->e)
f->e->destroy(f->e);
free(f);
}
static void _test_create(void *fixture)
{
// empty
}
struct io {
bool completed;
int error;
};
static void _io_init(struct io *io)
{
io->completed = false;
io->error = 0;
}
static void _complete_io(void *context, int io_error)
{
struct io *io = context;
io->completed = true;
io->error = io_error;
}
static void _test_read(void *fixture)
{
struct fixture *f = fixture;
struct io io;
_io_init(&io);
T_ASSERT(f->e->issue(f->e, DIR_READ, f->fd, 0, BLOCK_SIZE_SECTORS, f->data, &io));
T_ASSERT(f->e->wait(f->e, _complete_io));
T_ASSERT(io.completed);
T_ASSERT(!io.error);
_check_buffer(f->data, 123, sizeof(f->data));
}
static void _test_write(void *fixture)
{
struct fixture *f = fixture;
struct io io;
_io_init(&io);
T_ASSERT(f->e->issue(f->e, DIR_WRITE, f->fd, 0, BLOCK_SIZE_SECTORS, f->data, &io));
T_ASSERT(f->e->wait(f->e, _complete_io));
T_ASSERT(io.completed);
T_ASSERT(!io.error);
}
static void _test_write_bytes(void *fixture)
{
struct fixture *f = fixture;
unsigned offset = 345;
char buf_out[32];
char buf_in[32];
struct bcache *cache = bcache_create(8, BLOCK_SIZE_SECTORS, f->e);
T_ASSERT(cache);
// T_ASSERT(bcache_read_bytes(cache, f->fd, offset, sizeof(buf_in), buf_in));
_fill_buffer((uint8_t *) buf_out, 234, sizeof(buf_out));
T_ASSERT(bcache_write_bytes(cache, f->fd, offset, sizeof(buf_out), buf_out));
T_ASSERT(bcache_read_bytes(cache, f->fd, offset, sizeof(buf_in), buf_in));
_print_buffer("buf_out", (uint8_t *) buf_out, sizeof(buf_out));
_print_buffer("buf_in", (uint8_t *) buf_in, sizeof(buf_in));
T_ASSERT(!memcmp(buf_out, buf_in, sizeof(buf_out)));
bcache_destroy(cache);
f->e = NULL; // already destroyed
}
//----------------------------------------------------------------
#define T(path, desc, fn) register_test(ts, "/base/device/bcache/io-engine/" path, desc, fn)
static struct test_suite *_tests(void)
{
struct test_suite *ts = test_suite_create(_fix_init, _fix_exit);
if (!ts) {
fprintf(stderr, "out of memory\n");
exit(1);
}
T("create-destroy", "simple create/destroy", _test_create);
T("read", "read sanity check", _test_read);
T("write", "write sanity check", _test_write);
T("bcache-write-bytes", "test the utility fns", _test_write_bytes);
return ts;
}
void io_engine_tests(struct dm_list *all_tests)
{
dm_list_add(all_tests, &_tests()->list);
}

View File

@@ -21,13 +21,13 @@
// Declare the function that adds tests suites here ...
void activation_generator_tests(struct dm_list *suites);
void bcache_tests(struct dm_list *suites);
void bcache_utils_tests(struct dm_list *suites);
void bitset_tests(struct dm_list *suites);
void config_tests(struct dm_list *suites);
void dm_list_tests(struct dm_list *suites);
void dm_status_tests(struct dm_list *suites);
void io_engine_tests(struct dm_list *suites);
void io_manager_tests(struct dm_list *suites);
void io_manager_utils_tests(struct dm_list *suites);
void io_processor_tests(struct dm_list *suites);
void percent_tests(struct dm_list *suites);
void radix_tree_tests(struct dm_list *suites);
void regex_tests(struct dm_list *suites);
@@ -38,13 +38,13 @@ void vdo_tests(struct dm_list *suites);
static inline void register_all_tests(struct dm_list *suites)
{
activation_generator_tests(suites);
bcache_tests(suites);
bcache_utils_tests(suites);
bitset_tests(suites);
config_tests(suites);
dm_list_tests(suites);
dm_status_tests(suites);
io_engine_tests(suites);
io_manager_tests(suites);
io_manager_utils_tests(suites);
io_processor_tests(suites);
percent_tests(suites);
radix_tree_tests(suites);
regex_tests(suites);