Skip site navigation (1)Skip section navigation (2)
Date:      Tue, 26 Sep 2017 11:04:08 +0000 (UTC)
From:      Andriy Gapon <avg@FreeBSD.org>
To:        src-committers@freebsd.org, svn-src-all@freebsd.org, svn-src-head@freebsd.org
Subject:   svn commit: r324011 - in head: cddl/contrib/opensolaris/cmd/ztest sys/cddl/contrib/opensolaris/uts/common/fs/zfs sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys sys/cddl/contrib/opensolaris/uts/...
Message-ID:  <201709261104.v8QB480K002320@repo.freebsd.org>

next in thread | raw e-mail | index | archive | help
Author: avg
Date: Tue Sep 26 11:04:08 2017
New Revision: 324011
URL: https://svnweb.freebsd.org/changeset/base/324011

Log:
  MFV r323535: 8585 improve batching done in zil_commit()
  
  FreeBSD notes:
  - this MFV reverts FreeBSD commit r314549 to make the merge easier
  - at present our emulation of cv_timedwait_hires is rather poor,
    so I elected to use cv_timedwait_sbt directly
  Please see the differential revision for details.
  Unfortunately, I did not get any positive reviews, so there could be
  bugs in the FreeBSD-specific piece of the merge.
  Hence, the long MFC timeout.
  
  illumos/illumos-gate@1271e4b10dfaaed576c08a812f466f6e81370e5e
  https://github.com/illumos/illumos-gate/commit/1271e4b10dfaaed576c08a812f466f6e81370e5e
  
  https://www.illumos.org/issues/8585
    The current implementation of zil_commit() can introduce significant
    latency, beyond what is inherent due to the latency of the underlying
    storage. The additional latency comes from two main problems:
    1. When there's outstanding ZIL blocks being written (i.e. there's
        already a "writer thread" in progress), then any new calls to
        zil_commit() will block waiting for the currently oustanding ZIL
        blocks to complete. The blocks written for each "writer thread" is
        coined a "batch", and there can only ever be a single "batch" being
        written at a time. When a batch is being written, any new ZIL
        transactions will have to wait for the next batch to be written,
        which won't occur until the current batch finishes.
    As a result, the underlying storage may not be used as efficiently
        as possible. While "new" threads enter zil_commit() and are blocked
        waiting for the next batch, it's possible that the underlying
        storage isn't fully utilized by the current batch of ZIL blocks. In
        that case, it'd be better to allow these new threads to generate
        (and issue) a new ZIL block, such that it could be serviced by the
        underlying storage concurrently with the other ZIL blocks that are
        being serviced.
    2. Any call to zil_commit() must wait for all ZIL blocks in its "batch"
        to complete, prior to zil_commit() returning. The size of any given
        batch is proportional to the number of ZIL transaction in the queue
        at the time that the batch starts processing the queue; which
        doesn't occur until the previous batch completes. Thus, if there's a
        lot of transactions in the queue, the batch could be composed of
        many ZIL blocks, and each call to zil_commit() will have to wait for
        all of these writes to complete (even if the thread calling
        zil_commit() only cared about one of the transactions in the batch).
  
  Reviewed by: Brad Lewis <brad.lewis@delphix.com>
  Reviewed by: Matt Ahrens <mahrens@delphix.com>
  Reviewed by: George Wilson <george.wilson@delphix.com>
  Approved by: Dan McDonald <danmcd@joyent.com>
  Author: Prakash Surya <prakash.surya@delphix.com>
  
  MFC after:	1 month
  Differential Revision:	https://reviews.freebsd.org/D12355

Modified:
  head/cddl/contrib/opensolaris/cmd/ztest/ztest.c
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/dmu.c
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/dmu.h
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zil.h
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zil_impl.h
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zio.h
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/txg.c
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zfs_vnops.c
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zil.c
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zio.c
  head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zvol.c
  head/sys/cddl/contrib/opensolaris/uts/common/sys/debug.h
Directory Properties:
  head/cddl/contrib/opensolaris/   (props changed)
  head/sys/cddl/contrib/opensolaris/   (props changed)

Modified: head/cddl/contrib/opensolaris/cmd/ztest/ztest.c
==============================================================================
--- head/cddl/contrib/opensolaris/cmd/ztest/ztest.c	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/cddl/contrib/opensolaris/cmd/ztest/ztest.c	Tue Sep 26 11:04:08 2017	(r324011)
@@ -1825,13 +1825,14 @@ ztest_get_done(zgd_t *zgd, int error)
 	ztest_object_unlock(zd, object);
 
 	if (error == 0 && zgd->zgd_bp)
-		zil_add_block(zgd->zgd_zilog, zgd->zgd_bp);
+		zil_lwb_add_block(zgd->zgd_lwb, zgd->zgd_bp);
 
 	umem_free(zgd, sizeof (*zgd));
 }
 
 static int
