Add FASTWRITE algorithm for synchronous writes.
authorEtienne Dechamps <etienne.dechamps@ovh.net>
Wed, 27 Jun 2012 13:20:20 +0000 (15:20 +0200)
committerBrian Behlendorf <behlendorf1@llnl.gov>
Wed, 17 Oct 2012 15:56:41 +0000 (08:56 -0700)
Currently, ZIL blocks are spread over vdevs using hint block pointers
managed by the ZIL commit code and passed to metaslab_alloc(). Spreading
log blocks accross vdevs is important for performance: indeed, using
mutliple disks in parallel decreases the ZIL commit latency, which is
the main performance metric for synchronous writes. However, the current
implementation suffers from the following issues:

1) It would be best if the ZIL module was not aware of such low-level
details. They should be handled by the ZIO and metaslab modules;

2) Because the hint block pointer is managed per log, simultaneous
commits from multiple logs might use the same vdevs at the same time,
which is inefficient;

3) Because dmu_write() does not honor the block pointer hint, indirect
writes are not spread.

The naive solution of rotating the metaslab rotor each time a block is
allocated for the ZIL or dmu_sync() doesn't work in practice because the
first ZIL block to be written is actually allocated during the previous
commit. Consequently, when metaslab_alloc() decides the vdev for this
block, it will do so while a bunch of other allocations are happening at
the same time (from dmu_sync() and other ZILs). This means the vdev for
this block is chosen more or less at random. When the next commit
happens, there is a high chance (especially when the number of blocks
per commit is slightly less than the number of the disks) that one disk
will have to write two blocks (with a potential seek) while other disks
are sitting idle, which defeats spreading and increases the commit
latency.

This commit introduces a new concept in the metaslab allocator:
fastwrites. Basically, each top-level vdev maintains a counter
indicating the number of synchronous writes (from dmu_sync() and the
ZIL) which have been allocated but not yet completed. When the metaslab
is called with the FASTWRITE flag, it will choose the vdev with the
least amount of pending synchronous writes. If there are multiple vdevs
with the same value, the first matching vdev (starting from the rotor)
is used. Once metaslab_alloc() has decided which vdev the block is
allocated to, it updates the fastwrite counter for this vdev.

The rationale goes like this: when an allocation is done with
FASTWRITE, it "reserves" the vdev until the data is written. Until then,
all future allocations will naturally avoid this vdev, even after a full
rotation of the rotor. As a result, pending synchronous writes at a
given point in time will be nicely spread over all vdevs. This contrasts
with the previous algorithm, which is based on the implicit assumption
that blocks are written instantaneously after they're allocated.

metaslab_fastwrite_mark() and metaslab_fastwrite_unmark() are used to
manually increase or decrease fastwrite counters, respectively. They
should be used with caution, as there is no per-BP tracking of fastwrite
information, so leaks and "double-unmarks" are possible. There is,
however, an assert in the vdev teardown code which will fire if the
fastwrite counters are not zero when the pool is exported or the vdev
removed. Note that as stated above, marking is also done implictly by
metaslab_alloc().

ZIO also got a new FASTWRITE flag; when it is used, ZIO will pass it to
the metaslab when allocating (assuming ZIO does the allocation, which is
only true in the case of dmu_sync). This flag will also trigger an
unmark when zio_done() fires.

A side-effect of the new algorithm is that when a ZIL stops being used,
its last block can stay in the pending state (allocated but not yet
written) for a long time, polluting the fastwrite counters. To avoid
that, I've implemented a somewhat crude but working solution which
unmarks these pending blocks in zil_sync(), thus guaranteeing that
linguering fastwrites will get pruned at each sync event.

The best performance improvements are observed with pools using a large
number of top-level vdevs and heavy synchronous write workflows
(especially indirect writes and concurrent writes from multiple ZILs).
Real-life testing shows a 200% to 300% performance increase with
indirect writes and various commit sizes.

Signed-off-by: Brian Behlendorf <behlendorf1@llnl.gov>
Issue #1013

