]> git.proxmox.com Git - mirror_zfs.git/blobdiff - module/zfs/vdev_disk.c
OpenZFS 6531 - Provide mechanism to artificially limit disk performance
[mirror_zfs.git] / module / zfs / vdev_disk.c
index 863392544f84c2870317497dc0199b0e76c62395..1ef54d3899f9c2dadd6de5aa093cc44c0d0b9f3a 100644 (file)
@@ -23,6 +23,7 @@
  * Produced at Lawrence Livermore National Laboratory (cf, DISCLAIMER).
  * Rewritten for Linux by Brian Behlendorf <behlendorf1@llnl.gov>.
  * LLNL-CODE-403049.
+ * Copyright (c) 2012, 2015 by Delphix. All rights reserved.
  */
 
 #include <sys/zfs_context.h>
 #include <sys/zio.h>
 #include <sys/sunldi.h>
 
+char *zfs_vdev_scheduler = VDEV_SCHEDULER;
+static void *zfs_vdev_holder = VDEV_HOLDER;
+
 /*
  * Virtual device vector for disks.
  */
 typedef struct dio_request {
        struct completion       dr_comp;        /* Completion for sync IO */
-       atomic_t                dr_ref;         /* References */
        zio_t                   *dr_zio;        /* Parent ZIO */
-       int                     dr_rw;          /* Read/Write */
+       atomic_t                dr_ref;         /* References */
+       int                     dr_wait;        /* Wait for IO */
        int                     dr_error;       /* Bio error */
        int                     dr_bio_count;   /* Count of bio's */
-        struct bio             *dr_bio[0];     /* Attached bio's */
+       struct bio              *dr_bio[0];     /* Attached bio's */
 } dio_request_t;
 
 
@@ -61,7 +65,7 @@ vdev_bdev_mode(int smode)
        if (smode & FWRITE)
                mode |= FMODE_WRITE;
 
-       return mode;
+       return (mode);
 }
 #else
 static int
@@ -74,7 +78,7 @@ vdev_bdev_mode(int smode)
        if ((smode & FREAD) && !(smode & FWRITE))
                mode = MS_RDONLY;
 
-       return mode;
+       return (mode);
 }
 #endif /* HAVE_OPEN_BDEV_EXCLUSIVE */
 
@@ -85,89 +89,253 @@ bdev_capacity(struct block_device *bdev)
 
        /* The partition capacity referenced by the block device */
        if (part)
-              return part->nr_sects;
+               return (part->nr_sects << 9);
 
        /* Otherwise assume the full device capacity */
