Print this page
4045 zfs write throttle & i/o scheduler performance work
Reviewed by: George Wilson <george.wilson@delphix.com>
Reviewed by: Adam Leventhal <ahl@delphix.com>
Reviewed by: Christopher Siden <christopher.siden@delphix.com>
*** 125,134 ****
--- 125,135 ----
#include <sys/zfs_context.h>
#include <sys/arc.h>
#include <sys/refcount.h>
#include <sys/vdev.h>
#include <sys/vdev_impl.h>
+ #include <sys/dsl_pool.h>
#ifdef _KERNEL
#include <sys/vmsystm.h>
#include <vm/anon.h>
#include <sys/fs/swapnode.h>
#include <sys/dnlc.h>
*** 145,166 ****
static kmutex_t arc_reclaim_thr_lock;
static kcondvar_t arc_reclaim_thr_cv; /* used to signal reclaim thr */
static uint8_t arc_thread_exit;
- extern int zfs_write_limit_shift;
- extern uint64_t zfs_write_limit_max;
- extern kmutex_t zfs_write_limit_lock;
-
#define ARC_REDUCE_DNLC_PERCENT 3
uint_t arc_reduce_dnlc_percent = ARC_REDUCE_DNLC_PERCENT;
typedef enum arc_reclaim_strategy {
ARC_RECLAIM_AGGR, /* Aggressive reclaim strategy */
ARC_RECLAIM_CONS /* Conservative reclaim strategy */
} arc_reclaim_strategy_t;
/* number of seconds before growing cache again */
static int arc_grow_retry = 60;
/* shift of arc_c for calculating both min and max arc_p */
static int arc_p_min_shift = 4;
--- 146,169 ----
static kmutex_t arc_reclaim_thr_lock;
static kcondvar_t arc_reclaim_thr_cv; /* used to signal reclaim thr */
static uint8_t arc_thread_exit;
#define ARC_REDUCE_DNLC_PERCENT 3
uint_t arc_reduce_dnlc_percent = ARC_REDUCE_DNLC_PERCENT;
typedef enum arc_reclaim_strategy {
ARC_RECLAIM_AGGR, /* Aggressive reclaim strategy */
ARC_RECLAIM_CONS /* Conservative reclaim strategy */
} arc_reclaim_strategy_t;
+ /*
+ * The number of iterations through arc_evict_*() before we
+ * drop & reacquire the lock.
+ */
+ int arc_evict_iterations = 100;
+
/* number of seconds before growing cache again */
static int arc_grow_retry = 60;
/* shift of arc_c for calculating both min and max arc_p */
static int arc_p_min_shift = 4;
*** 172,181 ****
--- 175,189 ----
* minimum lifespan of a prefetch block in clock ticks
* (initialized in arc_init())
*/
static int arc_min_prefetch_lifespan;
+ /*
+ * If this percent of memory is free, don't throttle.
+ */
+ int arc_lotsfree_percent = 10;
+
static int arc_dead;
/*
* The arc has filled available memory and has now warmed up.
*/
*** 467,476 ****
--- 475,485 ----
typedef struct arc_write_callback arc_write_callback_t;
struct arc_write_callback {
void *awcb_private;
arc_done_func_t *awcb_ready;
+ arc_done_func_t *awcb_physdone;
arc_done_func_t *awcb_done;
arc_buf_t *awcb_buf;
};
struct arc_buf_hdr {
*** 1161,1171 ****
arc_state_t *old_state = ab->b_state;
int64_t refcnt = refcount_count(&ab->b_refcnt);
uint64_t from_delta, to_delta;
ASSERT(MUTEX_HELD(hash_lock));
! ASSERT(new_state != old_state);
ASSERT(refcnt == 0 || ab->b_datacnt > 0);
ASSERT(ab->b_datacnt == 0 || !GHOST_STATE(new_state));
ASSERT(ab->b_datacnt <= 1 || old_state != arc_anon);
from_delta = to_delta = ab->b_datacnt * ab->b_size;
--- 1170,1180 ----
arc_state_t *old_state = ab->b_state;
int64_t refcnt = refcount_count(&ab->b_refcnt);
uint64_t from_delta, to_delta;
ASSERT(MUTEX_HELD(hash_lock));
! ASSERT3P(new_state, !=, old_state);
ASSERT(refcnt == 0 || ab->b_datacnt > 0);
ASSERT(ab->b_datacnt == 0 || !GHOST_STATE(new_state));
ASSERT(ab->b_datacnt <= 1 || old_state != arc_anon);
from_delta = to_delta = ab->b_datacnt * ab->b_size;
*** 1776,1785 ****
--- 1785,1796 ----
arc_buf_hdr_t *ab, *ab_prev = NULL;
list_t *list = &state->arcs_list[type];
kmutex_t *hash_lock;
boolean_t have_lock;
void *stolen = NULL;
+ arc_buf_hdr_t marker = { 0 };
+ int count = 0;
ASSERT(state == arc_mru || state == arc_mfu);
evicted_state = (state == arc_mru) ? arc_mru_ghost : arc_mfu_ghost;
*** 1799,1808 ****
--- 1810,1846 ----
}
/* "lookahead" for better eviction candidate */
if (recycle && ab->b_size != bytes &&
ab_prev && ab_prev->b_size == bytes)
continue;
+
+ /* ignore markers */
+ if (ab->b_spa == 0)
+ continue;
+
+ /*
+ * It may take a long time to evict all the bufs requested.
+ * To avoid blocking all arc activity, periodically drop
+ * the arcs_mtx and give other threads a chance to run
+ * before reacquiring the lock.
+ *
+ * If we are looking for a buffer to recycle, we are in
+ * the hot code path, so don't sleep.
+ */
+ if (!recycle && count++ > arc_evict_iterations) {
+ list_insert_after(list, ab, &marker);
+ mutex_exit(&evicted_state->arcs_mtx);
+ mutex_exit(&state->arcs_mtx);
+ kpreempt(KPREEMPT_SYNC);
+ mutex_enter(&state->arcs_mtx);
+ mutex_enter(&evicted_state->arcs_mtx);
+ ab_prev = list_prev(list, &marker);
+ list_remove(list, &marker);
+ count = 0;
+ continue;
+ }
+
hash_lock = HDR_LOCK(ab);
have_lock = MUTEX_HELD(hash_lock);
if (have_lock || mutex_tryenter(hash_lock)) {
ASSERT0(refcount_count(&ab->b_refcnt));
ASSERT(ab->b_datacnt > 0);
*** 1880,1909 ****
if (missed)
ARCSTAT_INCR(arcstat_mutex_miss, missed);
/*
! * We have just evicted some data into the ghost state, make
! * sure we also adjust the ghost state size if necessary.
*/
- if (arc_no_grow &&
- arc_mru_ghost->arcs_size + arc_mfu_ghost->arcs_size > arc_c) {
- int64_t mru_over = arc_anon->arcs_size + arc_mru->arcs_size +
- arc_mru_ghost->arcs_size - arc_c;
- if (mru_over > 0 && arc_mru_ghost->arcs_lsize[type] > 0) {
- int64_t todelete =
- MIN(arc_mru_ghost->arcs_lsize[type], mru_over);
- arc_evict_ghost(arc_mru_ghost, NULL, todelete);
- } else if (arc_mfu_ghost->arcs_lsize[type] > 0) {
- int64_t todelete = MIN(arc_mfu_ghost->arcs_lsize[type],
- arc_mru_ghost->arcs_size +
- arc_mfu_ghost->arcs_size - arc_c);
- arc_evict_ghost(arc_mfu_ghost, NULL, todelete);
- }
- }
-
return (stolen);
}
/*
* Remove buffers from list until we've removed the specified number of
--- 1918,1933 ----
if (missed)
ARCSTAT_INCR(arcstat_mutex_miss, missed);
/*
! * Note: we have just evicted some data into the ghost state,
! * potentially putting the ghost size over the desired size. Rather
! * that evicting from the ghost list in this hot code path, leave
! * this chore to the arc_reclaim_thread().
*/
return (stolen);
}
/*
* Remove buffers from list until we've removed the specified number of
*** 1916,1931 ****
--- 1940,1958 ----
arc_buf_hdr_t marker = { 0 };
list_t *list = &state->arcs_list[ARC_BUFC_DATA];
kmutex_t *hash_lock;
uint64_t bytes_deleted = 0;
uint64_t bufs_skipped = 0;
+ int count = 0;
ASSERT(GHOST_STATE(state));
top:
mutex_enter(&state->arcs_mtx);
for (ab = list_tail(list); ab; ab = ab_prev) {
ab_prev = list_prev(list, ab);
+ if (ab->b_type > ARC_BUFC_NUMTYPES)
+ panic("invalid ab=%p", (void *)ab);
if (spa && ab->b_spa != spa)
continue;
/* ignore markers */
if (ab->b_spa == 0)
*** 1933,1942 ****
--- 1960,1986 ----
hash_lock = HDR_LOCK(ab);
/* caller may be trying to modify this buffer, skip it */
if (MUTEX_HELD(hash_lock))
continue;
+
+ /*
+ * It may take a long time to evict all the bufs requested.
+ * To avoid blocking all arc activity, periodically drop
+ * the arcs_mtx and give other threads a chance to run
+ * before reacquiring the lock.
+ */
+ if (count++ > arc_evict_iterations) {
+ list_insert_after(list, ab, &marker);
+ mutex_exit(&state->arcs_mtx);
+ kpreempt(KPREEMPT_SYNC);
+ mutex_enter(&state->arcs_mtx);
+ ab_prev = list_prev(list, &marker);
+ list_remove(list, &marker);
+ count = 0;
+ continue;
+ }
if (mutex_tryenter(hash_lock)) {
ASSERT(!HDR_IO_IN_PROGRESS(ab));
ASSERT(ab->b_buf == NULL);
ARCSTAT_BUMP(arcstat_deleted);
bytes_deleted += ab->b_size;
*** 1968,1980 ****
mutex_enter(hash_lock);
mutex_exit(hash_lock);
mutex_enter(&state->arcs_mtx);
ab_prev = list_prev(list, &marker);
list_remove(list, &marker);
! } else
bufs_skipped += 1;
}
mutex_exit(&state->arcs_mtx);
if (list == &state->arcs_list[ARC_BUFC_DATA] &&
(bytes < 0 || bytes_deleted < bytes)) {
list = &state->arcs_list[ARC_BUFC_METADATA];
--- 2012,2026 ----
mutex_enter(hash_lock);
mutex_exit(hash_lock);
mutex_enter(&state->arcs_mtx);
ab_prev = list_prev(list, &marker);
list_remove(list, &marker);
! } else {
bufs_skipped += 1;
}
+
+ }
mutex_exit(&state->arcs_mtx);
if (list == &state->arcs_list[ARC_BUFC_DATA] &&
(bytes < 0 || bytes_deleted < bytes)) {
list = &state->arcs_list[ARC_BUFC_METADATA];
*** 2823,2833 ****
* arc_read_done() will invoke all the requested "done" functions
* for readers of this block.
*/
int
arc_read(zio_t *pio, spa_t *spa, const blkptr_t *bp, arc_done_func_t *done,
! void *private, int priority, int zio_flags, uint32_t *arc_flags,
const zbookmark_t *zb)
{
arc_buf_hdr_t *hdr;
arc_buf_t *buf = NULL;
kmutex_t *hash_lock;
--- 2869,2879 ----
* arc_read_done() will invoke all the requested "done" functions
* for readers of this block.
*/
int
arc_read(zio_t *pio, spa_t *spa, const blkptr_t *bp, arc_done_func_t *done,
! void *private, zio_priority_t priority, int zio_flags, uint32_t *arc_flags,
const zbookmark_t *zb)
{
arc_buf_hdr_t *hdr;
arc_buf_t *buf = NULL;
kmutex_t *hash_lock;
*** 3426,3436 ****
--- 3472,3494 ----
}
arc_cksum_compute(buf, B_FALSE);
hdr->b_flags |= ARC_IO_IN_PROGRESS;
}
+ /*
+ * The SPA calls this callback for each physical write that happens on behalf
+ * of a logical write. See the comment in dbuf_write_physdone() for details.
+ */
static void
+ arc_write_physdone(zio_t *zio)
+ {
+ arc_write_callback_t *cb = zio->io_private;
+ if (cb->awcb_physdone != NULL)
+ cb->awcb_physdone(zio, cb->awcb_buf, cb->awcb_private);
+ }
+
+ static void
arc_write_done(zio_t *zio)
{
arc_write_callback_t *callback = zio->io_private;
arc_buf_t *buf = callback->awcb_buf;
arc_buf_hdr_t *hdr = buf->b_hdr;
*** 3506,3517 ****
}
zio_t *
arc_write(zio_t *pio, spa_t *spa, uint64_t txg,
blkptr_t *bp, arc_buf_t *buf, boolean_t l2arc, boolean_t l2arc_compress,
! const zio_prop_t *zp, arc_done_func_t *ready, arc_done_func_t *done,
! void *private, int priority, int zio_flags, const zbookmark_t *zb)
{
arc_buf_hdr_t *hdr = buf->b_hdr;
arc_write_callback_t *callback;
zio_t *zio;
--- 3564,3576 ----
}
zio_t *
arc_write(zio_t *pio, spa_t *spa, uint64_t txg,
blkptr_t *bp, arc_buf_t *buf, boolean_t l2arc, boolean_t l2arc_compress,
! const zio_prop_t *zp, arc_done_func_t *ready, arc_done_func_t *physdone,
! arc_done_func_t *done, void *private, zio_priority_t priority,
! int zio_flags, const zbookmark_t *zb)
{
arc_buf_hdr_t *hdr = buf->b_hdr;
arc_write_callback_t *callback;
zio_t *zio;
*** 3524,3545 ****
hdr->b_flags |= ARC_L2CACHE;
if (l2arc_compress)
hdr->b_flags |= ARC_L2COMPRESS;
callback = kmem_zalloc(sizeof (arc_write_callback_t), KM_SLEEP);
callback->awcb_ready = ready;
callback->awcb_done = done;
callback->awcb_private = private;
callback->awcb_buf = buf;
zio = zio_write(pio, spa, txg, bp, buf->b_data, hdr->b_size, zp,
! arc_write_ready, arc_write_done, callback, priority, zio_flags, zb);
return (zio);
}
static int
! arc_memory_throttle(uint64_t reserve, uint64_t inflight_data, uint64_t txg)
{
#ifdef _KERNEL
uint64_t available_memory = ptob(freemem);
static uint64_t page_load = 0;
static uint64_t last_txg = 0;
--- 3583,3606 ----
hdr->b_flags |= ARC_L2CACHE;
if (l2arc_compress)
hdr->b_flags |= ARC_L2COMPRESS;
callback = kmem_zalloc(sizeof (arc_write_callback_t), KM_SLEEP);
callback->awcb_ready = ready;
+ callback->awcb_physdone = physdone;
callback->awcb_done = done;
callback->awcb_private = private;
callback->awcb_buf = buf;
zio = zio_write(pio, spa, txg, bp, buf->b_data, hdr->b_size, zp,
! arc_write_ready, arc_write_physdone, arc_write_done, callback,
! priority, zio_flags, zb);
return (zio);
}
static int
! arc_memory_throttle(uint64_t reserve, uint64_t txg)
{
#ifdef _KERNEL
uint64_t available_memory = ptob(freemem);
static uint64_t page_load = 0;
static uint64_t last_txg = 0;
*** 3546,3556 ****
#if defined(__i386)
available_memory =
MIN(available_memory, vmem_size(heap_arena, VMEM_FREE));
#endif
! if (available_memory >= zfs_write_limit_max)
return (0);
if (txg > last_txg) {
last_txg = txg;
page_load = 0;
--- 3607,3618 ----
#if defined(__i386)
available_memory =
MIN(available_memory, vmem_size(heap_arena, VMEM_FREE));
#endif
!
! if (freemem > physmem * arc_lotsfree_percent / 100)
return (0);
if (txg > last_txg) {
last_txg = txg;
page_load = 0;
*** 3570,3593 ****
/* memory is low, delay before restarting */
ARCSTAT_INCR(arcstat_memory_throttle_count, 1);
return (SET_ERROR(EAGAIN));
}
page_load = 0;
-
- if (arc_size > arc_c_min) {
- uint64_t evictable_memory =
- arc_mru->arcs_lsize[ARC_BUFC_DATA] +
- arc_mru->arcs_lsize[ARC_BUFC_METADATA] +
- arc_mfu->arcs_lsize[ARC_BUFC_DATA] +
- arc_mfu->arcs_lsize[ARC_BUFC_METADATA];
- available_memory += MIN(evictable_memory, arc_size - arc_c_min);
- }
-
- if (inflight_data > available_memory / 4) {
- ARCSTAT_INCR(arcstat_memory_throttle_count, 1);
- return (SET_ERROR(ERESTART));
- }
#endif
return (0);
}
void
--- 3632,3641 ----
*** 3601,3619 ****
arc_tempreserve_space(uint64_t reserve, uint64_t txg)
{
int error;
uint64_t anon_size;
- #ifdef ZFS_DEBUG
- /*
- * Once in a while, fail for no reason. Everything should cope.
- */
- if (spa_get_random(10000) == 0) {
- dprintf("forcing random failure\n");
- return (SET_ERROR(ERESTART));
- }
- #endif
if (reserve > arc_c/4 && !arc_no_grow)
arc_c = MIN(arc_c_max, reserve * 4);
if (reserve > arc_c)
return (SET_ERROR(ENOMEM));
--- 3649,3658 ----
*** 3627,3637 ****
/*
* Writes will, almost always, require additional memory allocations
* in order to compress/encrypt/etc the data. We therefore need to
* make sure that there is sufficient available memory for this.
*/
! if (error = arc_memory_throttle(reserve, anon_size, txg))
return (error);
/*
* Throttle writes when the amount of dirty data in the cache
* gets too large. We try to keep the cache less than half full
--- 3666,3677 ----
/*
* Writes will, almost always, require additional memory allocations
* in order to compress/encrypt/etc the data. We therefore need to
* make sure that there is sufficient available memory for this.
*/
! error = arc_memory_throttle(reserve, txg);
! if (error != 0)
return (error);
/*
* Throttle writes when the amount of dirty data in the cache
* gets too large. We try to keep the cache less than half full
*** 3776,3790 ****
TS_RUN, minclsyspri);
arc_dead = FALSE;
arc_warm = B_FALSE;
! if (zfs_write_limit_max == 0)
! zfs_write_limit_max = ptob(physmem) >> zfs_write_limit_shift;
! else
! zfs_write_limit_shift = 0;
! mutex_init(&zfs_write_limit_lock, NULL, MUTEX_DEFAULT, NULL);
}
void
arc_fini(void)
{
--- 3816,3839 ----
TS_RUN, minclsyspri);
arc_dead = FALSE;
arc_warm = B_FALSE;
! /*
! * Calculate maximum amount of dirty data per pool.
! *
! * If it has been set by /etc/system, take that.
! * Otherwise, use a percentage of physical memory defined by
! * zfs_dirty_data_max_percent (default 10%) with a cap at
! * zfs_dirty_data_max_max (default 4GB).
! */
! if (zfs_dirty_data_max == 0) {
! zfs_dirty_data_max = physmem * PAGESIZE *
! zfs_dirty_data_max_percent / 100;
! zfs_dirty_data_max = MIN(zfs_dirty_data_max,
! zfs_dirty_data_max_max);
! }
}
void
arc_fini(void)
{
*** 3821,3832 ****
mutex_destroy(&arc_mru_ghost->arcs_mtx);
mutex_destroy(&arc_mfu->arcs_mtx);
mutex_destroy(&arc_mfu_ghost->arcs_mtx);
mutex_destroy(&arc_l2c_only->arcs_mtx);
- mutex_destroy(&zfs_write_limit_lock);
-
buf_fini();
ASSERT(arc_loaned_bytes == 0);
}
--- 3870,3879 ----