include/sys/metaslab.h
include/sys/metaslab_impl.h
include/sys/vdev_impl.h
include/sys/zil_impl.h
include/sys/zio.h
module/zfs/dmu.c
module/zfs/metaslab.c
module/zfs/vdev.c
module/zfs/zil.c
module/zfs/zio.c

index 2cf4d2b..9991242 100644 (file)
@@ -50,12 +50,15 @@ extern void metaslab_sync_reassess(metaslab_group_t *mg);
 #define        METASLAB_GANG_HEADER    0x2
 #define        METASLAB_GANG_CHILD     0x4
 #define        METASLAB_GANG_AVOID     0x8
+#define        METASLAB_FASTWRITE      0x10
 
 extern int metaslab_alloc(spa_t *spa, metaslab_class_t *mc, uint64_t psize,
     blkptr_t *bp, int ncopies, uint64_t txg, blkptr_t *hintbp, int flags);
 extern void metaslab_free(spa_t *spa, const blkptr_t *bp, uint64_t txg,
     boolean_t now);
 extern int metaslab_claim(spa_t *spa, const blkptr_t *bp, uint64_t txg);
+extern void metaslab_fastwrite_mark(spa_t *spa, const blkptr_t *bp);
+extern void metaslab_fastwrite_unmark(spa_t *spa, const blkptr_t *bp);
 
 extern metaslab_class_t *metaslab_class_create(spa_t *spa,
     space_map_ops_t *ops);
index 6c670a1..6583594 100644 (file)
@@ -46,6 +46,7 @@ struct metaslab_class {
        uint64_t                mc_deferred;    /* total deferred frees */
        uint64_t                mc_space;       /* total space (alloc + free) */
        uint64_t                mc_dspace;      /* total deflated space */
+       kmutex_t                mc_fastwrite_lock;
 };
 
 struct metaslab_group {
index 5bd432b..0b532dc 100644 (file)
@@ -156,6 +156,7 @@ struct vdev {
        uint64_t        vdev_ms_count;  /* number of metaslabs          */
        metaslab_group_t *vdev_mg;      /* metaslab group               */
        metaslab_t      **vdev_ms;      /* metaslab array               */
+       uint64_t        vdev_pending_fastwrite; /* allocated fastwrites */
        txg_list_t      vdev_ms_list;   /* per-txg dirty metaslab lists */
        txg_list_t      vdev_dtl_list;  /* per-txg dirty DTL lists      */
        txg_node_t      vdev_txg_node;  /* per-txg dirty vdev linkage   */
index 1d4c0cc..6c37c1a 100644 (file)
@@ -40,6 +40,7 @@ extern "C" {
 typedef struct lwb {
        zilog_t         *lwb_zilog;     /* back pointer to log struct */
        blkptr_t        lwb_blk;        /* on disk address of this log blk */
+       boolean_t       lwb_fastwrite;  /* is blk marked for fastwrite? */
        int             lwb_nused;      /* # used bytes in buffer */
        int             lwb_sz;         /* size of block and buffer */
        char            *lwb_buf;       /* log write buffer */
index 4f20cab..289238c 100644 (file)
@@ -193,7 +193,8 @@ enum zio_flag {
        ZIO_FLAG_RAW            = 1 << 21,
        ZIO_FLAG_GANG_CHILD     = 1 << 22,
        ZIO_FLAG_DDT_CHILD      = 1 << 23,
-       ZIO_FLAG_GODFATHER      = 1 << 24
+       ZIO_FLAG_GODFATHER      = 1 << 24,
+       ZIO_FLAG_FASTWRITE      = 1 << 25
 };
 
 #define        ZIO_FLAG_MUSTSUCCEED            0
@@ -475,7 +476,7 @@ extern zio_t *zio_free_sync(zio_t *pio, spa_t *spa, uint64_t txg,
     const blkptr_t *bp, enum zio_flag flags);
 
 extern int zio_alloc_zil(spa_t *spa, uint64_t txg, blkptr_t *new_bp,
-    blkptr_t *old_bp, uint64_t size, boolean_t use_slog);
+    uint64_t size, boolean_t use_slog);
 extern void zio_free_zil(spa_t *spa, uint64_t txg, blkptr_t *bp);
 extern void zio_flush(zio_t *zio, vdev_t *vd);
 extern void zio_shrink(zio_t *zio, uint64_t size);
index 1d4d125..e2abf8c 100644 (file)
@@ -1440,7 +1440,7 @@ dmu_sync_late_arrival(zio_t *pio, objset_t *os, dmu_sync_cb_t *done, zgd_t *zgd,
        zio_nowait(zio_write(pio, os->os_spa, dmu_tx_get_txg(tx), zgd->zgd_bp,
            zgd->zgd_db->db_data, zgd->zgd_db->db_size, zp,
            dmu_sync_late_arrival_ready, dmu_sync_late_arrival_done, dsa,
-           ZIO_PRIORITY_SYNC_WRITE, ZIO_FLAG_CANFAIL, zb));
+           ZIO_PRIORITY_SYNC_WRITE, ZIO_FLAG_CANFAIL | ZIO_FLAG_FASTWRITE, zb));
 
        return (0);
 }
@@ -1564,7 +1564,7 @@ dmu_sync(zio_t *pio, uint64_t txg, dmu_sync_cb_t *done, zgd_t *zgd)
        zio_nowait(arc_write(pio, os->os_spa, txg,
            bp, dr->dt.dl.dr_data, DBUF_IS_L2CACHEABLE(db), &zp,
            dmu_sync_ready, dmu_sync_done, dsa,
-           ZIO_PRIORITY_SYNC_WRITE, ZIO_FLAG_CANFAIL, &zb));
+           ZIO_PRIORITY_SYNC_WRITE, ZIO_FLAG_CANFAIL | ZIO_FLAG_FASTWRITE, &zb));
 
        return (0);
 }
