src/Makefile | 5 -
src/direct.c | 89 --------------------------
src/direct_lib.c | 107 ++++++++++++++++++++++++++++++++
src/diskio.c | 163 +++++++++++++++++++++++++++++++++----------------
src/host_id.c | 3
src/main.c | 71 ++++++++++++++-------
src/sanlock_internal.h | 18 +++++
src/token_manager.c | 3
tests/Makefile | 2
tests/devcount.c | 4 -
10 files changed, 294 insertions(+), 171 deletions(-)
New commits:
commit dc2b3c5982f7c46ebf93dc3ef58c894fb626c7af
Author: David Teigland <teigland(a)redhat.com>
Date: Tue May 17 15:58:15 2011 -0500
sanlock: aio changes
Move iocb structs off stack, into task struct, and don't reuse an
iocb until we reap an event for it. Also add task names so aio log
messages can show which tasks have errors.
Also enable both linux and posix aio (default linux). If one has
problems in a given situation, the other may not.
diff --git a/src/Makefile b/src/Makefile
index cade592..a93ced1 100644
--- a/src/Makefile
+++ b/src/Makefile
@@ -63,10 +63,9 @@ CFLAGS += -D_GNU_SOURCE -g \
-Wp,-D_FORTIFY_SOURCE=2 \
-fexceptions \
-fasynchronous-unwind-tables \
- -fdiagnostics-show-option \
- -DLINUX_AIO
+ -fdiagnostics-show-option
-CMD_LDFLAGS = -lpthread -laio -lblkid -lsanlock -lwdmd
+CMD_LDFLAGS = -lpthread -lrt -laio -lblkid -lsanlock -lwdmd
all: $(SHLIB_TARGET) $(CMD_TARGET) $(SHLIB2_TARGET)
diff --git a/src/direct.c b/src/direct.c
index 5ce625d..a95750d 100644
--- a/src/direct.c
+++ b/src/direct.c
@@ -29,7 +29,6 @@
#include "direct.h"
#include "paxos_lease.h"
#include "delta_lease.h"
-#include "sanlock_direct.h"
/*
* cli: sanlock direct init
@@ -78,17 +77,6 @@
* delta_lease_renew()
*/
-
-/* TODO: include from sanlock_internal */
-static struct task task_default = {
- DEFAULT_USE_AIO,
- DEFAULT_IO_TIMEOUT_SECONDS,
- DEFAULT_HOST_ID_TIMEOUT_SECONDS,
- DEFAULT_HOST_ID_RENEWAL_SECONDS,
- DEFAULT_HOST_ID_RENEWAL_FAIL_SECONDS,
- DEFAULT_HOST_ID_RENEWAL_WARN_SECONDS,
- 0 };
-
static int do_paxos_action(int action, struct task *task,
struct sanlk_resource *res,
int max_hosts, int num_hosts,
@@ -313,31 +301,6 @@ int direct_read_id(struct task *task,
return rv;
}
-int sanlock_direct_read_id(struct sanlk_lockspace *ls,
- uint64_t *timestamp,
- uint64_t *owner_id,
- uint64_t *owner_generation,
- int use_aio)
-{
- struct task task = task_default;
- task.use_aio = use_aio;
- int rv;
-
- if (use_aio) {
- memset(&task.aio_ctx, 0, sizeof(task.aio_ctx));
- rv = io_setup(1, &task.aio_ctx);
- if (rv < 0)
- return rv;
- }
-
- rv = direct_read_id(&task, ls, timestamp, owner_id, owner_generation);
-
- if (use_aio)
- io_destroy(task.aio_ctx);
-
- return rv;
-}
-
int direct_live_id(struct task *task,
struct sanlk_lockspace *ls,
uint64_t *timestamp,
@@ -390,32 +353,6 @@ int direct_live_id(struct task *task,
return 0;
}
-int sanlock_direct_live_id(struct sanlk_lockspace *ls,
- uint64_t *timestamp,
- uint64_t *owner_id,
- uint64_t *owner_generation,
- int *live,
- int use_aio)
-{
- struct task task = task_default;
- task.use_aio = use_aio;
- int rv;
-
- if (use_aio) {
- memset(&task.aio_ctx, 0, sizeof(task.aio_ctx));
- rv = io_setup(1, &task.aio_ctx);
- if (rv < 0)
- return rv;
- }
-
- rv = direct_live_id(&task, ls, timestamp, owner_id, owner_generation, live);
-
- if (use_aio)
- io_destroy(task.aio_ctx);
-
- return rv;
-}
-
/*
* sanlock direct init -n <num_hosts> [-s LOCKSPACE] [-r RESOURCE]
*
@@ -453,32 +390,6 @@ int direct_init(struct task *task,
return rv;
}
-int sanlock_direct_init(struct sanlk_lockspace *ls,
- struct sanlk_resource *res,
- int max_hosts, int num_hosts, int use_aio)
-{
- struct task task = task_default;
- task.use_aio = use_aio;
- int rv;
-
- if (use_aio) {
- memset(&task.aio_ctx, 0, sizeof(task.aio_ctx));
- rv = io_setup(1, &task.aio_ctx);
- if (rv < 0)
- return rv;
- }
-
- if (!max_hosts)
- max_hosts = DEFAULT_MAX_HOSTS;
-
- rv = direct_init(&task, ls, res, max_hosts, num_hosts);
-
- if (use_aio)
- io_destroy(task.aio_ctx);
-
- return rv;
-}
-
int direct_read_leader(struct task *task,
struct sanlk_lockspace *ls,
struct sanlk_resource *res,
diff --git a/src/direct_lib.c b/src/direct_lib.c
index 2c168ee..5045026 100644
--- a/src/direct_lib.c
+++ b/src/direct_lib.c
@@ -12,8 +12,11 @@
#include <stdlib.h>
#include <stdint.h>
#include <stddef.h>
+#include <errno.h>
#include "sanlock_internal.h"
+#include "sanlock_direct.h"
+#include "direct.h"
void log_level(int space_id GNUC_UNUSED, int token_id GNUC_UNUSED,
int level GNUC_UNUSED, const char *fmt GNUC_UNUSED, ...);
@@ -29,3 +32,107 @@ int host_id_disk_info(char *name GNUC_UNUSED, struct sync_disk *disk GNUC_UNUSED
{
return -1;
}
+
+static void setup_task_lib(struct task *task, int use_aio)
+{
+ int rv;
+
+ memset(task, 0, sizeof(struct task));
+
+ sprintf(task->name, "%s", "lib");
+
+ task->io_timeout_seconds = DEFAULT_IO_TIMEOUT_SECONDS;
+ task->host_id_timeout_seconds = DEFAULT_HOST_ID_TIMEOUT_SECONDS;
+ task->host_id_renewal_seconds = DEFAULT_HOST_ID_RENEWAL_SECONDS;
+ task->host_id_renewal_fail_seconds = DEFAULT_HOST_ID_RENEWAL_FAIL_SECONDS;
+ task->host_id_renewal_warn_seconds = DEFAULT_HOST_ID_RENEWAL_WARN_SECONDS;
+
+ task->use_aio = use_aio;
+
+ if (task->use_aio) {
+ rv = io_setup(LIB_AIO_CB_SIZE, &task->aio_ctx);
+ if (rv < 0)
+ goto fail;
+
+ task->cb_size = LIB_AIO_CB_SIZE;
+ task->callbacks = malloc(LIB_AIO_CB_SIZE * sizeof(struct aicb));
+ if (!task->callbacks) {
+ rv = -ENOMEM;
+ goto fail_setup;
+ }
+ memset(task->callbacks, 0, LIB_AIO_CB_SIZE * sizeof(struct aicb));
+ }
+ return;
+
+ fail_setup:
+ io_destroy(task->aio_ctx);
+ fail:
+ task->use_aio = 0;
+}
+
+static void close_task_lib(struct task *task)
+{
+ if (task->use_aio)
+ io_destroy(task->aio_ctx);
+
+ if (task->callbacks)
+ free(task->callbacks);
+ task->callbacks = NULL;
+}
+
+int sanlock_direct_read_id(struct sanlk_lockspace *ls,
+ uint64_t *timestamp,
+ uint64_t *owner_id,
+ uint64_t *owner_generation,
+ int use_aio)
+{
+ struct task task;
+ int rv;
+
+ setup_task_lib(&task, use_aio);
+
+ rv = direct_read_id(&task, ls, timestamp, owner_id, owner_generation);
+
+ close_task_lib(&task);
+
+ return rv;
+}
+
+int sanlock_direct_live_id(struct sanlk_lockspace *ls,
+ uint64_t *timestamp,
+ uint64_t *owner_id,
+ uint64_t *owner_generation,
+ int *live,
+ int use_aio)
+{
+ struct task task;
+ int rv;
+
+ setup_task_lib(&task, use_aio);
+
+ rv = direct_live_id(&task, ls, timestamp, owner_id, owner_generation, live);
+
+ close_task_lib(&task);
+
+ return rv;
+}
+
+int sanlock_direct_init(struct sanlk_lockspace *ls,
+ struct sanlk_resource *res,
+ int max_hosts, int num_hosts, int use_aio)
+{
+ struct task task;
+ int rv;
+
+ setup_task_lib(&task, use_aio);
+
+ if (!max_hosts)
+ max_hosts = DEFAULT_MAX_HOSTS;
+
+ rv = direct_init(&task, ls, res, max_hosts, num_hosts);
+
+ close_task_lib(&task);
+
+ return rv;
+}
+
diff --git a/src/diskio.c b/src/diskio.c
index 1046f03..d0e2873 100644
--- a/src/diskio.c
+++ b/src/diskio.c
@@ -23,11 +23,8 @@
#include <sys/stat.h>
#include <blkid/blkid.h>
-#ifdef LINUX_AIO
-#include <libaio.h>
-#else /* POSIX_AIO */
-#include <aio.h>
-#endif
+#include <libaio.h> /* linux aio */
+#include <aio.h> /* posix aio */
#include "sanlock_internal.h"
#include "diskio.h"
@@ -180,12 +177,15 @@ int open_disks(struct sync_disk *disks, int num_disks)
return 0;
}
-static int do_write(int fd, uint64_t offset, const char *buf, int len)
+static int do_write(int fd, uint64_t offset, const char *buf, int len, struct task *task)
{
off_t ret;
int rv;
int pos = 0;
+ if (task)
+ task->io_count++;
+
ret = lseek(fd, offset, SEEK_SET);
if (ret != offset)
return -1;
@@ -210,11 +210,14 @@ static int do_write(int fd, uint64_t offset, const char *buf, int len)
return 0;
}
-static int do_read(int fd, uint64_t offset, char *buf, int len)
+static int do_read(int fd, uint64_t offset, char *buf, int len, struct task *task)
{
off_t ret;
int rv, pos = 0;
+ if (task)
+ task->io_count++;
+
ret = lseek(fd, offset, SEEK_SET);
if (ret != offset)
return -1;
@@ -233,34 +236,81 @@ static int do_read(int fd, uint64_t offset, char *buf, int len)
return 0;
}
-#ifdef LINUX_AIO
-static int do_linux_aio(int fd, uint64_t offset, char *buf, int len,
- struct task *task, int cmd)
+static struct aicb *find_callback_slot(struct task *task)
{
struct timespec ts;
- struct iocb cb;
- struct iocb *p_cb;
struct io_event event;
+ int cleared = 0;
int rv;
+ int i;
+
+ find:
+ for (i = 0; i < task->cb_size; i++) {
+ if (task->callbacks[i].used)
+ continue;
+ return &task->callbacks[i];
+ }
+
+ if (cleared++)
+ return NULL;
memset(&ts, 0, sizeof(struct timespec));
ts.tv_sec = task->io_timeout_seconds;
+ retry:
+ memset(&event, 0, sizeof(event));
- memset(&cb, 0, sizeof(cb));
- p_cb = &cb;
+ rv = io_getevents(task->aio_ctx, 1, 1, &event, &ts);
+ if (rv == -EINTR)
+ goto retry;
+ if (rv < 0)
+ return NULL;
+ if (rv == 1) {
+ struct iocb *ev_iocb = event.obj;
+ struct aicb *ev_aicb = container_of(ev_iocb, struct aicb, iocb);
- cb.aio_fildes = fd;
- cb.aio_lio_opcode = cmd;
- cb.u.c.buf = buf;
- cb.u.c.nbytes = len;
- cb.u.c.offset = offset;
+ ev_aicb->used = 0;
+ goto find;
+ }
+ return NULL;
+}
- rv = io_submit(task->aio_ctx, 1, &p_cb);
+static int do_linux_aio(int fd, uint64_t offset, char *buf, int len,
+ struct task *task, int cmd)
+{
+ struct timespec ts;
+ struct aicb *aicb;
+ struct iocb *iocb;
+ struct io_event event;
+ int rv;
+
+ /* I expect this pre-emptively catches the io_submit EAGAIN case */
+
+ aicb = find_callback_slot(task);
+ if (!aicb)
+ return -ENOENT;
+
+ iocb = &aicb->iocb;
+
+ memset(iocb, 0, sizeof(struct iocb));
+ iocb->aio_fildes = fd;
+ iocb->aio_lio_opcode = cmd;
+ iocb->u.c.buf = buf;
+ iocb->u.c.nbytes = len;
+ iocb->u.c.offset = offset;
+
+ rv = io_submit(task->aio_ctx, 1, &iocb);
if (rv < 0) {
- log_error("aio %d io_submit error %d", cmd, rv);
+ log_error("aio %s io_submit error %d", task->name, rv);
goto out;
}
+ task->io_count++;
+
+ /* don't reuse aicb->iocb until we reap the event for it */
+ aicb->used = 1;
+
+ memset(&ts, 0, sizeof(struct timespec));
+ ts.tv_sec = task->io_timeout_seconds;
retry:
memset(&event, 0, sizeof(event));
@@ -268,23 +318,29 @@ static int do_linux_aio(int fd, uint64_t offset, char *buf, int len,
if (rv == -EINTR)
goto retry;
if (rv < 0) {
- log_error("aio %d io_getevents error %d", cmd, rv);
+ log_error("aio %s io_getevents error %d", task->name, rv);
goto out;
}
if (rv == 1) {
- if (event.obj != p_cb) {
- log_error("aio %d event for other io retry", cmd);
+ struct iocb *ev_iocb = event.obj;
+ struct aicb *ev_aicb = container_of(ev_iocb, struct aicb, iocb);
+
+ ev_aicb->used = 0;
+
+ if (ev_iocb != iocb) {
+ log_error("aio %s other iocb %p event result %ld %ld",
+ task->name, ev_iocb, event.res, event.res2);
goto retry;
}
if ((int)event.res < 0) {
- log_error("aio %d event res error %ld %ld",
- cmd, event.res, event.res2);
+ log_error("aio %s event result %ld %ld",
+ task->name, event.res, event.res2);
rv = event.res;
goto out;
}
if (event.res != len) {
- log_error("aio %d event len %d error %lu %lu",
- cmd, len, event.res, event.res2);
+ log_error("aio %s event len %d result %lu %lu",
+ task->name, len, event.res, event.res2);
rv = -EMSGSIZE;
goto out;
}
@@ -297,37 +353,39 @@ static int do_linux_aio(int fd, uint64_t offset, char *buf, int len,
/* Timed out waiting for result. If cancel fails, we could try retry
io_getevents indefinately, but that removes the whole point of using
aio, which is the timeout. So, we need to be prepared to reap the
- event the next time we call io_getevents for a different i/o. */
+ event the next time we call io_getevents for a different i/o. We
+ can't reuse the iocb for this timed out io until we get an event for
+ it because we need to compare the iocb to event.obj to distinguish
+ events for separate submissions.
- rv = io_cancel(task->aio_ctx, &cb, &event);
- if (!rv) {
- log_error("aio %d canceled", cmd);
- rv = -ECANCELED;
- goto out;
- }
-
- /* <phro> dct: io_cancel doesn't work, in general. you are very
+ <phro> dct: io_cancel doesn't work, in general. you are very
likely going to get -EINVAL from that call */
- log_error("aio %d error %d", cmd, rv);
+ task->to_count++;
+
+ log_error("aio %s iocb %p timeout %u io_count %u", task->name, iocb,
+ task->to_count, task->io_count);
- if (rv > 0)
+ rv = io_cancel(task->aio_ctx, iocb, &event);
+ if (!rv) {
+ rv = -ECANCELED;
+ } else if (rv > 0) {
rv = -EILSEQ;
+ }
out:
return rv;
}
-static int do_write_aio(int fd, uint64_t offset, char *buf, int len, struct task *task)
+static int do_write_aio_linux(int fd, uint64_t offset, char *buf, int len, struct task *task)
{
return do_linux_aio(fd, offset, buf, len, task, IO_CMD_PWRITE);
}
-static int do_read_aio(int fd, uint64_t offset, char *buf, int len, struct task *task)
+static int do_read_aio_linux(int fd, uint64_t offset, char *buf, int len, struct task *task)
{
return do_linux_aio(fd, offset, buf, len, task, IO_CMD_PREAD);
}
-#else
-static int do_write_aio(int fd, uint64_t offset, char *buf, int len, struct task *task)
+static int do_write_aio_posix(int fd, uint64_t offset, char *buf, int len, struct task *task)
{
struct timespec ts;
struct aiocb cb;
@@ -376,7 +434,7 @@ static int do_write_aio(int fd, uint64_t offset, char *buf, int len, struct task
return -1;
}
-static int do_read_aio(int fd, uint64_t offset, char *buf, int len, struct task *task)
+static int do_read_aio_posix(int fd, uint64_t offset, char *buf, int len, struct task *task)
{
struct timespec ts;
struct aiocb cb;
@@ -423,16 +481,17 @@ static int do_read_aio(int fd, uint64_t offset, char *buf, int len, struct task
/* undefined error condition */
return -1;
}
-#endif
/* write aligned io buffer */
int write_iobuf(int fd, uint64_t offset, char *iobuf, int iobuf_len, struct task *task)
{
- if (task && task->use_aio)
- return do_write_aio(fd, offset, iobuf, iobuf_len, task);
+ if (task && task->use_aio == 1)
+ return do_write_aio_linux(fd, offset, iobuf, iobuf_len, task);
+ else if (task && task->use_aio == 2)
+ return do_write_aio_posix(fd, offset, iobuf, iobuf_len, task);
else
- return do_write(fd, offset, iobuf, iobuf_len);
+ return do_write(fd, offset, iobuf, iobuf_len, task);
}
static int _write_sectors(const struct sync_disk *disk, uint64_t sector_nr,
@@ -543,10 +602,12 @@ int read_sectors(const struct sync_disk *disk, uint64_t sector_nr,
memset(iobuf, 0, iobuf_len);
- if (task && task->use_aio)
- rv = do_read_aio(disk->fd, offset, iobuf, iobuf_len, task);
+ if (task && task->use_aio == 1)
+ rv = do_read_aio_linux(disk->fd, offset, iobuf, iobuf_len, task);
+ else if (task && task->use_aio == 2)
+ rv = do_read_aio_posix(disk->fd, offset, iobuf, iobuf_len, task);
else
- rv = do_read(disk->fd, offset, iobuf, iobuf_len);
+ rv = do_read(disk->fd, offset, iobuf, iobuf_len, task);
if (!rv) {
memcpy(data, iobuf, data_len);
diff --git a/src/host_id.c b/src/host_id.c
index 97cf7d7..9cfe35f 100644
--- a/src/host_id.c
+++ b/src/host_id.c
@@ -167,7 +167,8 @@ static void *host_id_thread(void *arg_in)
host_id = sp->host_id;
memcpy(&space_name, sp->space_name, NAME_ID_SIZE);
- setup_task(&task);
+ setup_task(&task, HOSTID_AIO_CB_SIZE);
+ memcpy(task.name, sp->space_name, NAME_ID_SIZE);
last_attempt = time(NULL);
diff --git a/src/main.c b/src/main.c
index 6ad4c85..bfa6a1c 100644
--- a/src/main.c
+++ b/src/main.c
@@ -1430,12 +1430,13 @@ static void call_cmd(struct task *task, struct cmd_args *ca)
};
}
-static void *thread_pool_worker(void *data GNUC_UNUSED)
+static void *thread_pool_worker(void *data)
{
struct task task;
struct cmd_args *ca;
- setup_task(&task);
+ setup_task(&task, WORKER_AIO_CB_SIZE);
+ snprintf(task.name, NAME_ID_SIZE, "worker%ld", (long)data);
pthread_mutex_lock(&pool.mutex);
@@ -1484,7 +1485,8 @@ static int thread_pool_add_work(struct cmd_args *ca)
list_add_tail(&ca->list, &pool.work_data);
if (!pool.free_workers && pool.num_workers < pool.max_workers) {
- rv = pthread_create(&th, NULL, thread_pool_worker, &pool);
+ rv = pthread_create(&th, NULL, thread_pool_worker,
+ (void *)(long)pool.num_workers);
if (rv < 0) {
list_del(&ca->list);
pthread_mutex_unlock(&pool.mutex);
@@ -1522,7 +1524,8 @@ static int thread_pool_create(int min_workers, int max_workers)
pool.max_workers = max_workers;
for (i = 0; i < min_workers; i++) {
- rv = pthread_create(&th, NULL, thread_pool_worker, &pool);
+ rv = pthread_create(&th, NULL, thread_pool_worker,
+ (void *)(long)i);
if (rv < 0)
break;
pool.num_workers++;
@@ -1534,27 +1537,54 @@ static int thread_pool_create(int min_workers, int max_workers)
return rv;
}
-void setup_task(struct task *task)
+void setup_task(struct task *task, int cb_size)
{
int rv;
- memcpy(task, &main_task, sizeof(struct task));
+ memset(task, 0, sizeof(struct task));
- memset(&task->aio_ctx, 0, sizeof(io_context_t));
+ /* inherit configured timeouts from main_task */
+
+ task->use_aio = main_task.use_aio;
+ task->io_timeout_seconds = main_task.io_timeout_seconds;
+ task->host_id_timeout_seconds = main_task.host_id_timeout_seconds;
+ task->host_id_renewal_seconds = main_task.host_id_renewal_seconds;
+ task->host_id_renewal_fail_seconds = main_task.host_id_renewal_fail_seconds;
+ task->host_id_renewal_warn_seconds = main_task.host_id_renewal_warn_seconds;
if (task->use_aio) {
- rv = io_setup(1, &task->aio_ctx);
- if (rv < 0) {
- log_error("io_setup error %d, use_aio=0", rv);
- task->use_aio = 0;
+ if (!cb_size)
+ goto fail;
+
+ rv = io_setup(cb_size, &task->aio_ctx);
+ if (rv < 0)
+ goto fail;
+
+ task->cb_size = cb_size;
+ task->callbacks = malloc(cb_size * sizeof(struct aicb));
+ if (!task->callbacks) {
+ rv = -ENOMEM;
+ goto fail_setup;
}
+ memset(task->callbacks, 0, cb_size * sizeof(struct aicb));
}
+ return;
+
+ fail_setup:
+ io_destroy(task->aio_ctx);
+ fail:
+ task->use_aio = 0;
+ log_error("setup_task aio disabled %d", rv);
}
void close_task(struct task *task)
{
if (task->use_aio)
io_destroy(task->aio_ctx);
+
+ if (task->callbacks)
+ free(task->callbacks);
+ task->callbacks = NULL;
}
static int print_daemon_state(char *str)
@@ -2129,9 +2159,10 @@ static int do_daemon(void)
umask(0);
}
- /* in the daemon, the main_task should never do disk i/o, so we do not
- need to call io_setup() on main_task.aio_ctx */
-
+ /* no setup_task(&main_task) to set up aio in daemon because the daemon
+ main task should never do disk i/o. We do leave main_task.use_aio
+ set because other tasks copy their use_aio setting from there. */
+
rv = client_alloc();
if (rv < 0)
return rv;
@@ -2858,15 +2889,10 @@ static int do_direct(void)
int rv;
/* for direct commands, the main_task does disk i/o, so set up
- main_task.aio_ctx */
+ main_task for aio */
- if (main_task.use_aio) {
- rv = io_setup(1, &main_task.aio_ctx);
- if (rv < 0) {
- log_tool("io_setup error %d, use_aio=0", rv);
- main_task.use_aio = 0;
- }
- }
+ setup_task(&main_task, DIRECT_AIO_CB_SIZE);
+ sprintf(main_task.name, "%s", "main_direct");
switch (com.action) {
case ACT_INIT:
@@ -2990,6 +3016,7 @@ int main(int argc, char *argv[])
com.pid = -1;
memset(&main_task, 0, sizeof(main_task));
+ sprintf(main_task.name, "%s", "main");
main_task.use_aio = DEFAULT_USE_AIO;
main_task.io_timeout_seconds = DEFAULT_IO_TIMEOUT_SECONDS;
main_task.host_id_timeout_seconds = DEFAULT_HOST_ID_TIMEOUT_SECONDS;
diff --git a/src/sanlock_internal.h b/src/sanlock_internal.h
index 7a02b0a..5144188 100644
--- a/src/sanlock_internal.h
+++ b/src/sanlock_internal.h
@@ -251,14 +251,30 @@ struct sm_header {
#define DEFAULT_HOST_ID_RENEWAL_FAIL_SECONDS 30
#define DEFAULT_HOST_ID_RENEWAL_WARN_SECONDS 25
+#define HOSTID_AIO_CB_SIZE 64
+#define WORKER_AIO_CB_SIZE 8
+#define DIRECT_AIO_CB_SIZE 8
+#define RELEASE_AIO_CB_SIZE 64
+#define LIB_AIO_CB_SIZE 8
+
+struct aicb {
+ int used;
+ struct iocb iocb;
+};
+
struct task {
+ char name[NAME_ID_SIZE+1];
int use_aio;
int io_timeout_seconds;
int host_id_timeout_seconds;
int host_id_renewal_seconds;
int host_id_renewal_fail_seconds;
int host_id_renewal_warn_seconds;
+ unsigned int io_count;
+ unsigned int to_count;
+ int cb_size;
io_context_t aio_ctx;
+ struct aicb *callbacks;
};
EXTERN struct task main_task;
@@ -322,7 +338,7 @@ enum {
};
/* main.c */
-void setup_task(struct task *task);
+void setup_task(struct task *task, int cb_size);
void close_task(struct task *task);
#endif
diff --git a/src/token_manager.c b/src/token_manager.c
index 43a6507..12d14c5 100644
--- a/src/token_manager.c
+++ b/src/token_manager.c
@@ -190,7 +190,8 @@ static void *async_release_thread(void *arg GNUC_UNUSED)
struct resource *r;
struct token *token;
- setup_task(&task);
+ setup_task(&task, RELEASE_AIO_CB_SIZE);
+ sprintf(task.name, "%s", "release");
while (1) {
pthread_mutex_lock(&resource_mutex);
diff --git a/tests/Makefile b/tests/Makefile
index 80f5126..e656606 100644
--- a/tests/Makefile
+++ b/tests/Makefile
@@ -21,7 +21,7 @@ CFLAGS += -D_GNU_SOURCE -g \
-fasynchronous-unwind-tables \
-fdiagnostics-show-option
-LDFLAGS = -laio -lblkid -lsanlock -lsanlock_direct
+LDFLAGS = -lrt -laio -lblkid -lsanlock -lsanlock_direct
all: $(TARGET)
diff --git a/tests/devcount.c b/tests/devcount.c
index fc7a1fb..f7d049e 100644
--- a/tests/devcount.c
+++ b/tests/devcount.c
@@ -570,9 +570,9 @@ static int do_relock(int argc, char *argv[])
}
run_more:
- /* let the child run for 10 seconds before stopping it */
+ /* let the child run for 20 seconds before stopping it */
- for (i = 0; i < 10; i++) {
+ for (i = 0; i < 20; i++) {
rv = waitpid(pid, &status, WNOHANG);
if (rv == pid)
break;