]> git.proxmox.com Git - mirror_zfs.git/blobdiff - module/zfs/vdev_queue.c
OpenZFS 9102 - zfs should be able to initialize storage devices
[mirror_zfs.git] / module / zfs / vdev_queue.c
index 0c62a6fa3c7df4b3de618166dab6c710d7b6642a..939699cb837349f159867e7d032d05cfa68aac14 100644 (file)
@@ -24,7 +24,7 @@
  */
 
 /*
- * Copyright (c) 2012, 2014 by Delphix. All rights reserved.
+ * Copyright (c) 2012, 2018 by Delphix. All rights reserved.
  */
 
 #include <sys/zfs_context.h>
 #include <sys/zio.h>
 #include <sys/avl.h>
 #include <sys/dsl_pool.h>
+#include <sys/metaslab_impl.h>
 #include <sys/spa.h>
 #include <sys/spa_impl.h>
 #include <sys/kstat.h>
+#include <sys/abd.h>
 
 /*
  * ZFS I/O Scheduler
@@ -146,10 +148,14 @@ uint32_t zfs_vdev_sync_write_min_active = 10;
 uint32_t zfs_vdev_sync_write_max_active = 10;
 uint32_t zfs_vdev_async_read_min_active = 1;
 uint32_t zfs_vdev_async_read_max_active = 3;
-uint32_t zfs_vdev_async_write_min_active = 1;
+uint32_t zfs_vdev_async_write_min_active = 2;
 uint32_t zfs_vdev_async_write_max_active = 10;
 uint32_t zfs_vdev_scrub_min_active = 1;
 uint32_t zfs_vdev_scrub_max_active = 2;
+uint32_t zfs_vdev_removal_min_active = 1;
+uint32_t zfs_vdev_removal_max_active = 2;
+uint32_t zfs_vdev_initializing_min_active = 1;
+uint32_t zfs_vdev_initializing_max_active = 1;
 
 /*
  * When the pool has less than zfs_vdev_async_write_active_min_dirty_percent
@@ -167,27 +173,48 @@ int zfs_vdev_async_write_active_max_dirty_percent = 60;
  * we include spans of optional I/Os to aid aggregation at the disk even when
  * they aren't able to help us aggregate at this level.
  */
-int zfs_vdev_aggregation_limit = SPA_OLD_MAXBLOCKSIZE;
+int zfs_vdev_aggregation_limit = 1 << 20;
 int zfs_vdev_read_gap_limit = 32 << 10;
 int zfs_vdev_write_gap_limit = 4 << 10;
 
+/*
+ * Define the queue depth percentage for each top-level. This percentage is
+ * used in conjunction with zfs_vdev_async_max_active to determine how many
+ * allocations a specific top-level vdev should handle. Once the queue depth
+ * reaches zfs_vdev_queue_depth_pct * zfs_vdev_async_write_max_active / 100
+ * then allocator will stop allocating blocks on that top-level device.
+ * The default kernel setting is 1000% which will yield 100 allocations per
+ * device. For userland testing, the default setting is 300% which equates
+ * to 30 allocations per device.
+ */
+#ifdef _KERNEL
+int zfs_vdev_queue_depth_pct = 1000;
+#else
+int zfs_vdev_queue_depth_pct = 300;
+#endif
+
+/*
+ * When performing allocations for a given metaslab, we want to make sure that
+ * there are enough IOs to aggregate together to improve throughput. We want to
+ * ensure that there are at least 128k worth of IOs that can be aggregated, and
+ * we assume that the average allocation size is 4k, so we need the queue depth
+ * to be 32 per allocator to get good aggregation of sequential writes.
+ */
+int zfs_vdev_def_queue_depth = 32;
+
+
 int
 vdev_queue_offset_compare(const void *x1, const void *x2)
 {
-       const zio_t *z1 = x1;
-       const zio_t *z2 = x2;
+       const zio_t *z1 = (const zio_t *)x1;
+       const zio_t *z2 = (const zio_t *)x2;
 
-       if (z1->io_offset < z2->io_offset)
-               return (-1);
-       if (z1->io_offset > z2->io_offset)
-               return (1);
+       int cmp = AVL_CMP(z1->io_offset, z2->io_offset);
 
-       if (z1 < z2)
-               return (-1);
-       if (z1 > z2)
-               return (1);
+       if (likely(cmp))
+               return (cmp);
 
-       return (0);
+       return (AVL_PCMP(z1, z2));
 }
 
 static inline avl_tree_t *
