OpenZFS 8585 - improve batching done in zil_commit()

Authored by: Prakash Surya <prakash.surya@delphix.com>
Reviewed by: Brad Lewis <brad.lewis@delphix.com>
Reviewed by: Matt Ahrens <mahrens@delphix.com>
Reviewed by: George Wilson <george.wilson@delphix.com>
Reviewed-by: Brian Behlendorf <behlendorf1@llnl.gov>
Approved by: Dan McDonald <danmcd@joyent.com>
Ported-by: Prakash Surya <prakash.surya@delphix.com>

Problem
=======

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).

To further complicate the situation, these two issues result in the
following side effect:

 3. If a given batch takes longer to complete than normal, this results
    in larger batch sizes, which then take longer to complete and
    further drive up the latency of zil_commit(). This can occur for a
    number of reasons, including (but not limited to): transient changes
    in the workload, and storage latency irregularites.

Solution
========

The solution attempted by this change has the following goals:

 1. no on-disk changes; maintain current on-disk format.
 2. modify the "batch size" to be equal to the "ZIL block size".
 3. allow new batches to be generated and issued to disk, while there's
    already batches being serviced by the disk.
 4. allow zil_commit() to wait for as few ZIL blocks as possible.
 5. use as few ZIL blocks as possible, for the same amount of ZIL
    transactions, without introducing significant latency to any
    individual ZIL transaction. i.e. use fewer, but larger, ZIL blocks.

In theory, with these goals met, the new allgorithm will allow the
following improvements:

 1. new ZIL blocks can be generated and issued, while there's already
    oustanding ZIL blocks being serviced by the storage.
 2. the latency of zil_commit() should be proportional to the underlying
    storage latency, rather than the incoming synchronous workload.

Porting Notes
=============

Due to the changes made in commit 119a394ab0, the lifetime of an itx
structure differs than in OpenZFS. Specifically, the itx structure is
kept around until the data associated with the itx is considered to be
safe on disk; this is so that the itx's callback can be called after the
data is committed to stable storage. Since OpenZFS doesn't have this itx
callback mechanism, it's able to destroy the itx structure immediately
after the itx is committed to an lwb (before the lwb is written to
disk).

To support this difference, and to ensure the itx's callbacks can still
be called after the itx's data is on disk, a few changes had to be made:

  * A list of itxs was added to the lwb structure. This list contains
    all of the itxs that have been committed to the lwb, such that the
    callbacks for these itxs can be called from zil_lwb_flush_vdevs_done(),
    after the data for the itxs is committed to disk.

  * A list of itxs was added on the stack of the zil_process_commit_list()
    function; the "nolwb_itxs" list. In some circumstances, an itx may
    not be committed to an lwb (e.g. if allocating the "next" ZIL block
    on disk fails), so this list is used to keep track of which itxs
    fall into this state, such that their callbacks can be called after
    the ZIL's writer pipeline is "stalled".

  * The logic to actually call the itx's callback was moved into the
    zil_itx_destroy() function. Since all consumers of zil_itx_destroy()
    were effectively performing the same logic (i.e. if callback is
    non-null, call the callback), it seemed like useful code cleanup to
    consolidate this logic into a single function.

Additionally, the existing Linux tracepoint infrastructure dealing with
the ZIL's probes and structures had to be updated to reflect these code
changes. Specifically:

  * The "zil__cw1" and "zil__cw2" probes were removed, so they had to be
    removed from "trace_zil.h" as well.

  * Some of the zilog structure's fields were removed, which affected
    the tracepoint definitions of the structure.

  * New tracepoints had to be added for the following 3 new probes:
      * zil__process__commit__itx
      * zil__process__normal__itx
      * zil__commit__io__error

OpenZFS-issue: https://www.illumos.org/issues/8585
OpenZFS-commit: https://github.com/openzfs/openzfs/commit/5d95a3a
Closes #6566
This commit is contained in:
Prakash Surya 2017-12-05 09:39:16 -08:00 committed by Brian Behlendorf
parent 7b3407003f
commit 1ce23dcaff
15 changed files with 1610 additions and 401 deletions

View File