-       return get_capacity(bdev->bd_disk);
+       return (get_capacity(bdev->bd_disk) << 9);
 }
 
 static void
 vdev_disk_error(zio_t *zio)
 {
 #ifdef ZFS_DEBUG
-       printk("ZFS: zio error=%d type=%d offset=%llu "
-           "size=%llu flags=%x\n", zio->io_error, zio->io_type,
+       printk("ZFS: zio error=%d type=%d offset=%llu size=%llu "
+           "flags=%x\n", zio->io_error, zio->io_type,
            (u_longlong_t)zio->io_offset, (u_longlong_t)zio->io_size,
            zio->io_flags);
 #endif
 }
 
+/*
+ * Use the Linux 'noop' elevator for zfs managed block devices.  This
+ * strikes the ideal balance by allowing the zfs elevator to do all
+ * request ordering and prioritization.  While allowing the Linux
+ * elevator to do the maximum front/back merging allowed by the
+ * physical device.  This yields the largest possible requests for
+ * the device with the lowest total overhead.
+ */
 static int
-vdev_disk_open(vdev_t *v, uint64_t *psize, uint64_t *ashift)
+vdev_elevator_switch(vdev_t *v, char *elevator)
 {
-       struct block_device *bdev;
+       vdev_disk_t *vd = v->vdev_tsd;
+       struct block_device *bdev = vd->vd_bdev;
+       struct request_queue *q = bdev_get_queue(bdev);
+       char *device = bdev->bd_disk->disk_name;
+       int error;
+
+       /*
+        * Skip devices which are not whole disks (partitions).
+        * Device-mapper devices are excepted since they may be whole
+        * disks despite the vdev_wholedisk flag, in which case we can
+        * and should switch the elevator. If the device-mapper device
+        * does not have an elevator (i.e. dm-raid, dm-crypt, etc.) the
+        * "Skip devices without schedulers" check below will fail.
+        */
+       if (!v->vdev_wholedisk && strncmp(device, "dm-", 3) != 0)
+               return (0);
+
+       /* Skip devices without schedulers (loop, ram, dm, etc) */
+       if (!q->elevator || !blk_queue_stackable(q))
+               return (0);
+
+       /* Leave existing scheduler when set to "none" */
+       if ((strncmp(elevator, "none", 4) == 0) && (strlen(elevator) == 4))
+               return (0);
+
+#ifdef HAVE_ELEVATOR_CHANGE
+       error = elevator_change(q, elevator);
+#else
+       /*
+        * For pre-2.6.36 kernels elevator_change() is not available.
+        * Therefore we fall back to using a usermodehelper to echo the
+        * elevator into sysfs;  This requires /bin/echo and sysfs to be
+        * mounted which may not be true early in the boot process.
+        */
+#define        SET_SCHEDULER_CMD \
+       "exec 0</dev/null " \
+       "     1>/sys/block/%s/queue/scheduler " \
+       "     2>/dev/null; " \
+       "echo %s"
+
+       {
+               char *argv[] = { "/bin/sh", "-c", NULL, NULL };
+               char *envp[] = { NULL };
+
+               argv[2] = kmem_asprintf(SET_SCHEDULER_CMD, device, elevator);
+               error = call_usermodehelper(argv[0], argv, envp, UMH_WAIT_PROC);
+               strfree(argv[2]);
+       }
+#endif /* HAVE_ELEVATOR_CHANGE */
+       if (error)
+               printk("ZFS: Unable to set \"%s\" scheduler for %s (%s): %d\n",
+                   elevator, v->vdev_path, device, error);
+
+       return (error);
+}
+
+/*
+ * Expanding a whole disk vdev involves invoking BLKRRPART on the
+ * whole disk device. This poses a problem, because BLKRRPART will
+ * return EBUSY if one of the disk's partitions is open. That's why
+ * we have to do it here, just before opening the data partition.
+ * Unfortunately, BLKRRPART works by dropping all partitions and
+ * recreating them, which means that for a short time window, all
+ * /dev/sdxN device files disappear (until udev recreates them).
+ * This means two things:
+ *  - When we open the data partition just after a BLKRRPART, we
+ *    can't do it using the normal device file path because of the
+ *    obvious race condition with udev. Instead, we use reliable
+ *    kernel APIs to get a handle to the new partition device from
+ *    the whole disk device.
+ *  - Because vdev_disk_open() initially needs to find the device
+ *    using its path, multiple vdev_disk_open() invocations in
+ *    short succession on the same disk with BLKRRPARTs in the
+ *    middle have a high probability of failure (because of the
+ *    race condition with udev). A typical situation where this
+ *    might happen is when the zpool userspace tool does a
+ *    TRYIMPORT immediately followed by an IMPORT. For this
+ *    reason, we only invoke BLKRRPART in the module when strictly
+ *    necessary (zpool online -e case), and rely on userspace to
+ *    do it when possible.
+ */
+static struct block_device *
+vdev_disk_rrpart(const char *path, int mode, vdev_disk_t *vd)
+{
+#if defined(HAVE_3ARG_BLKDEV_GET) && defined(HAVE_GET_GENDISK)
+       struct block_device *bdev, *result = ERR_PTR(-ENXIO);
+       struct gendisk *disk;
+       int error, partno;
+
+       bdev = vdev_bdev_open(path, vdev_bdev_mode(mode), zfs_vdev_holder);
+       if (IS_ERR(bdev))
+               return (bdev);
+
+       disk = get_gendisk(bdev->bd_dev, &partno);
+       vdev_bdev_close(bdev, vdev_bdev_mode(mode));
+
+       if (disk) {
+               bdev = bdget(disk_devt(disk));
+               if (bdev) {
+                       error = blkdev_get(bdev, vdev_bdev_mode(mode), vd);
+                       if (error == 0)
+                               error = ioctl_by_bdev(bdev, BLKRRPART, 0);
+                       vdev_bdev_close(bdev, vdev_bdev_mode(mode));
+               }
+
+               bdev = bdget_disk(disk, partno);
+               if (bdev) {
+                       error = blkdev_get(bdev,
+                           vdev_bdev_mode(mode) | FMODE_EXCL, vd);
+                       if (error == 0)
+                               result = bdev;
+               }
+               put_disk(disk);
+       }
+
+       return (result);
+#else
+       return (ERR_PTR(-EOPNOTSUPP));
+#endif /* defined(HAVE_3ARG_BLKDEV_GET) && defined(HAVE_GET_GENDISK) */
+}
+
+static int
+vdev_disk_open(vdev_t *v, uint64_t *psize, uint64_t *max_psize,
+    uint64_t *ashift)
+{
+       struct block_device *bdev = ERR_PTR(-ENXIO);
        vdev_disk_t *vd;
-       int mode, block_size;
+       int count = 0, mode, block_size;
 
        /* Must have a pathname and it must be absolute. */
        if (v->vdev_path == NULL || v->vdev_path[0] != '/') {
                v->vdev_stat.vs_aux = VDEV_AUX_BAD_LABEL;
-               return EINVAL;
+               return (SET_ERROR(EINVAL));
        }
 
-       vd = kmem_zalloc(sizeof(vdev_disk_t), KM_SLEEP);
+       /*
+        * Reopen the device if it's not currently open. Otherwise,
+        * just update the physical size of the device.
+        */
+       if (v->vdev_tsd != NULL) {
+               ASSERT(v->vdev_reopening);
+               vd = v->vdev_tsd;
+               goto skip_open;
+       }
+
+       vd = kmem_zalloc(sizeof (vdev_disk_t), KM_SLEEP);
        if (vd == NULL)
-               return ENOMEM;
+               return (SET_ERROR(ENOMEM));
 
        /*
         * Devices are always opened by the path provided at configuration
         * time.  This means that if the provided path is a udev by-id path
         * then drives may be recabled without an issue.  If the provided
-        * path is a udev by-path path then the physical location information
+        * path is a udev by-path path, then the physical location information
         * will be preserved.  This can be critical for more complicated
         * configurations where drives are located in specific physical
         * locations to maximize the systems tolerence to component failure.
-        * Alternately you can provide your own udev rule to flexibly map
+        * Alternatively, you can provide your own udev rule to flexibly map
         * the drives as you see fit.  It is not advised that you use the
-        * /dev/[hd]d devices which may be reorder due to probing order.
+        * /dev/[hd]d devices which may be reordered due to probing order.
         * Devices in the wrong locations will be detected by the higher
         * level vdev validation.
+        *
+        * The specified paths may be briefly removed and recreated in
+        * response to udev events.  This should be exceptionally unlikely
+        * because the zpool command makes every effort to verify these paths
+        * have already settled prior to reaching this point.  Therefore,
+        * a ENOENT failure at this point is highly likely to be transient
+        * and it is reasonable to sleep and retry before giving up.  In
+        * practice delays have been observed to be on the order of 100ms.
         */
        mode = spa_mode(v->vdev_spa);
-       bdev = vdev_bdev_open(v->vdev_path, vdev_bdev_mode(mode), vd);
+       if (v->vdev_wholedisk && v->vdev_expanding)
+               bdev = vdev_disk_rrpart(v->vdev_path, mode, vd);
+
+       while (IS_ERR(bdev) && count < 50) {
+               bdev = vdev_bdev_open(v->vdev_path,
+                   vdev_bdev_mode(mode), zfs_vdev_holder);
+               if (unlikely(PTR_ERR(bdev) == -ENOENT)) {
+                       msleep(10);
+                       count++;
+               } else if (IS_ERR(bdev)) {
+                       break;
+               }
+       }
+
        if (IS_ERR(bdev)) {
-               kmem_free(vd, sizeof(vdev_disk_t));
-               return -PTR_ERR(bdev);
+               dprintf("failed open v->vdev_path=%s, error=%d count=%d\n",
+                   v->vdev_path, -PTR_ERR(bdev), count);
+               kmem_free(vd, sizeof (vdev_disk_t));
+               return (SET_ERROR(-PTR_ERR(bdev)));
        }
 
        v->vdev_tsd = vd;
        vd->vd_bdev = bdev;
-       block_size =  vdev_bdev_block_size(bdev);
-
-       /* We think the wholedisk property should always be set when this
-        * function is called.  ASSERT here so if any legitimate cases exist
-        * where it's not set, we'll find them during debugging.  If we never
-        * hit the ASSERT, this and the following conditional statement can be
-        * removed. */
-       ASSERT3S(v->vdev_wholedisk, !=, -1ULL);
-
-       /* The wholedisk property was initialized to -1 in vdev_alloc() if it
-        * was unspecified.  In that case, check if this is a whole device.
-        * When bdev->bd_contains == bdev we have a whole device and not simply
-        * a partition. */
-       if (v->vdev_wholedisk == -1ULL)
-               v->vdev_wholedisk = (bdev->bd_contains == bdev);
+
+skip_open:
+       /*  Determine the physical block size */
+       block_size = vdev_bdev_block_size(vd->vd_bdev);
 
        /* Clear the nowritecache bit, causes vdev_reopen() to try again. */
        v->vdev_nowritecache = B_FALSE;
 
+       /* Inform the ZIO pipeline that we are non-rotational */
+       v->vdev_nonrot = blk_queue_nonrot(bdev_get_queue(vd->vd_bdev));
+
        /* Physical volume size in bytes */
-       *psize = bdev_capacity(bdev) * block_size;
+       *psize = bdev_capacity(vd->vd_bdev);
+
+       /* TODO: report possible expansion size */
+       *max_psize = *psize;
 
        /* Based on the minimum sector size set the block size */
-       *ashift = highbit(MAX(block_size, SPA_MINBLOCKSIZE)) - 1;
+       *ashift = highbit64(MAX(block_size, SPA_MINBLOCKSIZE)) - 1;
+
+       /* Try to set the io scheduler elevator algorithm */
+       (void) vdev_elevator_switch(v, zfs_vdev_scheduler);
 
-       return 0;
+       return (0);
 }
 
 static void
@@ -175,14 +343,14 @@ vdev_disk_close(vdev_t *v)
 {
        vdev_disk_t *vd = v->vdev_tsd;
 
-       if (vd == NULL)
+       if (v->vdev_reopening || vd == NULL)
                return;
 
        if (vd->vd_bdev != NULL)
                vdev_bdev_close(vd->vd_bdev,
-                               vdev_bdev_mode(spa_mode(v->vdev_spa)));
+                   vdev_bdev_mode(spa_mode(v->vdev_spa)));
 
-       kmem_free(vd, sizeof(vdev_disk_t));
+       kmem_free(vd, sizeof (vdev_disk_t));
        v->vdev_tsd = NULL;
 }
 