@@ -209,20 +236,15 @@ vdev_queue_type_tree(vdev_queue_t *vq, zio_type_t t)
 int
 vdev_queue_timestamp_compare(const void *x1, const void *x2)
 {
-       const zio_t *z1 = x1;
-       const zio_t *z2 = x2;
+       const zio_t *z1 = (const zio_t *)x1;
+       const zio_t *z2 = (const zio_t *)x2;
 
-       if (z1->io_timestamp < z2->io_timestamp)
-               return (-1);
-       if (z1->io_timestamp > z2->io_timestamp)
-               return (1);
+       int cmp = AVL_CMP(z1->io_timestamp, z2->io_timestamp);
 
-       if (z1 < z2)
-               return (-1);
-       if (z1 > z2)
-               return (1);
+       if (likely(cmp))
+               return (cmp);
 
-       return (0);
+       return (AVL_PCMP(z1, z2));
 }
 
 static int
@@ -239,6 +261,10 @@ vdev_queue_class_min_active(zio_priority_t p)
                return (zfs_vdev_async_write_min_active);
        case ZIO_PRIORITY_SCRUB:
                return (zfs_vdev_scrub_min_active);
+       case ZIO_PRIORITY_REMOVAL:
+               return (zfs_vdev_removal_min_active);
+       case ZIO_PRIORITY_INITIALIZING:
+               return (zfs_vdev_initializing_min_active);
        default:
                panic("invalid priority %u", p);
                return (0);
@@ -249,20 +275,29 @@ static int
 vdev_queue_max_async_writes(spa_t *spa)
 {
        int writes;
-       uint64_t dirty = spa->spa_dsl_pool->dp_dirty_total;
+       uint64_t dirty = 0;
+       dsl_pool_t *dp = spa_get_dsl(spa);
        uint64_t min_bytes = zfs_dirty_data_max *
            zfs_vdev_async_write_active_min_dirty_percent / 100;
        uint64_t max_bytes = zfs_dirty_data_max *
            zfs_vdev_async_write_active_max_dirty_percent / 100;
 
+       /*
+        * Async writes may occur before the assignment of the spa's
+        * dsl_pool_t if a self-healing zio is issued prior to the
+        * completion of dmu_objset_open_impl().
+        */
+       if (dp == NULL)
+               return (zfs_vdev_async_write_max_active);
+
        /*
         * Sync tasks correspond to interactive user actions. To reduce the
         * execution time of those actions we push data out as fast as possible.
         */
-       if (spa_has_pending_synctask(spa)) {
+       if (spa_has_pending_synctask(spa))
                return (zfs_vdev_async_write_max_active);
-       }
 
+       dirty = dp->dp_dirty_total;
        if (dirty < min_bytes)
                return (zfs_vdev_async_write_min_active);
        if (dirty > max_bytes)
@@ -298,6 +333,10 @@ vdev_queue_class_max_active(spa_t *spa, zio_priority_t p)
                return (vdev_queue_max_async_writes(spa));
        case ZIO_PRIORITY_SCRUB:
                return (zfs_vdev_scrub_max_active);
+       case ZIO_PRIORITY_REMOVAL:
+               return (zfs_vdev_removal_max_active);
+       case ZIO_PRIORITY_INITIALIZING:
+               return (zfs_vdev_initializing_max_active);
        default:
                panic("invalid priority %u", p);
                return (0);
@@ -353,11 +392,11 @@ vdev_queue_init(vdev_t *vd)
        avl_create(&vq->vq_active_tree, vdev_queue_offset_compare,
            sizeof (zio_t), offsetof(struct zio, io_queue_node));
        avl_create(vdev_queue_type_tree(vq, ZIO_TYPE_READ),
-               vdev_queue_offset_compare, sizeof (zio_t),
-               offsetof(struct zio, io_offset_node));
+           vdev_queue_offset_compare, sizeof (zio_t),
+           offsetof(struct zio, io_offset_node));
        avl_create(vdev_queue_type_tree(vq, ZIO_TYPE_WRITE),
-               vdev_queue_offset_compare, sizeof (zio_t),
-               offsetof(struct zio, io_offset_node));
+           vdev_queue_offset_compare, sizeof (zio_t),
+           offsetof(struct zio, io_offset_node));
 
        for (p = 0; p < ZIO_PRIORITY_NUM_QUEUEABLE; p++) {
                int (*compfn) (const void *, const void *);
@@ -372,17 +411,18 @@ vdev_queue_init(vdev_t *vd)
                else
                        compfn = vdev_queue_offset_compare;
                avl_create(vdev_queue_class_tree(vq, p), compfn,
-                       sizeof (zio_t), offsetof(struct zio, io_queue_node));
+                   sizeof (zio_t), offsetof(struct zio, io_queue_node));
        }