@ -2144,14 +2144,15 @@ 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));
umem_free(zzp, sizeof (*zzp));
}
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;
@ -2166,6 +2167,10 @@ ztest_get_data(void *arg, lr_write_t *lr, char *buf, zio_t *zio)
int error;
ztest_zgd_private_t *zgd_private;
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) {
@ -2186,7 +2191,7 @@ ztest_get_data(void *arg, lr_write_t *lr, char *buf, zio_t *zio)
db = NULL;
zgd = umem_zalloc(sizeof (*zgd), UMEM_NOFAIL);
zgd->zgd_zilog = zd->zd_zilog;
zgd->zgd_lwb = lwb;
zgd_private = umem_zalloc(sizeof (ztest_zgd_private_t), UMEM_NOFAIL);
zgd_private->z_zd = zd;
zgd_private->z_object = object;

View File

@ -982,7 +982,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;

View File

@ -33,89 +33,181 @@
#include <linux/tracepoint.h>
#include <sys/types.h>
#define ZILOG_TP_STRUCT_ENTRY \
__field(uint64_t, zl_lr_seq) \
__field(uint64_t, zl_commit_lr_seq) \
__field(uint64_t, zl_destroy_txg) \
__field(uint64_t, zl_replaying_seq) \
__field(uint32_t, zl_suspend) \
__field(uint8_t, zl_suspending) \
__field(uint8_t, zl_keep_first) \
__field(uint8_t, zl_replay) \
__field(uint8_t, zl_stop_sync) \
__field(uint8_t, zl_logbias) \
__field(uint8_t, zl_sync) \
__field(int, zl_parse_error) \
__field(uint64_t, zl_parse_blk_seq) \
__field(uint64_t, zl_parse_lr_seq) \
__field(uint64_t, zl_parse_blk_count) \
__field(uint64_t, zl_parse_lr_count) \
__field(uint64_t, zl_cur_used) \
__field(clock_t, zl_replay_time) \
__field(uint64_t, zl_replay_blks)
#define ZILOG_TP_FAST_ASSIGN \
__entry->zl_lr_seq = zilog->zl_lr_seq; \
__entry->zl_commit_lr_seq = zilog->zl_commit_lr_seq; \
__entry->zl_destroy_txg = zilog->zl_destroy_txg; \
__entry->zl_replaying_seq = zilog->zl_replaying_seq; \
__entry->zl_suspend = zilog->zl_suspend; \
__entry->zl_suspending = zilog->zl_suspending; \
__entry->zl_keep_first = zilog->zl_keep_first; \
__entry->zl_replay = zilog->zl_replay; \
__entry->zl_stop_sync = zilog->zl_stop_sync; \
__entry->zl_logbias = zilog->zl_logbias; \
__entry->zl_sync = zilog->zl_sync; \
__entry->zl_parse_error = zilog->zl_parse_error; \
__entry->zl_parse_blk_seq = zilog->zl_parse_blk_seq; \
__entry->zl_parse_lr_seq = zilog->zl_parse_lr_seq; \
__entry->zl_parse_blk_count = zilog->zl_parse_blk_count;\
__entry->zl_parse_lr_count = zilog->zl_parse_lr_count; \
__entry->zl_cur_used = zilog->zl_cur_used; \
__entry->zl_replay_time = zilog->zl_replay_time; \
__entry->zl_replay_blks = zilog->zl_replay_blks;
#define ZILOG_TP_PRINTK_FMT \
"zl { lr_seq %llu commit_lr_seq %llu destroy_txg %llu " \
"replaying_seq %llu suspend %u suspending %u keep_first %u " \
"replay %u stop_sync %u logbias %u sync %u " \
"parse_error %u parse_blk_seq %llu parse_lr_seq %llu " \
"parse_blk_count %llu parse_lr_count %llu " \
"cur_used %llu replay_time %lu replay_blks %llu }"
#define ZILOG_TP_PRINTK_ARGS \
__entry->zl_lr_seq, __entry->zl_commit_lr_seq, \
__entry->zl_destroy_txg, __entry->zl_replaying_seq, \
__entry->zl_suspend, __entry->zl_suspending, \
__entry->zl_keep_first, __entry->zl_replay, \
__entry->zl_stop_sync, __entry->zl_logbias, __entry->zl_sync, \
__entry->zl_parse_error, __entry->zl_parse_blk_seq, \
__entry->zl_parse_lr_seq, __entry->zl_parse_blk_count, \
__entry->zl_parse_lr_count, __entry->zl_cur_used, \
__entry->zl_replay_time, __entry->zl_replay_blks
#define ITX_TP_STRUCT_ENTRY \
__field(itx_wr_state_t, itx_wr_state) \
__field(uint8_t, itx_sync) \
__field(zil_callback_t, itx_callback) \
__field(void *, itx_callback_data) \
__field(uint64_t, itx_oid) \
\
__field(uint64_t, lrc_txtype) \
__field(uint64_t, lrc_reclen) \
__field(uint64_t, lrc_txg) \
__field(uint64_t, lrc_seq)
#define ITX_TP_FAST_ASSIGN \
__entry->itx_wr_state = itx->itx_wr_state; \
__entry->itx_sync = itx->itx_sync; \
__entry->itx_callback = itx->itx_callback; \
__entry->itx_callback_data = itx->itx_callback_data; \
__entry->itx_oid = itx->itx_oid; \
\
__entry->lrc_txtype = itx->itx_lr.lrc_txtype; \
__entry->lrc_reclen = itx->itx_lr.lrc_reclen; \
__entry->lrc_txg = itx->itx_lr.lrc_txg; \
__entry->lrc_seq = itx->itx_lr.lrc_seq;
#define ITX_TP_PRINTK_FMT \
"itx { wr_state %u sync %u callback %p callback_data %p oid %llu" \
" { txtype %llu reclen %llu txg %llu seq %llu } }"
#define ITX_TP_PRINTK_ARGS \
__entry->itx_wr_state, __entry->itx_sync, __entry->itx_callback,\
__entry->itx_callback_data, __entry->itx_oid, \
__entry->lrc_txtype, __entry->lrc_reclen, __entry->lrc_txg, \
__entry->lrc_seq
#define ZCW_TP_STRUCT_ENTRY \
__field(lwb_t *, zcw_lwb) \
__field(boolean_t, zcw_done) \
__field(int, zcw_zio_error) \
#define ZCW_TP_FAST_ASSIGN \
__entry->zcw_lwb = zcw->zcw_lwb; \
__entry->zcw_done = zcw->zcw_done; \
__entry->zcw_zio_error = zcw->zcw_zio_error;
#define ZCW_TP_PRINTK_FMT \
"zcw { lwb %p done %u error %u }"
#define ZCW_TP_PRINTK_ARGS \
__entry->zcw_lwb, __entry->zcw_done, __entry->zcw_zio_error
/*
* Generic support for one argument tracepoints of the form:
* Generic support for two argument tracepoints of the form:
*
* DTRACE_PROBE1(...,
* zilog_t *, ...);
* DTRACE_PROBE2(...,
* zilog_t *, ...,
* itx_t *, ...);
*/
/* BEGIN CSTYLED */
DECLARE_EVENT_CLASS(zfs_zil_class,
TP_PROTO(zilog_t *zilog),
TP_ARGS(zilog),
DECLARE_EVENT_CLASS(zfs_zil_process_itx_class,
TP_PROTO(zilog_t *zilog, itx_t *itx),
TP_ARGS(zilog, itx),
TP_STRUCT__entry(
__field(uint64_t, zl_lr_seq)
__field(uint64_t, zl_commit_lr_seq)
__field(uint64_t, zl_destroy_txg)
__field(uint64_t, zl_replaying_seq)
__field(uint32_t, zl_suspend)
__field(uint8_t, zl_suspending)
__field(uint8_t, zl_keep_first)
__field(uint8_t, zl_replay)
__field(uint8_t, zl_stop_sync)
__field(uint8_t, zl_writer)
__field(uint8_t, zl_logbias)
__field(uint8_t, zl_sync)
__field(int, zl_parse_error)
__field(uint64_t, zl_parse_blk_seq)
__field(uint64_t, zl_parse_lr_seq)
__field(uint64_t, zl_parse_blk_count)
__field(uint64_t, zl_parse_lr_count)
__field(uint64_t, zl_next_batch)
__field(uint64_t, zl_com_batch)
__field(uint64_t, zl_cur_used)
__field(clock_t, zl_replay_time)
__field(uint64_t, zl_replay_blks)
ZILOG_TP_STRUCT_ENTRY
ITX_TP_STRUCT_ENTRY
),
TP_fast_assign(
__entry->zl_lr_seq = zilog->zl_lr_seq;
__entry->zl_commit_lr_seq = zilog->zl_commit_lr_seq;
__entry->zl_destroy_txg = zilog->zl_destroy_txg;
__entry->zl_replaying_seq = zilog->zl_replaying_seq;
__entry->zl_suspend = zilog->zl_suspend;
__entry->zl_suspending = zilog->zl_suspending;
__entry->zl_keep_first = zilog->zl_keep_first;
__entry->zl_replay = zilog->zl_replay;
__entry->zl_stop_sync = zilog->zl_stop_sync;
__entry->zl_writer = zilog->zl_writer;
__entry->zl_logbias = zilog->zl_logbias;
__entry->zl_sync = zilog->zl_sync;
__entry->zl_parse_error = zilog->zl_parse_error;
__entry->zl_parse_blk_seq = zilog->zl_parse_blk_seq;
__entry->zl_parse_lr_seq = zilog->zl_parse_lr_seq;
__entry->zl_parse_blk_count = zilog->zl_parse_blk_count;
__entry->zl_parse_lr_count = zilog->zl_parse_lr_count;
__entry->zl_next_batch = zilog->zl_next_batch;
__entry->zl_com_batch = zilog->zl_com_batch;
__entry->zl_cur_used = zilog->zl_cur_used;
__entry->zl_replay_time = zilog->zl_replay_time;
__entry->zl_replay_blks = zilog->zl_replay_blks;
ZILOG_TP_FAST_ASSIGN
ITX_TP_FAST_ASSIGN
),
TP_printk("zl { lr_seq %llu commit_lr_seq %llu destroy_txg %llu "
"replaying_seq %llu suspend %u suspending %u keep_first %u "
"replay %u stop_sync %u writer %u logbias %u sync %u "
"parse_error %u parse_blk_seq %llu parse_lr_seq %llu "
"parse_blk_count %llu parse_lr_count %llu next_batch %llu "
"com_batch %llu cur_used %llu replay_time %lu replay_blks %llu }",
__entry->zl_lr_seq, __entry->zl_commit_lr_seq,
__entry->zl_destroy_txg, __entry->zl_replaying_seq,
__entry->zl_suspend, __entry->zl_suspending, __entry->zl_keep_first,
__entry->zl_replay, __entry->zl_stop_sync, __entry->zl_writer,
__entry->zl_logbias, __entry->zl_sync, __entry->zl_parse_error,
__entry->zl_parse_blk_seq, __entry->zl_parse_lr_seq,
__entry->zl_parse_blk_count, __entry->zl_parse_lr_count,
__entry->zl_next_batch, __entry->zl_com_batch, __entry->zl_cur_used,
__entry->zl_replay_time, __entry->zl_replay_blks)
TP_printk(
ZILOG_TP_PRINTK_FMT " " ITX_TP_PRINTK_FMT,
ZILOG_TP_PRINTK_ARGS, ITX_TP_PRINTK_ARGS)
);
/* END CSTYLED */
/* BEGIN CSTYLED */
#define DEFINE_ZIL_EVENT(name) \
DEFINE_EVENT(zfs_zil_class, name, \
TP_PROTO(zilog_t *zilog), \
TP_ARGS(zilog))
DEFINE_ZIL_EVENT(zfs_zil__cw1);
DEFINE_ZIL_EVENT(zfs_zil__cw2);
#define DEFINE_ZIL_PROCESS_ITX_EVENT(name) \
DEFINE_EVENT(zfs_zil_process_itx_class, name, \
TP_PROTO(zilog_t *zilog, itx_t *itx), \
TP_ARGS(zilog, itx))
DEFINE_ZIL_PROCESS_ITX_EVENT(zfs_zil__process__commit__itx);
DEFINE_ZIL_PROCESS_ITX_EVENT(zfs_zil__process__normal__itx);
/* END CSTYLED */
/*
* Generic support for two argument tracepoints of the form:
*
* DTRACE_PROBE2(...,
* zilog_t *, ...,
* zil_commit_waiter_t *, ...);
*/
/* BEGIN CSTYLED */
DECLARE_EVENT_CLASS(zfs_zil_commit_io_error_class,
TP_PROTO(zilog_t *zilog, zil_commit_waiter_t *zcw),
TP_ARGS(zilog, zcw),
TP_STRUCT__entry(
ZILOG_TP_STRUCT_ENTRY
ZCW_TP_STRUCT_ENTRY
),
TP_fast_assign(
ZILOG_TP_FAST_ASSIGN
ZCW_TP_FAST_ASSIGN
),
TP_printk(
ZILOG_TP_PRINTK_FMT " " ZCW_TP_PRINTK_FMT,
ZILOG_TP_PRINTK_ARGS, ZCW_TP_PRINTK_ARGS)
);
/* BEGIN CSTYLED */
#define DEFINE_ZIL_COMMIT_IO_ERROR_EVENT(name) \
DEFINE_EVENT(zfs_zil_commit_io_error_class, name, \
TP_PROTO(zilog_t *zilog, zil_commit_waiter_t *zcw), \
TP_ARGS(zilog, zcw))
DEFINE_ZIL_COMMIT_IO_ERROR_EVENT(zfs_zil__commit__io__error);
/* END CSTYLED */
#endif /* _TRACE_ZIL_H */

View File

@ -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 */
@ -465,7 +467,8 @@ typedef int zil_parse_blk_func_t(zilog_t *zilog, blkptr_t *bp, void *arg,
typedef int zil_parse_lr_func_t(zilog_t *zilog, lr_t *lr, void *arg,
uint64_t txg);
typedef int zil_replay_func_t(void *arg1, void *arg2, boolean_t byteswap);
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,
@ -503,7 +506,8 @@ extern void zil_clean(zilog_t *zilog, uint64_t synced_txg);
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);

View File

@ -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.
*/
/* Portions Copyright 2010 Robert Milkowski */
@ -36,7 +36,30 @@ 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 */
@ -45,13 +68,45 @@ typedef struct lwb {
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_itxs; /* list of itx's */
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
*/
@ -94,20 +149,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 +170,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 +181,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 {

View File

@ -596,6 +596,7 @@ extern enum zio_checksum zio_checksum_dedup_select(spa_t *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);

View File

@ -76,6 +76,7 @@ do { \
(u_longlong_t)__left, #OP, (u_longlong_t)__right); \
} while (0)
#define VERIFY3B(x, y, z) VERIFY3_IMPL(x, y, z, boolean_t)
#define VERIFY3S(x, y, z) VERIFY3_IMPL(x, y, z, int64_t)
#define VERIFY3U(x, y, z) VERIFY3_IMPL(x, y, z, uint64_t)
#define VERIFY3P(x, y, z) VERIFY3_IMPL(x, y, z, uintptr_t)
@ -94,6 +95,7 @@ do { \
__compile_time_assertion__ ## y[(x) ? 1 : -1]
#ifdef NDEBUG
#define ASSERT3B(x, y, z) ((void)0)
#define ASSERT3S(x, y, z) ((void)0)
#define ASSERT3U(x, y, z) ((void)0)
#define ASSERT3P(x, y, z) ((void)0)
@ -104,6 +106,7 @@ do { \
#define IMPLY(A, B) ((void)0)
#define EQUIV(A, B) ((void)0)
#else
#define ASSERT3B(x, y, z) VERIFY3B(x, y, z)
#define ASSERT3S(x, y, z) VERIFY3S(x, y, z)
#define ASSERT3U(x, y, z) VERIFY3U(x, y, z)
#define ASSERT3P(x, y, z) VERIFY3P(x, y, z)

View File

@ -58,6 +58,14 @@
#define NSEC2MSEC(n) ((n) / (NANOSEC / MILLISEC))
#endif
#ifndef USEC2NSEC
#define USEC2NSEC(m) ((hrtime_t)(m) * (NANOSEC / MICROSEC))
#endif
#ifndef NSEC2USEC
#define NSEC2USEC(n) ((n) / (NANOSEC / MICROSEC))
#endif
#ifndef NSEC2SEC
#define NSEC2SEC(n) ((n) / (NANOSEC / SEC))
#endif

View File

@ -1783,6 +1783,13 @@ dmu_sync_late_arrival(zio_t *pio, objset_t *os, dmu_sync_cb_t *done, zgd_t *zgd,
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;

View File

@ -160,7 +160,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);
@ -201,7 +201,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;
@ -263,7 +263,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.
@ -275,7 +275,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;
@ -648,7 +648,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)
@ -673,7 +673,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)

View File

@ -78,6 +78,7 @@
#include <sys/cred.h>
#include <sys/attr.h>
#include <sys/zpl.h>
#include <sys/zil.h>
/*
* Programming rules.
@ -983,7 +984,7 @@ zfs_get_done(zgd_t *zgd, int error)
zfs_iput_async(ZTOI(zp));
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));
}
@ -996,7 +997,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;
@ -1008,8 +1009,9 @@ zfs_get_data(void *arg, lr_write_t *lr, char *buf, zio_t *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
@ -1026,7 +1028,7 @@ zfs_get_data(void *arg, lr_write_t *lr, char *buf, zio_t *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;
/*

File diff suppressed because it is too large Load Diff

View File

@ -568,7 +568,7 @@ zio_add_child(zio_t *pio, zio_t *cio)
* Vdev I/Os can only have vdev children.
* The following ASSERT captures all of these constraints.
*/
ASSERT(cio->io_child_type <= pio->io_child_type);
ASSERT3S(cio->io_child_type, <=, pio->io_child_type);
zl->zl_parent = pio;
zl->zl_child = cio;
@ -1281,9 +1281,9 @@ zio_flush(zio_t *zio, vdev_t *vd)
void
zio_shrink(zio_t *zio, uint64_t size)
{
ASSERT(zio->io_executor == NULL);
ASSERT(zio->io_orig_size == zio->io_size);
ASSERT(size <= zio->io_size);
ASSERT3P(zio->io_executor, ==, NULL);
ASSERT3U(zio->io_orig_size, ==, zio->io_size);
ASSERT3U(size, <=, zio->io_size);
/*
* We don't shrink for raidz because of problems with the
@ -1877,8 +1877,8 @@ zio_wait(zio_t *zio)
{
int error;
ASSERT(zio->io_stage == ZIO_STAGE_OPEN);
ASSERT(zio->io_executor == NULL);
ASSERT3S(zio->io_stage, ==, ZIO_STAGE_OPEN);
ASSERT3P(zio->io_executor, ==, NULL);
zio->io_waiter = curthread;
ASSERT0(zio->io_queued_timestamp);
@ -1900,7 +1900,7 @@ zio_wait(zio_t *zio)
void
zio_nowait(zio_t *zio)
{
ASSERT(zio->io_executor == NULL);
ASSERT3P(zio->io_executor, ==, NULL);
if (zio->io_child_type == ZIO_CHILD_LOGICAL &&
zio_unique_parent(zio) == NULL) {
@ -1926,7 +1926,7 @@ zio_nowait(zio_t *zio)
/*
* ==========================================================================
* Reexecute or suspend/resume failed I/O
* Reexecute, cancel, or suspend/resume failed I/O
* ==========================================================================
*/
@ -1983,6 +1983,20 @@ zio_reexecute(zio_t *pio)
}
}
void
zio_cancel(zio_t *zio)
{
/*
* Disallow cancellation of a zio that's already been issued.
*/
VERIFY3P(zio->io_executor, ==, NULL);
zio->io_pipeline = ZIO_INTERLOCK_PIPELINE;
zio->io_done = NULL;
zio_nowait(zio);
}
void
zio_suspend(spa_t *spa, zio_t *zio)
{
@ -3276,6 +3290,9 @@ zio_alloc_zil(spa_t *spa, objset_t *os, uint64_t txg, blkptr_t *new_bp,
zio_crypt_encode_params_bp(new_bp, salt, iv);
}
} else {
zfs_dbgmsg("%s: zil block allocation failure: "
"size %llu, error %d", spa_name(spa), size, error);
}
return (error);

View File

@ -1052,7 +1052,7 @@ zvol_get_done(zgd_t *zgd, int error)
zfs_range_unlock(zgd->zgd_rl);
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));
}
@ -1061,7 +1061,7 @@ zvol_get_done(zgd_t *zgd, int error)
* Get data to generate a TX_WRITE intent log record.
*/
static int
zvol_get_data(void *arg, lr_write_t *lr, char *buf, zio_t *zio)
zvol_get_data(void *arg, lr_write_t *lr, char *buf, struct lwb *lwb, zio_t *zio)
{
zvol_state_t *zv = arg;
uint64_t offset = lr->lr_offset;
@ -1070,11 +1070,12 @@ zvol_get_data(void *arg, lr_write_t *lr, char *buf, zio_t *zio)
zgd_t *zgd;
int error;
ASSERT(zio != NULL);
ASSERT(size != 0);
ASSERT3P(lwb, !=, NULL);
ASSERT3P(zio, !=, NULL);
ASSERT3U(size, !=, 0);
zgd = (zgd_t *)kmem_zalloc(sizeof (zgd_t), KM_SLEEP);
zgd->zgd_zilog = zv->zv_zilog;
zgd->zgd_lwb = lwb;
/*
* Write records come in two flavors: immediate and indirect.

View File

@ -254,7 +254,7 @@ while [[ $timeout -eq 0 ]] || [[ $curtime -le $((starttime + timeout)) ]]; do
$cmd >>ztest.out 2>&1
ztrc=$?
egrep '===|WARNING' ztest.out >>ztest.history
$ZDB -U "$workdir/zpool.cache" -DD ztest >>ztest.ddt
$ZDB -U "$workdir/zpool.cache" -DD ztest >>ztest.ddt 2>&1
store_core