@@ -192,8 +360,8 @@ vdev_disk_dio_alloc(int bio_count)
        dio_request_t *dr;
        int i;
 
-       dr = kmem_zalloc(sizeof(dio_request_t) +
-                        sizeof(struct bio *) * bio_count, KM_SLEEP);
+       dr = kmem_zalloc(sizeof (dio_request_t) +
+           sizeof (struct bio *) * bio_count, KM_SLEEP);
        if (dr) {
                init_completion(&dr->dr_comp);
                atomic_set(&dr->dr_ref, 0);
@@ -204,7 +372,7 @@ vdev_disk_dio_alloc(int bio_count)
                        dr->dr_bio[i] = NULL;
        }
 
-       return dr;
+       return (dr);
 }
 
 static void
@@ -216,8 +384,8 @@ vdev_disk_dio_free(dio_request_t *dr)
                if (dr->dr_bio[i])
                        bio_put(dr->dr_bio[i]);
 
-       kmem_free(dr, sizeof(dio_request_t) +
-                 sizeof(struct bio *) * dr->dr_bio_count);
+       kmem_free(dr, sizeof (dio_request_t) +
+           sizeof (struct bio *) * dr->dr_bio_count);
 }
 
 static void
@@ -246,53 +414,44 @@ vdev_disk_dio_put(dio_request_t *dr)
                        ASSERT3S(zio->io_error, >=, 0);
                        if (zio->io_error)
                                vdev_disk_error(zio);