index d06012f..d199921 100644 (file)
@@ -107,6 +107,7 @@ metaslab_class_create(spa_t *spa, space_map_ops_t *ops)
        mc->mc_spa = spa;
        mc->mc_rotor = NULL;
        mc->mc_ops = ops;
+       mutex_init(&mc->mc_fastwrite_lock, NULL, MUTEX_DEFAULT, NULL);
 
        return (mc);
 }
@@ -120,6 +121,7 @@ metaslab_class_destroy(metaslab_class_t *mc)
        ASSERT(mc->mc_space == 0);
        ASSERT(mc->mc_dspace == 0);
 
+       mutex_destroy(&mc->mc_fastwrite_lock);
        kmem_free(mc, sizeof (metaslab_class_t));
 }
 
@@ -1307,7 +1309,7 @@ static int
 metaslab_alloc_dva(spa_t *spa, metaslab_class_t *mc, uint64_t psize,
     dva_t *dva, int d, dva_t *hintdva, uint64_t txg, int flags)
 {
-       metaslab_group_t *mg, *rotor;
+       metaslab_group_t *mg, *fast_mg, *rotor;
        vdev_t *vd;
        int dshift = 3;
        int all_zero;
@@ -1325,6 +1327,9 @@ metaslab_alloc_dva(spa_t *spa, metaslab_class_t *mc, uint64_t psize,
        if (psize >= metaslab_gang_bang && (ddi_get_lbolt() & 3) == 0)
                return (ENOSPC);
 
+       if (flags & METASLAB_FASTWRITE)
+               mutex_enter(&mc->mc_fastwrite_lock);
+
        /*
         * Start at the rotor and loop through all mgs until we find something.
         * Note that there's no locking on mc_rotor or mc_aliquot because
@@ -1367,6 +1372,15 @@ metaslab_alloc_dva(spa_t *spa, metaslab_class_t *mc, uint64_t psize,
        } else if (d != 0) {
                vd = vdev_lookup_top(spa, DVA_GET_VDEV(&dva[d - 1]));
                mg = vd->vdev_mg->mg_next;
+       } else if (flags & METASLAB_FASTWRITE) {
+               mg = fast_mg = mc->mc_rotor;
+
+               do {
+                       if (fast_mg->mg_vd->vdev_pending_fastwrite <
+                           mg->mg_vd->vdev_pending_fastwrite)
+                               mg = fast_mg;
+               } while ((fast_mg = fast_mg->mg_next) != mc->mc_rotor);
+
        } else {
                mg = mc->mc_rotor;
        }
@@ -1453,7 +1467,8 @@ top:
                                    (int64_t)mg->mg_aliquot) / 100;
                        }
 
-                       if (atomic_add_64_nv(&mc->mc_aliquot, asize) >=
+                       if ((flags & METASLAB_FASTWRITE) ||
+                           atomic_add_64_nv(&mc->mc_aliquot, asize) >=
                            mg->mg_aliquot + mg->mg_bias) {
                                mc->mc_rotor = mg->mg_next;
                                mc->mc_aliquot = 0;
@@ -1464,6 +1479,12 @@ top:
                        DVA_SET_GANG(&dva[d], !!(flags & METASLAB_GANG_HEADER));
                        DVA_SET_ASIZE(&dva[d], asize);
 
+                       if (flags & METASLAB_FASTWRITE) {
+                               atomic_add_64(&vd->vdev_pending_fastwrite,
+                                   psize);
+                               mutex_exit(&mc->mc_fastwrite_lock);
+                       }
+
                        return (0);
                }
 next:
@@ -1485,6 +1506,8 @@ next:
 
        bzero(&dva[d], sizeof (dva_t));
 
+       if (flags & METASLAB_FASTWRITE)
+               mutex_exit(&mc->mc_fastwrite_lock);
        return (ENOSPC);
 }
 
@@ -1678,3 +1701,48 @@ metaslab_claim(spa_t *spa, const blkptr_t *bp, uint64_t txg)
 
        return (error);
 }
+
+void metaslab_fastwrite_mark(spa_t *spa, const blkptr_t *bp)
+{
+       const dva_t *dva = bp->blk_dva;
+       int ndvas = BP_GET_NDVAS(bp);
+       uint64_t psize = BP_GET_PSIZE(bp);
+       int d;
+       vdev_t *vd;
+
+       ASSERT(!BP_IS_HOLE(bp));
+       ASSERT(psize > 0);
+
+       spa_config_enter(spa, SCL_VDEV, FTAG, RW_READER);
+
+       for (d = 0; d < ndvas; d++) {
+               if ((vd = vdev_lookup_top(spa, DVA_GET_VDEV(&dva[d]))) == NULL)
+                       continue;
+               atomic_add_64(&vd->vdev_pending_fastwrite, psize);
+       }
+
+       spa_config_exit(spa, SCL_VDEV, FTAG);
+}
+
+void metaslab_fastwrite_unmark(spa_t *spa, const blkptr_t *bp)
+{
+       const dva_t *dva = bp->blk_dva;
+       int ndvas = BP_GET_NDVAS(bp);
+       uint64_t psize = BP_GET_PSIZE(bp);
+       int d;
+       vdev_t *vd;
+
+       ASSERT(!BP_IS_HOLE(bp));
+       ASSERT(psize > 0);
+
+       spa_config_enter(spa, SCL_VDEV, FTAG, RW_READER);
+
+       for (d = 0; d < ndvas; d++) {
+               if ((vd = vdev_lookup_top(spa, DVA_GET_VDEV(&dva[d]))) == NULL)
+                       continue;
+               ASSERT3U(vd->vdev_pending_fastwrite, >=, psize);
+               atomic_sub_64(&vd->vdev_pending_fastwrite, psize);
+       }
+
+       spa_config_exit(spa, SCL_VDEV, FTAG);
+}
index 205a1d1..7d6d527 100644 (file)
@@ -904,6 +904,8 @@ vdev_metaslab_fini(vdev_t *vd)
                kmem_free(vd->vdev_ms, count * sizeof (metaslab_t *));
                vd->vdev_ms = NULL;
        }
+
+       ASSERT3U(vd->vdev_pending_fastwrite, ==, 0);
 }
 
 typedef struct vdev_probe_stats {
index e76e5ec..6492dbc 100644 (file)
@@ -38,6 +38,7 @@
 #include <sys/vdev_impl.h>
 #include <sys/dmu_tx.h>
 #include <sys/dsl_pool.h>
+#include <sys/metaslab.h>
 
 /*
  * The zfs intent log (ZIL) saves transaction records of system calls
@@ -451,13 +452,14 @@ zil_free_log_record(zilog_t *zilog, lr_t *lrc, void *tx, uint64_t claim_txg)
 }
 
 static lwb_t *
-zil_alloc_lwb(zilog_t *zilog, blkptr_t *bp, uint64_t txg)
+zil_alloc_lwb(zilog_t *zilog, blkptr_t *bp, uint64_t txg, boolean_t fastwrite)
 {
        lwb_t *lwb;
 
        lwb = kmem_cache_alloc(zil_lwb_cache, KM_PUSHPAGE);
        lwb->lwb_zilog = zilog;
        lwb->lwb_blk = *bp;
+       lwb->lwb_fastwrite = fastwrite;
        lwb->lwb_buf = zio_buf_alloc(BP_GET_LSIZE(bp));
        lwb->lwb_max_txg = txg;
        lwb->lwb_zio = NULL;
@@ -489,6 +491,7 @@ zil_create(zilog_t *zilog)
        dmu_tx_t *tx = NULL;
        blkptr_t blk;
        int error = 0;
+       boolean_t fastwrite = FALSE;
 
        /*
         * Wait for any previous destroy to complete.
@@ -516,8 +519,9 @@ zil_create(zilog_t *zilog)
                        BP_ZERO(&blk);
                }
 
-               error = zio_alloc_zil(zilog->zl_spa, txg, &blk, NULL,
+               error = zio_alloc_zil(zilog->zl_spa, txg, &blk,
                    ZIL_MIN_BLKSZ, zilog->zl_logbias == ZFS_LOGBIAS_LATENCY);
+               fastwrite = TRUE;
 
                if (error == 0)
                        zil_init_log_chain(zilog, &blk);
@@ -527,7 +531,7 @@ zil_create(zilog_t *zilog)
         * Allocate a log write buffer (lwb) for the first log block.
         */
        if (error == 0)
-               lwb = zil_alloc_lwb(zilog, &blk, txg);
+               lwb = zil_alloc_lwb(zilog, &blk, txg, fastwrite);
 
        /*
         * If we just allocated the first log block, commit our transaction
@@ -586,6 +590,10 @@ zil_destroy(zilog_t *zilog, boolean_t keep_first)
                ASSERT(zh->zh_claim_txg == 0);
                VERIFY(!keep_first);
                while ((lwb = list_head(&zilog->zl_lwb_list)) != NULL) {
+                       ASSERT(lwb->lwb_zio == NULL);
+                       if (lwb->lwb_fastwrite)
+                               metaslab_fastwrite_unmark(zilog->zl_spa,
+                                   &lwb->lwb_blk);
                        list_remove(&zilog->zl_lwb_list, lwb);
                        if (lwb->lwb_buf != NULL)
                                zio_buf_free(lwb->lwb_buf, lwb->lwb_sz);
@@ -826,6 +834,8 @@ zil_lwb_write_done(zio_t *zio)
         */
        zio_buf_free(lwb->lwb_buf, lwb->lwb_sz);
        mutex_enter(&zilog->zl_lock);
+       lwb->lwb_zio = NULL;
+       lwb->lwb_fastwrite = FALSE;
        lwb->lwb_buf = NULL;
        lwb->lwb_tx = NULL;
        mutex_exit(&zilog->zl_lock);
@@ -854,12 +864,21 @@ zil_lwb_write_init(zilog_t *zilog, lwb_t *lwb)
                zilog->zl_root_zio = zio_root(zilog->zl_spa, NULL, NULL,
                    ZIO_FLAG_CANFAIL);
        }
+
+       /* Lock so zil_sync() doesn't fastwrite_unmark after zio is created */
+       mutex_enter(&zilog->zl_lock);
        if (lwb->lwb_zio == NULL) {
+               if (!lwb->lwb_fastwrite) {
+                       metaslab_fastwrite_mark(zilog->zl_spa, &lwb->lwb_blk);
+                       lwb->lwb_fastwrite = 1;
+               }
                lwb->lwb_zio = zio_rewrite(zilog->zl_root_zio, zilog->zl_spa,
                    0, &lwb->lwb_blk, lwb->lwb_buf, BP_GET_LSIZE(&lwb->lwb_blk),
                    zil_lwb_write_done, lwb, ZIO_PRIORITY_LOG_WRITE,
-                   ZIO_FLAG_CANFAIL | ZIO_FLAG_DONT_PROPAGATE, &zb);
+                   ZIO_FLAG_CANFAIL | ZIO_FLAG_DONT_PROPAGATE |
+                   ZIO_FLAG_FASTWRITE, &zb);
        }