-ztest_get_data(void *arg, lr_write_t *lr, char *buf, zio_t *zio)
+ztest_get_data(void *arg, lr_write_t *lr, char *buf, struct lwb *lwb,
+    zio_t *zio)
 {
 	ztest_ds_t *zd = arg;
 	objset_t *os = zd->zd_os;
@@ -1845,6 +1846,10 @@ ztest_get_data(void *arg, lr_write_t *lr, char *buf, z
 	zgd_t *zgd;
 	int error;
 
+	ASSERT3P(lwb, !=, NULL);
+	ASSERT3P(zio, !=, NULL);
+	ASSERT3U(size, !=, 0);
+
 	ztest_object_lock(zd, object, RL_READER);
 	error = dmu_bonus_hold(os, object, FTAG, &db);
 	if (error) {
@@ -1865,7 +1870,7 @@ ztest_get_data(void *arg, lr_write_t *lr, char *buf, z
 	db = NULL;
 
 	zgd = umem_zalloc(sizeof (*zgd), UMEM_NOFAIL);
-	zgd->zgd_zilog = zd->zd_zilog;
+	zgd->zgd_lwb = lwb;
 	zgd->zgd_private = zd;
 
 	if (buf != NULL) {	/* immediate write */

Modified: head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/dmu.c
==============================================================================
--- head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/dmu.c	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/dmu.c	Tue Sep 26 11:04:08 2017	(r324011)
@@ -1728,6 +1728,13 @@ dmu_sync_late_arrival(zio_t *pio, objset_t *os, dmu_sy
 		return (SET_ERROR(EIO));
 	}
 
+	/*
+	 * In order to prevent the zgd's lwb from being free'd prior to
+	 * dmu_sync_late_arrival_done() being called, we have to ensure
+	 * the lwb's "max txg" takes this tx's txg into account.
+	 */
+	zil_lwb_add_txg(zgd->zgd_lwb, dmu_tx_get_txg(tx));
+
 	dsa = kmem_alloc(sizeof (dmu_sync_arg_t), KM_SLEEP);
 	dsa->dsa_dr = NULL;
 	dsa->dsa_done = done;

Modified: head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/dmu.h
==============================================================================
--- head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/dmu.h	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/dmu.h	Tue Sep 26 11:04:08 2017	(r324011)
@@ -920,7 +920,7 @@ uint64_t dmu_tx_get_txg(dmu_tx_t *tx);
  * {zfs,zvol,ztest}_get_done() args
  */
 typedef struct zgd {
-	struct zilog	*zgd_zilog;
+	struct lwb	*zgd_lwb;
 	struct blkptr	*zgd_bp;
 	dmu_buf_t	*zgd_db;
 	struct rl	*zgd_rl;

Modified: head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zil.h
==============================================================================
--- head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zil.h	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zil.h	Tue Sep 26 11:04:08 2017	(r324011)
@@ -40,6 +40,7 @@ extern "C" {
 
 struct dsl_pool;
 struct dsl_dataset;
+struct lwb;
 
 /*
  * Intent log format:
@@ -140,6 +141,7 @@ typedef enum zil_create {
 /*
  * Intent log transaction types and record structures
  */
+#define	TX_COMMIT		0	/* Commit marker (no on-disk state) */
 #define	TX_CREATE		1	/* Create file */
 #define	TX_MKDIR		2	/* Make directory */
 #define	TX_MKXATTR		3	/* Make XATTR directory */
@@ -388,7 +390,8 @@ typedef int zil_parse_blk_func_t(zilog_t *zilog, blkpt
 typedef int zil_parse_lr_func_t(zilog_t *zilog, lr_t *lr, void *arg,
     uint64_t txg);
 typedef int zil_replay_func_t();
-typedef int zil_get_data_t(void *arg, lr_write_t *lr, char *dbuf, zio_t *zio);
+typedef int zil_get_data_t(void *arg, lr_write_t *lr, char *dbuf,
+    struct lwb *lwb, zio_t *zio);
 
 extern int zil_parse(zilog_t *zilog, zil_parse_blk_func_t *parse_blk_func,
     zil_parse_lr_func_t *parse_lr_func, void *arg, uint64_t txg);
@@ -427,7 +430,8 @@ extern void	zil_clean(zilog_t *zilog, uint64_t synced_
 extern int	zil_suspend(const char *osname, void **cookiep);
 extern void	zil_resume(void *cookie);
 
-extern void	zil_add_block(zilog_t *zilog, const blkptr_t *bp);
+extern void	zil_lwb_add_block(struct lwb *lwb, const blkptr_t *bp);
+extern void	zil_lwb_add_txg(struct lwb *lwb, uint64_t txg);
 extern int	zil_bp_tree_add(zilog_t *zilog, const blkptr_t *bp);
 
 extern void	zil_set_sync(zilog_t *zilog, uint64_t syncval);

Modified: head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zil_impl.h
==============================================================================
--- head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zil_impl.h	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zil_impl.h	Tue Sep 26 11:04:08 2017	(r324011)
@@ -20,7 +20,7 @@
  */
 /*
  * Copyright (c) 2005, 2010, Oracle and/or its affiliates. All rights reserved.
- * Copyright (c) 2012 by Delphix. All rights reserved.
+ * Copyright (c) 2012, 2017 by Delphix. All rights reserved.
  * Copyright (c) 2014 Integros [integros.com]
  */
 
@@ -37,22 +37,76 @@ extern "C" {
 #endif
 
 /*
- * Log write buffer.
+ * Possbile states for a given lwb structure. An lwb will start out in
+ * the "closed" state, and then transition to the "opened" state via a
+ * call to zil_lwb_write_open(). After the lwb is "open", it can
+ * transition into the "issued" state via zil_lwb_write_issue(). After
+ * the lwb's zio completes, and the vdev's are flushed, the lwb will
+ * transition into the "done" state via zil_lwb_write_done(), and the
+ * structure eventually freed.
  */
+typedef enum {
+    LWB_STATE_CLOSED,
+    LWB_STATE_OPENED,
+    LWB_STATE_ISSUED,
+    LWB_STATE_DONE,
+    LWB_NUM_STATES
+} lwb_state_t;
+
+/*
+ * Log write block (lwb)
+ *
+ * Prior to an lwb being issued to disk via zil_lwb_write_issue(), it
+ * will be protected by the zilog's "zl_writer_lock". Basically, prior
+ * to it being issued, it will only be accessed by the thread that's
+ * holding the "zl_writer_lock". After the lwb is issued, the zilog's
+ * "zl_lock" is used to protect the lwb against concurrent access.
+ */
 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_slog;	/* lwb_blk is on SLOG device */
 	int		lwb_nused;	/* # used bytes in buffer */
 	int		lwb_sz;		/* size of block and buffer */
+	lwb_state_t	lwb_state;	/* the state of this lwb */
 	char		*lwb_buf;	/* log write buffer */
-	zio_t		*lwb_zio;	/* zio for this buffer */
+	zio_t		*lwb_write_zio;	/* zio for the lwb buffer */
+	zio_t		*lwb_root_zio;	/* root zio for lwb write and flushes */
 	dmu_tx_t	*lwb_tx;	/* tx for log block allocation */
 	uint64_t	lwb_max_txg;	/* highest txg in this lwb */
 	list_node_t	lwb_node;	/* zilog->zl_lwb_list linkage */
+	list_t		lwb_waiters;	/* list of zil_commit_waiter's */
+	avl_tree_t	lwb_vdev_tree;	/* vdevs to flush after lwb write */
+	kmutex_t	lwb_vdev_lock;	/* protects lwb_vdev_tree */
+	hrtime_t	lwb_issued_timestamp; /* when was the lwb issued? */
 } lwb_t;
 
 /*
+ * ZIL commit waiter.
+ *
+ * This structure is allocated each time zil_commit() is called, and is
+ * used by zil_commit() to communicate with other parts of the ZIL, such
+ * that zil_commit() can know when it safe for it return. For more
+ * details, see the comment above zil_commit().
+ *
+ * The "zcw_lock" field is used to protect the commit waiter against
+ * concurrent access. This lock is often acquired while already holding
+ * the zilog's "zl_writer_lock" or "zl_lock"; see the functions
+ * zil_process_commit_list() and zil_lwb_flush_vdevs_done() as examples
+ * of this. Thus, one must be careful not to acquire the
+ * "zl_writer_lock" or "zl_lock" when already holding the "zcw_lock";
+ * e.g. see the zil_commit_waiter_timeout() function.
+ */
+typedef struct zil_commit_waiter {
+	kcondvar_t	zcw_cv;		/* signalled when "done" */
+	kmutex_t	zcw_lock;	/* protects fields of this struct */
+	list_node_t	zcw_node;	/* linkage in lwb_t:lwb_waiter list */
+	lwb_t		*zcw_lwb;	/* back pointer to lwb when linked */
+	boolean_t	zcw_done;	/* B_TRUE when "done", else B_FALSE */
+	int		zcw_zio_error;	/* contains the zio io_error value */
+} zil_commit_waiter_t;
+
+/*
  * Intent log transaction lists
  */
 typedef struct itxs {
@@ -94,20 +148,20 @@ struct zilog {
 	const zil_header_t *zl_header;	/* log header buffer */
 	objset_t	*zl_os;		/* object set we're logging */
 	zil_get_data_t	*zl_get_data;	/* callback to get object content */
-	zio_t		*zl_root_zio;	/* log writer root zio */
+	lwb_t		*zl_last_lwb_opened; /* most recent lwb opened */
+	hrtime_t	zl_last_lwb_latency; /* zio latency of last lwb done */
 	uint64_t	zl_lr_seq;	/* on-disk log record sequence number */
 	uint64_t	zl_commit_lr_seq; /* last committed on-disk lr seq */
 	uint64_t	zl_destroy_txg;	/* txg of last zil_destroy() */
 	uint64_t	zl_replayed_seq[TXG_SIZE]; /* last replayed rec seq */
 	uint64_t	zl_replaying_seq; /* current replay seq number */
 	uint32_t	zl_suspend;	/* log suspend count */
-	kcondvar_t	zl_cv_writer;	/* log writer thread completion */
 	kcondvar_t	zl_cv_suspend;	/* log suspend completion */
 	uint8_t		zl_suspending;	/* log is currently suspending */
 	uint8_t		zl_keep_first;	/* keep first log block in destroy */
 	uint8_t		zl_replay;	/* replaying records while set */
 	uint8_t		zl_stop_sync;	/* for debugging */
-	uint8_t		zl_writer;	/* boolean: write setup in progress */
+	kmutex_t	zl_writer_lock;	/* single writer, per ZIL, at a time */
 	uint8_t		zl_logbias;	/* latency or throughput */
 	uint8_t		zl_sync;	/* synchronous or asynchronous */
 	int		zl_parse_error;	/* last zil_parse() error */
@@ -115,15 +169,10 @@ struct zilog {
 	uint64_t	zl_parse_lr_seq; /* highest lr seq on last parse */
 	uint64_t	zl_parse_blk_count; /* number of blocks parsed */
 	uint64_t	zl_parse_lr_count; /* number of log records parsed */
-	uint64_t	zl_next_batch;	/* next batch number */
-	uint64_t	zl_com_batch;	/* committed batch number */
-	kcondvar_t	zl_cv_batch[2];	/* batch condition variables */
 	itxg_t		zl_itxg[TXG_SIZE]; /* intent log txg chains */
 	list_t		zl_itx_commit_list; /* itx list to be committed */
 	uint64_t	zl_cur_used;	/* current commit log size used */
 	list_t		zl_lwb_list;	/* in-flight log write list */
-	kmutex_t	zl_vdev_lock;	/* protects zl_vdev_tree */
-	avl_tree_t	zl_vdev_tree;	/* vdevs to flush in zil_commit() */
 	avl_tree_t	zl_bp_tree;	/* track bps during log parse */
 	clock_t		zl_replay_time;	/* lbolt of when replay started */
 	uint64_t	zl_replay_blks;	/* number of log blocks replayed */
@@ -131,6 +180,7 @@ struct zilog {
 	uint_t		zl_prev_blks[ZIL_PREV_BLKS]; /* size - sector rounded */
 	uint_t		zl_prev_rotor;	/* rotor for zl_prev[] */
 	txg_node_t	zl_dirty_link;	/* protected by dp_dirty_zilogs list */
+	uint64_t	zl_dirty_max_txg; /* highest txg used to dirty zilog */
 };
 
 typedef struct zil_bp_node {

Modified: head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zio.h
==============================================================================
--- head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zio.h	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/sys/zio.h	Tue Sep 26 11:04:08 2017	(r324011)
@@ -593,6 +593,7 @@ extern enum zio_checksum zio_checksum_dedup_select(spa
 extern enum zio_compress zio_compress_select(spa_t *spa,
     enum zio_compress child, enum zio_compress parent);
 
+extern void zio_cancel(zio_t *zio);
 extern void zio_suspend(spa_t *spa, zio_t *zio);
 extern int zio_resume(spa_t *spa);
 extern void zio_resume_wait(spa_t *spa);

Modified: head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/txg.c
==============================================================================
--- head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/txg.c	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/txg.c	Tue Sep 26 11:04:08 2017	(r324011)
@@ -163,7 +163,7 @@ txg_fini(dsl_pool_t *dp)
 	tx_state_t *tx = &dp->dp_tx;
 	int c;
 
-	ASSERT(tx->tx_threads == 0);
+	ASSERT0(tx->tx_threads);
 
 	mutex_destroy(&tx->tx_sync_lock);
 
@@ -204,7 +204,7 @@ txg_sync_start(dsl_pool_t *dp)
 
 	dprintf("pool %p\n", dp);
 
-	ASSERT(tx->tx_threads == 0);
+	ASSERT0(tx->tx_threads);
 
 	tx->tx_threads = 2;
 
@@ -265,7 +265,7 @@ txg_sync_stop(dsl_pool_t *dp)
 	/*
 	 * Finish off any work in progress.
 	 */
-	ASSERT(tx->tx_threads == 2);
+	ASSERT3U(tx->tx_threads, ==, 2);
 
 	/*
 	 * We need to ensure that we've vacated the deferred space_maps.
@@ -277,7 +277,7 @@ txg_sync_stop(dsl_pool_t *dp)
 	 */
 	mutex_enter(&tx->tx_sync_lock);
 
-	ASSERT(tx->tx_threads == 2);
+	ASSERT3U(tx->tx_threads, ==, 2);
 
 	tx->tx_exiting = 1;
 
@@ -616,7 +616,7 @@ txg_wait_synced(dsl_pool_t *dp, uint64_t txg)
 	ASSERT(!dsl_pool_config_held(dp));
 
 	mutex_enter(&tx->tx_sync_lock);
-	ASSERT(tx->tx_threads == 2);
+	ASSERT3U(tx->tx_threads, ==, 2);
 	if (txg == 0)
 		txg = tx->tx_open_txg + TXG_DEFER_SIZE;
 	if (tx->tx_sync_txg_waiting < txg)
@@ -641,7 +641,7 @@ txg_wait_open(dsl_pool_t *dp, uint64_t txg)
 	ASSERT(!dsl_pool_config_held(dp));
 
 	mutex_enter(&tx->tx_sync_lock);
-	ASSERT(tx->tx_threads == 2);
+	ASSERT3U(tx->tx_threads, ==, 2);
 	if (txg == 0)
 		txg = tx->tx_open_txg + 1;
 	if (tx->tx_quiesce_txg_waiting < txg)

Modified: head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zfs_vnops.c
==============================================================================
--- head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zfs_vnops.c	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zfs_vnops.c	Tue Sep 26 11:04:08 2017	(r324011)
@@ -76,6 +76,7 @@
 #include <sys/acl.h>
 #include <sys/vmmeter.h>
 #include <vm/vm_param.h>
+#include <sys/zil.h>
 
 /*
  * Programming rules.
@@ -1276,7 +1277,7 @@ zfs_get_done(zgd_t *zgd, int error)
 	VN_RELE_ASYNC(ZTOV(zp), dsl_pool_vnrele_taskq(dmu_objset_pool(os)));
 
 	if (error == 0 && zgd->zgd_bp)
-		zil_add_block(zgd->zgd_zilog, zgd->zgd_bp);
+		zil_lwb_add_block(zgd->zgd_lwb, zgd->zgd_bp);
 
 	kmem_free(zgd, sizeof (zgd_t));
 }
@@ -1289,7 +1290,7 @@ static int zil_fault_io = 0;
  * Get data to generate a TX_WRITE intent log record.
  */
 int
-zfs_get_data(void *arg, lr_write_t *lr, char *buf, zio_t *zio)
+zfs_get_data(void *arg, lr_write_t *lr, char *buf, struct lwb *lwb, zio_t *zio)
 {
 	zfsvfs_t *zfsvfs = arg;
 	objset_t *os = zfsvfs->z_os;
@@ -1301,8 +1302,9 @@ zfs_get_data(void *arg, lr_write_t *lr, char *buf, zio
 	zgd_t *zgd;
 	int error = 0;
 
-	ASSERT(zio != NULL);
-	ASSERT(size != 0);
+	ASSERT3P(lwb, !=, NULL);
+	ASSERT3P(zio, !=, NULL);
+	ASSERT3U(size, !=, 0);
 
 	/*
 	 * Nothing to do if the file has been removed
@@ -1320,7 +1322,7 @@ zfs_get_data(void *arg, lr_write_t *lr, char *buf, zio
 	}
 
 	zgd = (zgd_t *)kmem_zalloc(sizeof (zgd_t), KM_SLEEP);
-	zgd->zgd_zilog = zfsvfs->z_log;
+	zgd->zgd_lwb = lwb;
 	zgd->zgd_private = zp;
 
 	/*

Modified: head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zil.c
==============================================================================
--- head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zil.c	Tue Sep 26 09:34:18 2017	(r324010)
+++ head/sys/cddl/contrib/opensolaris/uts/common/fs/zfs/zil.c	Tue Sep 26 11:04:08 2017	(r324011)
@@ -42,32 +42,53 @@
 #include <sys/abd.h>
 
 /*
- * The zfs intent log (ZIL) saves transaction records of system calls
- * that change the file system in memory with enough information
- * to be able to replay them. These are stored in memory until
- * either the DMU transaction group (txg) commits them to the stable pool
- * and they can be discarded, or they are flushed to the stable log
- * (also in the pool) due to a fsync, O_DSYNC or other synchronous
- * requirement. In the event of a panic or power fail then those log
- * records (transactions) are replayed.
+ * The ZFS Intent Log (ZIL) saves "transaction records" (itxs) of system
+ * calls that change the file system. Each itx has enough information to
+ * be able to replay them after a system crash, power loss, or
+ * equivalent failure mode. These are stored in memory until either:
  *
- * There is one ZIL per file system. Its on-disk (pool) format consists
- * of 3 parts:
+ *   1. they are committed to the pool by the DMU transaction group
+ *      (txg), at which point they can be discarded; or
+ *   2. they are committed to the on-disk ZIL for the dataset being
+ *      modified (e.g. due to an fsync, O_DSYNC, or other synchronous
+ *      requirement).
  *
- * 	- ZIL header
- * 	- ZIL blocks
- * 	- ZIL records
+ * In the event of a crash or power loss, the itxs contained by each
+ * dataset's on-disk ZIL will be replayed when that dataset is first
+ * instantianted (e.g. if the dataset is a normal fileystem, when it is
+ * first mounted).
  *
- * A log record holds a system call transaction. Log blocks can
- * hold many log records and the blocks are chained together.
- * Each ZIL block contains a block pointer (blkptr_t) to the next
- * ZIL block in the chain. The ZIL header points to the first
- * block in the chain. Note there is not a fixed place in the pool
- * to hold blocks. They are dynamically allocated and freed as
- * needed from the blocks available. Figure X shows the ZIL structure:
+ * As hinted at above, there is one ZIL per dataset (both the in-memory
+ * representation, and the on-disk representation). The on-disk format
+ * consists of 3 parts:
+ *
+ * 	- a single, per-dataset, ZIL header; which points to a chain of
+ * 	- zero or more ZIL blocks; each of which contains
+ * 	- zero or more ZIL records
+ *
+ * A ZIL record holds the information necessary to replay a single
+ * system call transaction. A ZIL block can hold many ZIL records, and
+ * the blocks are chained together, similarly to a singly linked list.
+ *
+ * Each ZIL block contains a block pointer (blkptr_t) to the next ZIL
+ * block in the chain, and the ZIL header points to the first block in
+ * the chain.
+ *
+ * Note, there is not a fixed place in the pool to hold these ZIL
+ * blocks; they are dynamically allocated and freed as needed from the
+ * blocks available on the pool, though they can be preferentially
+ * allocated from a dedicated "log" vdev.
  */
 
 /*
+ * This controls the amount of time that a ZIL block (lwb) will remain
+ * "open" when it isn't "full", and it has a thread waiting for it to be
+ * committed to stable storage. Please refer to the zil_commit_waiter()
+ * function (and the comments within it) for more details.
+ */
+int zfs_commit_timeout_pct = 5;
+
+/*
  * Disable intent logging replay.  This global ZIL switch affects all pools.
  */
 int zil_replay_disable = 0;
@@ -98,6 +119,7 @@ SYSCTL_QUAD(_vfs_zfs, OID_AUTO, zil_slog_bulk, CTLFLAG
     &zil_slog_bulk, 0, "Maximal SLOG commit size with sync priority");
 
 static kmem_cache_t *zil_lwb_cache;
+static kmem_cache_t *zil_zcw_cache;
 
 #define	LWB_EMPTY(lwb) ((BP_GET_LSIZE(&lwb->lwb_blk) - \
     sizeof (zil_chain_t)) == (lwb->lwb_sz - lwb->lwb_nused))
@@ -445,6 +467,20 @@ zil_free_log_record(zilog_t *zilog, lr_t *lrc, void *t
 	return (0);
 }
 
+static int
+zil_lwb_vdev_compare(const void *x1, const void *x2)
+{
+	const uint64_t v1 = ((zil_vdev_node_t *)x1)->zv_vdev;
+	const uint64_t v2 = ((zil_vdev_node_t *)x2)->zv_vdev;
+
+	if (v1 < v2)
+		return (-1);
+	if (v1 > v2)
+		return (1);
+
+	return (0);
+}
+
 static lwb_t *
 zil_alloc_lwb(zilog_t *zilog, blkptr_t *bp, boolean_t slog, uint64_t txg)
 {
@@ -454,10 +490,13 @@ zil_alloc_lwb(zilog_t *zilog, blkptr_t *bp, boolean_t 
 	lwb->lwb_zilog = zilog;
 	lwb->lwb_blk = *bp;
 	lwb->lwb_slog = slog;
+	lwb->lwb_state = LWB_STATE_CLOSED;
 	lwb->lwb_buf = zio_buf_alloc(BP_GET_LSIZE(bp));
 	lwb->lwb_max_txg = txg;
-	lwb->lwb_zio = NULL;
+	lwb->lwb_write_zio = NULL;
+	lwb->lwb_root_zio = NULL;
 	lwb->lwb_tx = NULL;
+	lwb->lwb_issued_timestamp = 0;
 	if (BP_GET_CHECKSUM(bp) == ZIO_CHECKSUM_ZILOG2) {
 		lwb->lwb_nused = sizeof (zil_chain_t);
 		lwb->lwb_sz = BP_GET_LSIZE(bp);
@@ -470,9 +509,54 @@ zil_alloc_lwb(zilog_t *zilog, blkptr_t *bp, boolean_t 
 	list_insert_tail(&zilog->zl_lwb_list, lwb);
 	mutex_exit(&zilog->zl_lock);
 
+	ASSERT(!MUTEX_HELD(&lwb->lwb_vdev_lock));
+	ASSERT(avl_is_empty(&lwb->lwb_vdev_tree));
+	ASSERT(list_is_empty(&lwb->lwb_waiters));
+
 	return (lwb);
 }
 
+static void
+zil_free_lwb(zilog_t *zilog, lwb_t *lwb)
+{
+	ASSERT(MUTEX_HELD(&zilog->zl_lock));
+	ASSERT(!MUTEX_HELD(&lwb->lwb_vdev_lock));
+	ASSERT(list_is_empty(&lwb->lwb_waiters));
+
+	if (lwb->lwb_state == LWB_STATE_OPENED) {
+		avl_tree_t *t = &lwb->lwb_vdev_tree;
+		void *cookie = NULL;
+		zil_vdev_node_t *zv;
+
+		while ((zv = avl_destroy_nodes(t, &cookie)) != NULL)
+			kmem_free(zv, sizeof (*zv));
+
+		ASSERT3P(lwb->lwb_root_zio, !=, NULL);
+		ASSERT3P(lwb->lwb_write_zio, !=, NULL);
+
+		zio_cancel(lwb->lwb_root_zio);
+		zio_cancel(lwb->lwb_write_zio);
+
+		lwb->lwb_root_zio = NULL;
+		lwb->lwb_write_zio = NULL;
+	} else {
+		ASSERT3S(lwb->lwb_state, !=, LWB_STATE_ISSUED);
+	}
+
+	ASSERT(avl_is_empty(&lwb->lwb_vdev_tree));
+	ASSERT3P(lwb->lwb_write_zio, ==, NULL);
+	ASSERT3P(lwb->lwb_root_zio, ==, NULL);
+
+	/*
+	 * Clear the zilog's field to indicate this lwb is no longer
+	 * valid, and prevent use-after-free errors.
+	 */
+	if (zilog->zl_last_lwb_opened == lwb)
+		zilog->zl_last_lwb_opened = NULL;
+
+	kmem_cache_free(zil_lwb_cache, lwb);
+}
+
 /*
  * Called when we create in-memory log transactions so that we know
  * to cleanup the itxs at the end of spa_sync().
@@ -483,12 +567,16 @@ zilog_dirty(zilog_t *zilog, uint64_t txg)
 	dsl_pool_t *dp = zilog->zl_dmu_pool;
 	dsl_dataset_t *ds = dmu_objset_ds(zilog->zl_os);
 
+	ASSERT(spa_writeable(zilog->zl_spa));
+
 	if (ds->ds_is_snapshot)
 		panic("dirtying snapshot!");
 
 	if (txg_list_add(&dp->dp_dirty_zilogs, zilog, txg)) {
 		/* up the hold count until we can be written out */
 		dmu_buf_add_ref(ds->ds_dbuf, zilog);
+
+		zilog->zl_dirty_max_txg = MAX(txg, zilog->zl_dirty_max_txg);
 	}
 }
 
@@ -556,7 +644,7 @@ zil_create(zilog_t *zilog)
 	 */
 	if (BP_IS_HOLE(&blk) || BP_SHOULD_BYTESWAP(&blk)) {
 		tx = dmu_tx_create(zilog->zl_os);
-		VERIFY(dmu_tx_assign(tx, TXG_WAIT) == 0);
+		VERIFY0(dmu_tx_assign(tx, TXG_WAIT));
 		dsl_dataset_dirty(dmu_objset_ds(zilog->zl_os), tx);
 		txg = dmu_tx_get_txg(tx);
 
@@ -573,7 +661,7 @@ zil_create(zilog_t *zilog)
 	}
 
 	/*
-	 * Allocate a log write buffer (lwb) for the first log block.
+	 * Allocate a log write block (lwb) for the first log block.
 	 */
 	if (error == 0)
 		lwb = zil_alloc_lwb(zilog, &blk, slog, txg);
@@ -594,13 +682,13 @@ zil_create(zilog_t *zilog)
 }
 
 /*
- * In one tx, free all log blocks and clear the log header.
- * If keep_first is set, then we're replaying a log with no content.
- * We want to keep the first block, however, so that the first
- * synchronous transaction doesn't require a txg_wait_synced()
- * in zil_create().  We don't need to txg_wait_synced() here either
- * when keep_first is set, because both zil_create() and zil_destroy()
- * will wait for any in-progress destroys to complete.
+ * In one tx, free all log blocks and clear the log header. If keep_first
+ * is set, then we're replaying a log with no content. We want to keep the
+ * first block, however, so that the first synchronous transaction doesn't
+ * require a txg_wait_synced() in zil_create(). We don't need to
+ * txg_wait_synced() here either when keep_first is set, because both
+ * zil_create() and zil_destroy() will wait for any in-progress destroys
+ * to complete.
  */
 void
 zil_destroy(zilog_t *zilog, boolean_t keep_first)
@@ -621,7 +709,7 @@ zil_destroy(zilog_t *zilog, boolean_t keep_first)
 		return;
 
 	tx = dmu_tx_create(zilog->zl_os);
-	VERIFY(dmu_tx_assign(tx, TXG_WAIT) == 0);
+	VERIFY0(dmu_tx_assign(tx, TXG_WAIT));
 	dsl_dataset_dirty(dmu_objset_ds(zilog->zl_os), tx);
 	txg = dmu_tx_get_txg(tx);
 
@@ -638,8 +726,8 @@ zil_destroy(zilog_t *zilog, boolean_t keep_first)
 			list_remove(&zilog->zl_lwb_list, lwb);
 			if (lwb->lwb_buf != NULL)
 				zio_buf_free(lwb->lwb_buf, lwb->lwb_sz);
-			zio_free_zil(zilog->zl_spa, txg, &lwb->lwb_blk);
-			kmem_cache_free(zil_lwb_cache, lwb);
+			zio_free(zilog->zl_spa, txg, &lwb->lwb_blk);
+			zil_free_lwb(zilog, lwb);
 		}
 	} else if (!keep_first) {
 		zil_destroy_sync(zilog, tx);
@@ -777,24 +865,64 @@ zil_check_log_chain(dsl_pool_t *dp, dsl_dataset_t *ds,
 	return ((error == ECKSUM || error == ENOENT) ? 0 : error);
 }
 
-static int
-zil_vdev_compare(const void *x1, const void *x2)
+/*
+ * When an itx is "skipped", this function is used to properly mark the
+ * waiter as "done, and signal any thread(s) waiting on it. An itx can
+ * be skipped (and not committed to an lwb) for a variety of reasons,
+ * one of them being that the itx was committed via spa_sync(), prior to
+ * it being committed to an lwb; this can happen if a thread calling
+ * zil_commit() is racing with spa_sync().
+ */
+static void
+zil_commit_waiter_skip(zil_commit_waiter_t *zcw)
 {
-	const uint64_t v1 = ((zil_vdev_node_t *)x1)->zv_vdev;
-	const uint64_t v2 = ((zil_vdev_node_t *)x2)->zv_vdev;
+	mutex_enter(&zcw->zcw_lock);
+	ASSERT3B(zcw->zcw_done, ==, B_FALSE);
+	zcw->zcw_done = B_TRUE;
+	cv_broadcast(&zcw->zcw_cv);
+	mutex_exit(&zcw->zcw_lock);
+}
 
-	if (v1 < v2)
-		return (-1);
-	if (v1 > v2)
-		return (1);
+/*
+ * This function is used when the given waiter is to be linked into an
+ * lwb's "lwb_waiter" list; i.e. when the itx is committed to the lwb.
+ * At this point, the waiter will no longer be referenced by the itx,
+ * and instead, will be referenced by the lwb.
+ */
+static void
+zil_commit_waiter_link_lwb(zil_commit_waiter_t *zcw, lwb_t *lwb)
+{
+	mutex_enter(&zcw->zcw_lock);
+	ASSERT(!list_link_active(&zcw->zcw_node));
+	ASSERT3P(zcw->zcw_lwb, ==, NULL);
+	ASSERT3P(lwb, !=, NULL);
+	ASSERT(lwb->lwb_state == LWB_STATE_OPENED ||
+	    lwb->lwb_state == LWB_STATE_ISSUED);
 
-	return (0);
+	list_insert_tail(&lwb->lwb_waiters, zcw);
+	zcw->zcw_lwb = lwb;
+	mutex_exit(&zcw->zcw_lock);
 }
 
+/*
+ * This function is used when zio_alloc_zil() fails to allocate a ZIL
+ * block, and the given waiter must be linked to the "nolwb waiters"
+ * list inside of zil_process_commit_list().
+ */
+static void
+zil_commit_waiter_link_nolwb(zil_commit_waiter_t *zcw, list_t *nolwb)
+{
+	mutex_enter(&zcw->zcw_lock);
+	ASSERT(!list_link_active(&zcw->zcw_node));
+	ASSERT3P(zcw->zcw_lwb, ==, NULL);
+	list_insert_tail(nolwb, zcw);
+	mutex_exit(&zcw->zcw_lock);
+}
+
 void
-zil_add_block(zilog_t *zilog, const blkptr_t *bp)
+zil_lwb_add_block(lwb_t *lwb, const blkptr_t *bp)
 {
-	avl_tree_t *t = &zilog->zl_vdev_tree;
+	avl_tree_t *t = &lwb->lwb_vdev_tree;
 	avl_index_t where;
 	zil_vdev_node_t *zv, zvsearch;
 	int ndvas = BP_GET_NDVAS(bp);
@@ -803,14 +931,7 @@ zil_add_block(zilog_t *zilog, const blkptr_t *bp)
 	if (zfs_nocacheflush)
 		return;
 
-	ASSERT(zilog->zl_writer);
-
-	/*
-	 * Even though we're zl_writer, we still need a lock because the
-	 * zl_get_data() callbacks may have dmu_sync() done callbacks
-	 * that will run concurrently.
-	 */
-	mutex_enter(&zilog->zl_vdev_lock);
+	mutex_enter(&lwb->lwb_vdev_lock);
 	for (i = 0; i < ndvas; i++) {
 		zvsearch.zv_vdev = DVA_GET_VDEV(&bp->blk_dva[i]);
 		if (avl_find(t, &zvsearch, &where) == NULL) {
@@ -819,59 +940,117 @@ zil_add_block(zilog_t *zilog, const blkptr_t *bp)
 			avl_insert(t, zv, where);
 		}
 	}
-	mutex_exit(&zilog->zl_vdev_lock);
+	mutex_exit(&lwb->lwb_vdev_lock);
 }
 
+void
+zil_lwb_add_txg(lwb_t *lwb, uint64_t txg)
+{
+	lwb->lwb_max_txg = MAX(lwb->lwb_max_txg, txg);
+}
+
+/*
+ * This function is a called after all VDEVs associated with a given lwb
+ * write have completed their DKIOCFLUSHWRITECACHE command; or as soon
+ * as the lwb write completes, if "zfs_nocacheflush" is set.
+ *
+ * The intention is for this function to be called as soon as the
+ * contents of an lwb are considered "stable" on disk, and will survive
+ * any sudden loss of power. At this point, any threads waiting for the
+ * lwb to reach this state are signalled, and the "waiter" structures
+ * are marked "done".
+ */
 static void
-zil_flush_vdevs(zilog_t *zilog)
+zil_lwb_flush_vdevs_done(zio_t *zio)
 {
-	spa_t *spa = zilog->zl_spa;
-	avl_tree_t *t = &zilog->zl_vdev_tree;
-	void *cookie = NULL;
-	zil_vdev_node_t *zv;
-	zio_t *zio = NULL;
+	lwb_t *lwb = zio->io_private;
+	zilog_t *zilog = lwb->lwb_zilog;
+	dmu_tx_t *tx = lwb->lwb_tx;
+	zil_commit_waiter_t *zcw;
 
-	ASSERT(zilog->zl_writer);
+	spa_config_exit(zilog->zl_spa, SCL_STATE, lwb);
 
+	zio_buf_free(lwb->lwb_buf, lwb->lwb_sz);
+
+	mutex_enter(&zilog->zl_lock);
+
 	/*
-	 * We don't need zl_vdev_lock here because we're the zl_writer,
-	 * and all zl_get_data() callbacks are done.
+	 * Ensure the lwb buffer pointer is cleared before releasing the
+	 * txg. If we have had an allocation failure and the txg is
+	 * waiting to sync then we want zil_sync() to remove the lwb so
+	 * that it's not picked up as the next new one in
+	 * zil_process_commit_list(). zil_sync() will only remove the
+	 * lwb if lwb_buf is null.
 	 */
-	if (avl_numnodes(t) == 0)
-		return;
+	lwb->lwb_buf = NULL;
+	lwb->lwb_tx = NULL;
 
-	spa_config_enter(spa, SCL_STATE, FTAG, RW_READER);
+	ASSERT3U(lwb->lwb_issued_timestamp, >, 0);
+	zilog->zl_last_lwb_latency = gethrtime() - lwb->lwb_issued_timestamp;
 
-	while ((zv = avl_destroy_nodes(t, &cookie)) != NULL) {
-		vdev_t *vd = vdev_lookup_top(spa, zv->zv_vdev);
-		if (vd != NULL && !vd->vdev_nowritecache) {
-			if (zio == NULL)
-				zio = zio_root(spa, NULL, NULL, ZIO_FLAG_CANFAIL);
-			zio_flush(zio, vd);
-		}
-		kmem_free(zv, sizeof (*zv));
+	lwb->lwb_root_zio = NULL;
+	lwb->lwb_state = LWB_STATE_DONE;
+
+	if (zilog->zl_last_lwb_opened == lwb) {
+		/*
+		 * Remember the highest committed log sequence number
+		 * for ztest. We only update this value when all the log
+		 * writes succeeded, because ztest wants to ASSERT that
+		 * it got the whole log chain.
+		 */
+		zilog->zl_commit_lr_seq = zilog->zl_lr_seq;
 	}
 
+	while ((zcw = list_head(&lwb->lwb_waiters)) != NULL) {
+		mutex_enter(&zcw->zcw_lock);
+
+		ASSERT(list_link_active(&zcw->zcw_node));
+		list_remove(&lwb->lwb_waiters, zcw);
+
+		ASSERT3P(zcw->zcw_lwb, ==, lwb);
+		zcw->zcw_lwb = NULL;
+
+		zcw->zcw_zio_error = zio->io_error;
+
+		ASSERT3B(zcw->zcw_done, ==, B_FALSE);
+		zcw->zcw_done = B_TRUE;
+		cv_broadcast(&zcw->zcw_cv);
+
+		mutex_exit(&zcw->zcw_lock);
+	}
+
+	mutex_exit(&zilog->zl_lock);
+
 	/*
-	 * Wait for all the flushes to complete.  Not all devices actually
-	 * support the DKIOCFLUSHWRITECACHE ioctl, so it's OK if it fails.
+	 * Now that we've written this log block, we have a stable pointer
+	 * to the next block in the chain, so it's OK to let the txg in
+	 * which we allocated the next block sync.
 	 */
-	if (zio)
-		(void) zio_wait(zio);
-
-	spa_config_exit(spa, SCL_STATE, FTAG);
+	dmu_tx_commit(tx);
 }
 
 /*
- * Function called when a log block write completes
+ * This is called when an lwb write completes. This means, this specific
+ * lwb was written to disk, and all dependent lwb have also been
+ * written to disk.
+ *
+ * At this point, a DKIOCFLUSHWRITECACHE command hasn't been issued to
+ * the VDEVs involved in writing out this specific lwb. The lwb will be
+ * "done" once zil_lwb_flush_vdevs_done() is called, which occurs in the
+ * zio completion callback for the lwb's root zio.
  */
 static void
 zil_lwb_write_done(zio_t *zio)
 {
 	lwb_t *lwb = zio->io_private;
+	spa_t *spa = zio->io_spa;
 	zilog_t *zilog = lwb->lwb_zilog;
-	dmu_tx_t *tx = lwb->lwb_tx;
+	avl_tree_t *t = &lwb->lwb_vdev_tree;
+	void *cookie = NULL;
+	zil_vdev_node_t *zv;
 
+	ASSERT3S(spa_config_held(spa, SCL_STATE, RW_READER), !=, 0);
+
 	ASSERT(BP_GET_COMPRESS(zio->io_bp) == ZIO_COMPRESS_OFF);
 	ASSERT(BP_GET_TYPE(zio->io_bp) == DMU_OT_INTENT_LOG);
 	ASSERT(BP_GET_LEVEL(zio->io_bp) == 0);
@@ -880,58 +1059,115 @@ zil_lwb_write_done(zio_t *zio)
 	ASSERT(!BP_IS_HOLE(zio->io_bp));
 	ASSERT(BP_GET_FILL(zio->io_bp) == 0);
 
-	/*
-	 * Ensure the lwb buffer pointer is cleared before releasing
-	 * the txg. If we have had an allocation failure and
-	 * the txg is waiting to sync then we want want zil_sync()
-	 * to remove the lwb so that it's not picked up as the next new
-	 * one in zil_commit_writer(). zil_sync() will only remove
-	 * the lwb if lwb_buf is null.
-	 */
 	abd_put(zio->io_abd);
-	zio_buf_free(lwb->lwb_buf, lwb->lwb_sz);
+
+	ASSERT3S(lwb->lwb_state, ==, LWB_STATE_ISSUED);
+
 	mutex_enter(&zilog->zl_lock);
-	lwb->lwb_buf = NULL;
-	lwb->lwb_tx = NULL;
+	lwb->lwb_write_zio = NULL;
 	mutex_exit(&zilog->zl_lock);
 
+	if (avl_numnodes(t) == 0)
+		return;
+
 	/*
-	 * Now that we've written this log block, we have a stable pointer
-	 * to the next block in the chain, so it's OK to let the txg in
-	 * which we allocated the next block sync.
+	 * If there was an IO error, we're not going to call zio_flush()
+	 * on these vdevs, so we simply empty the tree and free the
+	 * nodes. We avoid calling zio_flush() since there isn't any
+	 * good reason for doing so, after the lwb block failed to be
+	 * written out.
 	 */
-	dmu_tx_commit(tx);
+	if (zio->io_error != 0) {
+		while ((zv = avl_destroy_nodes(t, &cookie)) != NULL)
+			kmem_free(zv, sizeof (*zv));
+		return;
+	}
+
+	while ((zv = avl_destroy_nodes(t, &cookie)) != NULL) {
+		vdev_t *vd = vdev_lookup_top(spa, zv->zv_vdev);
+		if (vd != NULL)
+			zio_flush(lwb->lwb_root_zio, vd);
+		kmem_free(zv, sizeof (*zv));
+	}
 }
 
 /*
- * Initialize the io for a log block.
+ * This function's purpose is to "open" an lwb such that it is ready to
+ * accept new itxs being committed to it. To do this, the lwb's zio
+ * structures are created, and linked to the lwb. This function is
+ * idempotent; if the passed in lwb has already been opened, this
+ * function is essentially a no-op.
  */
 static void
-zil_lwb_write_init(zilog_t *zilog, lwb_t *lwb)
+zil_lwb_write_open(zilog_t *zilog, lwb_t *lwb)
 {
 	zbookmark_phys_t zb;
 	zio_priority_t prio;
 
+	ASSERT(MUTEX_HELD(&zilog->zl_writer_lock));
+	ASSERT3P(lwb, !=, NULL);
+	EQUIV(lwb->lwb_root_zio == NULL, lwb->lwb_state == LWB_STATE_CLOSED);
+	EQUIV(lwb->lwb_root_zio != NULL, lwb->lwb_state == LWB_STATE_OPENED);
+
 	SET_BOOKMARK(&zb, lwb->lwb_blk.blk_cksum.zc_word[ZIL_ZC_OBJSET],
 	    ZB_ZIL_OBJECT, ZB_ZIL_LEVEL,
 	    lwb->lwb_blk.blk_cksum.zc_word[ZIL_ZC_SEQ]);
 
-	if (zilog->zl_root_zio == NULL) {
-		zilog->zl_root_zio = zio_root(zilog->zl_spa, NULL, NULL,
-		    ZIO_FLAG_CANFAIL);
-	}
-	if (lwb->lwb_zio == NULL) {
+	if (lwb->lwb_root_zio == NULL) {
 		abd_t *lwb_abd = abd_get_from_buf(lwb->lwb_buf,
 		    BP_GET_LSIZE(&lwb->lwb_blk));
+
 		if (!lwb->lwb_slog || zilog->zl_cur_used <= zil_slog_bulk)
 			prio = ZIO_PRIORITY_SYNC_WRITE;
 		else
 			prio = ZIO_PRIORITY_ASYNC_WRITE;
-		lwb->lwb_zio = zio_rewrite(zilog->zl_root_zio, zilog->zl_spa,
-		    0, &lwb->lwb_blk, lwb_abd, BP_GET_LSIZE(&lwb->lwb_blk),
-		    zil_lwb_write_done, lwb, prio,
-		    ZIO_FLAG_CANFAIL | ZIO_FLAG_DONT_PROPAGATE, &zb);
+
+		lwb->lwb_root_zio = zio_root(zilog->zl_spa,
+		    zil_lwb_flush_vdevs_done, lwb, ZIO_FLAG_CANFAIL);
+		ASSERT3P(lwb->lwb_root_zio, !=, NULL);
+
+		lwb->lwb_write_zio = zio_rewrite(lwb->lwb_root_zio,
+		    zilog->zl_spa, 0, &lwb->lwb_blk, lwb_abd,
+		    BP_GET_LSIZE(&lwb->lwb_blk), zil_lwb_write_done, lwb,
+		    prio, ZIO_FLAG_CANFAIL | ZIO_FLAG_DONT_PROPAGATE, &zb);
+		ASSERT3P(lwb->lwb_write_zio, !=, NULL);
+
+		lwb->lwb_state = LWB_STATE_OPENED;
+
+		mutex_enter(&zilog->zl_lock);
+
+		/*
+		 * The zilog's "zl_last_lwb_opened" field is used to
+		 * build the lwb/zio dependency chain, which is used to
+		 * preserve the ordering of lwb completions that is
+		 * required by the semantics of the ZIL. Each new lwb
+		 * zio becomes a parent of the "previous" lwb zio, such
+		 * that the new lwb's zio cannot complete until the
+		 * "previous" lwb's zio completes.
+		 *
+		 * This is required by the semantics of zil_commit();
+		 * the commit waiters attached to the lwbs will be woken
+		 * in the lwb zio's completion callback, so this zio
+		 * dependency graph ensures the waiters are woken in the
+		 * correct order (the same order the lwbs were created).
+		 */
+		lwb_t *last_lwb_opened = zilog->zl_last_lwb_opened;
+		if (last_lwb_opened != NULL &&
+		    last_lwb_opened->lwb_state != LWB_STATE_DONE) {
+			ASSERT(last_lwb_opened->lwb_state == LWB_STATE_OPENED ||
+			    last_lwb_opened->lwb_state == LWB_STATE_ISSUED);
+			ASSERT3P(last_lwb_opened->lwb_root_zio, !=, NULL);
+			zio_add_child(lwb->lwb_root_zio,
+			    last_lwb_opened->lwb_root_zio);
+		}
+		zilog->zl_last_lwb_opened = lwb;
+
+		mutex_exit(&zilog->zl_lock);
 	}
+
+	ASSERT3P(lwb->lwb_root_zio, !=, NULL);
+	ASSERT3P(lwb->lwb_write_zio, !=, NULL);
+	ASSERT3S(lwb->lwb_state, ==, LWB_STATE_OPENED);
 }
 
 /*
@@ -953,7 +1189,7 @@ uint64_t zil_block_buckets[] = {
  * Calls are serialized.
  */
 static lwb_t *
-zil_lwb_write_start(zilog_t *zilog, lwb_t *lwb, boolean_t last)
+zil_lwb_write_issue(zilog_t *zilog, lwb_t *lwb)
 {
 	lwb_t *nlwb = NULL;
 	zil_chain_t *zilc;
@@ -965,6 +1201,11 @@ zil_lwb_write_start(zilog_t *zilog, lwb_t *lwb, boolea
 	int i, error;

*** DIFF OUTPUT TRUNCATED AT 1000 LINES ***



Want to link to this message? Use this URL: <https://mail-archive.FreeBSD.org/cgi/mid.cgi?201709261104.v8QB480K002320>