-                       zio_interrupt(zio);
+                       zio_delay_interrupt(zio);
                }
        }
 
-       return rc;
+       return (rc);
 }
 
-BIO_END_IO_PROTO(vdev_disk_physio_completion, bio, size, error)
+BIO_END_IO_PROTO(vdev_disk_physio_completion, bio, error)
 {
        dio_request_t *dr = bio->bi_private;
        int rc;
+       int wait;
 
-       /* Fatal error but print some useful debugging before asserting */
-       if (dr == NULL)
-               PANIC("dr == NULL, bio->bi_private == NULL\n"
-                   "bi_next: %p, bi_flags: %lx, bi_rw: %lu, bi_vcnt: %d\n"
-                   "bi_idx: %d, bi_size: %d, bi_end_io: %p, bi_cnt: %d\n",
-                   bio->bi_next, bio->bi_flags, bio->bi_rw, bio->bi_vcnt,
-                   bio->bi_idx, bio->bi_size, bio->bi_end_io,
-                   atomic_read(&bio->bi_cnt));
-
-#ifndef HAVE_2ARGS_BIO_END_IO_T
-       if (bio->bi_size)
-               return 1;
-#endif /* HAVE_2ARGS_BIO_END_IO_T */
-
-       if (error == 0 && !test_bit(BIO_UPTODATE, &bio->bi_flags))
-               error = -EIO;
-
-       if (dr->dr_error == 0)
-               dr->dr_error = -error;
+       if (dr->dr_error == 0) {
+#ifdef HAVE_1ARG_BIO_END_IO_T
+               dr->dr_error = -(bio->bi_error);
+#else
+               if (error)
+                       dr->dr_error = -(error);
+               else if (!test_bit(BIO_UPTODATE, &bio->bi_flags))
+                       dr->dr_error = EIO;
+#endif
+       }
 
+       wait = dr->dr_wait;
        /* Drop reference aquired by __vdev_disk_physio */
        rc = vdev_disk_dio_put(dr);
 
        /* Wake up synchronous waiter this is the last outstanding bio */
-       if ((rc == 1) && (dr->dr_rw & (1 << DIO_RW_SYNCIO)))
+       if (wait && rc == 1)
                complete(&dr->dr_comp);
-
-       BIO_END_IO_RETURN(0);
 }
 
 static inline unsigned long
 bio_nr_pages(void *bio_ptr, unsigned int bio_size)
 {
        return ((((unsigned long)bio_ptr + bio_size + PAGE_SIZE - 1) >>
-               PAGE_SHIFT) - ((unsigned long)bio_ptr >> PAGE_SHIFT));
+           PAGE_SHIFT) - ((unsigned long)bio_ptr >> PAGE_SHIFT));
 }
 
 static unsigned int
