]> git.proxmox.com Git - mirror_zfs.git/blobdiff - module/zfs/zio.c
Provide macros for setting and getting blkptr birth times
[mirror_zfs.git] / module / zfs / zio.c
index 3b3b40fa73d81ca6359718724427db68f525bd57..e96bbda35a047e19ee8579239d089eb6cb7043d8 100644 (file)
@@ -158,23 +158,22 @@ zio_init(void)
        zio_link_cache = kmem_cache_create("zio_link_cache",
            sizeof (zio_link_t), 0, NULL, NULL, NULL, NULL, NULL, 0);
 
-       /*
-        * For small buffers, we want a cache for each multiple of
-        * SPA_MINBLOCKSIZE.  For larger buffers, we want a cache
-        * for each quarter-power of 2.
-        */
        for (c = 0; c < SPA_MAXBLOCKSIZE >> SPA_MINBLOCKSHIFT; c++) {
                size_t size = (c + 1) << SPA_MINBLOCKSHIFT;
-               size_t p2 = size;
-               size_t align = 0;
-               size_t data_cflags, cflags;
-
-               data_cflags = KMC_NODEBUG;
-               cflags = (zio_exclude_metadata || size > zio_buf_debug_limit) ?
-                   KMC_NODEBUG : 0;
+               size_t align, cflags, data_cflags;
+               char name[32];
 
+               /*
+                * Create cache for each half-power of 2 size, starting from
+                * SPA_MINBLOCKSIZE.  It should give us memory space efficiency
+                * of ~7/8, sufficient for transient allocations mostly using
+                * these caches.
+                */
+               size_t p2 = size;
                while (!ISP2(p2))
                        p2 &= p2 - 1;
+               if (!IS_P2ALIGNED(size, p2 / 2))
+                       continue;
 
 #ifndef _KERNEL
                /*
@@ -185,47 +184,37 @@ zio_init(void)
                 */
                if (arc_watch && !IS_P2ALIGNED(size, PAGESIZE))
                        continue;
-               /*
-                * Here's the problem - on 4K native devices in userland on
-                * Linux using O_DIRECT, buffers must be 4K aligned or I/O
-                * will fail with EINVAL, causing zdb (and others) to coredump.
-                * Since userland probably doesn't need optimized buffer caches,
-                * we just force 4K alignment on everything.
-                */
-               align = 8 * SPA_MINBLOCKSIZE;
-#else
-               if (size < PAGESIZE) {
-                       align = SPA_MINBLOCKSIZE;
-               } else if (IS_P2ALIGNED(size, p2 >> 2)) {
-                       align = PAGESIZE;
-               }
 #endif
 
-               if (align != 0) {
-                       char name[36];
-                       if (cflags == data_cflags) {
-                               /*
-                                * Resulting kmem caches would be identical.
-                                * Save memory by creating only one.
-                                */
-                               (void) snprintf(name, sizeof (name),
-                                   "zio_buf_comb_%lu", (ulong_t)size);
-                               zio_buf_cache[c] = kmem_cache_create(name,
-                                   size, align, NULL, NULL, NULL, NULL, NULL,
-                                   cflags);
-                               zio_data_buf_cache[c] = zio_buf_cache[c];
-                               continue;
-                       }
-                       (void) snprintf(name, sizeof (name), "zio_buf_%lu",
-                           (ulong_t)size);
-                       zio_buf_cache[c] = kmem_cache_create(name, size,
-                           align, NULL, NULL, NULL, NULL, NULL, cflags);
-
-                       (void) snprintf(name, sizeof (name), "zio_data_buf_%lu",
-                           (ulong_t)size);
-                       zio_data_buf_cache[c] = kmem_cache_create(name, size,
-                           align, NULL, NULL, NULL, NULL, NULL, data_cflags);
+               if (IS_P2ALIGNED(size, PAGESIZE))
+                       align = PAGESIZE;
+               else
+                       align = 1 << (highbit64(size ^ (size - 1)) - 1);
+
+               cflags = (zio_exclude_metadata || size > zio_buf_debug_limit) ?
+                   KMC_NODEBUG : 0;
+               data_cflags = KMC_NODEBUG;
+               if (cflags == data_cflags) {
+                       /*
+                        * Resulting kmem caches would be identical.
+                        * Save memory by creating only one.
+                        */
+                       (void) snprintf(name, sizeof (name),
+                           "zio_buf_comb_%lu", (ulong_t)size);
+                       zio_buf_cache[c] = kmem_cache_create(name, size, align,
+                           NULL, NULL, NULL, NULL, NULL, cflags);
+                       zio_data_buf_cache[c] = zio_buf_cache[c];
+                       continue;
                }
+               (void) snprintf(name, sizeof (name), "zio_buf_%lu",
+                   (ulong_t)size);
+               zio_buf_cache[c] = kmem_cache_create(name, size, align,
+                   NULL, NULL, NULL, NULL, NULL, cflags);
+
+               (void) snprintf(name, sizeof (name), "zio_data_buf_%lu",
+                   (ulong_t)size);
+               zio_data_buf_cache[c] = kmem_cache_create(name, size, align,
+                   NULL, NULL, NULL, NULL, NULL, data_cflags);
        }
 
        while (--c != 0) {
@@ -306,6 +295,53 @@ zio_fini(void)
  * ==========================================================================
  */
 
+#ifdef ZFS_DEBUG
+static const ulong_t zio_buf_canary = (ulong_t)0xdeadc0dedead210b;
+#endif
+
+/*
+ * Use empty space after the buffer to detect overflows.
+ *
+ * Since zio_init() creates kmem caches only for certain set of buffer sizes,
+ * allocations of different sizes may have some unused space after the data.
+ * Filling part of that space with a known pattern on allocation and checking
+ * it on free should allow us to detect some buffer overflows.
+ */
+static void
+zio_buf_put_canary(ulong_t *p, size_t size, kmem_cache_t **cache, size_t c)
+{
+#ifdef ZFS_DEBUG
+       size_t off = P2ROUNDUP(size, sizeof (ulong_t));
+       ulong_t *canary = p + off / sizeof (ulong_t);
+       size_t asize = (c + 1) << SPA_MINBLOCKSHIFT;
+       if (c + 1 < SPA_MAXBLOCKSIZE >> SPA_MINBLOCKSHIFT &&
+           cache[c] == cache[c + 1])
+               asize = (c + 2) << SPA_MINBLOCKSHIFT;
+       for (; off < asize; canary++, off += sizeof (ulong_t))
+               *canary = zio_buf_canary;
+#endif
+}
+
+static void
+zio_buf_check_canary(ulong_t *p, size_t size, kmem_cache_t **cache, size_t c)
+{
+#ifdef ZFS_DEBUG
+       size_t off = P2ROUNDUP(size, sizeof (ulong_t));
+       ulong_t *canary = p + off / sizeof (ulong_t);
+       size_t asize = (c + 1) << SPA_MINBLOCKSHIFT;
+       if (c + 1 < SPA_MAXBLOCKSIZE >> SPA_MINBLOCKSHIFT &&
+           cache[c] == cache[c + 1])
+               asize = (c + 2) << SPA_MINBLOCKSHIFT;
+       for (; off < asize; canary++, off += sizeof (ulong_t)) {
+               if (unlikely(*canary != zio_buf_canary)) {
+                       PANIC("ZIO buffer overflow %p (%zu) + %zu %#lx != %#lx",
+                           p, size, (canary - p) * sizeof (ulong_t),
+                           *canary, zio_buf_canary);
+               }
+       }
+#endif
+}
+
 /*
  * Use zio_buf_alloc to allocate ZFS metadata.  This data will appear in a
  * crashdump if the kernel panics, so use it judiciously.  Obviously, it's
@@ -322,7 +358,9 @@ zio_buf_alloc(size_t size)
        atomic_add_64(&zio_buf_cache_allocs[c], 1);
 #endif
 
-       return (kmem_cache_alloc(zio_buf_cache[c], KM_PUSHPAGE));
+       void *p = kmem_cache_alloc(zio_buf_cache[c], KM_PUSHPAGE);
+       zio_buf_put_canary(p, size, zio_buf_cache, c);
+       return (p);
 }
 
 /*
@@ -338,7 +376,9 @@ zio_data_buf_alloc(size_t size)
 
        VERIFY3U(c, <, SPA_MAXBLOCKSIZE >> SPA_MINBLOCKSHIFT);
 
-       return (kmem_cache_alloc(zio_data_buf_cache[c], KM_PUSHPAGE));
+       void *p = kmem_cache_alloc(zio_data_buf_cache[c], KM_PUSHPAGE);
+       zio_buf_put_canary(p, size, zio_data_buf_cache, c);
+       return (p);
 }
 
 void
@@ -351,6 +391,7 @@ zio_buf_free(void *buf, size_t size)
        atomic_add_64(&zio_buf_cache_frees[c], 1);
 #endif
 
+       zio_buf_check_canary(buf, size, zio_buf_cache, c);
        kmem_cache_free(zio_buf_cache[c], buf);
 }
 
@@ -361,6 +402,7 @@ zio_data_buf_free(void *buf, size_t size)
 
        VERIFY3U(c, <, SPA_MAXBLOCKSIZE >> SPA_MINBLOCKSHIFT);
 
+       zio_buf_check_canary(buf, size, zio_data_buf_cache, c);
        kmem_cache_free(zio_data_buf_cache[c], buf);
 }
 
@@ -571,7 +613,7 @@ error:
                zio->io_error = SET_ERROR(EIO);
                if ((zio->io_flags & ZIO_FLAG_SPECULATIVE) == 0) {
                        spa_log_error(spa, &zio->io_bookmark,
-                           &zio->io_bp->blk_birth);
+                           BP_GET_LOGICAL_BIRTH(zio->io_bp));
                        (void) zfs_ereport_post(FM_EREPORT_ZFS_AUTHENTICATION,
                            spa, NULL, &zio->io_bookmark, zio, 0);
                }
@@ -634,6 +676,11 @@ zio_add_child(zio_t *pio, zio_t *cio)
         */
        ASSERT3S(cio->io_child_type, <=, pio->io_child_type);
 
+       /* Parent should not have READY stage if child doesn't have it. */
+       IMPLY((cio->io_pipeline & ZIO_STAGE_READY) == 0 &&
+           (cio->io_child_type != ZIO_CHILD_VDEV),
+           (pio->io_pipeline & ZIO_STAGE_READY) == 0);
+
        zio_link_t *zl = kmem_cache_alloc(zio_link_cache, KM_SLEEP);
        zl->zl_parent = pio;
        zl->zl_child = cio;
@@ -665,6 +712,11 @@ zio_add_child_first(zio_t *pio, zio_t *cio)
         */
        ASSERT3S(cio->io_child_type, <=, pio->io_child_type);
 
+       /* Parent should not have READY stage if child doesn't have it. */
+       IMPLY((cio->io_pipeline & ZIO_STAGE_READY) == 0 &&
+           (cio->io_child_type != ZIO_CHILD_VDEV),
+           (pio->io_pipeline & ZIO_STAGE_READY) == 0);
+
        zio_link_t *zl = kmem_cache_alloc(zio_link_cache, KM_SLEEP);
        zl->zl_parent = pio;
        zl->zl_child = cio;
@@ -900,8 +952,10 @@ zio_create(zio_t *pio, spa_t *spa, uint64_t txg, const blkptr_t *bp,
        zio->io_orig_stage = zio->io_stage = stage;
        zio->io_orig_pipeline = zio->io_pipeline = pipeline;
        zio->io_pipeline_trace = ZIO_STAGE_OPEN;
+       zio->io_allocator = ZIO_ALLOCATOR_NONE;
 
-       zio->io_state[ZIO_WAIT_READY] = (stage >= ZIO_STAGE_READY);
+       zio->io_state[ZIO_WAIT_READY] = (stage >= ZIO_STAGE_READY) ||
+           (pipeline & ZIO_STAGE_READY) == 0;
        zio->io_state[ZIO_WAIT_DONE] = (stage >= ZIO_STAGE_DONE);
 
        if (zb != NULL)
@@ -932,6 +986,10 @@ zio_destroy(zio_t *zio)
        kmem_cache_free(zio_cache, zio);
 }
 
+/*
+ * ZIO intended to be between others.  Provides synchronization at READY
+ * and DONE pipeline stages and calls the respective callbacks.
+ */
 zio_t *
 zio_null(zio_t *pio, spa_t *spa, vdev_t *vd, zio_done_func_t *done,
     void *private, zio_flag_t flags)
@@ -945,10 +1003,22 @@ zio_null(zio_t *pio, spa_t *spa, vdev_t *vd, zio_done_func_t *done,
        return (zio);
 }
 
+/*
+ * ZIO intended to be a root of a tree.  Unlike null ZIO does not have a
+ * READY pipeline stage (is ready on creation), so it should not be used
+ * as child of any ZIO that may need waiting for grandchildren READY stage
+ * (any other ZIO type).
+ */
 zio_t *
 zio_root(spa_t *spa, zio_done_func_t *done, void *private, zio_flag_t flags)
 {
-       return (zio_null(NULL, spa, NULL, done, private, flags));
+       zio_t *zio;
+
+       zio = zio_create(NULL, spa, 0, NULL, NULL, 0, 0, done, private,
+           ZIO_TYPE_NULL, ZIO_PRIORITY_NOW, flags, NULL, 0, NULL,
+           ZIO_STAGE_OPEN, ZIO_ROOT_PIPELINE);
+
+       return (zio);
 }
 
 static int
@@ -982,8 +1052,8 @@ zfs_blkptr_verify_log(spa_t *spa, const blkptr_t *bp,
            (long long)bp->blk_prop,
            (long long)bp->blk_pad[0],
            (long long)bp->blk_pad[1],
-           (long long)bp->blk_phys_birth,
-           (long long)bp->blk_birth,
+           (long long)BP_GET_PHYSICAL_BIRTH(bp),
+           (long long)BP_GET_LOGICAL_BIRTH(bp),
            (long long)bp->blk_fill,
            (long long)bp->blk_cksum.zc_word[0],
            (long long)bp->blk_cksum.zc_word[1],
@@ -1086,10 +1156,11 @@ zfs_blkptr_verify(spa_t *spa, const blkptr_t *bp,
        /*
         * Pool-specific checks.
         *
-        * Note: it would be nice to verify that the blk_birth and
-        * BP_PHYSICAL_BIRTH() are not too large.  However, spa_freeze()
-        * allows the birth time of log blocks (and dmu_sync()-ed blocks
-        * that are in the log) to be arbitrarily large.
+        * Note: it would be nice to verify that the logical birth
+        * and physical birth are not too large.  However,
+        * spa_freeze() allows the birth time of log blocks (and
+        * dmu_sync()-ed blocks that are in the log) to be arbitrarily
+        * large.
         */
        for (int i = 0; i < BP_GET_NDVAS(bp); i++) {
                const dva_t *dva = &bp->blk_dva[i];
@@ -1176,7 +1247,7 @@ zio_read(zio_t *pio, spa_t *spa, const blkptr_t *bp,
 {
        zio_t *zio;
 
-       zio = zio_create(pio, spa, BP_PHYSICAL_BIRTH(bp), bp,
+       zio = zio_create(pio, spa, BP_GET_BIRTH(bp), bp,
            data, size, size, done, private,
            ZIO_TYPE_READ, priority, flags, NULL, 0, zb,
            ZIO_STAGE_OPEN, (flags & ZIO_FLAG_DDT_CHILD) ?
@@ -1365,7 +1436,7 @@ zio_claim(zio_t *pio, spa_t *spa, uint64_t txg, const blkptr_t *bp,
         * starts allocating blocks -- so that nothing is allocated twice.
         * If txg == 0 we just verify that the block is claimable.
         */
-       ASSERT3U(spa->spa_uberblock.ub_rootbp.blk_birth, <,
+       ASSERT3U(BP_GET_LOGICAL_BIRTH(&spa->spa_uberblock.ub_rootbp), <,
            spa_min_claim_txg(spa));
        ASSERT(txg == spa_min_claim_txg(spa) || txg == 0);
        ASSERT(!BP_GET_DEDUP(bp) || !spa_writeable(spa));       /* zdb(8) */
@@ -1382,23 +1453,10 @@ zio_t *
 zio_ioctl(zio_t *pio, spa_t *spa, vdev_t *vd, int cmd,
     zio_done_func_t *done, void *private, zio_flag_t flags)
 {
-       zio_t *zio;
-       int c;
-
-       if (vd->vdev_children == 0) {
-               zio = zio_create(pio, spa, 0, NULL, NULL, 0, 0, done, private,
-                   ZIO_TYPE_IOCTL, ZIO_PRIORITY_NOW, flags, vd, 0, NULL,
-                   ZIO_STAGE_OPEN, ZIO_IOCTL_PIPELINE);
-
-               zio->io_cmd = cmd;
-       } else {
-               zio = zio_null(pio, spa, NULL, NULL, NULL, flags);
-
-               for (c = 0; c < vd->vdev_children; c++)
-                       zio_nowait(zio_ioctl(zio, spa, vd->vdev_child[c], cmd,
-                           done, private, flags));
-       }
-
+       zio_t *zio = zio_create(pio, spa, 0, NULL, NULL, 0, 0, done, private,
+           ZIO_TYPE_IOCTL, ZIO_PRIORITY_NOW, flags, vd, 0, NULL,
+           ZIO_STAGE_OPEN, ZIO_IOCTL_PIPELINE);
+       zio->io_cmd = cmd;
        return (zio);
 }
 
@@ -1569,11 +1627,18 @@ zio_vdev_delegated_io(vdev_t *vd, uint64_t offset, abd_t *data, uint64_t size,
 }
 
 void
-zio_flush(zio_t *zio, vdev_t *vd)
+zio_flush(zio_t *pio, vdev_t *vd)
 {
-       zio_nowait(zio_ioctl(zio, zio->io_spa, vd, DKIOCFLUSHWRITECACHE,
-           NULL, NULL,
-           ZIO_FLAG_CANFAIL | ZIO_FLAG_DONT_PROPAGATE | ZIO_FLAG_DONT_RETRY));
+       if (vd->vdev_nowritecache)
+               return;
+       if (vd->vdev_children == 0) {
+               zio_nowait(zio_ioctl(pio, vd->vdev_spa, vd,
+                   DKIOCFLUSHWRITECACHE, NULL, NULL, ZIO_FLAG_CANFAIL |
+                   ZIO_FLAG_DONT_PROPAGATE | ZIO_FLAG_DONT_RETRY));
+       } else {
+               for (uint64_t c = 0; c < vd->vdev_children; c++)
+                       zio_flush(pio, vd->vdev_child[c]);
+       }
 }
 
 void
@@ -1667,7 +1732,7 @@ zio_write_bp_init(zio_t *zio)
                blkptr_t *bp = zio->io_bp;
                zio_prop_t *zp = &zio->io_prop;
 
-               ASSERT(bp->blk_birth != zio->io_txg);
+               ASSERT(BP_GET_LOGICAL_BIRTH(bp) != zio->io_txg);
 
                *bp = *zio->io_bp_override;
                zio->io_pipeline = ZIO_INTERLOCK_PIPELINE;
@@ -1755,7 +1820,7 @@ zio_write_compress(zio_t *zio)
        ASSERT(zio->io_child_type != ZIO_CHILD_DDT);
        ASSERT(zio->io_bp_override == NULL);
 
-       if (!BP_IS_HOLE(bp) && bp->blk_birth == zio->io_txg) {
+       if (!BP_IS_HOLE(bp) && BP_GET_LOGICAL_BIRTH(bp) == zio->io_txg) {
                /*
                 * We're rewriting an existing block, which means we're
                 * working on behalf of spa_sync().  For spa_sync() to
@@ -1802,7 +1867,7 @@ zio_write_compress(zio_t *zio)
                        BP_SET_TYPE(bp, zio->io_prop.zp_type);
                        BP_SET_LEVEL(bp, zio->io_prop.zp_level);
                        zio_buf_free(cbuf, lsize);
-                       bp->blk_birth = zio->io_txg;
+                       BP_SET_LOGICAL_BIRTH(bp, zio->io_txg);
                        zio->io_pipeline = ZIO_INTERLOCK_PIPELINE;
                        ASSERT(spa_feature_is_active(spa,
                            SPA_FEATURE_EMBEDDED_DATA));
@@ -1883,7 +1948,7 @@ zio_write_compress(zio_t *zio)
         * spa_sync() to allocate new blocks, but force rewrites after that.
         * There should only be a handful of blocks after pass 1 in any case.
         */
-       if (!BP_IS_HOLE(bp) && bp->blk_birth == zio->io_txg &&
+       if (!BP_IS_HOLE(bp) && BP_GET_LOGICAL_BIRTH(bp) == zio->io_txg &&
            BP_GET_PSIZE(bp) == psize &&
            pass >= zfs_sync_pass_rewrite) {
                VERIFY3U(psize, !=, 0);
@@ -1897,7 +1962,7 @@ zio_write_compress(zio_t *zio)
        }
 
        if (psize == 0) {
-               if (zio->io_bp_orig.blk_birth != 0 &&
+               if (BP_GET_LOGICAL_BIRTH(&zio->io_bp_orig) != 0 &&
                    spa_feature_is_active(spa, SPA_FEATURE_HOLE_BIRTH)) {
                        BP_SET_LSIZE(bp, lsize);
                        BP_SET_TYPE(bp, zp->zp_type);
@@ -1991,7 +2056,7 @@ zio_taskq_dispatch(zio_t *zio, zio_taskq_type_t q, boolean_t cutinline)
         */
        ASSERT(taskq_empty_ent(&zio->io_tqent));
        spa_taskq_dispatch_ent(spa, t, q, zio_execute, zio, flags,
-           &zio->io_tqent);
+           &zio->io_tqent, zio);
 }
 
 static boolean_t
@@ -2016,8 +2081,8 @@ zio_taskq_member(zio_t *zio, zio_taskq_type_t q)
 static zio_t *
 zio_issue_async(zio_t *zio)
 {
+       ASSERT((zio->io_type != ZIO_TYPE_WRITE) || ZIO_HAS_ALLOCATOR(zio));
        zio_taskq_dispatch(zio, ZIO_TASKQ_ISSUE, B_FALSE);
-
        return (NULL);
 }
 
@@ -2331,6 +2396,9 @@ zio_wait(zio_t *zio)
        ASSERT0(zio->io_queued_timestamp);
        zio->io_queued_timestamp = gethrtime();
 
+       if (zio->io_type == ZIO_TYPE_WRITE) {
+               spa_select_allocator(zio);
+       }
        __zio_execute(zio);
 
        mutex_enter(&zio->io_lock);
@@ -2383,6 +2451,9 @@ zio_nowait(zio_t *zio)
 
        ASSERT0(zio->io_queued_timestamp);
        zio->io_queued_timestamp = gethrtime();
+       if (zio->io_type == ZIO_TYPE_WRITE) {
+               spa_select_allocator(zio);
+       }
        __zio_execute(zio);
 }
 
@@ -2396,13 +2467,14 @@ static void
 zio_reexecute(void *arg)
 {
        zio_t *pio = arg;
-       zio_t *cio, *cio_next;
+       zio_t *cio, *cio_next, *gio;
 
        ASSERT(pio->io_child_type == ZIO_CHILD_LOGICAL);
        ASSERT(pio->io_orig_stage == ZIO_STAGE_OPEN);
        ASSERT(pio->io_gang_leader == NULL);
        ASSERT(pio->io_gang_tree == NULL);
 
+       mutex_enter(&pio->io_lock);
        pio->io_flags = pio->io_orig_flags;
        pio->io_stage = pio->io_orig_stage;
        pio->io_pipeline = pio->io_orig_pipeline;
@@ -2410,8 +2482,16 @@ zio_reexecute(void *arg)
        pio->io_flags |= ZIO_FLAG_REEXECUTED;
        pio->io_pipeline_trace = 0;
        pio->io_error = 0;
-       for (int w = 0; w < ZIO_WAIT_TYPES; w++)
-               pio->io_state[w] = 0;
+       pio->io_state[ZIO_WAIT_READY] = (pio->io_stage >= ZIO_STAGE_READY) ||
+           (pio->io_pipeline & ZIO_STAGE_READY) == 0;
+       pio->io_state[ZIO_WAIT_DONE] = (pio->io_stage >= ZIO_STAGE_DONE);
+       zio_link_t *zl = NULL;
+       while ((gio = zio_walk_parents(pio, &zl)) != NULL) {
+               for (int w = 0; w < ZIO_WAIT_TYPES; w++) {
+                       gio->io_children[pio->io_child_type][w] +=
+                           !pio->io_state[w];
+               }
+       }
        for (int c = 0; c < ZIO_CHILD_TYPES; c++)
                pio->io_child_error[c] = 0;
 
@@ -2425,12 +2505,9 @@ zio_reexecute(void *arg)
         * the remainder of pio's io_child_list, from 'cio_next' onward,
         * cannot be affected by any side effects of reexecuting 'cio'.
         */
-       zio_link_t *zl = NULL;
-       mutex_enter(&pio->io_lock);
+       zl = NULL;
        for (cio = zio_walk_children(pio, &zl); cio != NULL; cio = cio_next) {
                cio_next = zio_walk_children(pio, &zl);
-               for (int w = 0; w < ZIO_WAIT_TYPES; w++)
-                       pio->io_children[cio->io_child_type][w]++;
                mutex_exit(&pio->io_lock);
                zio_reexecute(cio);
                mutex_enter(&pio->io_lock);
@@ -2841,6 +2918,13 @@ zio_gang_issue(zio_t *zio)
        return (zio);
 }
 
+static void
+zio_gang_inherit_allocator(zio_t *pio, zio_t *cio)
+{
+       cio->io_allocator = pio->io_allocator;
+       cio->io_wr_iss_tq = pio->io_wr_iss_tq;
+}
+
 static void
 zio_write_gang_member_ready(zio_t *zio)
 {
@@ -2912,6 +2996,7 @@ zio_write_gang_block(zio_t *pio, metaslab_class_t *mc)
                gbh_copies = MIN(2, spa_max_replication(spa));
        }
 
+       ASSERT(ZIO_HAS_ALLOCATOR(pio));
        int flags = METASLAB_HINTBP_FAVOR | METASLAB_GANG_HEADER;
        if (pio->io_flags & ZIO_FLAG_IO_ALLOCATING) {
                ASSERT(pio->io_priority == ZIO_PRIORITY_ASYNC_WRITE);
@@ -2975,6 +3060,8 @@ zio_write_gang_block(zio_t *pio, metaslab_class_t *mc)
            zio_write_gang_done, NULL, pio->io_priority,
            ZIO_GANG_CHILD_FLAGS(pio), &pio->io_bookmark);
 
+       zio_gang_inherit_allocator(pio, zio);
+
        /*
         * Create and nowait the gang children.
         */
@@ -3005,6 +3092,8 @@ zio_write_gang_block(zio_t *pio, metaslab_class_t *mc)
                    zio_write_gang_done, &gn->gn_child[g], pio->io_priority,
                    ZIO_GANG_CHILD_FLAGS(pio), &pio->io_bookmark);
 
+               zio_gang_inherit_allocator(zio, cio);
+
                if (pio->io_flags & ZIO_FLAG_IO_ALLOCATING) {
                        ASSERT(pio->io_priority == ZIO_PRIORITY_ASYNC_WRITE);
                        ASSERT(has_data);
@@ -3451,7 +3540,7 @@ zio_ddt_write(zio_t *zio)
                else
                        ddt_phys_addref(ddp);
        } else if (zio->io_bp_override) {
-               ASSERT(bp->blk_birth == txg);
+               ASSERT(BP_GET_LOGICAL_BIRTH(bp) == txg);
                ASSERT(BP_EQUAL(bp, zio->io_bp_override));
                ddt_phys_fill(ddp, bp);
                ddt_phys_addref(ddp);
@@ -3517,6 +3606,7 @@ zio_io_to_allocate(spa_t *spa, int allocator)
                return (NULL);
 
        ASSERT(IO_IS_ALLOCATING(zio));
+       ASSERT(ZIO_HAS_ALLOCATOR(zio));
 
        /*
         * Try to place a reservation for this zio. If we're unable to
@@ -3553,21 +3643,12 @@ zio_dva_throttle(zio_t *zio)
        }
 
        ASSERT(zio->io_type == ZIO_TYPE_WRITE);
+       ASSERT(ZIO_HAS_ALLOCATOR(zio));
        ASSERT(zio->io_child_type > ZIO_CHILD_GANG);
        ASSERT3U(zio->io_queued_timestamp, >, 0);
        ASSERT(zio->io_stage == ZIO_STAGE_DVA_THROTTLE);
 
-       zbookmark_phys_t *bm = &zio->io_bookmark;
-       /*
-        * We want to try to use as many allocators as possible to help improve
-        * performance, but we also want logically adjacent IOs to be physically
-        * adjacent to improve sequential read performance. We chunk each object
-        * into 2^20 block regions, and then hash based on the objset, object,
-        * level, and region to accomplish both of these goals.
-        */
-       int allocator = (uint_t)cityhash4(bm->zb_objset, bm->zb_object,
-           bm->zb_level, bm->zb_blkid >> 20) % spa->spa_alloc_count;
-       zio->io_allocator = allocator;
+       int allocator = zio->io_allocator;
        zio->io_metaslab_class = mc;
        mutex_enter(&spa->spa_allocs[allocator].spaa_lock);
        avl_add(&spa->spa_allocs[allocator].spaa_tree, zio);
@@ -3641,6 +3722,7 @@ zio_dva_allocate(zio_t *zio)
         * sync write performance.  If a log allocation fails, we will fall
         * back to spa_sync() which is abysmal for performance.
         */
+       ASSERT(ZIO_HAS_ALLOCATOR(zio));
        error = metaslab_alloc(spa, mc, zio->io_size, bp,
            zio->io_prop.zp_copies, zio->io_txg, NULL, flags,
            &zio->io_alloc_list, zio, zio->io_allocator);
@@ -3729,11 +3811,13 @@ zio_dva_claim(zio_t *zio)
 static void
 zio_dva_unallocate(zio_t *zio, zio_gang_node_t *gn, blkptr_t *bp)
 {
-       ASSERT(bp->blk_birth == zio->io_txg || BP_IS_HOLE(bp));
+       ASSERT(BP_GET_LOGICAL_BIRTH(bp) == zio->io_txg || BP_IS_HOLE(bp));
        ASSERT(zio->io_bp_override == NULL);
 
-       if (!BP_IS_HOLE(bp))
-               metaslab_free(zio->io_spa, bp, bp->blk_birth, B_TRUE);
+       if (!BP_IS_HOLE(bp)) {
+               metaslab_free(zio->io_spa, bp, BP_GET_LOGICAL_BIRTH(bp),
+                   B_TRUE);
+       }
 
        if (gn != NULL) {
                for (int g = 0; g < SPA_GBH_NBLKPTRS; g++) {
@@ -4474,8 +4558,8 @@ zio_ready(zio_t *zio)
 
        if (zio->io_ready) {
                ASSERT(IO_IS_ALLOCATING(zio));
-               ASSERT(bp->blk_birth == zio->io_txg || BP_IS_HOLE(bp) ||
-                   (zio->io_flags & ZIO_FLAG_NOPWRITE));
+               ASSERT(BP_GET_LOGICAL_BIRTH(bp) == zio->io_txg ||
+                   BP_IS_HOLE(bp) || (zio->io_flags & ZIO_FLAG_NOPWRITE));
                ASSERT(zio->io_children[ZIO_CHILD_GANG][ZIO_WAIT_READY] == 0);
 
                zio->io_ready(zio);
@@ -4493,6 +4577,7 @@ zio_ready(zio_t *zio)
                        ASSERT(IO_IS_ALLOCATING(zio));
                        ASSERT(zio->io_priority == ZIO_PRIORITY_ASYNC_WRITE);
                        ASSERT(zio->io_metaslab_class != NULL);
+                       ASSERT(ZIO_HAS_ALLOCATOR(zio));
 
                        /*
                         * We were unable to allocate anything, unreserve and
@@ -4579,6 +4664,7 @@ zio_dva_throttle_done(zio_t *zio)
        }
 
        ASSERT(IO_IS_ALLOCATING(pio));
+       ASSERT(ZIO_HAS_ALLOCATOR(pio));
        ASSERT3P(zio, !=, zio->io_logical);
        ASSERT(zio->io_logical != NULL);
        ASSERT(!(zio->io_flags & ZIO_FLAG_IO_REPAIR));
@@ -4641,6 +4727,7 @@ zio_done(zio_t *zio)
                ASSERT(zio->io_type == ZIO_TYPE_WRITE);
                ASSERT(zio->io_priority == ZIO_PRIORITY_ASYNC_WRITE);
                ASSERT(zio->io_bp != NULL);
+               ASSERT(ZIO_HAS_ALLOCATOR(zio));
 
                metaslab_group_alloc_verify(zio->io_spa, zio->io_bp, zio,
                    zio->io_allocator);
@@ -4768,7 +4855,7 @@ zio_done(zio_t *zio)
                         * error and generate a logical data ereport.
                         */
                        spa_log_error(zio->io_spa, &zio->io_bookmark,
-                           &zio->io_bp->blk_birth);
+                           BP_GET_LOGICAL_BIRTH(zio->io_bp));
                        (void) zfs_ereport_post(FM_EREPORT_ZFS_DATA,
                            zio->io_spa, NULL, &zio->io_bookmark, zio, 0);
                }
@@ -4906,7 +4993,7 @@ zio_done(zio_t *zio)
                        ASSERT(taskq_empty_ent(&zio->io_tqent));
                        spa_taskq_dispatch_ent(zio->io_spa,
                            ZIO_TYPE_CLAIM, ZIO_TASKQ_ISSUE,
-                           zio_reexecute, zio, 0, &zio->io_tqent);
+                           zio_reexecute, zio, 0, &zio->io_tqent, NULL);
                }
                return (NULL);
        }