summaryrefslogtreecommitdiffstats
path: root/xlators/performance/io-threads/src/io-threads.c
diff options
context:
space:
mode:
Diffstat (limited to 'xlators/performance/io-threads/src/io-threads.c')
-rw-r--r--xlators/performance/io-threads/src/io-threads.c3666
1 files changed, 2309 insertions, 1357 deletions
diff --git a/xlators/performance/io-threads/src/io-threads.c b/xlators/performance/io-threads/src/io-threads.c
index 66aec51a3..bbcf4ed26 100644
--- a/xlators/performance/io-threads/src/io-threads.c
+++ b/xlators/performance/io-threads/src/io-threads.c
@@ -1,20 +1,11 @@
/*
- Copyright (c) 2006-2009 Z RESEARCH, Inc. <http://www.zresearch.com>
+ Copyright (c) 2008-2012 Red Hat, Inc. <http://www.redhat.com>
This file is part of GlusterFS.
- GlusterFS is free software; you can redistribute it and/or modify
- it under the terms of the GNU General Public License as published
- by the Free Software Foundation; either version 3 of the License,
- or (at your option) any later version.
-
- GlusterFS is distributed in the hope that it will be useful, but
- WITHOUT ANY WARRANTY; without even the implied warranty of
- MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- General Public License for more details.
-
- You should have received a copy of the GNU General Public License
- along with this program. If not, see
- <http://www.gnu.org/licenses/>.
+ This file is licensed to you under your choice of the GNU Lesser
+ General Public License, version 3 or any later version (LGPLv3 or
+ later), or the GNU General Public License, version 2 (GPLv2), in all
+ cases as published by the Free Software Foundation.
*/
#ifndef _CONFIG_H
@@ -31,1804 +22,2736 @@
#include <stdlib.h>
#include <sys/time.h>
#include <time.h>
+#include "locking.h"
-typedef void *(*iot_worker_fn)(void*);
-
-void _iot_queue (iot_worker_t *worker, iot_request_t *req);
-iot_request_t * iot_init_request (call_stub_t *stub);
-void iot_startup_workers (iot_worker_t **workers, int start_idx, int count,
- iot_worker_fn workerfunc);
-void * iot_worker_unordered (void *arg);
-void * iot_worker_ordered (void *arg);
-void iot_startup_worker (iot_worker_t *worker, iot_worker_fn workerfunc);
-void iot_destroy_request (iot_request_t * req);
-
-/* I know this function modularizes things a bit too much,
- * but it is easier on the eyes to read this than see all that locking,
- * queueing, and thread firing in the same curly block, as was the
- * case before this function.
- */
-void
-iot_request_queue_and_thread_fire (iot_worker_t *worker,
- iot_worker_fn workerfunc,
- iot_request_t *req)
-{
- pthread_mutex_lock (&worker->qlock);
- {
- if (iot_worker_active (worker))
- _iot_queue (worker, req);
- else {
- iot_startup_worker (worker, workerfunc);
- _iot_queue (worker, req);
- }
+void *iot_worker (void *arg);
+int iot_workers_scale (iot_conf_t *conf);
+int __iot_workers_scale (iot_conf_t *conf);
+struct volume_options options[];
+
+call_stub_t *
+__iot_dequeue (iot_conf_t *conf, int *pri, struct timespec *sleep)
+{
+ call_stub_t *stub = NULL;
+ int i = 0;
+ struct timeval curtv = {0,}, difftv = {0,};
+
+ *pri = -1;
+ sleep->tv_sec = 0;
+ sleep->tv_nsec = 0;
+ for (i = 0; i < IOT_PRI_MAX; i++) {
+ if (list_empty (&conf->reqs[i]) ||
+ (conf->ac_iot_count[i] >= conf->ac_iot_limit[i]))
+ continue;
+
+ if (i == IOT_PRI_LEAST) {
+ pthread_mutex_lock(&conf->throttle.lock);
+ if (!conf->throttle.sample_time.tv_sec) {
+ /* initialize */
+ gettimeofday(&conf->throttle.sample_time, NULL);
+ } else {
+ /*
+ * Maintain a running count of least priority
+ * operations that are handled over a particular
+ * time interval. The count is provided via
+ * state dump and is used as a measure against
+ * least priority op throttling.
+ */
+ gettimeofday(&curtv, NULL);
+ timersub(&curtv, &conf->throttle.sample_time,
+ &difftv);
+ if (difftv.tv_sec >= IOT_LEAST_THROTTLE_DELAY) {
+ conf->throttle.cached_rate =
+ conf->throttle.sample_cnt;
+ conf->throttle.sample_cnt = 0;
+ conf->throttle.sample_time = curtv;
+ }
+
+ /*
+ * If we're over the configured rate limit,
+ * provide an absolute time to the caller that
+ * represents the soonest we're allowed to
+ * return another least priority request.
+ */
+ if (conf->throttle.rate_limit &&
+ conf->throttle.sample_cnt >=
+ conf->throttle.rate_limit) {
+ struct timeval delay;
+ delay.tv_sec = IOT_LEAST_THROTTLE_DELAY;
+ delay.tv_usec = 0;
+
+ timeradd(&conf->throttle.sample_time,
+ &delay, &curtv);
+ TIMEVAL_TO_TIMESPEC(&curtv, sleep);
+
+ pthread_mutex_unlock(
+ &conf->throttle.lock);
+ break;
+ }
+ }
+ conf->throttle.sample_cnt++;
+ pthread_mutex_unlock(&conf->throttle.lock);
+ }
+
+ stub = list_entry (conf->reqs[i].next, call_stub_t, list);
+ conf->ac_iot_count[i]++;
+ *pri = i;
+ break;
}
- pthread_mutex_unlock (&worker->qlock);
-}
+ if (!stub)
+ return NULL;
+ conf->queue_size--;
+ conf->queue_sizes[*pri]--;
+ list_del_init (&stub->list);
-int
-iot_unordered_request_balancer (iot_conf_t *conf)
+ return stub;
+}
+
+
+void
+__iot_enqueue (iot_conf_t *conf, call_stub_t *stub, int pri)
{
- long int rand = 0;
- int idx = 0;
+ if (pri < 0 || pri >= IOT_PRI_MAX)
+ pri = IOT_PRI_MAX-1;
- /* Decide which thread will service the request.
- * FIXME: This should change into some form of load-balancing.
- * */
- rand = random ();
+ list_add_tail (&stub->list, &conf->reqs[pri]);
- /* If scaling is on, we can choose from any thread
- * that has been allocated upto, max_o_threads, but
- * with scaling off, we'll never have threads more
- * than min_o_threads.
- */
- if (iot_unordered_scaling_on (conf))
- idx = (rand % conf->max_u_threads);
- else
- idx = (rand % conf->min_u_threads);
+ conf->queue_size++;
+ conf->queue_sizes[pri]++;
- return idx;
+ return;
}
-void
-iot_schedule_unordered (iot_conf_t *conf,
- inode_t *inode,
- call_stub_t *stub)
-{
- int32_t idx = 0;
- iot_worker_t *selected_worker = NULL;
- iot_request_t *req = NULL;
-
- idx = iot_unordered_request_balancer (conf);
- selected_worker = conf->uworkers[idx];
-
- req = iot_init_request (stub);
- iot_request_queue_and_thread_fire (selected_worker,
- iot_worker_unordered, req);
-}
-
-/* Only to be used with ordered requests.
- */
-uint64_t
-iot_create_inode_worker_assoc (iot_conf_t * conf, inode_t * inode)
-{
- long int rand = 0;
- uint64_t idx = 0;
-
- rand = random ();
- /* If scaling is on, we can choose from any thread
- * that has been allocated upto, max_o_threads, but
- * with scaling off, we'll never have threads more
- * than min_o_threads.
- */
- if (iot_ordered_scaling_on (conf))
- idx = (rand % conf->max_o_threads);
- else
- idx = (rand % conf->min_o_threads);
-
- __inode_ctx_put (inode, conf->this, idx);
-
- return idx;
-}
-
-/* Assumes inode lock is held. */
-int
-iot_ordered_request_balancer (iot_conf_t *conf, inode_t *inode, uint64_t *idx)
-{
- int ret = 0;
-
- if (__inode_ctx_get (inode, conf->this, idx) < 0)
- *idx = iot_create_inode_worker_assoc (conf, inode);
- else {
- /* Sanity check to ensure the idx received from the inode
- * context is within bounds. We're a bit optimistic in
- * assuming that if an index is within bounds, it is
- * not corrupted. idx is uint so we dont check for less
- * than 0.
- */
- if ((*idx >= (uint64_t)conf->max_o_threads)) {
- gf_log (conf->this->name, GF_LOG_ERROR,
- "inode context returned insane thread index %"
- PRIu64, *idx);
- ret = -1;
+
+void *
+iot_worker (void *data)
+{
+ iot_conf_t *conf = NULL;
+ xlator_t *this = NULL;
+ call_stub_t *stub = NULL;
+ struct timespec sleep_till = {0, };
+ int ret = 0;
+ int pri = -1;
+ char timeout = 0;
+ char bye = 0;
+ struct timespec sleep = {0,};
+
+ conf = data;
+ this = conf->this;
+ THIS = this;
+
+ for (;;) {
+ sleep_till.tv_sec = time (NULL) + conf->idle_time;
+
+ pthread_mutex_lock (&conf->mutex);
+ {
+ if (pri != -1) {
+ conf->ac_iot_count[pri]--;
+ pri = -1;
+ }
+ while (conf->queue_size == 0) {
+ conf->sleep_count++;
+
+ ret = pthread_cond_timedwait (&conf->cond,
+ &conf->mutex,
+ &sleep_till);
+ conf->sleep_count--;
+
+ if (ret == ETIMEDOUT) {
+ timeout = 1;
+ break;
+ }
+ }
+
+ if (timeout) {
+ if (conf->curr_count > IOT_MIN_THREADS) {
+ conf->curr_count--;
+ bye = 1;
+ gf_log (conf->this->name, GF_LOG_DEBUG,
+ "timeout, terminated. conf->curr_count=%d",
+ conf->curr_count);
+ } else {
+ timeout = 0;
+ }
+ }
+
+ stub = __iot_dequeue (conf, &pri, &sleep);
+ if (!stub && (sleep.tv_sec || sleep.tv_nsec)) {
+ pthread_cond_timedwait(&conf->cond,
+ &conf->mutex, &sleep);
+ pthread_mutex_unlock(&conf->mutex);
+ continue;
+ }
}
- }
+ pthread_mutex_unlock (&conf->mutex);
- return ret;
-}
+ if (stub) /* guard against spurious wakeups */
+ call_resume (stub);
-void
-iot_schedule_ordered (iot_conf_t *conf,
- inode_t *inode,
- call_stub_t *stub)
-{
- uint64_t idx = 0;
- iot_worker_t *selected_worker = NULL;
- iot_request_t * req = NULL;
- int balstatus = 0;
-
- if (inode == NULL) {
- gf_log (conf->this->name, GF_LOG_ERROR,
- "Got NULL inode for ordered request");
- STACK_UNWIND (stub->frame, -1, EINVAL, NULL);
- call_stub_destroy (stub);
- return;
- }
- req = iot_init_request (stub);
- LOCK (&inode->lock);
- {
- balstatus = iot_ordered_request_balancer (conf, inode, &idx);
- if (balstatus < 0) {
- gf_log (conf->this->name, GF_LOG_ERROR,
- "Insane worker index. Unwinding stack");
- STACK_UNWIND (stub->frame, -1, ECANCELED, NULL);
- iot_destroy_request (req);
- call_stub_destroy (stub);
- goto unlock_out;
+ if (bye)
+ break;
+ }
+
+ if (pri != -1) {
+ pthread_mutex_lock (&conf->mutex);
+ {
+ conf->ac_iot_count[pri]--;
}
- /* inode lock once acquired, cannot be left here
- * because other gluster main threads might be
- * contending on it to append a request for this file.
- * So we'll also leave the lock only after we've
- * added the request to the worker queue.
- */
- selected_worker = conf->oworkers[idx];
- iot_request_queue_and_thread_fire (selected_worker,
- iot_worker_ordered, req);
+ pthread_mutex_unlock (&conf->mutex);
}
-unlock_out:
- UNLOCK (&inode->lock);
+ return NULL;
}
-int32_t
-iot_lookup_cbk (call_frame_t *frame,
- void * cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- inode_t *inode,
- struct stat *buf,
- dict_t *xattr)
-{
- STACK_UNWIND (frame, op_ret, op_errno, inode, buf, xattr);
- return 0;
-}
-int32_t
-iot_lookup_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- dict_t *xattr_req)
+int
+do_iot_schedule (iot_conf_t *conf, call_stub_t *stub, int pri)
{
- STACK_WIND (frame, iot_lookup_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->lookup, loc, xattr_req);
- return 0;
-}
+ int ret = 0;
-int32_t
-iot_lookup (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- dict_t *xattr_req)
-{
- call_stub_t *stub = NULL;
+ pthread_mutex_lock (&conf->mutex);
+ {
+ __iot_enqueue (conf, stub, pri);
- stub = fop_lookup_stub (frame, iot_lookup_wrapper, loc, xattr_req);
- if (!stub) {
- gf_log (this->name, GF_LOG_ERROR,
- "cannot get lookup stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL, NULL, NULL);
- return 0;
- }
+ pthread_cond_signal (&conf->cond);
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
- return 0;
-}
+ ret = __iot_workers_scale (conf);
+ }
+ pthread_mutex_unlock (&conf->mutex);
-int32_t
-iot_chmod_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
-{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
- return 0;
+ return ret;
}
-int32_t
-iot_chmod_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- mode_t mode)
-{
- STACK_WIND (frame, iot_chmod_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->chmod, loc, mode);
- return 0;
+char*
+iot_get_pri_meaning (iot_pri_t pri)
+{
+ char *name = NULL;
+ switch (pri) {
+ case IOT_PRI_HI:
+ name = "fast";
+ break;
+ case IOT_PRI_NORMAL:
+ name = "normal";
+ break;
+ case IOT_PRI_LO:
+ name = "slow";
+ break;
+ case IOT_PRI_LEAST:
+ name = "least priority";
+ break;
+ case IOT_PRI_MAX:
+ name = "invalid";
+ break;
+ }
+ return name;
}
-int32_t
-iot_chmod (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- mode_t mode)
+int
+iot_schedule (call_frame_t *frame, xlator_t *this, call_stub_t *stub)
{
- call_stub_t *stub = NULL;
- fd_t *fd = NULL;
+ int ret = -1;
+ iot_pri_t pri = IOT_PRI_MAX - 1;
+ iot_conf_t *conf = this->private;
- stub = fop_chmod_stub (frame, iot_chmod_wrapper, loc, mode);
- if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get chmod stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ if ((frame->root->pid < GF_CLIENT_PID_MAX) && conf->least_priority) {
+ pri = IOT_PRI_LEAST;
+ goto out;
}
- fd = fd_lookup (loc->inode, frame->root->pid);
- if (fd == NULL)
- iot_schedule_unordered ((iot_conf_t *)this->private,
- loc->inode, stub);
- else {
- iot_schedule_ordered ((iot_conf_t *)this->private, loc->inode,
- stub);
- fd_unref (fd);
+ switch (stub->fop) {
+ case GF_FOP_OPEN:
+ case GF_FOP_STAT:
+ case GF_FOP_FSTAT:
+ case GF_FOP_LOOKUP:
+ case GF_FOP_ACCESS:
+ case GF_FOP_READLINK:
+ case GF_FOP_OPENDIR:
+ case GF_FOP_STATFS:
+ case GF_FOP_READDIR:
+ case GF_FOP_READDIRP:
+ pri = IOT_PRI_HI;
+ break;
+
+ case GF_FOP_CREATE:
+ case GF_FOP_FLUSH:
+ case GF_FOP_LK:
+ case GF_FOP_INODELK:
+ case GF_FOP_FINODELK:
+ case GF_FOP_ENTRYLK:
+ case GF_FOP_FENTRYLK:
+ case GF_FOP_UNLINK:
+ case GF_FOP_SETATTR:
+ case GF_FOP_FSETATTR:
+ case GF_FOP_MKNOD:
+ case GF_FOP_MKDIR:
+ case GF_FOP_RMDIR:
+ case GF_FOP_SYMLINK:
+ case GF_FOP_RENAME:
+ case GF_FOP_LINK:
+ case GF_FOP_SETXATTR:
+ case GF_FOP_GETXATTR:
+ case GF_FOP_FGETXATTR:
+ case GF_FOP_FSETXATTR:
+ case GF_FOP_REMOVEXATTR:
+ case GF_FOP_FREMOVEXATTR:
+ pri = IOT_PRI_NORMAL;
+ break;
+
+ case GF_FOP_READ:
+ case GF_FOP_WRITE:
+ case GF_FOP_FSYNC:
+ case GF_FOP_TRUNCATE:
+ case GF_FOP_FTRUNCATE:
+ case GF_FOP_FSYNCDIR:
+ case GF_FOP_XATTROP:
+ case GF_FOP_FXATTROP:
+ case GF_FOP_RCHECKSUM:
+ case GF_FOP_FALLOCATE:
+ case GF_FOP_DISCARD:
+ case GF_FOP_ZEROFILL:
+ pri = IOT_PRI_LO;
+ break;
+
+ case GF_FOP_NULL:
+ case GF_FOP_FORGET:
+ case GF_FOP_RELEASE:
+ case GF_FOP_RELEASEDIR:
+ case GF_FOP_GETSPEC:
+ case GF_FOP_MAXVALUE:
+ //fail compilation on missing fop
+ //new fop must choose priority.
+ break;
}
- return 0;
+out:
+ gf_log (this->name, GF_LOG_DEBUG, "%s scheduled as %s fop",
+ gf_fop_list[stub->fop], iot_get_pri_meaning (pri));
+ ret = do_iot_schedule (this->private, stub, pri);
+ return ret;
}
-int32_t
-iot_fchmod_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
+int
+iot_lookup_cbk (call_frame_t *frame, void * cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno,
+ inode_t *inode, struct iatt *buf, dict_t *xdata,
+ struct iatt *postparent)
{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+ STACK_UNWIND_STRICT (lookup, frame, op_ret, op_errno, inode, buf, xdata,
+ postparent);
return 0;
}
-int32_t
-iot_fchmod_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- mode_t mode)
+
+int
+iot_lookup_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ dict_t *xdata)
{
- STACK_WIND (frame, iot_fchmod_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->fchmod, fd, mode);
+ STACK_WIND (frame, iot_lookup_cbk,
+ FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->lookup,
+ loc, xdata);
return 0;
}
-int32_t
-iot_fchmod (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- mode_t mode)
+
+int
+iot_lookup (call_frame_t *frame, xlator_t *this, loc_t *loc, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_fchmod_stub (frame, iot_fchmod_wrapper, fd, mode);
+ stub = fop_lookup_stub (frame, iot_lookup_wrapper, loc, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fchmod stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create lookup stub (out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ STACK_UNWIND_STRICT (lookup, frame, -1, -ret, NULL, NULL, NULL,
+ NULL);
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
return 0;
}
-int32_t
-iot_chown_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
+
+int
+iot_setattr_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno,
+ struct iatt *preop, struct iatt *postop, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+ STACK_UNWIND_STRICT (setattr, frame, op_ret, op_errno, preop, postop,
+ xdata);
return 0;
}
-int32_t
-iot_chown_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- uid_t uid,
- gid_t gid)
+
+int
+iot_setattr_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ struct iatt *stbuf, int32_t valid, dict_t *xdata)
{
- STACK_WIND (frame, iot_chown_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->chown, loc, uid, gid);
+ STACK_WIND (frame, iot_setattr_cbk,
+ FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->setattr,
+ loc, stbuf, valid, xdata);
return 0;
}
-int32_t
-iot_chown (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- uid_t uid,
- gid_t gid)
+
+int
+iot_setattr (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ struct iatt *stbuf, int32_t valid, dict_t *xdata)
{
call_stub_t *stub = NULL;
- fd_t *fd = NULL;
+ int ret = -1;
- stub = fop_chown_stub (frame, iot_chown_wrapper, loc, uid, gid);
+ stub = fop_setattr_stub (frame, iot_setattr_wrapper, loc, stbuf, valid,
+ xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get chown stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR, "Cannot create setattr stub"
+ "(Out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- fd = fd_lookup (loc->inode, frame->root->pid);
- if (fd == NULL)
- iot_schedule_unordered ((iot_conf_t *)this->private,
- loc->inode, stub);
- else {
- iot_schedule_ordered ((iot_conf_t *)this->private, loc->inode,
- stub);
- fd_unref (fd);
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+
+ STACK_UNWIND_STRICT (setattr, frame, -1, -ret, NULL, NULL, NULL);
}
return 0;
}
-int32_t
-iot_fchown_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
+
+int
+iot_fsetattr_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno,
+ struct iatt *preop, struct iatt *postop, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+ STACK_UNWIND_STRICT (fsetattr, frame, op_ret, op_errno, preop, postop,
+ xdata);
return 0;
}
-int32_t
-iot_fchown_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- uid_t uid,
- gid_t gid)
+
+int
+iot_fsetattr_wrapper (call_frame_t *frame, xlator_t *this,
+ fd_t *fd, struct iatt *stbuf, int32_t valid, dict_t *xdata)
{
- STACK_WIND (frame, iot_fchown_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->fchown, fd, uid, gid);
+ STACK_WIND (frame, iot_fsetattr_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->fsetattr, fd, stbuf, valid,
+ xdata);
return 0;
}
-int32_t
-iot_fchown (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- uid_t uid,
- gid_t gid)
+
+int
+iot_fsetattr (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ struct iatt *stbuf, int32_t valid, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_fchown_stub (frame, iot_fchown_wrapper, fd, uid, gid);
+ stub = fop_fsetattr_stub (frame, iot_fsetattr_wrapper, fd, stbuf,
+ valid, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fchown stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR, "cannot create fsetattr stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fsetattr, frame, -1, -ret, NULL, NULL,
+ NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_access_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno)
+
+int
+iot_access_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno);
+ STACK_UNWIND_STRICT (access, frame, op_ret, op_errno, xdata);
return 0;
}
-int32_t
-iot_access_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- int32_t mask)
+
+int
+iot_access_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ int32_t mask, dict_t *xdata)
{
STACK_WIND (frame, iot_access_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->access, loc, mask);
+ FIRST_CHILD (this)->fops->access, loc, mask, xdata);
return 0;
}
-int32_t
-iot_access (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- int32_t mask)
+
+int
+iot_access (call_frame_t *frame, xlator_t *this, loc_t *loc, int32_t mask,
+ dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_access_stub (frame, iot_access_wrapper, loc, mask);
+ stub = fop_access_stub (frame, iot_access_wrapper, loc, mask, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get access stub");
- STACK_UNWIND (frame, -1, ENOMEM);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR, "cannot create access stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (access, frame, -1, -ret, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_readlink_cbk (call_frame_t *frame,
- void * cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- const char *path)
+
+int
+iot_readlink_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, const char *path,
+ struct iatt *stbuf, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, path);
+ STACK_UNWIND_STRICT (readlink, frame, op_ret, op_errno, path, stbuf,
+ xdata);
return 0;
}
-int32_t
-iot_readlink_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- size_t size)
+
+int
+iot_readlink_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ size_t size, dict_t *xdata)
{
- STACK_WIND (frame, iot_readlink_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->readlink, loc, size);
+ STACK_WIND (frame, iot_readlink_cbk,
+ FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->readlink,
+ loc, size, xdata);
return 0;
}
-int32_t
-iot_readlink (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- size_t size)
+
+int
+iot_readlink (call_frame_t *frame, xlator_t *this, loc_t *loc, size_t size, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_readlink_stub (frame, iot_readlink_wrapper, loc, size);
+ stub = fop_readlink_stub (frame, iot_readlink_wrapper, loc, size, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get readlink stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR, "cannot create readlink stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (readlink, frame, -1, -ret, NULL, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
}
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
return 0;
}
-int32_t
-iot_mknod_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- inode_t *inode,
- struct stat *buf)
+
+int
+iot_mknod_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, inode_t *inode,
+ struct iatt *buf, struct iatt *preparent,
+ struct iatt *postparent, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, inode, buf);
+ STACK_UNWIND_STRICT (mknod, frame, op_ret, op_errno, inode, buf,
+ preparent, postparent, xdata);
return 0;
}
-int32_t
-iot_mknod_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- mode_t mode,
- dev_t rdev)
+
+int
+iot_mknod_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc, mode_t mode,
+ dev_t rdev, mode_t umask, dict_t *xdata)
{
STACK_WIND (frame, iot_mknod_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->mknod, loc, mode, rdev);
+ FIRST_CHILD (this)->fops->mknod, loc, mode, rdev, umask,
+ xdata);
return 0;
}
-int32_t
-iot_mknod (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- mode_t mode,
- dev_t rdev)
+
+int
+iot_mknod (call_frame_t *frame, xlator_t *this, loc_t *loc, mode_t mode,
+ dev_t rdev, mode_t umask, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_mknod_stub (frame, iot_mknod_wrapper, loc, mode, rdev);
+ stub = fop_mknod_stub (frame, iot_mknod_wrapper, loc, mode, rdev,
+ umask, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get mknod stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR, "cannot create mknod stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (mknod, frame, -1, -ret, NULL, NULL, NULL,
+ NULL, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_mkdir_cbk (call_frame_t *frame,
- void * cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- inode_t *inode,
- struct stat *buf)
+
+int
+iot_mkdir_cbk (call_frame_t *frame, void * cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, inode_t *inode,
+ struct iatt *buf, struct iatt *preparent,
+ struct iatt *postparent, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, inode, buf);
+ STACK_UNWIND_STRICT (mkdir, frame, op_ret, op_errno, inode, buf,
+ preparent, postparent, xdata);
return 0;
}
-int32_t
-iot_mkdir_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- mode_t mode)
+
+int
+iot_mkdir_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc, mode_t mode,
+ mode_t umask, dict_t *xdata)
{
STACK_WIND (frame, iot_mkdir_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->mkdir, loc, mode);
+ FIRST_CHILD (this)->fops->mkdir, loc, mode, umask, xdata);
return 0;
}
-int32_t
-iot_mkdir (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- mode_t mode)
+
+int
+iot_mkdir (call_frame_t *frame, xlator_t *this, loc_t *loc, mode_t mode,
+ mode_t umask, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_mkdir_stub (frame, iot_mkdir_wrapper, loc, mode);
+ stub = fop_mkdir_stub (frame, iot_mkdir_wrapper, loc, mode, umask,
+ xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get mkdir stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR, "cannot create mkdir stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (mkdir, frame, -1, -ret, NULL, NULL, NULL,
+ NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_rmdir_cbk (call_frame_t *frame,
- void * cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno)
+
+int
+iot_rmdir_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *preparent,
+ struct iatt *postparent, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno);
+ STACK_UNWIND_STRICT (rmdir, frame, op_ret, op_errno, preparent,
+ postparent, xdata);
return 0;
}
-int32_t
-iot_rmdir_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc)
+
+int
+iot_rmdir_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc, int flags, dict_t *xdata)
{
STACK_WIND (frame, iot_rmdir_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->rmdir, loc);
+ FIRST_CHILD (this)->fops->rmdir, loc, flags, xdata);
return 0;
}
-int32_t
-iot_rmdir (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc)
+
+int
+iot_rmdir (call_frame_t *frame, xlator_t *this, loc_t *loc, int flags, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_rmdir_stub (frame, iot_rmdir_wrapper, loc);
+ stub = fop_rmdir_stub (frame, iot_rmdir_wrapper, loc, flags, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get rmdir stub");
- STACK_UNWIND (frame, -1, ENOMEM);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR, "cannot create rmdir stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (rmdir, frame, -1, -ret, NULL, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_symlink_cbk (call_frame_t *frame,
- void * cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- inode_t *inode,
- struct stat *buf)
+
+int
+iot_symlink_cbk (call_frame_t *frame, void * cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, inode_t *inode,
+ struct iatt *buf, struct iatt *preparent,
+ struct iatt *postparent, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, inode, buf);
+ STACK_UNWIND_STRICT (symlink, frame, op_ret, op_errno, inode, buf,
+ preparent, postparent, xdata);
return 0;
}
-int32_t
-iot_symlink_wrapper (call_frame_t *frame,
- xlator_t *this,
- const char *linkname,
- loc_t *loc)
+
+int
+iot_symlink_wrapper (call_frame_t *frame, xlator_t *this, const char *linkname,
+ loc_t *loc, mode_t umask, dict_t *xdata)
{
STACK_WIND (frame, iot_symlink_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->symlink, linkname, loc);
+ FIRST_CHILD (this)->fops->symlink, linkname, loc, umask,
+ xdata);
return 0;
}
-int32_t
-iot_symlink (call_frame_t *frame,
- xlator_t *this,
- const char *linkname,
- loc_t *loc)
+
+int
+iot_symlink (call_frame_t *frame, xlator_t *this, const char *linkname,
+ loc_t *loc, mode_t umask, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_symlink_stub (frame, iot_symlink_wrapper, linkname, loc);
+ stub = fop_symlink_stub (frame, iot_symlink_wrapper, linkname, loc,
+ umask, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get symlink stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR, "cannot create symlink stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (symlink, frame, -1, -ret, NULL, NULL, NULL,
+ NULL, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
}
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
return 0;
}
-int32_t
-iot_rename_cbk (call_frame_t *frame,
- void * cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
-{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+
+int
+iot_rename_cbk (call_frame_t *frame, void * cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *buf,
+ struct iatt *preoldparent, struct iatt *postoldparent,
+ struct iatt *prenewparent, struct iatt *postnewparent,
+ dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (rename, frame, op_ret, op_errno, buf, preoldparent,
+ postoldparent, prenewparent, postnewparent, xdata);
return 0;
}
-int32_t
-iot_rename_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *oldloc,
- loc_t *newloc)
+
+int
+iot_rename_wrapper (call_frame_t *frame, xlator_t *this, loc_t *oldloc,
+ loc_t *newloc, dict_t *xdata)
{
STACK_WIND (frame, iot_rename_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->rename, oldloc, newloc);
+ FIRST_CHILD (this)->fops->rename, oldloc, newloc, xdata);
return 0;
}
-int32_t
-iot_rename (call_frame_t *frame,
- xlator_t *this,
- loc_t *oldloc,
- loc_t *newloc)
+
+int
+iot_rename (call_frame_t *frame, xlator_t *this, loc_t *oldloc, loc_t *newloc,
+ dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_rename_stub (frame, iot_rename_wrapper, oldloc, newloc);
+ stub = fop_rename_stub (frame, iot_rename_wrapper, oldloc, newloc, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get rename stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_DEBUG, "cannot create rename stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (rename, frame, -1, -ret, NULL, NULL, NULL,
+ NULL, NULL, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
}
- iot_schedule_unordered ((iot_conf_t *)this->private, oldloc->inode,
- stub);
return 0;
}
-int32_t
-iot_open_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- fd_t *fd)
-{
- STACK_UNWIND (frame, op_ret, op_errno, fd);
+
+int
+iot_open_cbk (call_frame_t *frame, void *cookie, xlator_t *this, int32_t op_ret,
+ int32_t op_errno, fd_t *fd, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (open, frame, op_ret, op_errno, fd, xdata);
return 0;
}
-static int32_t
-iot_open_wrapper (call_frame_t * frame,
- xlator_t * this,
- loc_t *loc,
- int32_t flags,
- fd_t * fd)
+
+int
+iot_open_wrapper (call_frame_t * frame, xlator_t * this, loc_t *loc,
+ int32_t flags, fd_t * fd, dict_t *xdata)
{
STACK_WIND (frame, iot_open_cbk, FIRST_CHILD (this),
- FIRST_CHILD (this)->fops->open, loc, flags, fd);
+ FIRST_CHILD (this)->fops->open, loc, flags, fd,
+ xdata);
return 0;
}
-int32_t
-iot_open (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- int32_t flags,
- fd_t *fd)
+
+int
+iot_open (call_frame_t *frame, xlator_t *this, loc_t *loc, int32_t flags,
+ fd_t *fd, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_open_stub (frame, iot_open_wrapper, loc, flags, fd);
+ stub = fop_open_stub (frame, iot_open_wrapper, loc, flags, fd,
+ xdata);
if (!stub) {
gf_log (this->name, GF_LOG_ERROR,
- "cannot get open call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL, 0);
- return 0;
+ "cannot create open call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (open, frame, -1, -ret, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
}
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
return 0;
}
-int32_t
-iot_create_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- fd_t *fd,
- inode_t *inode,
- struct stat *stbuf)
-{
- STACK_UNWIND (frame, op_ret, op_errno, fd, inode, stbuf);
+int
+iot_create_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, fd_t *fd, inode_t *inode,
+ struct iatt *stbuf, struct iatt *preparent,
+ struct iatt *postparent, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (create, frame, op_ret, op_errno, fd, inode, stbuf,
+ preparent, postparent, xdata);
return 0;
}
-int32_t
-iot_create_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- int32_t flags,
- mode_t mode,
- fd_t *fd)
-{
- STACK_WIND (frame,
- iot_create_cbk,
+
+int
+iot_create_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ int32_t flags, mode_t mode, mode_t umask, fd_t *fd,
+ dict_t *xdata)
+{
+ STACK_WIND (frame, iot_create_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->create,
- loc,
- flags,
- mode,
- fd);
+ loc, flags, mode, umask, fd, xdata);
return 0;
}
-int32_t
-iot_create (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- int32_t flags,
- mode_t mode,
- fd_t *fd)
+
+int
+iot_create (call_frame_t *frame, xlator_t *this, loc_t *loc, int32_t flags,
+ mode_t mode, mode_t umask, fd_t *fd, dict_t *xdata)
{
call_stub_t *stub = NULL;
+ int ret = -1;
stub = fop_create_stub (frame, iot_create_wrapper, loc, flags, mode,
- fd);
+ umask, fd, xdata);
if (!stub) {
gf_log (this->name, GF_LOG_ERROR,
- "cannot get create call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL, 0);
- return 0;
+ "cannot create \"create\" call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (create, frame, -1, -ret, NULL, NULL, NULL,
+ NULL, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
}
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
+
return 0;
}
-int32_t
-iot_readv_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct iovec *vector,
- int32_t count,
- struct stat *stbuf)
+
+int
+iot_readv_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iovec *vector,
+ int32_t count, struct iatt *stbuf, struct iobref *iobref,
+ dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, vector, count, stbuf);
+ STACK_UNWIND_STRICT (readv, frame, op_ret, op_errno, vector, count,
+ stbuf, iobref, xdata);
return 0;
}
-static int32_t
-iot_readv_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- size_t size,
- off_t offset)
+
+int
+iot_readv_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd, size_t size,
+ off_t offset, uint32_t flags, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_readv_cbk,
+ STACK_WIND (frame, iot_readv_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->readv,
- fd,
- size,
- offset);
+ fd, size, offset, flags, xdata);
return 0;
}
-int32_t
-iot_readv (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- size_t size,
- off_t offset)
+
+int
+iot_readv (call_frame_t *frame, xlator_t *this, fd_t *fd, size_t size,
+ off_t offset, uint32_t flags, dict_t *xdata)
{
- call_stub_t *stub;
- stub = fop_readv_stub (frame,
- iot_readv_wrapper,
- fd,
- size,
- offset);
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_readv_stub (frame, iot_readv_wrapper, fd, size, offset,
+ flags, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR,
- "cannot get readv call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL, 0);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create readv call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (readv, frame, -1, -ret, NULL, -1, NULL,
+ NULL, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_flush_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno)
+
+int
+iot_flush_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno);
+ STACK_UNWIND_STRICT (flush, frame, op_ret, op_errno, xdata);
return 0;
}
-static int32_t
-iot_flush_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd)
+
+int
+iot_flush_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_flush_cbk,
+ STACK_WIND (frame, iot_flush_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->flush,
- fd);
+ fd, xdata);
return 0;
}
-int32_t
-iot_flush (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd)
+
+int
+iot_flush (call_frame_t *frame, xlator_t *this, fd_t *fd, dict_t *xdata)
{
- call_stub_t *stub;
- stub = fop_flush_stub (frame,
- iot_flush_wrapper,
- fd);
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_flush_stub (frame, iot_flush_wrapper, fd, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get flush_cbk call stub");
- STACK_UNWIND (frame, -1, ENOMEM);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create flush_cbk call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (flush, frame, -1, -ret, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_fsync_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno)
+
+int
+iot_fsync_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *prebuf,
+ struct iatt *postbuf, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno);
+ STACK_UNWIND_STRICT (fsync, frame, op_ret, op_errno, prebuf, postbuf,
+ xdata);
return 0;
}
-static int32_t
-iot_fsync_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- int32_t datasync)
+
+int
+iot_fsync_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ int32_t datasync, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_fsync_cbk,
+ STACK_WIND (frame, iot_fsync_cbk,
FIRST_CHILD (this),
FIRST_CHILD (this)->fops->fsync,
- fd,
- datasync);
+ fd, datasync, xdata);
return 0;
}
-int32_t
-iot_fsync (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- int32_t datasync)
+
+int
+iot_fsync (call_frame_t *frame, xlator_t *this, fd_t *fd, int32_t datasync,
+ dict_t *xdata)
{
- call_stub_t *stub;
- stub = fop_fsync_stub (frame,
- iot_fsync_wrapper,
- fd,
- datasync);
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_fsync_stub (frame, iot_fsync_wrapper, fd, datasync, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fsync_cbk call stub");
- STACK_UNWIND (frame, -1, ENOMEM);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create fsync_cbk call stub"
+ "(out of memory)");
+ ret = -1;
+ goto out;
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fsync, frame, -1, -ret, NULL, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_writev_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *stbuf)
-{
- STACK_UNWIND (frame, op_ret, op_errno, stbuf);
+
+int
+iot_writev_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *prebuf,
+ struct iatt *postbuf, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (writev, frame, op_ret, op_errno, prebuf, postbuf,
+ xdata);
return 0;
}
-static int32_t
-iot_writev_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- struct iovec *vector,
- int32_t count,
- off_t offset)
+
+int
+iot_writev_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ struct iovec *vector, int32_t count,
+ off_t offset, uint32_t flags, struct iobref *iobref,
+ dict_t *xdata)
{
- STACK_WIND (frame,
- iot_writev_cbk,
+ STACK_WIND (frame, iot_writev_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->writev,
- fd,
- vector,
- count,
- offset);
+ fd, vector, count, offset, flags, iobref, xdata);
return 0;
}
-int32_t
-iot_writev (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- struct iovec *vector,
- int32_t count,
- off_t offset)
+
+int
+iot_writev (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ struct iovec *vector, int32_t count, off_t offset,
+ uint32_t flags, struct iobref *iobref, dict_t *xdata)
{
- call_stub_t *stub;
- stub = fop_writev_stub (frame, iot_writev_wrapper,
- fd, vector, count, offset);
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_writev_stub (frame, iot_writev_wrapper, fd, vector,
+ count, offset, flags, iobref, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get writev call stub");
- STACK_UNWIND (frame, -1, ENOMEM);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create writev call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (writev, frame, -1, -ret, NULL, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
int32_t
-iot_lk_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct flock *flock)
-{
- STACK_UNWIND (frame, op_ret, op_errno, flock);
+iot_lk_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct gf_flock *flock,
+ dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (lk, frame, op_ret, op_errno, flock, xdata);
return 0;
}
-static int32_t
-iot_lk_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- int32_t cmd,
- struct flock *flock)
+int
+iot_lk_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ int32_t cmd, struct gf_flock *flock, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_lk_cbk,
+ STACK_WIND (frame, iot_lk_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->lk,
- fd,
- cmd,
- flock);
+ fd, cmd, flock, xdata);
return 0;
}
-int32_t
-iot_lk (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- int32_t cmd,
- struct flock *flock)
+int
+iot_lk (call_frame_t *frame, xlator_t *this, fd_t *fd, int32_t cmd,
+ struct gf_flock *flock, dict_t *xdata)
{
- call_stub_t *stub;
- stub = fop_lk_stub (frame, iot_lk_wrapper,
- fd, cmd, flock);
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_lk_stub (frame, iot_lk_wrapper, fd, cmd, flock, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fop_lk call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create fop_lk call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (lk, frame, -1, -ret, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_stat_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
+int
+iot_stat_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *buf, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+ STACK_UNWIND_STRICT (stat, frame, op_ret, op_errno, buf, xdata);
return 0;
}
-static int32_t
-iot_stat_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc)
+int
+iot_stat_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_stat_cbk,
+ STACK_WIND (frame, iot_stat_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->stat,
- loc);
+ loc, xdata);
return 0;
}
-int32_t
-iot_stat (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc)
+
+int
+iot_stat (call_frame_t *frame, xlator_t *this, loc_t *loc, dict_t *xdata)
{
- call_stub_t *stub;
- fd_t *fd = NULL;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_stat_stub (frame,
- iot_stat_wrapper,
- loc);
+ stub = fop_stat_stub (frame, iot_stat_wrapper, loc, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fop_stat call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create fop_stat call stub"
+ "(out of memory)");
+ ret = -1;
+ goto out;
}
- fd = fd_lookup (loc->inode, frame->root->pid);
- /* File is not open, so we can send it through unordered pool.
- */
- if (fd == NULL)
- iot_schedule_unordered ((iot_conf_t *)this->private,
- loc->inode, stub);
- else {
- iot_schedule_ordered ((iot_conf_t *)this->private, loc->inode,
- stub);
- fd_unref (fd);
- }
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (stat, frame, -1, -ret, NULL, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_fstat_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
+int
+iot_fstat_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *buf, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+ STACK_UNWIND_STRICT (fstat, frame, op_ret, op_errno, buf, xdata);
return 0;
}
-static int32_t
-iot_fstat_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd)
+
+int
+iot_fstat_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_fstat_cbk,
+ STACK_WIND (frame, iot_fstat_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->fstat,
- fd);
+ fd, xdata);
return 0;
}
-int32_t
-iot_fstat (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd)
+
+int
+iot_fstat (call_frame_t *frame, xlator_t *this, fd_t *fd, dict_t *xdata)
{
- call_stub_t *stub;
- stub = fop_fstat_stub (frame,
- iot_fstat_wrapper,
- fd);
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_fstat_stub (frame, iot_fstat_wrapper, fd, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fop_fstat call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create fop_fstat call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fstat, frame, -1, -ret, NULL, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_truncate_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
+
+int
+iot_truncate_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *prebuf,
+ struct iatt *postbuf, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+ STACK_UNWIND_STRICT (truncate, frame, op_ret, op_errno, prebuf,
+ postbuf, xdata);
return 0;
}
-static int32_t
-iot_truncate_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- off_t offset)
+
+int
+iot_truncate_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ off_t offset, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_truncate_cbk,
+ STACK_WIND (frame, iot_truncate_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->truncate,
- loc,
- offset);
+ loc, offset, xdata);
return 0;
}
-int32_t
-iot_truncate (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- off_t offset)
+
+int
+iot_truncate (call_frame_t *frame, xlator_t *this, loc_t *loc, off_t offset,
+ dict_t *xdata)
{
call_stub_t *stub;
- fd_t *fd = NULL;
+ int ret = -1;
- stub = fop_truncate_stub (frame,
- iot_truncate_wrapper,
- loc,
- offset);
+ stub = fop_truncate_stub (frame, iot_truncate_wrapper, loc, offset,
+ xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fop_stat call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create fop_stat call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- fd = fd_lookup (loc->inode, frame->root->pid);
- if (fd == NULL)
- iot_schedule_unordered ((iot_conf_t *)this->private,
- loc->inode, stub);
- else {
- iot_schedule_ordered ((iot_conf_t *)this->private, loc->inode,
- stub);
- fd_unref (fd);
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (truncate, frame, -1, -ret, NULL, NULL,
+ NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
}
return 0;
}
-int32_t
-iot_ftruncate_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
+
+int
+iot_ftruncate_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *prebuf,
+ struct iatt *postbuf, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+ STACK_UNWIND_STRICT (ftruncate, frame, op_ret, op_errno, prebuf,
+ postbuf, xdata);
return 0;
}
-static int32_t
-iot_ftruncate_wrapper (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- off_t offset)
+
+int
+iot_ftruncate_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ off_t offset, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_ftruncate_cbk,
+ STACK_WIND (frame, iot_ftruncate_cbk,
FIRST_CHILD(this),
FIRST_CHILD(this)->fops->ftruncate,
- fd,
- offset);
+ fd, offset, xdata);
return 0;
}
-int32_t
-iot_ftruncate (call_frame_t *frame,
- xlator_t *this,
- fd_t *fd,
- off_t offset)
+
+int
+iot_ftruncate (call_frame_t *frame, xlator_t *this, fd_t *fd, off_t offset,
+ dict_t *xdata)
{
- call_stub_t *stub;
- stub = fop_ftruncate_stub (frame,
- iot_ftruncate_wrapper,
- fd,
- offset);
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_ftruncate_stub (frame, iot_ftruncate_wrapper, fd, offset,
+ xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fop_ftruncate call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create fop_ftruncate call stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- iot_schedule_ordered ((iot_conf_t *)this->private, fd->inode, stub);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (ftruncate, frame, -1, -ret, NULL, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
return 0;
}
-int32_t
-iot_utimens_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- struct stat *buf)
+
+
+int
+iot_unlink_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct iatt *preparent,
+ struct iatt *postparent, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, buf);
+ STACK_UNWIND_STRICT (unlink, frame, op_ret, op_errno, preparent,
+ postparent, xdata);
return 0;
}
-static int32_t
-iot_utimens_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- struct timespec tv[2])
+
+int
+iot_unlink_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ int32_t xflag, dict_t *xdata)
{
- STACK_WIND (frame,
- iot_utimens_cbk,
+ STACK_WIND (frame, iot_unlink_cbk,
FIRST_CHILD(this),
- FIRST_CHILD(this)->fops->utimens,
- loc,
- tv);
-
+ FIRST_CHILD(this)->fops->unlink,
+ loc, xflag, xdata);
return 0;
}
-int32_t
-iot_utimens (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- struct timespec tv[2])
+
+int
+iot_unlink (call_frame_t *frame, xlator_t *this, loc_t *loc, int32_t xflag,
+ dict_t *xdata)
{
- call_stub_t *stub;
- fd_t *fd = NULL;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_utimens_stub (frame,
- iot_utimens_wrapper,
- loc,
- tv);
+ stub = fop_unlink_stub (frame, iot_unlink_wrapper, loc, xflag, xdata);
if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fop_utimens call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL);
- return 0;
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot create fop_unlink call stub"
+ "(out of memory)");
+ ret = -1;
+ goto out;
}
- fd = fd_lookup (loc->inode, frame->root->pid);
- if (fd == NULL)
- iot_schedule_unordered ((iot_conf_t *)this->private,
- loc->inode, stub);
- else {
- iot_schedule_ordered ((iot_conf_t *)this->private, loc->inode, stub);
- fd_unref (fd);
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (unlink, frame, -1, -ret, NULL, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
}
return 0;
}
-int32_t
-iot_checksum_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno,
- uint8_t *file_checksum,
- uint8_t *dir_checksum)
+int
+iot_link_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, inode_t *inode,
+ struct iatt *buf, struct iatt *preparent, struct iatt *postparent,
+ dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno, file_checksum, dir_checksum);
- return 0;
+ STACK_UNWIND_STRICT (link, frame, op_ret, op_errno, inode, buf,
+ preparent, postparent, xdata);
+ return 0;
}
-static int32_t
-iot_checksum_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- int32_t flags)
+
+int
+iot_link_wrapper (call_frame_t *frame, xlator_t *this, loc_t *old, loc_t *new,
+ dict_t *xdata)
{
- STACK_WIND (frame,
- iot_checksum_cbk,
- FIRST_CHILD(this),
- FIRST_CHILD(this)->fops->checksum,
- loc,
- flags);
-
- return 0;
+ STACK_WIND (frame, iot_link_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->link, old, new, xdata);
+
+ return 0;
}
-int32_t
-iot_checksum (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc,
- int32_t flags)
+
+int
+iot_link (call_frame_t *frame, xlator_t *this, loc_t *oldloc, loc_t *newloc,
+ dict_t *xdata)
{
- call_stub_t *stub = NULL;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- stub = fop_checksum_stub (frame,
- iot_checksum_wrapper,
- loc,
- flags);
- if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fop_checksum call stub");
- STACK_UNWIND (frame, -1, ENOMEM, NULL, NULL);
- return 0;
- }
- iot_schedule_unordered ((iot_conf_t *)this->private, loc->inode, stub);
+ stub = fop_link_stub (frame, iot_link_wrapper, oldloc, newloc, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create link stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
- return 0;
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (link, frame, -1, -ret, NULL, NULL, NULL,
+ NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
-int32_t
-iot_unlink_cbk (call_frame_t *frame,
- void *cookie,
- xlator_t *this,
- int32_t op_ret,
- int32_t op_errno)
+int
+iot_opendir_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, fd_t *fd, dict_t *xdata)
{
- STACK_UNWIND (frame, op_ret, op_errno);
- return 0;
+ STACK_UNWIND_STRICT (opendir, frame, op_ret, op_errno, fd, xdata);
+ return 0;
}
-static int32_t
-iot_unlink_wrapper (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc)
+
+int
+iot_opendir_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc, fd_t *fd,
+ dict_t *xdata)
{
- STACK_WIND (frame,
- iot_unlink_cbk,
- FIRST_CHILD(this),
- FIRST_CHILD(this)->fops->unlink,
- loc);
-
- return 0;
+ STACK_WIND (frame, iot_opendir_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->opendir, loc, fd, xdata);
+ return 0;
}
-int32_t
-iot_unlink (call_frame_t *frame,
- xlator_t *this,
- loc_t *loc)
+
+int
+iot_opendir (call_frame_t *frame, xlator_t *this, loc_t *loc, fd_t *fd,
+ dict_t *xdata)
{
- call_stub_t *stub = NULL;
- stub = fop_unlink_stub (frame, iot_unlink_wrapper, loc);
- if (!stub) {
- gf_log (this->name, GF_LOG_ERROR, "cannot get fop_unlink call stub");
- STACK_UNWIND (frame, -1, ENOMEM);
- return 0;
- }
- iot_schedule_unordered((iot_conf_t *)this->private, loc->inode, stub);
+ call_stub_t *stub = NULL;
+ int ret = -1;
- return 0;
+ stub = fop_opendir_stub (frame, iot_opendir_wrapper, loc, fd, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create opendir stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (opendir, frame, -1, -ret, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
-/* Must be called with worker lock held */
-void
-_iot_queue (iot_worker_t *worker,
- iot_request_t *req)
+
+int
+iot_fsyncdir_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
{
- list_add_tail (&req->list, &worker->rqlist);
+ STACK_UNWIND_STRICT (fsyncdir, frame, op_ret, op_errno, xdata);
+ return 0;
+}
+
- /* dq_cond */
- worker->queue_size++;
- pthread_cond_broadcast (&worker->dq_cond);
+int
+iot_fsyncdir_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ int datasync, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_fsyncdir_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->fsyncdir, fd, datasync, xdata);
+ return 0;
}
-iot_request_t *
-iot_init_request (call_stub_t *stub)
+
+int
+iot_fsyncdir (call_frame_t *frame, xlator_t *this, fd_t *fd, int datasync,
+ dict_t *xdata)
{
- iot_request_t *req = NULL;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- req = CALLOC (1, sizeof (iot_request_t));
- ERR_ABORT (req);
- req->stub = stub;
+ stub = fop_fsyncdir_stub (frame, iot_fsyncdir_wrapper, fd, datasync,
+ xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create fsyncdir stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
- return req;
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fsyncdir, frame, -1, -ret, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
-void
-iot_destroy_request (iot_request_t * req)
+
+int
+iot_statfs_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, struct statvfs *buf,
+ dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (statfs, frame, op_ret, op_errno, buf, xdata);
+ return 0;
+}
+
+
+int
+iot_statfs_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ dict_t *xdata)
{
- if (req == NULL)
- return;
+ STACK_WIND (frame, iot_statfs_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->statfs, loc, xdata);
+ return 0;
+}
+
+
+int
+iot_statfs (call_frame_t *frame, xlator_t *this, loc_t *loc, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_statfs_stub (frame, iot_statfs_wrapper, loc, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create statfs stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (statfs, frame, -1, -ret, NULL, NULL);
- FREE (req);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
-/* Must be called with worker lock held. */
-gf_boolean_t
-iot_can_ordered_exit (iot_worker_t * worker)
+
+int
+iot_setxattr_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
{
- gf_boolean_t allow_exit = _gf_false;
- iot_conf_t *conf = NULL;
+ STACK_UNWIND_STRICT (setxattr, frame, op_ret, op_errno, xdata);
+ return 0;
+}
- conf = worker->conf;
- /* We dont want this thread to exit if its index is
- * below the min thread count.
- */
- if (worker->thread_idx >= conf->min_o_threads)
- allow_exit = _gf_true;
- return allow_exit;
+int
+iot_setxattr_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ dict_t *dict, int32_t flags, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_setxattr_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->setxattr, loc, dict, flags, xdata);
+ return 0;
}
-/* Must be called with worker lock held. */
-gf_boolean_t
-iot_ordered_exit (iot_worker_t *worker)
+
+int
+iot_setxattr (call_frame_t *frame, xlator_t *this, loc_t *loc, dict_t *dict,
+ int32_t flags, dict_t *xdata)
{
- gf_boolean_t allow_exit = _gf_false;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- allow_exit = iot_can_ordered_exit (worker);
- if (allow_exit) {
- worker->state = IOT_STATE_DEAD;
- worker->thread = 0;
+ stub = fop_setxattr_stub (frame, iot_setxattr_wrapper, loc, dict,
+ flags, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create setxattr stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- return allow_exit;
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (setxattr, frame, -1, -ret, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
+
int
-iot_ordered_request_wait (iot_worker_t * worker)
+iot_getxattr_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *dict, dict_t *xdata)
{
- struct timeval tv;
- struct timespec ts;
- int waitres = 0;
- int retstat = 0;
+ STACK_UNWIND_STRICT (getxattr, frame, op_ret, op_errno, dict, xdata);
+ return 0;
+}
- gettimeofday (&tv, NULL);
- ts.tv_sec = tv.tv_sec + worker->conf->o_idle_time;
- /* Slightly skew the idle time for threads so that, we dont
- * have all of them rushing to exit at the same time, if
- * they've been idle.
- */
- ts.tv_nsec = skew_usec_idle_time (tv.tv_usec) * 1000;
- waitres = pthread_cond_timedwait (&worker->dq_cond, &worker->qlock,
- &ts);
- if (waitres == ETIMEDOUT)
- if (iot_ordered_exit (worker))
- retstat = -1;
- return retstat;
+int
+iot_getxattr_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ const char *name, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_getxattr_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->getxattr, loc, name, xdata);
+ return 0;
}
-call_stub_t *
-iot_dequeue_ordered (iot_worker_t *worker)
+
+int
+iot_getxattr (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ const char *name, dict_t *xdata)
{
- call_stub_t *stub = NULL;
- iot_request_t *req = NULL;
- int waitstat = 0;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- pthread_mutex_lock (&worker->qlock);
- {
- while (!worker->queue_size) {
- waitstat = 0;
- waitstat = iot_ordered_request_wait (worker);
- /* We must've timed out and are now required to
- * exit.
- */
- if (waitstat == -1)
- goto out;
+ stub = fop_getxattr_stub (frame, iot_getxattr_wrapper, loc, name, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create getxattr stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (getxattr, frame, -1, -ret, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
}
+ }
+ return 0;
+}
- list_for_each_entry (req, &worker->rqlist, list)
- break;
- list_del (&req->list);
- stub = req->stub;
- worker->queue_size--;
+int
+iot_fgetxattr_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *dict,
+ dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (fgetxattr, frame, op_ret, op_errno, dict, xdata);
+ return 0;
+}
+
+
+int
+iot_fgetxattr_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ const char *name, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_fgetxattr_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->fgetxattr, fd, name, xdata);
+ return 0;
+}
+
+
+int
+iot_fgetxattr (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ const char *name, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_fgetxattr_stub (frame, iot_fgetxattr_wrapper, fd, name, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create fgetxattr stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
+
+ ret = iot_schedule (frame, this, stub);
out:
- pthread_mutex_unlock (&worker->qlock);
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fgetxattr, frame, -1, -ret, NULL, NULL);
- FREE (req);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
+}
- return stub;
+
+int
+iot_fsetxattr_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (fsetxattr, frame, op_ret, op_errno, xdata);
+ return 0;
}
-void *
-iot_worker_ordered (void *arg)
+
+int
+iot_fsetxattr_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ dict_t *dict, int32_t flags, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_fsetxattr_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->fsetxattr, fd, dict, flags,
+ xdata);
+ return 0;
+}
+
+
+int
+iot_fsetxattr (call_frame_t *frame, xlator_t *this, fd_t *fd, dict_t *dict,
+ int32_t flags, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_fsetxattr_stub (frame, iot_fsetxattr_wrapper, fd, dict,
+ flags, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create fsetxattr stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fsetxattr, frame, -1, -ret, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
+}
+
+
+int
+iot_removexattr_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (removexattr, frame, op_ret, op_errno, xdata);
+ return 0;
+}
+
+
+int
+iot_removexattr_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ const char *name, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_removexattr_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->removexattr, loc, name, xdata);
+ return 0;
+}
+
+
+int
+iot_removexattr (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ const char *name, dict_t *xdata)
{
- iot_worker_t *worker = arg;
call_stub_t *stub = NULL;
+ int ret = -1;
- while (1) {
+ stub = fop_removexattr_stub (frame, iot_removexattr_wrapper, loc,
+ name, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR,"cannot get removexattr fop"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
- stub = iot_dequeue_ordered (worker);
- /* If stub is NULL, we must've timed out waiting for a
- * request and have now been allowed to exit.
- */
- if (stub == NULL)
- break;
- call_resume (stub);
- }
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (removexattr, frame, -1, -ret, NULL);
- return NULL;
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
-/* Must be called with worker lock held. */
-gf_boolean_t
-iot_can_unordered_exit (iot_worker_t * worker)
+int
+iot_fremovexattr_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
{
- gf_boolean_t allow_exit = _gf_false;
- iot_conf_t *conf = NULL;
+ STACK_UNWIND_STRICT (fremovexattr, frame, op_ret, op_errno, xdata);
+ return 0;
+}
- conf = worker->conf;
- /* We dont want this thread to exit if its index is
- * below the min thread count.
- */
- if (worker->thread_idx >= conf->min_u_threads)
- allow_exit = _gf_true;
- return allow_exit;
+int
+iot_fremovexattr_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ const char *name, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_fremovexattr_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->fremovexattr, fd, name, xdata);
+ return 0;
}
-/* Must be called with worker lock held. */
-gf_boolean_t
-iot_unordered_exit (iot_worker_t *worker)
+
+int
+iot_fremovexattr (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ const char *name, dict_t *xdata)
{
- gf_boolean_t allow_exit = _gf_false;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- allow_exit = iot_can_unordered_exit (worker);
- if (allow_exit) {
- worker->state = IOT_STATE_DEAD;
- worker->thread = 0;
+ stub = fop_fremovexattr_stub (frame, iot_fremovexattr_wrapper, fd,
+ name, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR,"cannot get fremovexattr fop"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
- return allow_exit;
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fremovexattr, frame, -1, -ret, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
int
-iot_unordered_request_wait (iot_worker_t * worker)
+iot_readdirp_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, gf_dirent_t *entries,
+ dict_t *xdata)
{
- struct timeval tv;
- struct timespec ts;
- int waitres = 0;
- int retstat = 0;
+ STACK_UNWIND_STRICT (readdirp, frame, op_ret, op_errno, entries, xdata);
+ return 0;
+}
- gettimeofday (&tv, NULL);
- ts.tv_sec = tv.tv_sec + worker->conf->u_idle_time;
- /* Slightly skew the idle time for threads so that, we dont
- * have all of them rushing to exit at the same time, if
- * they've been idle.
- */
- ts.tv_nsec = skew_usec_idle_time (tv.tv_usec) * 1000;
- waitres = pthread_cond_timedwait (&worker->dq_cond, &worker->qlock,
- &ts);
- if (waitres == ETIMEDOUT)
- if (iot_unordered_exit (worker))
- retstat = -1;
- return retstat;
+int
+iot_readdirp_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ size_t size, off_t offset, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_readdirp_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->readdirp, fd, size, offset, xdata);
+ return 0;
}
-call_stub_t *
-iot_dequeue_unordered (iot_worker_t *worker)
+int
+iot_readdirp (call_frame_t *frame, xlator_t *this, fd_t *fd, size_t size,
+ off_t offset, dict_t *xdata)
{
- call_stub_t *stub= NULL;
- iot_request_t *req = NULL;
- int waitstat = 0;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- pthread_mutex_lock (&worker->qlock);
- {
- while (!worker->queue_size) {
- waitstat = 0;
- waitstat = iot_unordered_request_wait (worker);
- /* If -1, request wait must've timed
- * out.
- */
- if (waitstat == -1)
- goto out;
+ stub = fop_readdirp_stub (frame, iot_readdirp_wrapper, fd, size,
+ offset, xdata);
+ if (!stub) {
+ gf_log (this->private, GF_LOG_ERROR,"cannot get readdir stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (readdirp, frame, -1, -ret, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
}
+ }
+ return 0;
+}
- list_for_each_entry (req, &worker->rqlist, list)
- break;
- list_del (&req->list);
- stub = req->stub;
- worker->queue_size--;
+int
+iot_readdir_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, gf_dirent_t *entries,
+ dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (readdir, frame, op_ret, op_errno, entries, xdata);
+ return 0;
+}
+
+
+int
+iot_readdir_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ size_t size, off_t offset, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_readdir_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->readdir, fd, size, offset, xdata);
+ return 0;
+}
+
+
+int
+iot_readdir (call_frame_t *frame, xlator_t *this, fd_t *fd, size_t size,
+ off_t offset, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_readdir_stub (frame, iot_readdir_wrapper, fd, size, offset,
+ xdata);
+ if (!stub) {
+ gf_log (this->private, GF_LOG_ERROR,"cannot get readdir stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
+
+ ret = iot_schedule (frame, this, stub);
out:
- pthread_mutex_unlock (&worker->qlock);
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (readdir, frame, -1, -ret, NULL, NULL);
- FREE (req);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
+}
- return stub;
+int
+iot_inodelk_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (inodelk, frame, op_ret, op_errno, xdata);
+ return 0;
}
-void *
-iot_worker_unordered (void *arg)
+int
+iot_inodelk_wrapper (call_frame_t *frame, xlator_t *this, const char *volume,
+ loc_t *loc, int32_t cmd, struct gf_flock *lock,
+ dict_t *xdata)
+{
+ STACK_WIND (frame, iot_inodelk_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->inodelk, volume, loc, cmd, lock,
+ xdata);
+ return 0;
+}
+
+
+int
+iot_inodelk (call_frame_t *frame, xlator_t *this,
+ const char *volume, loc_t *loc, int32_t cmd, struct gf_flock *lock,
+ dict_t *xdata)
{
- iot_worker_t *worker = arg;
call_stub_t *stub = NULL;
+ int ret = -1;
- while (1) {
+ stub = fop_inodelk_stub (frame, iot_inodelk_wrapper,
+ volume, loc, cmd, lock, xdata);
+ if (!stub) {
+ ret = -ENOMEM;
+ goto out;
+ }
- stub = iot_dequeue_unordered (worker);
- /* If no request was received, we must've timed out,
- * and can exit. */
- if (stub == NULL)
- break;
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (inodelk, frame, -1, -ret, NULL);
- call_resume (stub);
- }
- return NULL;
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
+int
+iot_finodelk_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (finodelk, frame, op_ret, op_errno, xdata);
+ return 0;
+}
-static iot_worker_t **
-allocate_worker_array (int count)
+
+int
+iot_finodelk_wrapper (call_frame_t *frame, xlator_t *this,
+ const char *volume, fd_t *fd, int32_t cmd,
+ struct gf_flock *lock, dict_t *xdata)
{
- iot_worker_t ** warr = NULL;
+ STACK_WIND (frame, iot_finodelk_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->finodelk, volume, fd, cmd, lock,
+ xdata);
+ return 0;
+}
- warr = CALLOC (count, sizeof(iot_worker_t *));
- ERR_ABORT (warr);
- return warr;
+int
+iot_finodelk (call_frame_t *frame, xlator_t *this,
+ const char *volume, fd_t *fd, int32_t cmd, struct gf_flock *lock,
+ dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_finodelk_stub (frame, iot_finodelk_wrapper,
+ volume, fd, cmd, lock, xdata);
+ if (!stub) {
+ gf_log (this->private, GF_LOG_ERROR,"cannot get finodelk stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (finodelk, frame, -1, -ret, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
-static iot_worker_t *
-allocate_worker (iot_conf_t * conf)
+int
+iot_entrylk_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
{
- iot_worker_t *wrk = NULL;
+ STACK_UNWIND_STRICT (entrylk, frame, op_ret, op_errno, xdata);
+ return 0;
+}
+
+
+int
+iot_entrylk_wrapper (call_frame_t *frame, xlator_t *this,
+ const char *volume, loc_t *loc, const char *basename,
+ entrylk_cmd cmd, entrylk_type type, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_entrylk_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->entrylk,
+ volume, loc, basename, cmd, type, xdata);
+ return 0;
+}
+
+
+int
+iot_entrylk (call_frame_t *frame, xlator_t *this,
+ const char *volume, loc_t *loc, const char *basename,
+ entrylk_cmd cmd, entrylk_type type, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_entrylk_stub (frame, iot_entrylk_wrapper,
+ volume, loc, basename, cmd, type, xdata);
+ if (!stub) {
+ gf_log (this->private, GF_LOG_ERROR,"cannot get entrylk stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (entrylk, frame, -1, -ret, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
+}
- wrk = CALLOC (1, sizeof (iot_worker_t));
- ERR_ABORT (wrk);
+int
+iot_fentrylk_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (fentrylk, frame, op_ret, op_errno, xdata);
+ return 0;
+}
- INIT_LIST_HEAD (&wrk->rqlist);
- wrk->conf = conf;
- pthread_cond_init (&wrk->dq_cond, NULL);
- pthread_mutex_init (&wrk->qlock, NULL);
- wrk->state = IOT_STATE_DEAD;
- return wrk;
+int
+iot_fentrylk_wrapper (call_frame_t *frame, xlator_t *this,
+ const char *volume, fd_t *fd, const char *basename,
+ entrylk_cmd cmd, entrylk_type type, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_fentrylk_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->fentrylk,
+ volume, fd, basename, cmd, type, xdata);
+ return 0;
}
-static void
-allocate_workers (iot_conf_t *conf,
- iot_worker_t ** workers,
- int start_alloc_idx,
- int count)
+
+int
+iot_fentrylk (call_frame_t *frame, xlator_t *this,
+ const char *volume, fd_t *fd, const char *basename,
+ entrylk_cmd cmd, entrylk_type type, dict_t *xdata)
{
- int i, end_count;
+ call_stub_t *stub = NULL;
+ int ret = -1;
- end_count = count + start_alloc_idx;
- for (i = start_alloc_idx; i < end_count; i++) {
- workers[i] = allocate_worker (conf);
- workers[i]->thread_idx = i;
+ stub = fop_fentrylk_stub (frame, iot_fentrylk_wrapper,
+ volume, fd, basename, cmd, type, xdata);
+ if (!stub) {
+ gf_log (this->private, GF_LOG_ERROR,"cannot get fentrylk stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
}
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fentrylk, frame, -1, -ret, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
-void
-iot_startup_worker (iot_worker_t *worker, iot_worker_fn workerfunc)
+
+int
+iot_xattrop_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xattr, dict_t *xdata)
{
- worker->state = IOT_STATE_ACTIVE;
- pthread_create (&worker->thread, &worker->conf->w_attr, workerfunc,
- worker);
+ STACK_UNWIND_STRICT (xattrop, frame, op_ret, op_errno, xattr, xdata);
+ return 0;
}
-void
-iot_startup_workers (iot_worker_t **workers, int start_idx, int count,
- iot_worker_fn workerfunc)
+int
+iot_xattrop_wrapper (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ gf_xattrop_flags_t optype, dict_t *xattr, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_xattrop_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->xattrop, loc, optype, xattr, xdata);
+ return 0;
+}
+
+
+int
+iot_xattrop (call_frame_t *frame, xlator_t *this, loc_t *loc,
+ gf_xattrop_flags_t optype, dict_t *xattr, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_xattrop_stub (frame, iot_xattrop_wrapper, loc, optype,
+ xattr, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create xattrop stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (xattrop, frame, -1, -ret, NULL, NULL);
+
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
+}
+
+
+int
+iot_fxattrop_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, dict_t *xattr, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (fxattrop, frame, op_ret, op_errno, xattr, xdata);
+ return 0;
+}
+
+int
+iot_fxattrop_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ gf_xattrop_flags_t optype, dict_t *xattr, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_fxattrop_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->fxattrop, fd, optype, xattr, xdata);
+ return 0;
+}
+
+
+int
+iot_fxattrop (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ gf_xattrop_flags_t optype, dict_t *xattr, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_fxattrop_stub (frame, iot_fxattrop_wrapper, fd, optype,
+ xattr, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create fxattrop stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fxattrop, frame, -1, -ret, NULL, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
+}
+
+
+int32_t
+iot_rchecksum_cbk (call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno, uint32_t weak_checksum,
+ uint8_t *strong_checksum, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (rchecksum, frame, op_ret, op_errno, weak_checksum,
+ strong_checksum, xdata);
+ return 0;
+}
+
+
+int32_t
+iot_rchecksum_wrapper (call_frame_t *frame, xlator_t *this, fd_t *fd,
+ off_t offset, int32_t len, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_rchecksum_cbk, FIRST_CHILD(this),
+ FIRST_CHILD(this)->fops->rchecksum, fd, offset, len, xdata);
+ return 0;
+}
+
+
+int32_t
+iot_rchecksum (call_frame_t *frame, xlator_t *this, fd_t *fd, off_t offset,
+ int32_t len, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_rchecksum_stub (frame, iot_rchecksum_wrapper, fd, offset,
+ len, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create rchecksum stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (rchecksum, frame, -1, -ret, -1, NULL, NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+
+ return 0;
+}
+
+int
+iot_fallocate_cbk(call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno,
+ struct iatt *preop, struct iatt *postop, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (fallocate, frame, op_ret, op_errno, preop, postop,
+ xdata);
+ return 0;
+}
+
+
+int
+iot_fallocate_wrapper(call_frame_t *frame, xlator_t *this, fd_t *fd, int32_t mode,
+ off_t offset, size_t len, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_fallocate_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->fallocate, fd, mode, offset, len,
+ xdata);
+ return 0;
+}
+
+
+int
+iot_fallocate(call_frame_t *frame, xlator_t *this, fd_t *fd, int32_t mode,
+ off_t offset, size_t len, dict_t *xdata)
{
- int i = 0;
- int end_idx = 0;
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_fallocate_stub(frame, iot_fallocate_wrapper, fd, mode, offset,
+ len, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create fallocate stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
- end_idx = start_idx + count;
- for (i = start_idx; i < end_idx; i++)
- iot_startup_worker (workers[i], workerfunc);
+ ret = iot_schedule (frame, this, stub);
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (fallocate, frame, -1, -ret, NULL, NULL,
+ NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
}
-static void
+int
+iot_discard_cbk(call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno,
+ struct iatt *preop, struct iatt *postop, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (discard, frame, op_ret, op_errno, preop, postop,
+ xdata);
+ return 0;
+}
+
+
+int
+iot_discard_wrapper(call_frame_t *frame, xlator_t *this, fd_t *fd, off_t offset,
+ size_t len, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_discard_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->discard, fd, offset, len, xdata);
+ return 0;
+}
+
+
+int
+iot_discard(call_frame_t *frame, xlator_t *this, fd_t *fd, off_t offset,
+ size_t len, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_discard_stub(frame, iot_discard_wrapper, fd, offset, len,
+ xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create discard stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (discard, frame, -1, -ret, NULL, NULL,
+ NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
+}
+
+int
+iot_zerofill_cbk(call_frame_t *frame, void *cookie, xlator_t *this,
+ int32_t op_ret, int32_t op_errno,
+ struct iatt *preop, struct iatt *postop, dict_t *xdata)
+{
+ STACK_UNWIND_STRICT (zerofill, frame, op_ret, op_errno, preop, postop,
+ xdata);
+ return 0;
+}
+
+int
+iot_zerofill_wrapper(call_frame_t *frame, xlator_t *this, fd_t *fd,
+ off_t offset, size_t len, dict_t *xdata)
+{
+ STACK_WIND (frame, iot_zerofill_cbk, FIRST_CHILD (this),
+ FIRST_CHILD (this)->fops->zerofill, fd, offset, len, xdata);
+ return 0;
+}
+
+int
+iot_zerofill(call_frame_t *frame, xlator_t *this, fd_t *fd, off_t offset,
+ size_t len, dict_t *xdata)
+{
+ call_stub_t *stub = NULL;
+ int ret = -1;
+
+ stub = fop_zerofill_stub(frame, iot_zerofill_wrapper, fd,
+ offset, len, xdata);
+ if (!stub) {
+ gf_log (this->name, GF_LOG_ERROR, "cannot create zerofill stub"
+ "(out of memory)");
+ ret = -ENOMEM;
+ goto out;
+ }
+
+ ret = iot_schedule (frame, this, stub);
+
+out:
+ if (ret < 0) {
+ STACK_UNWIND_STRICT (zerofill, frame, -1, -ret, NULL, NULL,
+ NULL);
+ if (stub != NULL) {
+ call_stub_destroy (stub);
+ }
+ }
+ return 0;
+}
+
+
+int
+__iot_workers_scale (iot_conf_t *conf)
+{
+ int scale = 0;
+ int diff = 0;
+ pthread_t thread;
+ int ret = 0;
+ int i = 0;
+
+ for (i = 0; i < IOT_PRI_MAX; i++)
+ scale += min (conf->queue_sizes[i], conf->ac_iot_limit[i]);
+
+ if (scale < IOT_MIN_THREADS)
+ scale = IOT_MIN_THREADS;
+
+ if (scale > conf->max_count)
+ scale = conf->max_count;
+
+ if (conf->curr_count < scale) {
+ diff = scale - conf->curr_count;
+ }
+
+ while (diff) {
+ diff --;
+
+ ret = gf_thread_create (&thread, &conf->w_attr, iot_worker, conf);
+ if (ret == 0) {
+ conf->curr_count++;
+ gf_log (conf->this->name, GF_LOG_DEBUG,
+ "scaled threads to %d (queue_size=%d/%d)",
+ conf->curr_count, conf->queue_size, scale);
+ } else {
+ break;
+ }
+ }
+
+ return diff;
+}
+
+
+int
+iot_workers_scale (iot_conf_t *conf)
+{
+ int ret = -1;
+
+ if (conf == NULL) {
+ ret = -EINVAL;
+ goto out;
+ }
+
+ pthread_mutex_lock (&conf->mutex);
+ {
+ ret = __iot_workers_scale (conf);
+ }
+ pthread_mutex_unlock (&conf->mutex);
+
+out:
+ return ret;
+}
+
+
+void
set_stack_size (iot_conf_t *conf)
{
int err = 0;
size_t stacksize = IOT_THREAD_STACK_SIZE;
+ xlator_t *this = NULL;
+
+ this = THIS;
pthread_attr_init (&conf->w_attr);
err = pthread_attr_setstacksize (&conf->w_attr, stacksize);
if (err == EINVAL) {
- gf_log (conf->this->name, GF_LOG_WARNING,
+ err = pthread_attr_getstacksize (&conf->w_attr, &stacksize);
+ if (!err)
+ gf_log (this->name, GF_LOG_WARNING,
+ "Using default thread stack size %zd",
+ stacksize);
+ else
+ gf_log (this->name, GF_LOG_WARNING,
"Using default thread stack size");
- stacksize = 0;
}
- pthread_attr_setstacksize (&conf->w_attr, stacksize);
+ conf->stack_size = stacksize;
+}
+
+int32_t
+mem_acct_init (xlator_t *this)
+{
+ int ret = -1;
+
+ if (!this)
+ return ret;
+
+ ret = xlator_mem_acct_init (this, gf_iot_mt_end + 1);
+
+ if (ret != 0) {
+ gf_log (this->name, GF_LOG_ERROR, "Memory accounting init"
+ "failed");
+ return ret;
+ }
+
+ return ret;
}
-static void
-workers_init (iot_conf_t *conf)
+int
+iot_priv_dump (xlator_t *this)
{
- /* Initialize un-ordered workers */
- conf->uworkers = allocate_worker_array (conf->max_u_threads);
- allocate_workers (conf, conf->uworkers, 0, conf->max_u_threads);
+ iot_conf_t *conf = NULL;
+ char key_prefix[GF_DUMP_MAX_BUF_LEN];
- /* Initialize ordered workers */
- conf->oworkers = allocate_worker_array (conf->max_o_threads);
- allocate_workers (conf, conf->oworkers, 0, conf->max_o_threads);
+ if (!this)
+ return 0;
- set_stack_size (conf);
- iot_startup_workers (conf->oworkers, 0, conf->min_o_threads,
- iot_worker_ordered);
- iot_startup_workers (conf->uworkers, 0, conf->min_u_threads,
- iot_worker_unordered);
+ conf = this->private;
+ if (!conf)
+ return 0;
+
+ snprintf (key_prefix, GF_DUMP_MAX_BUF_LEN, "%s.%s", this->type,
+ this->name);
+
+ gf_proc_dump_add_section(key_prefix);
+
+ gf_proc_dump_write("maximum_threads_count", "%d", conf->max_count);
+ gf_proc_dump_write("current_threads_count", "%d", conf->curr_count);
+ gf_proc_dump_write("sleep_count", "%d", conf->sleep_count);
+ gf_proc_dump_write("idle_time", "%d", conf->idle_time);
+ gf_proc_dump_write("stack_size", "%zd", conf->stack_size);
+ gf_proc_dump_write("high_priority_threads", "%d",
+ conf->ac_iot_limit[IOT_PRI_HI]);
+ gf_proc_dump_write("normal_priority_threads", "%d",
+ conf->ac_iot_limit[IOT_PRI_NORMAL]);
+ gf_proc_dump_write("low_priority_threads", "%d",
+ conf->ac_iot_limit[IOT_PRI_LO]);
+ gf_proc_dump_write("least_priority_threads", "%d",
+ conf->ac_iot_limit[IOT_PRI_LEAST]);
+
+ gf_proc_dump_write("cached least rate", "%u",
+ conf->throttle.cached_rate);
+ gf_proc_dump_write("least rate limit", "%u", conf->throttle.rate_limit);
+
+ return 0;
}
+int
+reconfigure (xlator_t *this, dict_t *options)
+{
+ iot_conf_t *conf = NULL;
+ int ret = -1;
+
+ conf = this->private;
+ if (!conf)
+ goto out;
+
+ GF_OPTION_RECONF ("thread-count", conf->max_count, options, int32, out);
+
+ GF_OPTION_RECONF ("high-prio-threads",
+ conf->ac_iot_limit[IOT_PRI_HI], options, int32, out);
+
+ GF_OPTION_RECONF ("normal-prio-threads",
+ conf->ac_iot_limit[IOT_PRI_NORMAL], options, int32,
+ out);
-int32_t
+ GF_OPTION_RECONF ("low-prio-threads",
+ conf->ac_iot_limit[IOT_PRI_LO], options, int32, out);
+
+ GF_OPTION_RECONF ("least-prio-threads",
+ conf->ac_iot_limit[IOT_PRI_LEAST], options, int32,
+ out);
+ GF_OPTION_RECONF ("enable-least-priority", conf->least_priority,
+ options, bool, out);
+
+ GF_OPTION_RECONF("least-rate-limit", conf->throttle.rate_limit, options,
+ int32, out);
+
+ ret = 0;
+out:
+ return ret;
+}
+
+
+int
init (xlator_t *this)
{
- iot_conf_t *conf = NULL;
- dict_t *options = this->options;
- int thread_count = IOT_MIN_THREADS;
- gf_boolean_t autoscaling = IOT_SCALING_OFF;
- char *scalestr = NULL;
- int min_threads, max_threads;
+ iot_conf_t *conf = NULL;
+ int ret = -1;
+ int i = 0;
if (!this->children || this->children->next) {
- gf_log ("io-threads",
- GF_LOG_ERROR,
+ gf_log ("io-threads", GF_LOG_ERROR,
"FATAL: iot not configured with exactly one child");
- return -1;
+ goto out;
}
if (!this->parents) {
@@ -1836,177 +2759,206 @@ init (xlator_t *this)
"dangling volume. check volfile ");
}
- conf = (void *) CALLOC (1, sizeof (*conf));
- ERR_ABORT (conf);
+ conf = (void *) GF_CALLOC (1, sizeof (*conf),
+ gf_iot_mt_iot_conf_t);
+ if (conf == NULL) {
+ gf_log (this->name, GF_LOG_ERROR,
+ "out of memory");
+ goto out;
+ }
- if ((dict_get_str (options, "autoscaling", &scalestr)) == 0) {
- if ((gf_string2boolean (scalestr, &autoscaling)) == -1) {
- gf_log (this->name, GF_LOG_ERROR,
- "'autoscaling' option must be"
- " boolean");
- return -1;
- }
+ if ((ret = pthread_cond_init(&conf->cond, NULL)) != 0) {
+ gf_log (this->name, GF_LOG_ERROR,
+ "pthread_cond_init failed (%d)", ret);
+ goto out;
+ }
+
+ if ((ret = pthread_mutex_init(&conf->mutex, NULL)) != 0) {
+ gf_log (this->name, GF_LOG_ERROR,
+ "pthread_mutex_init failed (%d)", ret);
+ goto out;
+ }
+
+ set_stack_size (conf);
+
+ GF_OPTION_INIT ("thread-count", conf->max_count, int32, out);
+
+ GF_OPTION_INIT ("high-prio-threads",
+ conf->ac_iot_limit[IOT_PRI_HI], int32, out);
+
+ GF_OPTION_INIT ("normal-prio-threads",
+ conf->ac_iot_limit[IOT_PRI_NORMAL], int32, out);
+
+ GF_OPTION_INIT ("low-prio-threads",
+ conf->ac_iot_limit[IOT_PRI_LO], int32, out);
+
+ GF_OPTION_INIT ("least-prio-threads",
+ conf->ac_iot_limit[IOT_PRI_LEAST], int32, out);
+
+ GF_OPTION_INIT ("idle-time", conf->idle_time, int32, out);
+ GF_OPTION_INIT ("enable-least-priority", conf->least_priority,
+ bool, out);
+
+ GF_OPTION_INIT("least-rate-limit", conf->throttle.rate_limit, int32,
+ out);
+ if ((ret = pthread_mutex_init(&conf->throttle.lock, NULL)) != 0) {
+ gf_log (this->name, GF_LOG_ERROR,
+ "pthread_mutex_init failed (%d)", ret);
+ goto out;
}
- if (dict_get (options, "thread-count")) {
- thread_count = data_to_int32 (dict_get (options,
- "thread-count"));
- if (scalestr != NULL)
- gf_log (this->name, GF_LOG_WARNING,
- "'thread-count' is specified with "
- "'autoscaling' on. Ignoring"
- "'thread-count' option.");
- if (thread_count < 2)
- thread_count = 2;
- }
-
- min_threads = IOT_MIN_THREADS;
- max_threads = IOT_MAX_THREADS;
- if (dict_get (options, "min-threads"))
- min_threads = data_to_int32 (dict_get (options,
- "min-threads"));
-
- if (dict_get (options, "max-threads"))
- max_threads = data_to_int32 (dict_get (options,
- "max-threads"));
-
- if (min_threads > max_threads) {
- gf_log (this->name, GF_LOG_ERROR, " min-threads must be less "
- "than max-threads");
- return -1;
- }
-
- /* If autoscaling is off, then adjust the min and max
- * threads according to thread-count.
- * This is based on the assumption that despite autoscaling
- * being off, we still want to have separate pools for data
- * and meta-data threads.
- */
- if (!autoscaling)
- max_threads = min_threads = thread_count;
-
- /* If user specifies an odd number of threads, increase it by
- * one. The reason for having an even number of threads is
- * explained later.
- */
- if (max_threads % 2)
- max_threads++;
-
- if(min_threads % 2)
- min_threads++;
-
- /* If the user wants to have only a single thread for
- * some strange reason, make sure we set this count to
- * 2. Explained later.
- */
- if (min_threads < 2)
- min_threads = 2;
-
- /* Again, have atleast two. Read on. */
- if (max_threads < 2)
- max_threads = 2;
-
- /* This is why we need atleast two threads.
- * We're dividing the specified thread pool into
- * 2 halves, equally between ordered and unordered
- * pools.
- */
-
- /* Init params for un-ordered workers. */
- pthread_mutex_init (&conf->utlock, NULL);
- conf->max_u_threads = max_threads / 2;
- conf->min_u_threads = min_threads / 2;
- conf->u_idle_time = IOT_DEFAULT_IDLE;
- conf->u_scaling = autoscaling;
-
- /* Init params for ordered workers. */
- pthread_mutex_init (&conf->otlock, NULL);
- conf->max_o_threads = max_threads / 2;
- conf->min_o_threads = min_threads / 2;
- conf->o_idle_time = IOT_DEFAULT_IDLE;
- conf->o_scaling = autoscaling;
-
- gf_log (this->name, GF_LOG_DEBUG, "io-threads: Autoscaling: %s, "
- "min_threads: %d, max_threads: %d",
- (autoscaling) ? "on":"off", min_threads, max_threads);
conf->this = this;
- workers_init (conf);
+
+ for (i = 0; i < IOT_PRI_MAX; i++) {
+ INIT_LIST_HEAD (&conf->reqs[i]);
+ }
+
+ ret = iot_workers_scale (conf);
+
+ if (ret == -1) {
+ gf_log (this->name, GF_LOG_ERROR,
+ "cannot initialize worker threads, exiting init");
+ goto out;
+ }
this->private = conf;
- return 0;
+ ret = 0;
+out:
+ if (ret)
+ GF_FREE (conf);
+
+ return ret;
}
+
void
fini (xlator_t *this)
{
iot_conf_t *conf = this->private;
- FREE (conf);
+ GF_FREE (conf);
this->private = NULL;
return;
}
-/*
- * O - Goes to ordered threadpool.
- * U - Goes to un-ordered threadpool.
- * V - Variable, depends on whether the file is open.
- * If it is, then goes to ordered, otherwise to
- * un-ordered.
- */
-struct xlator_fops fops = {
- .open = iot_open, /* U */
- .create = iot_create, /* U */
- .readv = iot_readv, /* O */
- .writev = iot_writev, /* O */
- .flush = iot_flush, /* O */
- .fsync = iot_fsync, /* O */
- .lk = iot_lk, /* O */
- .stat = iot_stat, /* V */
- .fstat = iot_fstat, /* O */
- .truncate = iot_truncate, /* V */
- .ftruncate = iot_ftruncate, /* O */
- .utimens = iot_utimens, /* V */
- .checksum = iot_checksum, /* U */
- .unlink = iot_unlink, /* U */
- .lookup = iot_lookup, /* U */
- .chmod = iot_chmod, /* V */
- .fchmod = iot_fchmod, /* O */
- .chown = iot_chown, /* V */
- .fchown = iot_fchown, /* O */
- .access = iot_access, /* U */
- .readlink = iot_readlink, /* U */
- .mknod = iot_mknod, /* U */
- .mkdir = iot_mkdir, /* U */
- .rmdir = iot_rmdir, /* U */
- .symlink = iot_symlink, /* U */
- .rename = iot_rename, /* U */
+struct xlator_dumpops dumpops = {
+ .priv = iot_priv_dump,
};
-struct xlator_mops mops = {
+struct xlator_fops fops = {
+ .open = iot_open,
+ .create = iot_create,
+ .readv = iot_readv,
+ .writev = iot_writev,
+ .flush = iot_flush,
+ .fsync = iot_fsync,
+ .lk = iot_lk,
+ .stat = iot_stat,
+ .fstat = iot_fstat,
+ .truncate = iot_truncate,
+ .ftruncate = iot_ftruncate,
+ .unlink = iot_unlink,
+ .lookup = iot_lookup,
+ .setattr = iot_setattr,
+ .fsetattr = iot_fsetattr,
+ .access = iot_access,
+ .readlink = iot_readlink,
+ .mknod = iot_mknod,
+ .mkdir = iot_mkdir,
+ .rmdir = iot_rmdir,
+ .symlink = iot_symlink,
+ .rename = iot_rename,
+ .link = iot_link,
+ .opendir = iot_opendir,
+ .fsyncdir = iot_fsyncdir,
+ .statfs = iot_statfs,
+ .setxattr = iot_setxattr,
+ .getxattr = iot_getxattr,
+ .fgetxattr = iot_fgetxattr,
+ .fsetxattr = iot_fsetxattr,
+ .removexattr = iot_removexattr,
+ .fremovexattr = iot_fremovexattr,
+ .readdir = iot_readdir,
+ .readdirp = iot_readdirp,
+ .inodelk = iot_inodelk,
+ .finodelk = iot_finodelk,
+ .entrylk = iot_entrylk,
+ .fentrylk = iot_fentrylk,
+ .xattrop = iot_xattrop,
+ .fxattrop = iot_fxattrop,
+ .rchecksum = iot_rchecksum,
+ .fallocate = iot_fallocate,
+ .discard = iot_discard,
+ .zerofill = iot_zerofill,
};
-struct xlator_cbks cbks = {
-};
+struct xlator_cbks cbks;
struct volume_options options[] = {
- { .key = {"thread-count"},
- .type = GF_OPTION_TYPE_INT,
- .min = IOT_MIN_THREADS,
- .max = IOT_MAX_THREADS
+ { .key = {"thread-count"},
+ .type = GF_OPTION_TYPE_INT,
+ .min = IOT_MIN_THREADS,
+ .max = IOT_MAX_THREADS,
+ .default_value = "16",
+ .description = "Number of threads in IO threads translator which "
+ "perform concurrent IO operations"
+
},
- { .key = {"autoscaling"},
- .type = GF_OPTION_TYPE_BOOL
+ { .key = {"high-prio-threads"},
+ .type = GF_OPTION_TYPE_INT,
+ .min = IOT_MIN_THREADS,
+ .max = IOT_MAX_THREADS,
+ .default_value = "16",
+ .description = "Max number of threads in IO threads translator which "
+ "perform high priority IO operations at a given time"
+
+ },
+ { .key = {"normal-prio-threads"},
+ .type = GF_OPTION_TYPE_INT,
+ .min = IOT_MIN_THREADS,
+ .max = IOT_MAX_THREADS,
+ .default_value = "16",
+ .description = "Max number of threads in IO threads translator which "
+ "perform normal priority IO operations at a given time"
+
+ },
+ { .key = {"low-prio-threads"},
+ .type = GF_OPTION_TYPE_INT,
+ .min = IOT_MIN_THREADS,
+ .max = IOT_MAX_THREADS,
+ .default_value = "16",
+ .description = "Max number of threads in IO threads translator which "
+ "perform low priority IO operations at a given time"
+
+ },
+ { .key = {"least-prio-threads"},
+ .type = GF_OPTION_TYPE_INT,
+ .min = IOT_MIN_THREADS,
+ .max = IOT_MAX_THREADS,
+ .default_value = "1",
+ .description = "Max number of threads in IO threads translator which "
+ "perform least priority IO operations at a given time"
+ },
+ { .key = {"enable-least-priority"},
+ .type = GF_OPTION_TYPE_BOOL,
+ .default_value = "on",
+ .description = "Enable/Disable least priority"
},
- { .key = {"min-threads"},
- .type = GF_OPTION_TYPE_INT,
- .min = IOT_MIN_THREADS,
- .max = IOT_MAX_THREADS
+ {.key = {"idle-time"},
+ .type = GF_OPTION_TYPE_INT,
+ .min = 1,
+ .max = 0x7fffffff,
+ .default_value = "120",
},
- { .key = {"max-threads"},
- .type = GF_OPTION_TYPE_INT,
- .min = IOT_MIN_THREADS,
- .max = IOT_MAX_THREADS
+ {.key = {"least-rate-limit"},
+ .type = GF_OPTION_TYPE_INT,
+ .min = 0,
+ .max = INT_MAX,
+ .default_value = "0",
+ .description = "Max number of least priority operations to handle "
+ "per-second"
+ },
+ { .key = {NULL},
},
- { .key = {NULL} },
};