+
+       vq->vq_last_offset = 0;
 }
 
 void
 vdev_queue_fini(vdev_t *vd)
 {
        vdev_queue_t *vq = &vd->vdev_queue;
-       zio_priority_t p;
 
-       for (p = 0; p < ZIO_PRIORITY_NUM_QUEUEABLE; p++)
+       for (zio_priority_t p = 0; p < ZIO_PRIORITY_NUM_QUEUEABLE; p++)
                avl_destroy(vdev_queue_class_tree(vq, p));
        avl_destroy(&vq->vq_active_tree);
        avl_destroy(vdev_queue_type_tree(vq, ZIO_TYPE_READ));
@@ -395,16 +435,16 @@ static void
 vdev_queue_io_add(vdev_queue_t *vq, zio_t *zio)
 {
        spa_t *spa = zio->io_spa;
-       spa_stats_history_t *ssh = &spa->spa_stats.io_history;
+       spa_history_kstat_t *shk = &spa->spa_stats.io_history;
 
        ASSERT3U(zio->io_priority, <, ZIO_PRIORITY_NUM_QUEUEABLE);
        avl_add(vdev_queue_class_tree(vq, zio->io_priority), zio);
        avl_add(vdev_queue_type_tree(vq, zio->io_type), zio);
 
-       if (ssh->kstat != NULL) {
-               mutex_enter(&ssh->lock);
-               kstat_waitq_enter(ssh->kstat->ks_data);
-               mutex_exit(&ssh->lock);
+       if (shk->kstat != NULL) {
+               mutex_enter(&shk->lock);
+               kstat_waitq_enter(shk->kstat->ks_data);
+               mutex_exit(&shk->lock);
        }
 }
 
@@ -412,16 +452,16 @@ static void
 vdev_queue_io_remove(vdev_queue_t *vq, zio_t *zio)
 {
        spa_t *spa = zio->io_spa;
-       spa_stats_history_t *ssh = &spa->spa_stats.io_history;
+       spa_history_kstat_t *shk = &spa->spa_stats.io_history;
 
        ASSERT3U(zio->io_priority, <, ZIO_PRIORITY_NUM_QUEUEABLE);
        avl_remove(vdev_queue_class_tree(vq, zio->io_priority), zio);
        avl_remove(vdev_queue_type_tree(vq, zio->io_type), zio);
 
-       if (ssh->kstat != NULL) {
-               mutex_enter(&ssh->lock);
-               kstat_waitq_exit(ssh->kstat->ks_data);
-               mutex_exit(&ssh->lock);
+       if (shk->kstat != NULL) {
+               mutex_enter(&shk->lock);
+               kstat_waitq_exit(shk->kstat->ks_data);
+               mutex_exit(&shk->lock);
        }
 }
 
@@ -429,17 +469,17 @@ static void
 vdev_queue_pending_add(vdev_queue_t *vq, zio_t *zio)
 {
        spa_t *spa = zio->io_spa;
-       spa_stats_history_t *ssh = &spa->spa_stats.io_history;
+       spa_history_kstat_t *shk = &spa->spa_stats.io_history;
 
        ASSERT(MUTEX_HELD(&vq->vq_lock));
        ASSERT3U(zio->io_priority, <, ZIO_PRIORITY_NUM_QUEUEABLE);
        vq->vq_class[zio->io_priority].vqc_active++;
        avl_add(&vq->vq_active_tree, zio);
 
-       if (ssh->kstat != NULL) {
-               mutex_enter(&ssh->lock);
-               kstat_runq_enter(ssh->kstat->ks_data);
-               mutex_exit(&ssh->lock);
+       if (shk->kstat != NULL) {
+               mutex_enter(&shk->lock);
+               kstat_runq_enter(shk->kstat->ks_data);
+               mutex_exit(&shk->lock);
        }
 }
 
@@ -447,17 +487,17 @@ static void
 vdev_queue_pending_remove(vdev_queue_t *vq, zio_t *zio)
 {
        spa_t *spa = zio->io_spa;
-       spa_stats_history_t *ssh = &spa->spa_stats.io_history;
+       spa_history_kstat_t *shk = &spa->spa_stats.io_history;
 
        ASSERT(MUTEX_HELD(&vq->vq_lock));
        ASSERT3U(zio->io_priority, <, ZIO_PRIORITY_NUM_QUEUEABLE);
        vq->vq_class[zio->io_priority].vqc_active--;
        avl_remove(&vq->vq_active_tree, zio);
 
-       if (ssh->kstat != NULL) {
-               kstat_io_t *ksio = ssh->kstat->ks_data;
+       if (shk->kstat != NULL) {
+               kstat_io_t *ksio = shk->kstat->ks_data;
 
-               mutex_enter(&ssh->lock);
+               mutex_enter(&shk->lock);
                kstat_runq_exit(ksio);
                if (zio->io_type == ZIO_TYPE_READ) {
                        ksio->reads++;
@@ -466,7 +506,7 @@ vdev_queue_pending_remove(vdev_queue_t *vq, zio_t *zio)
                        ksio->writes++;
                        ksio->nwritten += zio->io_size;
                }
-               mutex_exit(&ssh->lock);
+               mutex_exit(&shk->lock);
        }
 }
 
@@ -475,13 +515,14 @@ vdev_queue_agg_io_done(zio_t *aio)
 {
        if (aio->io_type == ZIO_TYPE_READ) {
                zio_t *pio;
-               while ((pio = zio_walk_parents(aio)) != NULL) {
-                       bcopy((char *)aio->io_data + (pio->io_offset -
-                           aio->io_offset), pio->io_data, pio->io_size);
+               zio_link_t *zl = NULL;
+               while ((pio = zio_walk_parents(aio, &zl)) != NULL) {
+                       abd_copy_off(pio->io_abd, aio->io_abd,
+                           0, pio->io_offset - aio->io_offset, pio->io_size);
                }
        }
 
-       zio_buf_free(aio->io_data, aio->io_size);
+       abd_free(aio->io_abd);
 }
 
 /*
@@ -497,22 +538,21 @@ static zio_t *
 vdev_queue_aggregate(vdev_queue_t *vq, zio_t *zio)
 {
        zio_t *first, *last, *aio, *dio, *mandatory, *nio;
+       zio_link_t *zl = NULL;
        uint64_t maxgap = 0;
        uint64_t size;
+       uint64_t limit;
+       int maxblocksize;
        boolean_t stretch = B_FALSE;
        avl_tree_t *t = vdev_queue_type_tree(vq, zio->io_type);
        enum zio_flag flags = zio->io_flags & ZIO_FLAG_AGG_INHERIT;
-       void *buf;
+       abd_t *abd;
 
-       if (zio->io_flags & ZIO_FLAG_DONT_AGGREGATE)
-               return (NULL);
+       maxblocksize = spa_maxblocksize(vq->vq_vdev->vdev_spa);
+       limit = MAX(MIN(zfs_vdev_aggregation_limit, maxblocksize), 0);
 
-       /*
-        * Prevent users from setting the zfs_vdev_aggregation_limit
-        * tuning larger than SPA_MAXBLOCKSIZE.
-        */
-       zfs_vdev_aggregation_limit =
-           MIN(zfs_vdev_aggregation_limit, SPA_MAXBLOCKSIZE);
+       if (zio->io_flags & ZIO_FLAG_DONT_AGGREGATE || limit == 0)
+               return (NULL);
 
        first = last = zio;
 
@@ -536,12 +576,13 @@ vdev_queue_aggregate(vdev_queue_t *vq, zio_t *zio)
 
        /*
         * Walk backwards through sufficiently contiguous I/Os
-        * recording the last non-option I/O.
+        * recording the last non-optional I/O.
         */
        while ((dio = AVL_PREV(t, first)) != NULL &&
            (dio->io_flags & ZIO_FLAG_AGG_INHERIT) == flags &&
-           IO_SPAN(dio, last) <= zfs_vdev_aggregation_limit &&
-           IO_GAP(dio, first) <= maxgap) {
+           IO_SPAN(dio, last) <= limit &&
+           IO_GAP(dio, first) <= maxgap &&
+           dio->io_type == zio->io_type) {
                first = dio;
                if (mandatory == NULL && !(first->io_flags & ZIO_FLAG_OPTIONAL))
                        mandatory = first;
@@ -558,11 +599,17 @@ vdev_queue_aggregate(vdev_queue_t *vq, zio_t *zio)
 
        /*
         * Walk forward through sufficiently contiguous I/Os.
+        * The aggregation limit does not apply to optional i/os, so that
+        * we can issue contiguous writes even if they are larger than the
+        * aggregation limit.
         */
        while ((dio = AVL_NEXT(t, last)) != NULL &&
            (dio->io_flags & ZIO_FLAG_AGG_INHERIT) == flags &&
-           IO_SPAN(first, dio) <= zfs_vdev_aggregation_limit &&
-           IO_GAP(last, dio) <= maxgap) {
+           (IO_SPAN(first, dio) <= limit ||
+           (dio->io_flags & ZIO_FLAG_OPTIONAL)) &&
+           IO_SPAN(first, dio) <= maxblocksize &&
+           IO_GAP(last, dio) <= maxgap &&
+           dio->io_type == zio->io_type) {
                last = dio;
                if (!(last->io_flags & ZIO_FLAG_OPTIONAL))
                        mandatory = last;
@@ -592,10 +639,16 @@ vdev_queue_aggregate(vdev_queue_t *vq, zio_t *zio)
        }
 
        if (stretch) {
-               /* This may be a no-op. */
+               /*
+                * We are going to include an optional io in our aggregated
+                * span, thus closing the write gap.  Only mandatory i/os can
+                * start aggregated spans, so make sure that the next i/o
+                * after our span is mandatory.
+                */
                dio = AVL_NEXT(t, last);
                dio->io_flags &= ~ZIO_FLAG_OPTIONAL;
        } else {
+               /* do not include the optional i/o */
                while (last != mandatory && last != first) {
                        ASSERT(last->io_flags & ZIO_FLAG_OPTIONAL);
                        last = AVL_PREV(t, last);
@@ -607,14 +660,14 @@ vdev_queue_aggregate(vdev_queue_t *vq, zio_t *zio)
                return (NULL);
 
        size = IO_SPAN(first, last);
-       ASSERT3U(size, <=, zfs_vdev_aggregation_limit);
+       ASSERT3U(size, <=, maxblocksize);
 
-       buf = zio_buf_alloc_flags(size, KM_NOSLEEP);
-       if (buf == NULL)
+       abd = abd_alloc_for_io(size, B_TRUE);
+       if (abd == NULL)
                return (NULL);
 
        aio = zio_vdev_delegated_io(first->io_vd, first->io_offset,
-           buf, size, first->io_type, zio->io_priority,
+           abd, size, first->io_type, zio->io_priority,
            flags | ZIO_FLAG_DONT_CACHE | ZIO_FLAG_DONT_QUEUE,
            vdev_queue_agg_io_done, NULL);
        aio->io_timestamp = first->io_timestamp;
@@ -627,19 +680,27 @@ vdev_queue_aggregate(vdev_queue_t *vq, zio_t *zio)
 
                if (dio->io_flags & ZIO_FLAG_NODATA) {
                        ASSERT3U(dio->io_type, ==, ZIO_TYPE_WRITE);
-                       bzero((char *)aio->io_data + (dio->io_offset -
-                           aio->io_offset), dio->io_size);
+                       abd_zero_off(aio->io_abd,
+                           dio->io_offset - aio->io_offset, dio->io_size);
                } else if (dio->io_type == ZIO_TYPE_WRITE) {
-                       bcopy(dio->io_data, (char *)aio->io_data +
-                           (dio->io_offset - aio->io_offset),
-                           dio->io_size);
+                       abd_copy_off(aio->io_abd, dio->io_abd,
+                           dio->io_offset - aio->io_offset, 0, dio->io_size);
                }
 
                zio_add_child(dio, aio);
                vdev_queue_io_remove(vq, dio);
+       } while (dio != last);
+
+       /*
+        * We need to drop the vdev queue's lock to avoid a deadlock that we
+        * could encounter since this I/O will complete immediately.
+        */
+       mutex_exit(&vq->vq_lock);
+       while ((dio = zio_walk_parents(aio, &zl)) != NULL) {
                zio_vdev_io_bypass(dio);
                zio_execute(dio);
-       } while (dio != last);
+       }
+       mutex_enter(&vq->vq_lock);
 
        return (aio);
 }
@@ -663,16 +724,15 @@ again:
        }
 
        /*
-        * For LBA-ordered queues (async / scrub), issue the i/o which follows
-        * the most recently issued i/o in LBA (offset) order.
+        * For LBA-ordered queues (async / scrub / initializing), issue the
+        * i/o which follows the most recently issued i/o in LBA (offset) order.
         *
         * For FIFO queues (sync), issue the i/o with the lowest timestamp.
         */
        tree = vdev_queue_class_tree(vq, p);
        vq->vq_io_search.io_timestamp = 0;
-       vq->vq_io_search.io_offset = vq->vq_last_offset + 1;
-       VERIFY3P(avl_find(tree, &vq->vq_io_search,
-           &idx), ==, NULL);
+       vq->vq_io_search.io_offset = vq->vq_last_offset - 1;
+       VERIFY3P(avl_find(tree, &vq->vq_io_search, &idx), ==, NULL);
        zio = avl_nearest(tree, idx, AVL_AFTER);
        if (zio == NULL)
                zio = avl_first(tree);
@@ -699,7 +759,7 @@ again:
        }
 
        vdev_queue_pending_add(vq, zio);
-       vq->vq_last_offset = zio->io_offset;
+       vq->vq_last_offset = zio->io_offset + zio->io_size;
 
        return (zio);
 }
@@ -720,12 +780,16 @@ vdev_queue_io(zio_t *zio)
        if (zio->io_type == ZIO_TYPE_READ) {
                if (zio->io_priority != ZIO_PRIORITY_SYNC_READ &&
                    zio->io_priority != ZIO_PRIORITY_ASYNC_READ &&
-                   zio->io_priority != ZIO_PRIORITY_SCRUB)
+                   zio->io_priority != ZIO_PRIORITY_SCRUB &&
+                   zio->io_priority != ZIO_PRIORITY_REMOVAL &&
+                   zio->io_priority != ZIO_PRIORITY_INITIALIZING)
                        zio->io_priority = ZIO_PRIORITY_ASYNC_READ;
        } else {
                ASSERT(zio->io_type == ZIO_TYPE_WRITE);
                if (zio->io_priority != ZIO_PRIORITY_SYNC_WRITE &&
-                   zio->io_priority != ZIO_PRIORITY_ASYNC_WRITE)
+                   zio->io_priority != ZIO_PRIORITY_ASYNC_WRITE &&
+                   zio->io_priority != ZIO_PRIORITY_REMOVAL &&
+                   zio->io_priority != ZIO_PRIORITY_INITIALIZING)
                        zio->io_priority = ZIO_PRIORITY_ASYNC_WRITE;
        }
 
@@ -754,9 +818,6 @@ vdev_queue_io_done(zio_t *zio)
        vdev_queue_t *vq = &zio->io_vd->vdev_queue;
        zio_t *nio;
 
-       if (zio_injection_enabled)
-               delay(SEC_TO_TICK(zio_handle_io_delay(zio)));
-
        mutex_enter(&vq->vq_lock);
 
        vdev_queue_pending_remove(vq, zio);
@@ -779,7 +840,76 @@ vdev_queue_io_done(zio_t *zio)
        mutex_exit(&vq->vq_lock);
 }
 