+       mutex_exit(&zilog->zl_lock);
 }
 
 /*
@@ -956,10 +975,8 @@ zil_lwb_write_start(zilog_t *zilog, lwb_t *lwb)
        zilog->zl_prev_rotor = (zilog->zl_prev_rotor + 1) & (ZIL_PREV_BLKS - 1);
 
        BP_ZERO(bp);
-       /* pass the old blkptr in order to spread log blocks across devs */
        use_slog = USE_SLOG(zilog);
-       error = zio_alloc_zil(spa, txg, bp, &lwb->lwb_blk, zil_blksz,
-           use_slog);
+       error = zio_alloc_zil(spa, txg, bp, zil_blksz, USE_SLOG(zilog));
        if (use_slog)
        {
                ZIL_STAT_BUMP(zil_itx_metaslab_slog_count);
@@ -978,7 +995,7 @@ zil_lwb_write_start(zilog_t *zilog, lwb_t *lwb)
                /*
                 * Allocate a new log write buffer (lwb).
                 */
-               nlwb = zil_alloc_lwb(zilog, bp, txg);
+               nlwb = zil_alloc_lwb(zilog, bp, txg, TRUE);
 
                /* Record the block for later vdev flushing */
                zil_add_block(zilog, &lwb->lwb_blk);
@@ -1625,6 +1642,9 @@ zil_sync(zilog_t *zilog, dmu_tx_t *tx)
                zh->zh_log = lwb->lwb_blk;
                if (lwb->lwb_buf != NULL || lwb->lwb_max_txg > txg)
                        break;
+
+               ASSERT(lwb->lwb_zio == NULL);
+
                list_remove(&zilog->zl_lwb_list, lwb);
                zio_free_zil(spa, txg, &lwb->lwb_blk);
                kmem_cache_free(zil_lwb_cache, lwb);
@@ -1638,6 +1658,19 @@ zil_sync(zilog_t *zilog, dmu_tx_t *tx)
                if (list_head(&zilog->zl_lwb_list) == NULL)
                        BP_ZERO(&zh->zh_log);
        }