@@ -311,11 +470,18 @@ bio_map(struct bio *bio, void *bio_ptr, unsigned int bio_size)
                if (size > bio_size)
                        size = bio_size;
 
-               if (kmem_virt(bio_ptr))
+               if (is_vmalloc_addr(bio_ptr))
                        page = vmalloc_to_page(bio_ptr);
                else
                        page = virt_to_page(bio_ptr);
 
+               /*
+                * Some network related block device uses tcp_sendpage, which
+                * doesn't behave well when using 0-count page, this is a
+                * safety net to catch them.
+                */
+               ASSERT3S(page_count(page), >, 0);
+
                if (bio_add_page(bio, page, size, offset) != size)
                        break;
 
@@ -324,30 +490,48 @@ bio_map(struct bio *bio, void *bio_ptr, unsigned int bio_size)
                offset = 0;
        }
 
-        return bio_size;
+       return (bio_size);
+}
+
+static inline void
+vdev_submit_bio(int rw, struct bio *bio)
+{
+#ifdef HAVE_CURRENT_BIO_TAIL
+       struct bio **bio_tail = current->bio_tail;
+       current->bio_tail = NULL;
+       submit_bio(rw, bio);
+       current->bio_tail = bio_tail;
+#else
+       struct bio_list *bio_list = current->bio_list;
+       current->bio_list = NULL;
+       submit_bio(rw, bio);
+       current->bio_list = bio_list;
+#endif
 }
 
 static int
 __vdev_disk_physio(struct block_device *bdev, zio_t *zio, caddr_t kbuf_ptr,
-                   size_t kbuf_size, uint64_t kbuf_offset, int flags)
+    size_t kbuf_size, uint64_t kbuf_offset, int flags, int wait)
 {
-        dio_request_t *dr;
+       dio_request_t *dr;
        caddr_t bio_ptr;
        uint64_t bio_offset;
-       int bio_size, bio_count = 16;
-       int i = 0, error = 0, block_size;
+       int rw, bio_size, bio_count = 16;
+       int i = 0, error = 0;
+
+       ASSERT3U(kbuf_offset + kbuf_size, <=, bdev->bd_inode->i_size);
 
 retry:
        dr = vdev_disk_dio_alloc(bio_count);
        if (dr == NULL)
-               return ENOMEM;
+               return (ENOMEM);
 
        if (zio && !(zio->io_flags & (ZIO_FLAG_IO_RETRY | ZIO_FLAG_TRYHARD)))
-                       bio_set_flags_failfast(bdev, &flags);
+               bio_set_flags_failfast(bdev, &flags);
 
+       rw = flags;
        dr->dr_zio = zio;
-       dr->dr_rw = flags;
-       block_size = vdev_bdev_block_size(bdev);
+       dr->dr_wait = wait;
 
        /*
         * When the IO size exceeds the maximum bio size for the request
@@ -373,23 +557,23 @@ retry:
                if (dr->dr_bio_count == i) {
                        vdev_disk_dio_free(dr);
                        bio_count *= 2;
-                       printk("WARNING: Resized bio's/dio to %d\n",bio_count);
                        goto retry;
                }
 
+               /* bio_alloc() with __GFP_WAIT never returns NULL */
                dr->dr_bio[i] = bio_alloc(GFP_NOIO,
-                                         bio_nr_pages(bio_ptr, bio_size));
-               if (dr->dr_bio[i] == NULL) {
+                   MIN(bio_nr_pages(bio_ptr, bio_size), BIO_MAX_PAGES));
+               if (unlikely(dr->dr_bio[i] == NULL)) {
                        vdev_disk_dio_free(dr);
-                       return ENOMEM;
+                       return (ENOMEM);
                }
 
                /* Matching put called by vdev_disk_physio_completion */
                vdev_disk_dio_get(dr);
 
                dr->dr_bio[i]->bi_bdev = bdev;
-               dr->dr_bio[i]->bi_sector = bio_offset / block_size;
-               dr->dr_bio[i]->bi_rw = dr->dr_rw;
+               BIO_BI_SECTOR(dr->dr_bio[i]) = bio_offset >> 9;
+               dr->dr_bio[i]->bi_rw = rw;
                dr->dr_bio[i]->bi_end_io = vdev_disk_physio_completion;
                dr->dr_bio[i]->bi_private = dr;
 
@@ -397,17 +581,17 @@ retry:
                bio_size = bio_map(dr->dr_bio[i], bio_ptr, bio_size);
 
                /* Advance in buffer and construct another bio if needed */
-               bio_ptr    += dr->dr_bio[i]->bi_size;
-               bio_offset += dr->dr_bio[i]->bi_size;
+               bio_ptr    += BIO_BI_SIZE(dr->dr_bio[i]);
+               bio_offset += BIO_BI_SIZE(dr->dr_bio[i]);
        }
 