-#if defined(_KERNEL) && defined(HAVE_SPL)
+void
+vdev_queue_change_io_priority(zio_t *zio, zio_priority_t priority)
+{
+       vdev_queue_t *vq = &zio->io_vd->vdev_queue;
+       avl_tree_t *tree;
+
+       /*
+        * ZIO_PRIORITY_NOW is used by the vdev cache code and the aggregate zio
+        * code to issue IOs without adding them to the vdev queue. In this
+        * case, the zio is already going to be issued as quickly as possible
+        * and so it doesn't need any reprioitization to help.
+        */
+       if (zio->io_priority == ZIO_PRIORITY_NOW)
+               return;
+
+       ASSERT3U(zio->io_priority, <, ZIO_PRIORITY_NUM_QUEUEABLE);
+       ASSERT3U(priority, <, ZIO_PRIORITY_NUM_QUEUEABLE);
+
+       if (zio->io_type == ZIO_TYPE_READ) {
+               if (priority != ZIO_PRIORITY_SYNC_READ &&
+                   priority != ZIO_PRIORITY_ASYNC_READ &&
+                   priority != ZIO_PRIORITY_SCRUB)
+                       priority = ZIO_PRIORITY_ASYNC_READ;
+       } else {
+               ASSERT(zio->io_type == ZIO_TYPE_WRITE);
+               if (priority != ZIO_PRIORITY_SYNC_WRITE &&
+                   priority != ZIO_PRIORITY_ASYNC_WRITE)
+                       priority = ZIO_PRIORITY_ASYNC_WRITE;
+       }
+
+       mutex_enter(&vq->vq_lock);
+
+       /*
+        * If the zio is in none of the queues we can simply change
+        * the priority. If the zio is waiting to be submitted we must
+        * remove it from the queue and re-insert it with the new priority.
+        * Otherwise, the zio is currently active and we cannot change its
+        * priority.
+        */
+       tree = vdev_queue_class_tree(vq, zio->io_priority);
+       if (avl_find(tree, zio, NULL) == zio) {
+               avl_remove(vdev_queue_class_tree(vq, zio->io_priority), zio);
+               zio->io_priority = priority;
+               avl_add(vdev_queue_class_tree(vq, zio->io_priority), zio);
+       } else if (avl_find(&vq->vq_active_tree, zio, NULL) != zio) {
+               zio->io_priority = priority;
+       }
+
+       mutex_exit(&vq->vq_lock);
+}
+
+/*
+ * As these two methods are only used for load calculations we're not
+ * concerned if we get an incorrect value on 32bit platforms due to lack of
+ * vq_lock mutex use here, instead we prefer to keep it lock free for
+ * performance.
+ */
+int
+vdev_queue_length(vdev_t *vd)
+{
+       return (avl_numnodes(&vd->vdev_queue.vq_active_tree));
+}
+
+uint64_t
+vdev_queue_last_offset(vdev_t *vd)
+{
+       return (vd->vdev_queue.vq_last_offset);
+}
+
+#if defined(_KERNEL)
 module_param(zfs_vdev_aggregation_limit, int, 0644);
 MODULE_PARM_DESC(zfs_vdev_aggregation_limit, "Max vdev I/O aggregation size");
 