+
+       /*
+        * Remove fastwrite on any blocks that have been pre-allocated for
+        * the next commit. This prevents fastwrite counter pollution by
+        * unused, long-lived LWBs.
+        */
+       for (; lwb != NULL; lwb = list_next(&zilog->zl_lwb_list, lwb)) {
+               if (lwb->lwb_fastwrite && !lwb->lwb_zio) {
+                       metaslab_fastwrite_unmark(zilog->zl_spa, &lwb->lwb_blk);
+                       lwb->lwb_fastwrite = 0;
+               }
+       }
+
        mutex_exit(&zilog->zl_lock);
 }
 
@@ -1817,6 +1850,9 @@ zil_close(zilog_t *zilog)
        lwb = list_head(&zilog->zl_lwb_list);
        if (lwb != NULL) {
                ASSERT(lwb == list_tail(&zilog->zl_lwb_list));
+               ASSERT(lwb->lwb_zio == NULL);
+               if (lwb->lwb_fastwrite)
+                       metaslab_fastwrite_unmark(zilog->zl_spa, &lwb->lwb_blk);
                list_remove(&zilog->zl_lwb_list, lwb);
                zio_buf_free(lwb->lwb_buf, lwb->lwb_sz);
                kmem_cache_free(zil_lwb_cache, lwb);
index ace72a0..ce76e01 100644 (file)
@@ -1861,6 +1861,11 @@ zio_write_gang_block(zio_t *pio)
         */
        pio->io_pipeline = ZIO_INTERLOCK_PIPELINE;
 
+       /*
+        * We didn't allocate this bp, so make sure it doesn't get unmarked.
+        */
+       pio->io_flags &= ~ZIO_FLAG_FASTWRITE;
+
        zio_nowait(zio);
 
        return (ZIO_PIPELINE_CONTINUE);
@@ -2270,6 +2275,7 @@ zio_dva_allocate(zio_t *zio)
        flags |= (zio->io_flags & ZIO_FLAG_NODATA) ? METASLAB_GANG_AVOID : 0;
        flags |= (zio->io_flags & ZIO_FLAG_GANG_CHILD) ?
            METASLAB_GANG_CHILD : 0;
+       flags |= (zio->io_flags & ZIO_FLAG_FASTWRITE) ? METASLAB_FASTWRITE : 0;
        error = metaslab_alloc(spa, mc, zio->io_size, bp,
            zio->io_prop.zp_copies, zio->io_txg, NULL, flags);
 
@@ -2333,8 +2339,8 @@ zio_dva_unallocate(zio_t *zio, zio_gang_node_t *gn, blkptr_t *bp)
  * Try to allocate an intent log block.  Return 0 on success, errno on failure.
  */
 int
-zio_alloc_zil(spa_t *spa, uint64_t txg, blkptr_t *new_bp, blkptr_t *old_bp,
-    uint64_t size, boolean_t use_slog)
+zio_alloc_zil(spa_t *spa, uint64_t txg, blkptr_t *new_bp, uint64_t size,
+    boolean_t use_slog)
 {
        int error = 1;
 
@@ -2347,14 +2353,14 @@ zio_alloc_zil(spa_t *spa, uint64_t txg, blkptr_t *new_bp, blkptr_t *old_bp,
         */
        if (use_slog) {
                error = metaslab_alloc(spa, spa_log_class(spa), size,
-                   new_bp, 1, txg, old_bp,
-                   METASLAB_HINTBP_AVOID | METASLAB_GANG_AVOID);
+                   new_bp, 1, txg, NULL,
+                   METASLAB_FASTWRITE | METASLAB_GANG_AVOID);
        }
 
        if (error) {
                error = metaslab_alloc(spa, spa_normal_class(spa), size,
-                   new_bp, 1, txg, old_bp,
-                   METASLAB_HINTBP_AVOID | METASLAB_GANG_AVOID);
+                   new_bp, 1, txg, NULL,
+                   METASLAB_FASTWRITE | METASLAB_GANG_AVOID);
        }
 
        if (error == 0) {
@@ -3066,6 +3072,11 @@ zio_done(zio_t *zio)
                zfs_ereport_free_checksum(zcr);
        }
 
+       if (zio->io_flags & ZIO_FLAG_FASTWRITE && zio->io_bp &&
+           !BP_IS_HOLE(zio->io_bp)) {
+               metaslab_fastwrite_unmark(zio->io_spa, zio->io_bp);
+       }
+
        /*
         * It is the responsibility of the done callback to ensure that this
         * particular zio is no longer discoverable for adoption, and as