-       /* Extra reference to protect dio_request during submit_bio */
+       /* Extra reference to protect dio_request during vdev_submit_bio */
        vdev_disk_dio_get(dr);
 
        /* Submit all bio's associated with this dio */
        for (i = 0; i < dr->dr_bio_count; i++)
                if (dr->dr_bio[i])
-                       submit_bio(dr->dr_rw, dr->dr_bio[i]);
+                       vdev_submit_bio(rw, dr->dr_bio[i]);
 
        /*
         * On synchronous blocking requests we wait for all bio the completion
@@ -417,30 +601,31 @@ retry:
         * only synchronous consumer is vdev_disk_read_rootlabel() all other
         * IO originating from vdev_disk_io_start() is asynchronous.
         */
-       if (dr->dr_rw & (1 << DIO_RW_SYNCIO)) {
+       if (wait) {
                wait_for_completion(&dr->dr_comp);
                error = dr->dr_error;
                ASSERT3S(atomic_read(&dr->dr_ref), ==, 1);
        }
 
-       (void)vdev_disk_dio_put(dr);
+       (void) vdev_disk_dio_put(dr);
 
-       return error;
+       return (error);
 }
 
 int
 vdev_disk_physio(struct block_device *bdev, caddr_t kbuf,
-                size_t size, uint64_t offset, int flags)
+    size_t size, uint64_t offset, int flags)
 {
        bio_set_flags_failfast(bdev, &flags);
-       return __vdev_disk_physio(bdev, NULL, kbuf, size, offset, flags);
+       return (__vdev_disk_physio(bdev, NULL, kbuf, size, offset, flags, 1));
 }
 