@@ -816,11 +946,29 @@ module_param(zfs_vdev_async_write_min_active, int, 0644);
 MODULE_PARM_DESC(zfs_vdev_async_write_min_active,
        "Min active async write I/Os per vdev");
 
+module_param(zfs_vdev_initializing_max_active, int, 0644);
+MODULE_PARM_DESC(zfs_vdev_initializing_max_active,
+       "Max active initializing I/Os per vdev");
+
+module_param(zfs_vdev_initializing_min_active, int, 0644);
+MODULE_PARM_DESC(zfs_vdev_initializing_min_active,
+       "Min active initializing I/Os per vdev");
+
+module_param(zfs_vdev_removal_max_active, int, 0644);
+MODULE_PARM_DESC(zfs_vdev_removal_max_active,
+       "Max active removal I/Os per vdev");
+
+module_param(zfs_vdev_removal_min_active, int, 0644);
+MODULE_PARM_DESC(zfs_vdev_removal_min_active,
+       "Min active removal I/Os per vdev");
+
 module_param(zfs_vdev_scrub_max_active, int, 0644);
-MODULE_PARM_DESC(zfs_vdev_scrub_max_active, "Max active scrub I/Os per vdev");
+MODULE_PARM_DESC(zfs_vdev_scrub_max_active,
+       "Max active scrub I/Os per vdev");
 
 module_param(zfs_vdev_scrub_min_active, int, 0644);
-MODULE_PARM_DESC(zfs_vdev_scrub_min_active, "Min active scrub I/Os per vdev");
+MODULE_PARM_DESC(zfs_vdev_scrub_min_active,
+       "Min active scrub I/Os per vdev");
 
 module_param(zfs_vdev_sync_read_max_active, int, 0644);
 MODULE_PARM_DESC(zfs_vdev_sync_read_max_active,
@@ -837,4 +985,8 @@ MODULE_PARM_DESC(zfs_vdev_sync_write_max_active,
 module_param(zfs_vdev_sync_write_min_active, int, 0644);
 MODULE_PARM_DESC(zfs_vdev_sync_write_min_active,
        "Min active sync write I/Os per vdev");
+
+module_param(zfs_vdev_queue_depth_pct, int, 0644);
+MODULE_PARM_DESC(zfs_vdev_queue_depth_pct,
+       "Queue depth percentage for each top-level vdev");
 #endif