diff options
Diffstat (limited to 'xlators/performance')
| -rw-r--r-- | xlators/performance/io-cache/src/io-cache.c | 102 | ||||
| -rw-r--r-- | xlators/performance/io-cache/src/io-cache.h | 41 | ||||
| -rw-r--r-- | xlators/performance/io-threads/src/io-threads.c | 75 | ||||
| -rw-r--r-- | xlators/performance/io-threads/src/io-threads.h | 13 | ||||
| -rw-r--r-- | xlators/performance/md-cache/src/md-cache.c | 51 | ||||
| -rw-r--r-- | xlators/performance/write-behind/src/write-behind.c | 14 |
6 files changed, 255 insertions, 41 deletions
diff --git a/xlators/performance/io-cache/src/io-cache.c b/xlators/performance/io-cache/src/io-cache.c index 98c37746921..f199b229bc2 100644 --- a/xlators/performance/io-cache/src/io-cache.c +++ b/xlators/performance/io-cache/src/io-cache.c @@ -1479,6 +1479,74 @@ ioc_zerofill(call_frame_t *frame, xlator_t *this, fd_t *fd, off_t offset, return 0; } +int32_t +ioc_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) +{ + ioc_table_t *table = NULL; + struct ioc_statvfs *cache = NULL; + + if (op_ret != 0) + goto out; + + table = this->private; + cache = &table->statfs_cache; + + LOCK (&cache->lock); + + gettimeofday (&cache->tv, NULL); + cache->buf = *buf; + + UNLOCK (&cache->lock); + +out: + STACK_UNWIND_STRICT (statfs, frame, op_ret, op_errno, buf, xdata); + return 0; +} + +int +ioc_statfs (call_frame_t *frame, xlator_t *this, loc_t *loc, dict_t *xdata) +{ + ioc_table_t *table = NULL; + struct ioc_statvfs *cache = NULL; + struct statvfs buf; + struct timeval tv = {0,}; + + table = this->private; + cache = &table->statfs_cache; + + if (!cache->enabled) + goto disabled; + + gettimeofday (&tv, NULL); + + LOCK (&cache->lock); + + if (time_elapsed (&tv, &cache->tv) >= cache->timeout) { + UNLOCK (&cache->lock); + goto uncached; + } + + buf = cache->buf; + + UNLOCK (&cache->lock); + + STACK_UNWIND_STRICT (statfs, frame, 0, 0, &buf, xdata); + + return 0; + +disabled: + STACK_WIND_TAIL (frame, FIRST_CHILD (frame->this), + FIRST_CHILD (frame->this)->fops->statfs, loc, xdata); + return 0; + +uncached: + STACK_WIND (frame, ioc_statfs_cbk, + FIRST_CHILD (frame->this), + FIRST_CHILD (frame->this)->fops->statfs, loc, xdata); + return 0; +} int32_t ioc_get_priority_list (const char *opt_str, struct list_head *first) @@ -1696,6 +1764,13 @@ reconfigure (xlator_t *this, dict_t *options) } table->cache_size = cache_size_new; + GF_OPTION_RECONF ("statfs-cache", table->statfs_cache.enabled, + options, bool, unlock); + + GF_OPTION_RECONF ("statfs-cache-timeout", + table->statfs_cache.timeout, + options, int32, unlock); + ret = 0; } unlock: @@ -1755,6 +1830,10 @@ init (xlator_t *this) GF_OPTION_INIT ("max-file-size", table->max_file_size, size_uint64, out); + GF_OPTION_INIT ("statfs-cache", table->statfs_cache.enabled, bool, out); + + GF_OPTION_INIT ("statfs-cache-timeout", table->statfs_cache.timeout, int32, out); + if (!check_cache_size_ok (this, table->cache_size)) { ret = -1; goto out; @@ -1827,6 +1906,11 @@ init (xlator_t *this) ctx = this->ctx; ioc_log2_page_size = log_base2 (ctx->page_size); + LOCK_INIT (&table->statfs_cache.lock); + /* Invalidate statfs cache */ + table->statfs_cache.tv.tv_sec = 0; + table->statfs_cache.tv.tv_usec = 0; + out: if (ret == -1) { if (table != NULL) { @@ -2096,6 +2180,7 @@ fini (xlator_t *this) GF_ASSERT (list_empty (&table->inode_lru[i])); } + LOCK_DESTROY (&table->statfs_cache.lock); GF_ASSERT (list_empty (&table->inodes)); */ pthread_mutex_destroy (&table->table_lock); @@ -2120,6 +2205,7 @@ struct xlator_fops fops = { .readdirp = ioc_readdirp, .discard = ioc_discard, .zerofill = ioc_zerofill, + .statfs = ioc_statfs, }; @@ -2171,5 +2257,21 @@ struct volume_options options[] = { .description = "Maximum file size which would be cached by the " "io-cache translator." }, + { .key = {"statfs-cache"}, + .type = GF_OPTION_TYPE_BOOL, + .default_value = "0", + .description = "The cached statfs for a filesystem will be " + "till 'statfs-cache-timeout' seconds, after which re-validation " + "is performed." + }, + { .key = {"statfs-cache-timeout"}, + .type = GF_OPTION_TYPE_INT, + .min = 0, + .max = 60, + .default_value = "1", + .description = "The cached statfs for a filesystem will be " + "till 'statfs-cache-timeout' seconds, after which re-validation " + "is performed." + }, { .key = {NULL} }, }; diff --git a/xlators/performance/io-cache/src/io-cache.h b/xlators/performance/io-cache/src/io-cache.h index d7c823fe962..da71b2f2371 100644 --- a/xlators/performance/io-cache/src/io-cache.h +++ b/xlators/performance/io-cache/src/io-cache.h @@ -148,23 +148,32 @@ struct ioc_inode { inode_t *inode; }; +struct ioc_statvfs { + struct statvfs buf; + int32_t timeout; + struct timeval tv; + gf_boolean_t enabled; + gf_lock_t lock; +}; + struct ioc_table { - uint64_t page_size; - uint64_t cache_size; - uint64_t cache_used; - uint64_t min_file_size; - uint64_t max_file_size; - struct list_head inodes; /* list of inodes cached */ - struct list_head active; - struct list_head *inode_lru; - struct list_head priority_list; - int32_t readv_count; - pthread_mutex_t table_lock; - xlator_t *xl; - uint32_t inode_count; - int32_t cache_timeout; - int32_t max_pri; - struct mem_pool *mem_pool; + uint64_t page_size; + uint64_t cache_size; + uint64_t cache_used; + uint64_t min_file_size; + uint64_t max_file_size; + struct list_head inodes; /* list of inodes cached */ + struct list_head active; + struct list_head *inode_lru; + struct list_head priority_list; + int32_t readv_count; + pthread_mutex_t table_lock; + xlator_t *xl; + uint32_t inode_count; + int32_t cache_timeout; + int32_t max_pri; + struct mem_pool *mem_pool; + struct ioc_statvfs statfs_cache; }; typedef struct ioc_table ioc_table_t; diff --git a/xlators/performance/io-threads/src/io-threads.c b/xlators/performance/io-threads/src/io-threads.c index 72a82082563..7f9dc5f82a8 100644 --- a/xlators/performance/io-threads/src/io-threads.c +++ b/xlators/performance/io-threads/src/io-threads.c @@ -161,8 +161,6 @@ iot_worker (void *data) THIS = this; for (;;) { - sleep_till.tv_sec = time (NULL) + conf->idle_time; - pthread_mutex_lock (&conf->mutex); { if (pri != -1) { @@ -175,8 +173,11 @@ iot_worker (void *data) break; } - conf->sleep_count++; + clock_gettime (CLOCK_REALTIME_COARSE, + &sleep_till); + sleep_till.tv_sec += conf->idle_time; + conf->sleep_count++; ret = pthread_cond_timedwait (&conf->cond, &conf->mutex, &sleep_till); @@ -232,14 +233,25 @@ int do_iot_schedule (iot_conf_t *conf, call_stub_t *stub, int pri) { int ret = 0; + int active_count = 0; pthread_mutex_lock (&conf->mutex); { __iot_enqueue (conf, stub, pri); - pthread_cond_signal (&conf->cond); - - ret = __iot_workers_scale (conf); + /* If we have an ample supply of threads alive already + * it's massively more efficient to keep the ones you have + * busy vs making new ones and signaling everyone + */ + active_count = conf->curr_count - conf->sleep_count; + if (conf->fops_per_thread_ratio == 0 || active_count == 0 || + (conf->queue_size/active_count > + conf->fops_per_thread_ratio && + active_count < conf->max_count)) { + pthread_cond_signal (&conf->cond); + + ret = __iot_workers_scale (conf); + } } pthread_mutex_unlock (&conf->mutex); @@ -266,6 +278,9 @@ iot_get_pri_meaning (iot_pri_t pri) case IOT_PRI_MAX: name = "invalid"; break; + case IOT_PRI_UNSPEC: + name = "unspecified"; + break; } return name; } @@ -598,6 +613,34 @@ int iot_getxattr (call_frame_t *frame, xlator_t *this, loc_t *loc, const char *name, dict_t *xdata) { + iot_conf_t *conf = NULL; + dict_t *depths = NULL; + int i = 0; + + conf = this->private; + + if (conf && name && strcmp (name, IO_THREADS_QUEUE_SIZE_KEY) == 0) { + // We explicitly do not want a reference count + // for this dict in this translator + depths = get_new_dict (); + if (!depths) + goto unwind_special_getxattr; + + for (i = 0; i < IOT_PRI_MAX; i++) { + if (dict_set_int32 (depths, + (char *)fop_pri_to_string (i), + conf->queue_sizes[i]) != 0) { + dict_destroy (depths); + depths = NULL; + goto unwind_special_getxattr; + } + } + +unwind_special_getxattr: + STACK_UNWIND_STRICT (getxattr, frame, 0, 0, depths, xdata); + return 0; + } + IOT_FOP (getxattr, frame, this, loc, name, xdata); return 0; } @@ -904,6 +947,9 @@ reconfigure (xlator_t *this, dict_t *options) GF_OPTION_RECONF ("thread-count", conf->max_count, options, int32, out); + GF_OPTION_RECONF ("fops-per-thread-ratio", conf->fops_per_thread_ratio, + options, int32, out); + GF_OPTION_RECONF ("high-prio-threads", conf->ac_iot_limit[IOT_PRI_HI], options, int32, out); @@ -978,6 +1024,9 @@ init (xlator_t *this) GF_OPTION_INIT ("thread-count", conf->max_count, int32, out); + GF_OPTION_INIT ("fops-per-thread-ratio", conf->fops_per_thread_ratio, + int32, out); + GF_OPTION_INIT ("high-prio-threads", conf->ac_iot_limit[IOT_PRI_HI], int32, out); @@ -1140,6 +1189,20 @@ struct volume_options options[] = { "perform concurrent IO operations" }, + { .key = {"fops-per-thread-ratio"}, + .type = GF_OPTION_TYPE_INT, + .min = IOT_MIN_FOP_PER_THREAD, + .max = IOT_MAX_FOP_PER_THREAD, + .default_value = "20", + .description = "The optimal ratio of threads to FOPs in the queue " + "we wish to achieve before creating a new thread. " + "The idea here is it's far cheaper to keep our " + "currently running threads busy than spin up " + "new threads or cause a stampeding herd of threads " + "to service a singlular FOP when you have a thread " + "which will momentarily become available to do the " + "work." + }, { .key = {"high-prio-threads"}, .type = GF_OPTION_TYPE_INT, .min = IOT_MIN_THREADS, diff --git a/xlators/performance/io-threads/src/io-threads.h b/xlators/performance/io-threads/src/io-threads.h index fa955b5954b..011d4a00f7f 100644 --- a/xlators/performance/io-threads/src/io-threads.h +++ b/xlators/performance/io-threads/src/io-threads.h @@ -34,20 +34,14 @@ struct iot_conf; #define IOT_MIN_THREADS 1 #define IOT_DEFAULT_THREADS 16 -#define IOT_MAX_THREADS 64 +#define IOT_MAX_THREADS 256 +#define IOT_MIN_FOP_PER_THREAD 0 +#define IOT_MAX_FOP_PER_THREAD 2000 #define IOT_THREAD_STACK_SIZE ((size_t)(1024*1024)) -typedef enum { - IOT_PRI_HI = 0, /* low latency */ - IOT_PRI_NORMAL, /* normal */ - IOT_PRI_LO, /* bulk */ - IOT_PRI_LEAST, /* least */ - IOT_PRI_MAX, -} iot_pri_t; - #define IOT_LEAST_THROTTLE_DELAY 1 /* sample interval in seconds */ struct iot_least_throttle { struct timeval sample_time; /* timestamp of current sample */ @@ -62,6 +56,7 @@ struct iot_conf { pthread_cond_t cond; int32_t max_count; /* configured maximum */ + int32_t fops_per_thread_ratio; int32_t curr_count; /* actual number of threads running */ int32_t sleep_count; diff --git a/xlators/performance/md-cache/src/md-cache.c b/xlators/performance/md-cache/src/md-cache.c index 30443761c56..c3baafdc1b6 100644 --- a/xlators/performance/md-cache/src/md-cache.c +++ b/xlators/performance/md-cache/src/md-cache.c @@ -33,6 +33,7 @@ struct mdc_conf { gf_boolean_t cache_selinux; gf_boolean_t force_readdirp; gf_boolean_t cache_swift_metadata; + gf_boolean_t cache_all_xattrs; }; @@ -792,6 +793,7 @@ struct checkpair { static int is_mdc_key_satisfied (const char *key) { + unsigned int checked_keys = 0; const char *mdc_key = NULL; int i = 0; @@ -801,11 +803,13 @@ is_mdc_key_satisfied (const char *key) for (mdc_key = mdc_keys[i].name; (mdc_key = mdc_keys[i].name); i++) { if (!mdc_keys[i].load) continue; + + checked_keys++; if (strcmp (mdc_key, key) == 0) return 1; } - return 0; + return 0; } @@ -875,7 +879,7 @@ mdc_lookup (call_frame_t *frame, xlator_t *this, loc_t *loc, dict_t *xattr_rsp = NULL; dict_t *xattr_alloc = NULL; mdc_local_t *local = NULL; - + struct mdc_conf *conf = this->private; local = mdc_local_get (frame); if (!local) @@ -899,10 +903,17 @@ mdc_lookup (call_frame_t *frame, xlator_t *this, loc_t *loc, if (ret != 0) goto uncached; - if (!mdc_xattr_satisfied (this, xdata, xattr_rsp)) + /* Only check the keys if we are not caching all the xattrs */ + if (!conf->cache_all_xattrs && + !mdc_xattr_satisfied (this, xdata, xattr_rsp)) { goto uncached; + } } + gf_msg (this->name, GF_LOG_TRACE, 0, 0, + "Returning lookup from cache for gfid %s", + uuid_utoa(loc->inode->gfid)); + MDC_STACK_UNWIND (lookup, frame, 0, 0, loc->inode, &stbuf, xattr_rsp, &postparent); @@ -1882,6 +1893,7 @@ mdc_getxattr (call_frame_t *frame, xlator_t *this, loc_t *loc, const char *key, int op_errno = ENODATA; mdc_local_t *local = NULL; dict_t *xattr = NULL; + struct mdc_conf *conf = this->private; local = mdc_local_get (frame); if (!local) @@ -1897,7 +1909,18 @@ mdc_getxattr (call_frame_t *frame, xlator_t *this, loc_t *loc, const char *key, goto uncached; if (!xattr || !dict_get (xattr, (char *)key)) { - ret = -1; + /* If we can't find the extended attribute, & cache-all-xattrs + * is enabled, we should wind and try to find them. + * + * NOTE: Quota & AFR queries through the mount + * (i.e, virtual Gluster xattrs) + * won't work unless we do this. + */ + if (conf->cache_all_xattrs) { + goto uncached; + } + + ret = -1; op_errno = ENODATA; } @@ -2363,7 +2386,8 @@ reconfigure (xlator_t *this, dict_t *options) GF_OPTION_RECONF("force-readdirp", conf->force_readdirp, options, bool, out); - + GF_OPTION_RECONF("cache-all-xattrs", conf->cache_all_xattrs, options, + bool, out); out: return 0; } @@ -2404,6 +2428,7 @@ init (xlator_t *this) conf->cache_swift_metadata); GF_OPTION_INIT("force-readdirp", conf->force_readdirp, bool, out); + GF_OPTION_INIT ("cache-all-xattrs", conf->cache_all_xattrs, bool, out); out: this->private = conf; @@ -2474,7 +2499,7 @@ struct volume_options options[] = { { .key = {"md-cache-timeout"}, .type = GF_OPTION_TYPE_INT, .min = 0, - .max = 60, + .max = 300, .default_value = "1", .description = "Time period after which cache has to be refreshed", }, @@ -2484,5 +2509,19 @@ struct volume_options options[] = { .description = "Convert all readdir requests to readdirplus to " "collect stat info on each entry.", }, + { .key = {"strict-xattrs"}, + .type = GF_OPTION_TYPE_BOOL, + .default_value = "true", + .description = "When reading extended attributes from the cache, " + "if an xattr is not found, attempt to find it by winding " + "instead of returning ENODATA. This is necessary to query " + "the special extended attributes (trusted.glusterfs.quota.size) " + "through a FUSE mount with md-cache enabled." + }, + { .key = {"cache-all-xattrs"}, + .type = GF_OPTION_TYPE_BOOL, + .default_value = "on", + .description = "Cache all the extended attributes for an inode.", + }, { .key = {NULL} }, }; diff --git a/xlators/performance/write-behind/src/write-behind.c b/xlators/performance/write-behind/src/write-behind.c index 7f5719b1e48..bc59036ff88 100644 --- a/xlators/performance/write-behind/src/write-behind.c +++ b/xlators/performance/write-behind/src/write-behind.c @@ -169,6 +169,7 @@ typedef struct wb_request { typedef struct wb_conf { uint64_t aggregate_size; + uint64_t page_size; uint64_t window_size; gf_boolean_t flush_behind; gf_boolean_t trickling_writes; @@ -1207,18 +1208,21 @@ __wb_collapse_small_writes (wb_request_t *holder, wb_request_t *req) char *ptr = NULL; struct iobuf *iobuf = NULL; struct iobref *iobref = NULL; + struct wb_conf *conf = NULL; int ret = -1; ssize_t required_size = 0; size_t holder_len = 0; size_t req_len = 0; + conf = req->wb_inode->this->private; + if (!holder->iobref) { holder_len = iov_length (holder->stub->args.vector, holder->stub->args.count); req_len = iov_length (req->stub->args.vector, req->stub->args.count); - required_size = max ((THIS->ctx->page_size), + required_size = max ((conf->page_size), (holder_len + req_len)); iobuf = iobuf_get2 (req->wb_inode->this->ctx->iobuf_pool, required_size); @@ -1281,7 +1285,6 @@ __wb_preprocess_winds (wb_inode_t *wb_inode) wb_request_t *holder = NULL; wb_conf_t *conf = NULL; int ret = 0; - ssize_t page_size = 0; /* With asynchronous IO from a VM guest (as a file), there can be two sequential writes happening in two regions @@ -1292,7 +1295,6 @@ __wb_preprocess_winds (wb_inode_t *wb_inode) through the interleaved ops */ - page_size = wb_inode->this->ctx->page_size; conf = wb_inode->this->private; list_for_each_entry_safe (req, tmp, &wb_inode->todo, todo) { @@ -1343,7 +1345,7 @@ __wb_preprocess_winds (wb_inode_t *wb_inode) continue; } - space_left = page_size - holder->write_size; + space_left = wb_inode->window_conf - holder->write_size; if (space_left < req->write_size) { holder->ordering.go = 1; @@ -2471,6 +2473,9 @@ reconfigure (xlator_t *this, dict_t *options) GF_OPTION_RECONF ("cache-size", conf->window_size, options, size_uint64, out); + GF_OPTION_RECONF ("cache-size", conf->page_size, options, size_uint64, + out); + GF_OPTION_RECONF ("flush-behind", conf->flush_behind, options, bool, out); @@ -2522,6 +2527,7 @@ init (xlator_t *this) /* configure 'option window-size <size>' */ GF_OPTION_INIT ("cache-size", conf->window_size, size_uint64, out); + GF_OPTION_INIT ("cache-size", conf->page_size, size_uint64, out); if (!conf->window_size && conf->aggregate_size) { gf_msg (this->name, GF_LOG_WARNING, 0, |