-/* 2.6.24 API change */
-#ifdef HAVE_BIO_EMPTY_BARRIER
-BIO_END_IO_PROTO(vdev_disk_io_flush_completion, bio, size, rc)
+BIO_END_IO_PROTO(vdev_disk_io_flush_completion, bio, rc)
 {
        zio_t *zio = bio->bi_private;
+#ifdef HAVE_1ARG_BIO_END_IO_T
+       int rc = bio->bi_error;
+#endif
 
        zio->io_error = -rc;
        if (rc && (rc == -EOPNOTSUPP))
@@ -451,8 +636,6 @@ BIO_END_IO_PROTO(vdev_disk_io_flush_completion, bio, size, rc)
        if (zio->io_error)
                vdev_disk_error(zio);
        zio_interrupt(zio);
-
-       BIO_END_IO_RETURN(0);
 }
 
 static int
@@ -463,40 +646,37 @@ vdev_disk_io_flush(struct block_device *bdev, zio_t *zio)
 
        q = bdev_get_queue(bdev);
        if (!q)
-               return ENXIO;
+               return (ENXIO);
 
-       bio = bio_alloc(GFP_KERNEL, 0);
-       if (!bio)
-               return ENOMEM;
+       bio = bio_alloc(GFP_NOIO, 0);
+       /* bio_alloc() with __GFP_WAIT never returns NULL */
+       if (unlikely(bio == NULL))
+               return (ENOMEM);
 
        bio->bi_end_io = vdev_disk_io_flush_completion;
        bio->bi_private = zio;
        bio->bi_bdev = bdev;
-       submit_bio(WRITE_BARRIER, bio);
+       vdev_submit_bio(VDEV_WRITE_FLUSH_FUA, bio);
+       invalidate_bdev(bdev);
 
-       return 0;
-}
-#else
-static int
-vdev_disk_io_flush(struct block_device *bdev, zio_t *zio)
-{
-       return ENOTSUP;
+       return (0);
 }
-#endif /* HAVE_BIO_EMPTY_BARRIER */
 
