Introduce image names and metadata storage in etcd

Each inode has: image name, parent inode number & pool, size and readonly flag

Snapshots are created by switching image name to a different inode number
while using the older inode as parent.
rdma-zerocopy
Vitaliy Filippov 2021-02-10 01:21:57 +03:00
parent ab39ce2bbb
commit 2612d3198a
9 changed files with 301 additions and 70 deletions

View File

@ -35,6 +35,8 @@ breaking changes in the future. However, the following is implemented:
- Inode removal tool (vitastor-rm)
- Packaging for Debian and CentOS
- Per-inode I/O and space usage statistics
- Inode metadata storage in etcd
- Snapshots and copy-on-write image clones
## Roadmap
@ -42,8 +44,6 @@ breaking changes in the future. However, the following is implemented:
- Other administrative tools
- Proxmox and OpenNebula plugins
- iSCSI proxy
- Inode metadata storage in etcd
- Snapshots and copy-on-write image clones
- Operation timeouts and better failure detection
- Scrubbing without checksums (verification of replicas)
- Checksums

View File

@ -149,6 +149,7 @@ const etcd_tree = {
<pool_id>: {
<inode_t>: {
name: string,
size?: uint64_t, // bytes
parent_pool?: <pool_id>,
parent_id?: <inode_t>,
readonly?: boolean,

View File

@ -11,6 +11,11 @@
etcd_state_client_t::~etcd_state_client_t()
{
for (auto watch: watches)
{
delete watch;
}
watches.clear();
etcd_watches_initialised = -1;
#ifndef __MOCK__
if (etcd_watch_ws)
@ -662,6 +667,22 @@ void etcd_state_client_t::parse_state(const std::string & key, const json11::Jso
else
{
inode_num |= (pool_id << (64-POOL_ID_BITS));
auto it = this->inode_config.find(inode_num);
if (it != this->inode_config.end() && it->second.name != "")
{
auto n_it = this->inode_by_name.find(it->second.name);
if (n_it->second == inode_num)
{
this->inode_by_name.erase(n_it);
for (auto w: watches)
{
if (w->name == it->second.name)
{
w->cfg = { 0 };
}
}
}
}
if (!value.is_object())
{
this->inode_config.erase(inode_num);
@ -685,12 +706,52 @@ void etcd_state_client_t::parse_state(const std::string & key, const json11::Jso
else
parent_inode_num |= parent_pool_id << (64-POOL_ID_BITS);
}
this->inode_config[inode_num] = (inode_config_t){
inode_config_t cfg = (inode_config_t){
.num = inode_num,
.name = value["name"].string_value(),
.size = value["size"].uint64_value(),
.parent_id = parent_inode_num,
.readonly = value["readonly"].bool_value(),
};
this->inode_config[inode_num] = cfg;
if (cfg.name != "")
{
this->inode_by_name[cfg.name] = inode_num;
for (auto w: watches)
{
if (w->name == value["name"].string_value())
{
w->cfg = cfg;
}
}
}
}
}
}
}
inode_watch_t* etcd_state_client_t::watch_inode(std::string name)
{
inode_watch_t *watch = new inode_watch_t;
watch->name = name;
watches.push_back(watch);
auto it = inode_by_name.find(name);
if (it != inode_by_name.end())
{
watch->cfg = inode_config[it->second];
}
return watch;
}
void etcd_state_client_t::close_watch(inode_watch_t* watch)
{
for (int i = 0; i < watches.size(); i++)
{
if (watches[i] == watch)
{
watches.erase(watches.begin()+i, watches.begin()+i+1);
break;
}
}
delete watch;
}

View File

@ -54,16 +54,25 @@ struct pool_config_t
struct inode_config_t
{
uint64_t num;
std::string name;
uint64_t size;
inode_t parent_id;
bool readonly;
};
struct inode_watch_t
{
std::string name;
inode_config_t cfg;
};
struct websocket_t;
struct etcd_state_client_t
{
protected:
std::vector<inode_watch_t*> watches;
websocket_t *etcd_watch_ws = NULL;
uint64_t bs_block_size = DEFAULT_BLOCK_SIZE;
void add_etcd_url(std::string);
@ -78,6 +87,7 @@ public:
std::map<pool_id_t, pool_config_t> pool_config;
std::map<osd_num_t, json11::Json> peer_states;
std::map<inode_t, inode_config_t> inode_config;
std::map<std::string, inode_t> inode_by_name;
std::function<void(json11::Json::object &)> on_change_hook;
std::function<void(json11::Json::object &)> on_load_config_hook;
@ -95,5 +105,7 @@ public:
void parse_state(const json_kv_t & kv);
void parse_state(const std::string & key, const json11::Json & value);
void parse_config(json11::Json & config);
inode_watch_t* watch_inode(std::string name);
void close_watch(inode_watch_t* watch);
~etcd_state_client_t();
};

View File

@ -6,17 +6,17 @@
// Random write:
//
// fio -thread -ioengine=./libfio_cluster.so -name=test -bs=4k -direct=1 -fsync=16 -iodepth=16 -rw=randwrite \
// -etcd=127.0.0.1:2379 [-etcd_prefix=/vitastor] -pool=1 -inode=1 -size=1000M
// -etcd=127.0.0.1:2379 [-etcd_prefix=/vitastor] (-image=testimg | -pool=1 -inode=1 -size=1000M)
//
// Linear write:
//
// fio -thread -ioengine=./libfio_cluster.so -name=test -bs=128k -direct=1 -fsync=32 -iodepth=32 -rw=write \
// -etcd=127.0.0.1:2379 [-etcd_prefix=/vitastor] -pool=1 -inode=1 -size=1000M
// -etcd=127.0.0.1:2379 [-etcd_prefix=/vitastor] -image=testimg
//
// Random read (run with -iodepth=32 or -iodepth=1):
//
// fio -thread -ioengine=./libfio_cluster.so -name=test -bs=4k -direct=1 -iodepth=32 -rw=randread \
// -etcd=127.0.0.1:2379 [-etcd_prefix=/vitastor] -pool=1 -inode=1 -size=1000M
// -etcd=127.0.0.1:2379 [-etcd_prefix=/vitastor] -image=testimg
#include <sys/types.h>
#include <sys/socket.h>
@ -35,6 +35,7 @@ struct sec_data
ring_loop_t *ringloop = NULL;
epoll_manager_t *epmgr = NULL;
cluster_client_t *cli = NULL;
inode_watch_t *watch = NULL;
bool last_sync = false;
/* The list of completed io_u structs. */
std::vector<io_u*> completed;
@ -47,6 +48,7 @@ struct sec_options
int __pad;
char *etcd_host = NULL;
char *etcd_prefix = NULL;
char *image = NULL;
uint64_t pool = 0;
uint64_t inode = 0;
int cluster_log = 0;
@ -64,7 +66,7 @@ static struct fio_option options[] = {
.group = FIO_OPT_G_FILENAME,
},
{
.name = "etcd",
.name = "etcd_prefix",
.lname = "etcd key prefix",
.type = FIO_OPT_STR_STORE,
.off1 = offsetof(struct sec_options, etcd_prefix),
@ -72,6 +74,15 @@ static struct fio_option options[] = {
.category = FIO_OPT_C_ENGINE,
.group = FIO_OPT_G_FILENAME,
},
{
.name = "image",
.lname = "Vitastor image name",
.type = FIO_OPT_STR_STORE,
.off1 = offsetof(struct sec_options, image),
.help = "Vitastor image name to run tests on",
.category = FIO_OPT_C_ENGINE,
.group = FIO_OPT_G_FILENAME,
},
{
.name = "pool",
.lname = "pool number for the inode",
@ -86,7 +97,7 @@ static struct fio_option options[] = {
.lname = "inode to run tests on",
.type = FIO_OPT_INT,
.off1 = offsetof(struct sec_options, inode),
.help = "inode to run tests on (1 by default)",
.help = "inode number to run tests on",
.category = FIO_OPT_C_ENGINE,
.group = FIO_OPT_G_FILENAME,
},
@ -141,6 +152,51 @@ static int sec_setup(struct thread_data *td)
td->o.open_files++;
}
json11::Json cfg = json11::Json::object {
{ "etcd_address", std::string(o->etcd_host) },
{ "etcd_prefix", std::string(o->etcd_prefix ? o->etcd_prefix : "/vitastor") },
{ "log_level", o->cluster_log },
};
if (!o->image)
{
if (!(o->inode & ((1l << (64-POOL_ID_BITS)) - 1)))
{
td_verror(td, EINVAL, "inode number is missing");
return 1;
}
if (o->pool)
{
o->inode = (o->inode & ((1l << (64-POOL_ID_BITS)) - 1)) | (o->pool << (64-POOL_ID_BITS));
}
if (!(o->inode >> (64-POOL_ID_BITS)))
{
td_verror(td, EINVAL, "pool is missing");
return 1;
}
}
else
{
o->inode = 0;
}
bsd->ringloop = new ring_loop_t(512);
bsd->epmgr = new epoll_manager_t(bsd->ringloop);
bsd->cli = new cluster_client_t(bsd->ringloop, bsd->epmgr->tfd, cfg);
if (o->image)
{
while (!bsd->cli->is_ready())
{
bsd->ringloop->loop();
if (bsd->cli->is_ready())
break;
bsd->ringloop->wait();
}
bsd->watch = bsd->cli->st_cli.watch_inode(std::string(o->image));
td->files[0]->real_file_size = bsd->watch->cfg.size;
}
bsd->trace = o->trace ? true : false;
return 0;
}
@ -149,6 +205,10 @@ static void sec_cleanup(struct thread_data *td)
sec_data *bsd = (sec_data*)td->io_ops_data;
if (bsd)
{
if (bsd->watch)
{
bsd->cli->st_cli.close_watch(bsd->watch);
}
delete bsd->cli;
delete bsd->epmgr;
delete bsd->ringloop;
@ -159,28 +219,6 @@ static void sec_cleanup(struct thread_data *td)
/* Connect to the server from each thread. */
static int sec_init(struct thread_data *td)
{
sec_options *o = (sec_options*)td->eo;
sec_data *bsd = (sec_data*)td->io_ops_data;
json11::Json cfg = json11::Json::object {
{ "etcd_address", std::string(o->etcd_host) },
{ "etcd_prefix", std::string(o->etcd_prefix ? o->etcd_prefix : "/vitastor") },
{ "log_level", o->cluster_log },
};
if (o->pool)
o->inode = (o->inode & ((1l << (64-POOL_ID_BITS)) - 1)) | (o->pool << (64-POOL_ID_BITS));
if (!(o->inode >> (64-POOL_ID_BITS)))
{
td_verror(td, EINVAL, "pool is missing");
return 1;
}
bsd->ringloop = new ring_loop_t(512);
bsd->epmgr = new epoll_manager_t(bsd->ringloop);
bsd->cli = new cluster_client_t(bsd->ringloop, bsd->epmgr->tfd, cfg);
bsd->trace = o->trace ? true : false;
return 0;
}
@ -200,19 +238,23 @@ static enum fio_q_status sec_queue(struct thread_data *td, struct io_u *io)
io->engine_data = bsd;
cluster_op_t *op = new cluster_op_t;
op->inode = opt->image ? bsd->watch->cfg.num : opt->inode;
switch (io->ddir)
{
case DDIR_READ:
op->opcode = OSD_OP_READ;
op->inode = opt->inode;
op->offset = io->offset;
op->len = io->xfer_buflen;
op->iov.push_back(io->xfer_buf, io->xfer_buflen);
bsd->last_sync = false;
break;
case DDIR_WRITE:
if (opt->image && bsd->watch->cfg.readonly)
{
io->error = EROFS;
return FIO_Q_COMPLETED;
}
op->opcode = OSD_OP_WRITE;
op->inode = opt->inode;
op->offset = io->offset;
op->len = io->xfer_buflen;
op->iov.push_back(io->xfer_buf, io->xfer_buflen);

View File

@ -39,12 +39,14 @@ void DSO_STAMP_FUN(void)
typedef struct VitastorClient
{
void *proxy;
void *watch;
char *etcd_host;
char *etcd_prefix;
char *image;
uint64_t inode;
uint64_t pool;
uint64_t size;
int readonly;
long readonly;
QemuMutex mutex;
} VitastorClient;
@ -53,10 +55,14 @@ typedef struct VitastorRPC
BlockDriverState *bs;
Coroutine *co;
QEMUIOVector *iov;
int ret;
long ret;
int complete;
} VitastorRPC;
static void vitastor_co_init_task(BlockDriverState *bs, VitastorRPC *task);
static void vitastor_co_generic_bh_cb(long retval, void *opaque);
static void vitastor_close(BlockDriverState *bs);
static char *qemu_rbd_next_tok(char *src, char delim, char **p)
{
char *end;
@ -132,22 +138,25 @@ static void vitastor_parse_filename(const char *filename, QDict *options, Error
qdict_put_str(options, name, value);
}
}
if (!qdict_get_try_str(options, "image"))
{
if (!qdict_get_try_int(options, "inode", 0))
{
error_setg(errp, "inode is missing");
error_setg(errp, "one of image (name) and inode (number) must be specified");
goto out;
}
if (!(qdict_get_try_int(options, "inode", 0) >> (64-POOL_ID_BITS)) &&
!qdict_get_try_int(options, "pool", 0))
{
error_setg(errp, "pool number is missing");
error_setg(errp, "pool number must be specified or included in the inode number");
goto out;
}
if (!qdict_get_try_int(options, "size", 0))
{
error_setg(errp, "size is missing");
error_setg(errp, "size must be specified when inode number is used instead of image name");
goto out;
}
}
if (!qdict_get_str(options, "etcd_host"))
{
error_setg(errp, "etcd_host is missing");
@ -159,27 +168,86 @@ out:
return;
}
static void coroutine_fn vitastor_co_get_metadata(VitastorRPC *task)
{
BlockDriverState *bs = task->bs;
VitastorClient *client = bs->opaque;
task->co = qemu_coroutine_self();
qemu_mutex_lock(&client->mutex);
vitastor_proxy_watch_metadata(client->proxy, client->image, vitastor_co_generic_bh_cb, task);
qemu_mutex_unlock(&client->mutex);
while (!task->complete)
{
qemu_coroutine_yield();
}
}
static int vitastor_file_open(BlockDriverState *bs, QDict *options, int flags, Error **errp)
{
VitastorClient *client = bs->opaque;
int64_t ret = 0;
qemu_mutex_init(&client->mutex);
client->etcd_host = g_strdup(qdict_get_try_str(options, "etcd_host"));
client->etcd_prefix = g_strdup(qdict_get_try_str(options, "etcd_prefix"));
client->proxy = vitastor_proxy_create(bdrv_get_aio_context(bs), client->etcd_host, client->etcd_prefix);
client->image = g_strdup(qdict_get_try_str(options, "image"));
client->readonly = (flags & BDRV_O_RDWR) ? 1 : 0;
if (client->image)
{
// Get image metadata (size and readonly flag)
VitastorRPC task;
task.complete = 0;
task.bs = bs;
if (qemu_in_coroutine())
{
vitastor_co_get_metadata(&task);
}
else
{
assert(qemu_get_current_aio_context() == qemu_get_aio_context());
qemu_coroutine_enter(qemu_coroutine_create((void(*)(void*))vitastor_co_get_metadata, &task));
}
BDRV_POLL_WHILE(bs, !task.complete);
client->watch = (void*)task.ret;
client->readonly = client->readonly || vitastor_proxy_get_readonly(client->watch);
client->size = vitastor_proxy_get_size(client->watch);
if (!vitastor_proxy_get_inode_num(client->watch))
{
error_setg(errp, "image does not exist");
vitastor_close(bs);
}
if (!client->size)
{
client->size = qdict_get_int(options, "size");
}
}
else
{
client->watch = NULL;
client->inode = qdict_get_int(options, "inode");
client->pool = qdict_get_int(options, "pool");
if (client->pool)
{
client->inode = (client->inode & ((1l << (64-POOL_ID_BITS)) - 1)) | (client->pool << (64-POOL_ID_BITS));
}
client->size = qdict_get_int(options, "size");
client->readonly = (flags & BDRV_O_RDWR) ? 1 : 0;
client->proxy = vitastor_proxy_create(bdrv_get_aio_context(bs), client->etcd_host, client->etcd_prefix);
//client->aio_context = bdrv_get_aio_context(bs);
}
if (!client->size)
{
error_setg(errp, "image size not specified");
vitastor_close(bs);
return -1;
}
bs->total_sectors = client->size / BDRV_SECTOR_SIZE;
//client->aio_context = bdrv_get_aio_context(bs);
qdict_del(options, "etcd_host");
qdict_del(options, "etcd_prefix");
qdict_del(options, "image");
qdict_del(options, "inode");
qdict_del(options, "pool");
qdict_del(options, "size");
qemu_mutex_init(&client->mutex);
return ret;
}
@ -191,6 +259,8 @@ static void vitastor_close(BlockDriverState *bs)
g_free(client->etcd_host);
if (client->etcd_prefix)
g_free(client->etcd_prefix);
if (client->image)
g_free(client->image);
}
#if QEMU_VERSION_MAJOR >= 3
@ -296,7 +366,7 @@ static void vitastor_co_init_task(BlockDriverState *bs, VitastorRPC *task)
};
}
static void vitastor_co_generic_bh_cb(int retval, void *opaque)
static void vitastor_co_generic_bh_cb(long retval, void *opaque)
{
VitastorRPC *task = opaque;
task->ret = retval;
@ -319,8 +389,9 @@ static int coroutine_fn vitastor_co_preadv(BlockDriverState *bs, uint64_t offset
vitastor_co_init_task(bs, &task);
task.iov = iov;
uint64_t inode = client->watch ? vitastor_proxy_get_inode_num(client->watch) : client->inode;
qemu_mutex_lock(&client->mutex);
vitastor_proxy_rw(0, client->proxy, client->inode, offset, bytes, iov->iov, iov->niov, vitastor_co_generic_bh_cb, &task);
vitastor_proxy_rw(0, client->proxy, inode, offset, bytes, iov->iov, iov->niov, vitastor_co_generic_bh_cb, &task);
qemu_mutex_unlock(&client->mutex);
while (!task.complete)
@ -338,8 +409,9 @@ static int coroutine_fn vitastor_co_pwritev(BlockDriverState *bs, uint64_t offse
vitastor_co_init_task(bs, &task);
task.iov = iov;
uint64_t inode = client->watch ? vitastor_proxy_get_inode_num(client->watch) : client->inode;
qemu_mutex_lock(&client->mutex);
vitastor_proxy_rw(1, client->proxy, client->inode, offset, bytes, iov->iov, iov->niov, vitastor_co_generic_bh_cb, &task);
vitastor_proxy_rw(1, client->proxy, inode, offset, bytes, iov->iov, iov->niov, vitastor_co_generic_bh_cb, &task);
qemu_mutex_unlock(&client->mutex);
while (!task.complete)

View File

@ -126,4 +126,38 @@ void vitastor_proxy_sync(void *client, VitastorIOHandler cb, void *opaque)
p->cli->execute(op);
}
void vitastor_proxy_watch_metadata(void *client, char *image, VitastorIOHandler cb, void *opaque)
{
QemuProxy *p = (QemuProxy*)client;
p->cli->on_ready([&]()
{
auto watch = p->cli->st_cli.watch_inode(std::string(image));
cb((long)watch, opaque);
});
}
void vitastor_proxy_close_watch(void *client, void *watch)
{
QemuProxy *p = (QemuProxy*)client;
p->cli->st_cli.close_watch((inode_watch_t*)watch);
}
uint64_t vitastor_proxy_get_size(void *watch_ptr)
{
inode_watch_t *watch = (inode_watch_t*)watch_ptr;
return watch->cfg.size;
}
uint64_t vitastor_proxy_get_inode_num(void *watch_ptr)
{
inode_watch_t *watch = (inode_watch_t*)watch_ptr;
return watch->cfg.num;
}
int vitastor_proxy_get_readonly(void *watch_ptr)
{
inode_watch_t *watch = (inode_watch_t*)watch_ptr;
return watch->cfg.readonly;
}
}

View File

@ -15,12 +15,17 @@ extern "C" {
#endif
// Our exports
typedef void VitastorIOHandler(int retval, void *opaque);
typedef void VitastorIOHandler(long retval, void *opaque);
void* vitastor_proxy_create(AioContext *ctx, const char *etcd_host, const char *etcd_prefix);
void vitastor_proxy_destroy(void *client);
void vitastor_proxy_rw(int write, void *client, uint64_t inode, uint64_t offset, uint64_t len,
struct iovec *iov, int iovcnt, VitastorIOHandler cb, void *opaque);
void vitastor_proxy_sync(void *client, VitastorIOHandler cb, void *opaque);
void vitastor_proxy_watch_metadata(void *client, char *image, VitastorIOHandler cb, void *opaque);
void vitastor_proxy_close_watch(void *client, void *watch);
uint64_t vitastor_proxy_get_size(void *watch);
uint64_t vitastor_proxy_get_inode_num(void *watch);
int vitastor_proxy_get_readonly(void *watch);
#ifdef __cplusplus
}

View File

@ -38,14 +38,18 @@ fi
# Test basic write and snapshot
LD_PRELOAD=libasan.so.5 \
fio -thread -name=test -ioengine=build/src/libfio_vitastor.so -bs=4M -direct=1 -iodepth=1 -fsync=1 -rw=write -etcd=$ETCD_URL -pool=1 -inode=2 -size=32M -cluster_log_level=10
$ETCDCTL put /vitastor/config/inode/1/2 '{"name":"testimg@0"}'
$ETCDCTL put /vitastor/config/inode/1/3 '{"parent_id":2,"name":"testimg"}'
$ETCDCTL put /vitastor/config/inode/1/2 '{"name":"testimg","size":'$((32*1024*1024))'}'
LD_PRELOAD=libasan.so.5 \
fio -thread -name=test -ioengine=build/src/libfio_vitastor.so -bs=4k -direct=1 -iodepth=1 -fsync=32 -buffer_pattern=0xdeadface -rw=randwrite -etcd=$ETCD_URL -pool=1 -inode=3 -size=32M -number_ios=1024
fio -thread -name=test -ioengine=build/src/libfio_vitastor.so -bs=4M -direct=1 -iodepth=1 -fsync=1 -rw=write \
-etcd=$ETCD_URL -pool=1 -inode=2 -size=32M -cluster_log_level=10
$ETCDCTL put /vitastor/config/inode/1/2 '{"name":"testimg@0","size":'$((32*1024*1024))'}'
$ETCDCTL put /vitastor/config/inode/1/3 '{"parent_id":2,"name":"testimg","size":'$((32*1024*1024))'}'
LD_PRELOAD=libasan.so.5 \
fio -thread -name=test -ioengine=build/src/libfio_vitastor.so -bs=4k -direct=1 -iodepth=1 -fsync=32 -buffer_pattern=0xdeadface \
-rw=randwrite -etcd=$ETCD_URL -image=testimg -number_ios=1024
LD_PRELOAD=libasan.so.5 \
fio -thread -name=test -ioengine=build/src/libfio_vitastor.so -bs=4M -direct=1 -iodepth=1 -rw=read -etcd=$ETCD_URL -pool=1 -inode=3 -size=32M
@ -55,13 +59,13 @@ qemu-img convert -S 4096 -p \
-O raw ./testdata/merged.bin
qemu-img convert -S 4096 -p \
-f raw "vitastor:etcd_host=127.0.0.1\:$ETCD_PORT/v3:pool=1:inode=2:size=$((32*1024*1024))" \
-f raw "vitastor:etcd_host=127.0.0.1\:$ETCD_PORT/v3:image=testimg@0" \
-O raw ./testdata/layer0.bin
$ETCDCTL put /vitastor/config/inode/1/3 '{"name":"testimg"}'
$ETCDCTL put /vitastor/config/inode/1/3 '{"name":"testimg","size":'$((32*1024*1024))'}'
qemu-img convert -S 4096 -p \
-f raw "vitastor:etcd_host=127.0.0.1\:$ETCD_PORT/v3:pool=1:inode=3:size=$((32*1024*1024))" \
-f raw "vitastor:etcd_host=127.0.0.1\:$ETCD_PORT/v3:image=testimg" \
-O raw ./testdata/layer1.bin
node mon/merge.js ./testdata/layer0.bin ./testdata/layer1.bin ./testdata/check.bin