-static int
+static void
 vdev_disk_io_start(zio_t *zio)
 {
        vdev_t *v = zio->io_vd;
        vdev_disk_t *vd = v->vdev_tsd;
+       zio_priority_t pri = zio->io_priority;
        int flags, error;
 
        switch (zio->io_type) {
        case ZIO_TYPE_IOCTL:
 
                if (!vdev_readable(v)) {
-                       zio->io_error = ENXIO;
-                       return ZIO_PIPELINE_CONTINUE;
+                       zio->io_error = SET_ERROR(ENXIO);
+                       zio_interrupt(zio);
+                       return;
                }
 
                switch (zio->io_cmd) {
@@ -506,13 +686,13 @@ vdev_disk_io_start(zio_t *zio)
                                break;
 
                        if (v->vdev_nowritecache) {
-                               zio->io_error = ENOTSUP;
+                               zio->io_error = SET_ERROR(ENOTSUP);
                                break;
                        }
 
                        error = vdev_disk_io_flush(vd->vd_bdev, zio);
                        if (error == 0)
-                               return ZIO_PIPELINE_STOP;
+                               return;
 
                        zio->io_error = error;
                        if (error == ENOTSUP)
@@ -521,32 +701,39 @@ vdev_disk_io_start(zio_t *zio)
                        break;
 
                default:
-                       zio->io_error = ENOTSUP;
+                       zio->io_error = SET_ERROR(ENOTSUP);
                }
 
-               return ZIO_PIPELINE_CONTINUE;
-
+               zio_execute(zio);
+               return;
        case ZIO_TYPE_WRITE:
-               flags = WRITE;
+               if ((pri == ZIO_PRIORITY_SYNC_WRITE) && (v->vdev_nonrot))
+                       flags = WRITE_SYNC;
+               else
+                       flags = WRITE;
                break;
 
        case ZIO_TYPE_READ:
-               flags = READ;
+               if ((pri == ZIO_PRIORITY_SYNC_READ) && (v->vdev_nonrot))
+                       flags = READ_SYNC;
+               else
+                       flags = READ;
                break;
 
        default:
-               zio->io_error = ENOTSUP;
-               return ZIO_PIPELINE_CONTINUE;
+               zio->io_error = SET_ERROR(ENOTSUP);
+               zio_interrupt(zio);
+               return;
        }
 
+       zio->io_target_timestamp = zio_handle_io_delay(zio);
        error = __vdev_disk_physio(vd->vd_bdev, zio, zio->io_data,
-                                  zio->io_size, zio->io_offset, flags);
+           zio->io_size, zio->io_offset, flags, 0);
        if (error) {
                zio->io_error = error;
-               return ZIO_PIPELINE_CONTINUE;
+               zio_interrupt(zio);
+               return;
        }
-
-       return ZIO_PIPELINE_STOP;
 }
 
 static void
@@ -558,7 +745,7 @@ vdev_disk_io_done(zio_t *zio)
         * removal of the device from the configuration.
         */
        if (zio->io_error == EIO) {
-               vdev_t *v = zio->io_vd;
+               vdev_t *v = zio->io_vd;
                vdev_disk_t *vd = v->vdev_tsd;
 
                if (check_disk_change(vd->vd_bdev)) {
@@ -623,21 +810,21 @@ vdev_disk_read_rootlabel(char *devpath, char *devid, nvlist_t **config)
        uint64_t s, size;
        int i;
 
-       bdev = vdev_bdev_open(devpath, vdev_bdev_mode(FREAD), NULL);
+       bdev = vdev_bdev_open(devpath, vdev_bdev_mode(FREAD), zfs_vdev_holder);
        if (IS_ERR(bdev))
-               return -PTR_ERR(bdev);
+               return (-PTR_ERR(bdev));
 
-       s = bdev_capacity(bdev) * vdev_bdev_block_size(bdev);
+       s = bdev_capacity(bdev);
        if (s == 0) {
                vdev_bdev_close(bdev, vdev_bdev_mode(FREAD));
-               return EIO;
+               return (EIO);
        }
 
-       size = P2ALIGN_TYPED(s, sizeof(vdev_label_t), uint64_t);
-       label = vmem_alloc(sizeof(vdev_label_t), KM_SLEEP);
+       size = P2ALIGN_TYPED(s, sizeof (vdev_label_t), uint64_t);
+       label = vmem_alloc(sizeof (vdev_label_t), KM_SLEEP);
 
        for (i = 0; i < VDEV_LABELS; i++) {
-               uint64_t offset, state, txg = 0;
+               uint64_t offset, state, txg = 0;
 
                /* read vdev label */
                offset = vdev_label_offset(size, i, 0);
@@ -668,8 +855,11 @@ vdev_disk_read_rootlabel(char *devpath, char *devid, nvlist_t **config)
                break;
        }
 
-       vmem_free(label, sizeof(vdev_label_t));
+       vmem_free(label, sizeof (vdev_label_t));
        vdev_bdev_close(bdev, vdev_bdev_mode(FREAD));
 
-       return 0;
+       return (0);
 }
+
+module_param(zfs_vdev_scheduler, charp, 0644);
+MODULE_PARM_DESC(zfs_vdev_scheduler, "I/O scheduler");