From 2724bcb3d6ed2621fe7f0c8da841735b731d9e6a Mon Sep 17 00:00:00 2001 From: Rob Norris Date: Wed, 10 May 2023 15:44:40 +1000 Subject: [PATCH] zil: allow the ZIL to fail and restart independently of the pool zil_commit() has always returned void, and thus, cannot fail. Everything inside it assumed that if anything ever went wrong, it could fall back on txg_wait_synced() until the txg covering the operations being flushed from the ZIL has fully committed. This meant that if the pool failed and failmode=continue was set, syncing operations like fsync() would still block. Unblocking zil_commit() means largely the same approach. The difficulty is that the ZIL carries the record of uncommitted VFS operations (vs the changed data), and attached to those, callbacks and cvs that will release userspace callers once the data is on disk. So if we can't write the ZIL, we also can't release those records until the data is on disk. This wasn't a problem before, because the zil_commit() would block. If we change zil_commit() to return error, we still need to track those entries until the data they represent hits the disk. We also need to accept new records; just because the ZIL fails may not necessarily mean the pool itself is unavailable. This commit reorganises the ZIL to allow zil_commit() to return failure. If ZIL writes or flushes fail, the ZIL is moved into a "failed" state, and no further writes are done; all zil_commit() calls are serviced by the regular txg mechanism. Outstanding records (itx_ts) are held until the main pool writes their associated txg out. The records are then released. Once all records are cleared, the ZIL is reset and reopened. Signed-off-by: Rob Norris (cherry picked from commit af821006f6602261e690fe6635689cabdeefcadf) --- include/sys/zil.h | 4 +- include/sys/zil_impl.h | 3 + module/os/linux/zfs/zfs_vfsops.c | 19 +- module/zfs/dmu.c | 3 +- module/zfs/zfs_vnops.c | 5 +- module/zfs/zil.c | 919 +++++++++++++++++++++++++++---- 6 files changed, 833 insertions(+), 120 deletions(-) diff --git a/include/sys/zil.h b/include/sys/zil.h index cefbccb32f..b5d65ab95c 100644 --- a/include/sys/zil.h +++ b/include/sys/zil.h @@ -495,8 +495,8 @@ extern void zil_itx_destroy(itx_t *itx); extern void zil_itx_assign(zilog_t *zilog, itx_t *itx, dmu_tx_t *tx); extern void zil_async_to_sync(zilog_t *zilog, uint64_t oid); -extern void zil_commit(zilog_t *zilog, uint64_t oid); -extern void zil_commit_impl(zilog_t *zilog, uint64_t oid); +extern int zil_commit(zilog_t *zilog, uint64_t oid); +extern int zil_commit_impl(zilog_t *zilog, uint64_t oid); extern void zil_remove_async(zilog_t *zilog, uint64_t oid); extern int zil_reset(const char *osname, void *txarg); diff --git a/include/sys/zil_impl.h b/include/sys/zil_impl.h index d2f4018653..6d43035fe9 100644 --- a/include/sys/zil_impl.h +++ b/include/sys/zil_impl.h @@ -146,6 +146,7 @@ typedef struct itxg { kmutex_t itxg_lock; /* lock for this structure */ uint64_t itxg_txg; /* txg for this chain */ itxs_t *itxg_itxs; /* sync and async itxs */ + boolean_t itxg_failed; /* ZIL failed, don't touch */ } itxg_t; /* for async nodes we build up an AVL tree of lists of async itxs per file */ @@ -198,6 +199,8 @@ struct zilog { uint64_t zl_parse_blk_count; /* number of blocks parsed */ uint64_t zl_parse_lr_count; /* number of log records parsed */ itxg_t zl_itxg[TXG_SIZE]; /* intent log txg chains */ + itxg_t zl_fail_itxg; /* holding space for failed itxs */ + uint64_t zl_unfail_txg; /* txg to unfail ZIL at */ 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 */ diff --git a/module/os/linux/zfs/zfs_vfsops.c b/module/os/linux/zfs/zfs_vfsops.c index b8fbff0ff0..28df811bc9 100644 --- a/module/os/linux/zfs/zfs_vfsops.c +++ b/module/os/linux/zfs/zfs_vfsops.c @@ -280,18 +280,29 @@ zfs_sync(struct super_block *sb, int wait, cred_t *cr) dp = dmu_objset_pool(zfsvfs->z_os); /* - * If the system is shutting down, then skip any - * filesystems which may exist on a suspended pool. + * If the system is shutting down, then skip any filesystems + * which may exist on a suspended pool. We don't do this if + * failmode=continue becase zil_commit might have a better + * error for us. */ - if (spa_suspended(dp->dp_spa)) { + if (spa_suspended(dp->dp_spa) && + spa_get_failmode(dp->dp_spa) != ZIO_FAILURE_MODE_CONTINUE) { ZFS_EXIT(zfsvfs); return (0); } + /* + * If there's a ZIL, try to flush it. If the pool is in some + * unflushable state, this will get us an approprate error + * return. + */ + int err = 0; if (zfsvfs->z_log != NULL) - zil_commit(zfsvfs->z_log, 0); + err = zil_commit(zfsvfs->z_log, 0); ZFS_EXIT(zfsvfs); + + return (err); } else { /* * Sync all ZFS filesystems. This is what happens when you diff --git a/module/zfs/dmu.c b/module/zfs/dmu.c index 0592e6a303..25923417cf 100644 --- a/module/zfs/dmu.c +++ b/module/zfs/dmu.c @@ -1648,7 +1648,8 @@ dmu_sync_late_arrival(zio_t *pio, objset_t *os, dmu_sync_cb_t *done, zgd_t *zgd, tx = dmu_tx_create(os); dmu_tx_hold_space(tx, zgd->zgd_db->db_size); - if (dmu_tx_assign(tx, DMU_TX_ASSIGN_WAIT) != 0) { + if (dmu_tx_assign(tx, + DMU_TX_ASSIGN_WAIT | DMU_TX_ASSIGN_CONTINUE) != 0) { dmu_tx_abort(tx); /* Make zl_get_data do txg_waited_synced() */ return (SET_ERROR(EIO)); diff --git a/module/zfs/zfs_vnops.c b/module/zfs/zfs_vnops.c index 77740e59fb..f00021a298 100644 --- a/module/zfs/zfs_vnops.c +++ b/module/zfs/zfs_vnops.c @@ -61,6 +61,7 @@ static ulong_t zfs_fsync_sync_cnt = 4; int zfs_fsync(znode_t *zp, int syncflag, cred_t *cr) { + int err = 0; zfsvfs_t *zfsvfs = ZTOZSB(zp); if (zfsvfs->z_os->os_sync == ZFS_SYNC_DISABLED) @@ -71,13 +72,13 @@ zfs_fsync(znode_t *zp, int syncflag, cred_t *cr) (void) tsd_set(zfs_fsyncer_key, (void *)zfs_fsync_sync_cnt); - zil_commit(zfsvfs->z_log, zp->z_id); + err = zil_commit(zfsvfs->z_log, zp->z_id); tsd_set(zfs_fsyncer_key, NULL); ZFS_EXIT(zfsvfs); - return (0); + return (err); } diff --git a/module/zfs/zil.c b/module/zfs/zil.c index efb8c8761f..bf8a6b29bb 100644 --- a/module/zfs/zil.c +++ b/module/zfs/zil.c @@ -583,6 +583,8 @@ zil_alloc_lwb(zilog_t *zilog, blkptr_t *bp, boolean_t slog, uint64_t txg, return (lwb); } +static boolean_t zil_failed(zilog_t *zilog); + static void zil_free_lwb(zilog_t *zilog, lwb_t *lwb) { @@ -593,9 +595,16 @@ zil_free_lwb(zilog_t *zilog, lwb_t *lwb) ASSERT(avl_is_empty(&lwb->lwb_vdev_tree)); ASSERT3P(lwb->lwb_write_zio, ==, NULL); ASSERT3P(lwb->lwb_root_zio, ==, NULL); - ASSERT3U(lwb->lwb_max_txg, <=, spa_syncing_txg(zilog->zl_spa)); - ASSERT(lwb->lwb_state == LWB_STATE_CLOSED || - lwb->lwb_state == LWB_STATE_FLUSH_DONE); + + /* + * If we can't write, then we could be in zil_close() destroying + * unissued lwbs; in that case, don't assert that they're completed. + */ + if (!(zil_failed(zilog) || spa_exiting_any(zilog->zl_spa))) { + ASSERT3U(lwb->lwb_max_txg, <=, spa_syncing_txg(zilog->zl_spa)); + ASSERT(lwb->lwb_state == LWB_STATE_CLOSED || + lwb->lwb_state == LWB_STATE_FLUSH_DONE); + } /* * Clear the zilog's field to indicate this lwb is no longer @@ -681,7 +690,12 @@ zil_create(zilog_t *zilog) /* * Wait for any previous destroy to complete. */ - txg_wait_synced(zilog->zl_dmu_pool, zilog->zl_destroy_txg); + error = txg_wait_synced_flags(zilog->zl_dmu_pool, zilog->zl_destroy_txg, + TXG_WAIT_F_NOSUSPEND); + if (error != 0) { + ASSERT3S(error, ==, EAGAIN); + return (NULL); + } ASSERT(zh->zh_claim_txg == 0); ASSERT(zh->zh_replay_seq == 0); @@ -695,9 +709,10 @@ zil_create(zilog_t *zilog) */ if (BP_IS_HOLE(&blk) || BP_SHOULD_BYTESWAP(&blk)) { tx = dmu_tx_create(zilog->zl_os); - error = dmu_tx_assign(tx, DMU_TX_ASSIGN_WAIT); + error = dmu_tx_assign(tx, DMU_TX_ASSIGN_WAIT | + DMU_TX_ASSIGN_NOSUSPEND); if (error != 0) { - ASSERT(dmu_objset_exiting(zilog->zl_os)); + ASSERT3S(error, ==, EAGAIN); dmu_tx_abort(tx); return (NULL); } @@ -730,7 +745,27 @@ zil_create(zilog_t *zilog) */ if (tx != NULL) { dmu_tx_commit(tx); - txg_wait_synced(zilog->zl_dmu_pool, txg); + error = txg_wait_synced_flags(zilog->zl_dmu_pool, txg, + TXG_WAIT_F_NOSUSPEND); + if (error != 0) { + /* + * Pool suspended at the last moment, so we have to + * unwind everything we've just done. + */ + ASSERT3S(error, ==, EAGAIN); + + metaslab_fastwrite_unmark(zilog->zl_spa, &lwb->lwb_blk); + + mutex_enter(&zilog->zl_lock); + list_remove(&zilog->zl_lwb_list, lwb); + mutex_exit(&zilog->zl_lock); + + zio_buf_free(lwb->lwb_buf, lwb->lwb_sz); + zio_free(zilog->zl_spa, txg, &lwb->lwb_blk); + zil_free_lwb(zilog, lwb); + + return (NULL); + } } ASSERT(error != 0 || bcmp(&blk, &zh->zh_log, sizeof (blk)) == 0); @@ -758,7 +793,9 @@ zil_destroy(zilog_t *zilog, boolean_t keep_first) int error; /* - * Wait for any previous destroy to complete. + * Wait for any previous destroy to complete. Its ok to block here, + * as this is always arrived at via administrative operations, never + * appication IO. Its also not obvious how we'd recover. */ txg_wait_synced(zilog->zl_dmu_pool, zilog->zl_destroy_txg); @@ -991,6 +1028,282 @@ zil_check_log_chain(dsl_pool_t *dp, dsl_dataset_t *ds, void *tx) return ((error == ECKSUM || error == ENOENT) ? 0 : error); } +/* + * True if the ZIL has failed. + */ +static boolean_t +zil_failed(zilog_t *zilog) +{ + return ((zilog->zl_unfail_txg > 0) ? B_TRUE : B_FALSE); +} + +static void zil_commit_waiter_skip(zil_commit_waiter_t *zcw); +static itxs_t *zil_alloc_itxs(void); + +/* + * Fail the ZIL. This will collect up all failed itxs, and note that the ZIL + * cannot be unfailed until all their txgs are synced. After this, all + * zil_commit() calls will be serviced by the regular txg sync, and no new itxs + * will be assigned or committed. + */ +static void +zil_fail(zilog_t *zilog) +{ + ASSERT(MUTEX_HELD(&zilog->zl_issuer_lock)); + ASSERT(MUTEX_HELD(&zilog->zl_lock)); + ASSERT(!zil_failed(zilog)); + + itx_t *itx; + list_t *l; + + list_t waiters; + list_create(&waiters, sizeof (zil_commit_waiter_t), + offsetof(zil_commit_waiter_t, zcw_node)); + + /* + * We have to take the namespace lock to prevent the txg being moved + * forward while we're processing the itxgs. + */ + mutex_enter(&spa_namespace_lock); + + uint64_t last_synced_txg = spa_last_synced_txg(zilog->zl_spa); + + /* + * A ZIL failure occurs when an LWB write or flush fails, or an LWB + * can't be issued. This usually occurs when the pool suspends during + * zil_commit(). + * + * The LWBs and the ZIL proper have outstanding itxs attached to them, + * with callbacks on them to be called when the data is safely on disk. + * If the pool never returns it doesn't matter; that data is never + * going to disk anyway. If the pool does return, and the txg covering + * those itxs is committed, then we need to fire those callbacks to + * make sure the rest of the system knows that its data is properly on + * disk. + * + * Outstandng itxs exist in in the live LWB list. The tail LWB is open, + * any others have issued IO. Once we set failure, zil_lwb_write_done() + * and zil_lwb_flush_vdevs_done() will quietly ignore any responses + * from live IO, so these LWBs will not progress any further. There may + * also be itxs on the commit list, that have been moved out of + * zl_itxg[] but not yet assigned to an itx. + * + * We collect up all those outstanding itxs, and put them onto + * fail_itxg. As we take each itx, we check its txg, and record the + * highest one we see. In the future, when that highest txg is + * committed, we know that all outstanding work is completed and the + * ZIL can be unfailed. See zil_sync() for details. + * + * In addition, each LWB has waiters attached, which threads are + * currently waiting on for zil_commit() to complete. We gather those + * so we can fire them via zil_commit_waiter_skip() after the unfail + * txg has been set. The first thing they'll do when they wake is to + * check the fail state, so we need to be done with our work before + * that. + */ + + uint64_t highest_txg = last_synced_txg; + + /* + * Prepare the fail itxg. This is not a real itxg, just a convenient + * holder for the itxs that couldn't be written and associated metadata + * until their transaction is committed and we can fire their + * callbacks. + */ + itxg_t *fail_itxg = &zilog->zl_fail_itxg; + mutex_enter(&fail_itxg->itxg_lock); + + ASSERT3U(fail_itxg->itxg_txg, ==, 0); + ASSERT3P(fail_itxg->itxg_itxs, ==, NULL); + + fail_itxg->itxg_itxs = zil_alloc_itxs(); + + /* + * Unlike the live itxgs, fail_itxg can carry itxs from multiple txgs, + * as zil_commit will collect all pending changes for a given foid, + * regardless of which txg its on. To handle this, we set its itxg_txg + * to the lowest txg of all the itxs it holds, as a cheap reference for + * what's left to process. + */ + fail_itxg->itxg_txg = UINT64_MAX; + + /* Process failed LWBs, oldest first */ + for (lwb_t *lwb = list_head(&zilog->zl_lwb_list); lwb; + lwb = list_next(&zilog->zl_lwb_list, lwb)) { + /* + * All transactions for unissued or completed LWBs should be + * aborted by the time we get here. Conversely, transactions + * on in-flight LWBs will still be live, to be aborted in + * zil_lwb_flush_vdevs_done(). + */ + IMPLY(lwb->lwb_state == LWB_STATE_CLOSED, lwb->lwb_tx == NULL); + IMPLY(lwb->lwb_state == LWB_STATE_OPENED, lwb->lwb_tx == NULL); + IMPLY(lwb->lwb_state == LWB_STATE_ISSUED, lwb->lwb_tx != NULL); + IMPLY(lwb->lwb_state == LWB_STATE_WRITE_DONE, + lwb->lwb_tx != NULL); + IMPLY(lwb->lwb_state == LWB_STATE_FLUSH_DONE, + lwb->lwb_tx == NULL); + + l = &lwb->lwb_itxs; + while ((itx = list_head(l)) != NULL) { + list_remove(l, itx); + if (itx->itx_lr.lrc_txtype == TX_COMMIT) { + /* + * zil_process_commit_list() already put the + * zcw for commit itxs on the waiter list, so + * we don't need to do anything special here. + */ + zil_itx_destroy(itx); + } else if (itx->itx_lr.lrc_txg <= last_synced_txg) { + /* + * txg covering this itx is alread on the main + * pool, so we can safely complete it now. + */ + zil_itx_destroy(itx); + } else { + /* + * Put the itx on the fail itxg sync list, + * and bump the unfail txg. + */ + list_insert_tail( + &fail_itxg->itxg_itxs->i_sync_list, itx); + fail_itxg->itxg_txg = MIN(fail_itxg->itxg_txg, + itx->itx_lr.lrc_txg); + + highest_txg = + MAX(highest_txg, itx->itx_lr.lrc_txg); + } + } + + /* Take any waiters */ + list_move_tail(&waiters, &lwb->lwb_waiters); + + ASSERT(list_is_empty(&lwb->lwb_itxs)); + ASSERT(list_is_empty(&lwb->lwb_waiters)); + } + + /* + * Failures in zil_process_commit_list() will leave stuff on the + * commit list; bring those over too. + */ + l = &zilog->zl_itx_commit_list; + while ((itx = list_head(l)) != NULL) { + list_remove(l, itx); + if (itx->itx_lr.lrc_txtype == TX_COMMIT) { + /* Take commit waiters */ + list_insert_tail(&waiters, itx->itx_private); + itx->itx_private = NULL; + zil_itx_destroy(itx); + } else if (itx->itx_lr.lrc_txg <= last_synced_txg) { + /* + * txg covering this itx is alread on the main pool, so + * we can safely complete it now. + */ + zil_itx_destroy(itx); + } else { + + /* + * Put the itx on the fail itxg sync list, and bump the + * unfail txg. + */ + list_insert_tail( + &fail_itxg->itxg_itxs->i_sync_list, itx); + fail_itxg->itxg_txg = MIN(fail_itxg->itxg_txg, + itx->itx_lr.lrc_txg); + + highest_txg = MAX(highest_txg, itx->itx_lr.lrc_txg); + } + } + ASSERT(list_is_empty(&zilog->zl_itx_commit_list)); + + /* + * If this fails, then we didn't take any itxs at all. If that's true, + * how did we end up here? + */ + ASSERT3U(fail_itxg->itxg_txg, <, UINT64_MAX); + + /* Prepare the live itxgs for failure */ + for (int i = 0; i < TXG_SIZE; i++) { + itxg_t *itxg = &zilog->zl_itxg[i]; + + mutex_enter(&itxg->itxg_lock); + ASSERT(!itxg->itxg_failed); + + /* + * Flag itxgs as failed. Most itxg users (eg zil_itx_assign()) + * take itxg_lock but not zl_lock, to avoid contention. They + * need a cheap way to test for failure; this is it. + */ + itxg->itxg_failed = B_TRUE; + + if (itxg->itxg_txg == 0) { + /* Previously cleaned itxg, nothing to do. */ + ASSERT3U(itxg->itxg_itxs, ==, NULL); + mutex_exit(&itxg->itxg_lock); + continue; + } + + /* Ensure unfail covers this itxg */ + highest_txg = MAX(highest_txg, itxg->itxg_txg); + + /* + * Scan the sync list for commit itxs, so we can signal their + * waiters. This shouldn't happen, but see zil_itxg_clean() for + * how they can legitimately end up here. + */ + l = &itxg->itxg_itxs->i_sync_list; + while ((itx = list_head(l)) != NULL) { + if (itx->itx_lr.lrc_txtype == TX_COMMIT) { + list_remove(l, itx); + list_insert_tail(&waiters, itx->itx_private); + itx->itx_private = NULL; + zil_itx_destroy(itx); + } + } + + mutex_exit(&itxg->itxg_lock); + } + + /* We've seen every outstanding itx, and have our unfail point */ + zilog->zl_unfail_txg = highest_txg + 1; + + mutex_exit(&fail_itxg->itxg_lock); + + char ds_name[ZFS_MAX_DATASET_NAME_LEN]; + dsl_dataset_name(dmu_objset_ds(zilog->zl_os), ds_name); + cmn_err(CE_WARN, + "ZIL failed on pool '%s' dataset '%s': " + "last_txg=%llu; unfail_txg=%llu", + spa_name(zilog->zl_spa), ds_name, + last_synced_txg, zilog->zl_unfail_txg); + + /* Now inform all the waiters */ + zil_commit_waiter_t *zcw; + while ((zcw = list_head(&waiters)) != NULL) { + mutex_enter(&zcw->zcw_lock); + + ASSERT(list_link_active(&zcw->zcw_node)); + list_remove(&waiters, zcw); + + zcw->zcw_lwb = NULL; + + /* + * Set a reasonable error for zil_commit_impl() to find when + * zil_commit_waiter() returns. + */ + zcw->zcw_zio_error = EIO; + + ASSERT3B(zcw->zcw_done, ==, B_FALSE); + zcw->zcw_done = B_TRUE; + + cv_broadcast(&zcw->zcw_cv); + + mutex_exit(&zcw->zcw_lock); + } + + mutex_exit(&spa_namespace_lock); +} + /* * 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 @@ -1146,6 +1459,19 @@ zil_lwb_flush_vdevs_done(zio_t *zio) zio_buf_free(lwb->lwb_buf, lwb->lwb_sz); + /* + * If the IO failed, then either the ZIL is already failed, or we need + * to call zil_fail() to fail it. To call zil_fail() we have to hold + * both zl_issuer_lock and zl_lock, and zl_issuer_lock must be taken + * before zl_lock, so we have to take it now. + * + * If zl_issuer_lock is already taken its because new IO is in the + * process of being issued. That's fine, as once we've called + * zil_fail() those IOs will be come through here and hit the + * zil_failed() test below anyway. + */ + if (zio->io_error != 0) + mutex_enter(&zilog->zl_issuer_lock); mutex_enter(&zilog->zl_lock); /* @@ -1167,6 +1493,50 @@ zil_lwb_flush_vdevs_done(zio_t *zio) ASSERT3S(lwb->lwb_state, ==, LWB_STATE_WRITE_DONE); lwb->lwb_state = LWB_STATE_FLUSH_DONE; + if (zil_failed(zilog)) { + /* + * The ZIL failed some time ago, so itxs have already been + * emptied and waiters informed, so there's nothing else + * to do here. + */ + mutex_exit(&zilog->zl_lock); + if (MUTEX_HELD(&zilog->zl_issuer_lock)) + mutex_exit(&zilog->zl_issuer_lock); + + /* + * This is the transaction for the next block; we have to + * commit it here otherwise txg_quiesce will block if the pool + * returns. + */ + dmu_tx_commit(tx); + + return; + } + + if (zio->io_error != 0) { + /* + * We expect any ZIO errors from child ZIOs to have been + * propagated "up" to this specific LWB's root ZIO, in + * order for this error handling to work correctly. This + * includes ZIO errors from either this LWB's write or + * flush, as well as any errors from other dependent LWBs + * (e.g. a root LWB ZIO that might be a child of this LWB). + */ + ASSERT(MUTEX_HELD(&zilog->zl_issuer_lock)); + ASSERT(MUTEX_HELD(&zilog->zl_lock)); + + zil_fail(zilog); + + mutex_exit(&zilog->zl_lock); + mutex_exit(&zilog->zl_issuer_lock); + + dmu_tx_commit(tx); + + return; + } + + ASSERT(!MUTEX_HELD(&zilog->zl_issuer_lock)); + if (zilog->zl_last_lwb_opened == lwb) { /* * Remember the highest committed log sequence number @@ -1190,16 +1560,6 @@ zil_lwb_flush_vdevs_done(zio_t *zio) ASSERT3P(zcw->zcw_lwb, ==, lwb); zcw->zcw_lwb = NULL; - /* - * We expect any ZIO errors from child ZIOs to have been - * propagated "up" to this specific LWB's root ZIO, in - * order for this error handling to work correctly. This - * includes ZIO errors from either this LWB's write or - * flush, as well as any errors from other dependent LWBs - * (e.g. a root LWB ZIO that might be a child of this LWB). - */ - - zcw->zcw_zio_error = zio->io_error; ASSERT3B(zcw->zcw_done, ==, B_FALSE); zcw->zcw_done = B_TRUE; @@ -1278,7 +1638,7 @@ zil_lwb_write_done(zio_t *zio) * we expect any error seen here, to have been propagated to * that function). */ - if (zio->io_error != 0) { + if (zil_failed(zilog) || zio->io_error != 0) { while ((zv = avl_destroy_nodes(t, &cookie)) != NULL) kmem_free(zv, sizeof (*zv)); return; @@ -1524,9 +1884,10 @@ zil_lwb_write_issue(zilog_t *zilog, lwb_t *lwb) * should not be subject to the dirty data based delays. We * use DMU_TX_ASSIGN_NOTHROTTLE to bypass the delay mechanism. */ - if (dmu_tx_assign(tx, - DMU_TX_ASSIGN_WAIT | DMU_TX_ASSIGN_NOTHROTTLE) != 0) { - ASSERT(dmu_objset_exiting(zilog->zl_os)); + error = dmu_tx_assign(tx, DMU_TX_ASSIGN_WAIT | + DMU_TX_ASSIGN_NOTHROTTLE | DMU_TX_ASSIGN_NOSUSPEND); + if (error != 0) { + ASSERT3S(error, ==, EAGAIN); dmu_tx_abort(tx); return (NULL); } @@ -1562,7 +1923,9 @@ zil_lwb_write_issue(zilog_t *zilog, lwb_t *lwb) zilog->zl_prev_rotor = (zilog->zl_prev_rotor + 1) & (ZIL_PREV_BLKS - 1); BP_ZERO(bp); - error = zio_alloc_zil(spa, zilog->zl_os, txg, bp, zil_blksz, &slog); + if (error == 0) + error = zio_alloc_zil(spa, zilog->zl_os, txg, bp, zil_blksz, + &slog); if (slog) { ZIL_STAT_BUMP(zil_itx_metaslab_slog_count); ZIL_STAT_INCR(zil_itx_metaslab_slog_bytes, lwb->lwb_nused); @@ -1743,6 +2106,11 @@ cont: */ if (lrc->lrc_txtype == TX_WRITE) { if (txg > spa_freeze_txg(zilog->zl_spa)) + /* + * We're allowed to block here as freeze is only used by + * ztest, which should always be a stable and controlled + * environment. + */ txg_wait_synced(zilog->zl_dmu_pool, txg); if (itx->itx_wr_state == WR_COPIED) { ZIL_STAT_BUMP(zil_itx_copied_count); @@ -1791,9 +2159,33 @@ cont: bzero((char *)dbuf + lrwb->lr_length, dpad); if (error == EIO) { - txg_wait_synced(zilog->zl_dmu_pool, txg); - return (lwb); + /* + * The zl_get_data callback couldn't give us + * the data. This is almost certainly because + * the pool suspended in dmu_sync_late_arrival() + * but we attempt to force the txg out anyway + * just to be sure. If it really is suspended, + * it'll return immediately. + */ + error = txg_wait_synced_flags( + zilog->zl_dmu_pool, txg, + TXG_WAIT_F_NOSUSPEND); + if (error == 0) + /* Surprising, but we'll take it. */ + return (lwb); + + /* + * The pool has suspended, and we're being + * asked to commit an itx whose associated data + * has no block pointer. This can't be + * resolved, so the itx can't be committed to + * this lwb. All we can do is signal this fact + * to the caller and let it clean up. + */ + ASSERT3S(error, ==, EAGAIN); + return (NULL); } + if (error != 0) { ASSERT(error == ENOENT || error == EEXIST || error == EALREADY); @@ -1976,6 +2368,17 @@ zil_remove_async(zilog_t *zilog, uint64_t oid) list_destroy(&clean_list); } +static itxs_t * +zil_alloc_itxs(void) +{ + itxs_t *itxs = kmem_zalloc(sizeof (itxs_t), KM_SLEEP); + list_create(&itxs->i_sync_list, sizeof (itx_t), + offsetof(itx_t, itx_node)); + avl_create(&itxs->i_async_tree, zil_aitx_compare, + sizeof (itx_async_node_t), offsetof(itx_async_node_t, ia_node)); + return itxs; +} + void zil_itx_assign(zilog_t *zilog, itx_t *itx, dmu_tx_t *tx) { @@ -1996,6 +2399,7 @@ zil_itx_assign(zilog_t *zilog, itx_t *itx, dmu_tx_t *tx) itxg = &zilog->zl_itxg[txg & TXG_MASK]; mutex_enter(&itxg->itxg_lock); + itxs = itxg->itxg_itxs; if (itxg->itxg_txg != txg) { if (itxs != NULL) { @@ -2009,14 +2413,7 @@ zil_itx_assign(zilog_t *zilog, itx_t *itx, dmu_tx_t *tx) clean = itxg->itxg_itxs; } itxg->itxg_txg = txg; - itxs = itxg->itxg_itxs = kmem_zalloc(sizeof (itxs_t), - KM_SLEEP); - - list_create(&itxs->i_sync_list, sizeof (itx_t), - offsetof(itx_t, itx_node)); - avl_create(&itxs->i_async_tree, zil_aitx_compare, - sizeof (itx_async_node_t), - offsetof(itx_async_node_t, ia_node)); + itxs = itxg->itxg_itxs = zil_alloc_itxs(); } if (itx->itx_sync) { list_insert_tail(&itxs->i_sync_list, itx); @@ -2055,6 +2452,48 @@ zil_itx_assign(zilog_t *zilog, itx_t *itx, dmu_tx_t *tx) zil_itxg_clean(clean); } +static void +zil_itxg_clean_failed(zilog_t *zilog, uint64_t synced_txg) +{ + itxg_t *fail_itxg = &zilog->zl_fail_itxg; + + if (fail_itxg->itxg_txg == 0 || fail_itxg->itxg_txg > synced_txg) + return; + + ASSERT3U(fail_itxg->itxg_txg, ==, synced_txg); + + uint64_t next_txg = UINT64_MAX; + + mutex_enter(&fail_itxg->itxg_lock); + + itx_t *itx, *next; + list_t *l = &fail_itxg->itxg_itxs->i_sync_list; + + next = list_head(l); + while (next != NULL) { + itx = next; + next = list_next(l, itx); + ASSERT3U(itx->itx_lr.lrc_txtype, !=, TX_COMMIT); + if (itx->itx_lr.lrc_txg <= synced_txg) { + list_remove(l, itx); + zil_itx_destroy(itx); + } else + next_txg = MIN(next_txg, itx->itx_lr.lrc_txg); + } + + if (next_txg < UINT64_MAX) + fail_itxg->itxg_txg = next_txg; + else { + ASSERT(list_is_empty(&fail_itxg->itxg_itxs->i_sync_list)); + avl_destroy(&fail_itxg->itxg_itxs->i_async_tree); + kmem_free(fail_itxg->itxg_itxs, sizeof (itxs_t)); + fail_itxg->itxg_itxs = NULL; + fail_itxg->itxg_txg = 0; + } + + mutex_exit(&fail_itxg->itxg_lock); +} + /* * If there are any in-memory intent log transactions which have now been * synced then start up a taskq to free them. We should only do this after we @@ -2071,6 +2510,14 @@ zil_clean(zilog_t *zilog, uint64_t synced_txg) ASSERT3U(synced_txg, <, ZILTEST_TXG); mutex_enter(&itxg->itxg_lock); + + /* + * Clean up the failed itxg if it has anything for this txg. This will + * be fast and lock-free if it doesn't, so its ok to do this directly + * on this thread. + */ + zil_itxg_clean_failed(zilog, synced_txg); + if (itxg->itxg_itxs == NULL || itxg->itxg_txg == ZILTEST_TXG) { mutex_exit(&itxg->itxg_lock); return; @@ -2125,6 +2572,15 @@ zil_get_commit_list(zilog_t *zilog) mutex_exit(&itxg->itxg_lock); continue; } + if (itxg->itxg_failed) { + /* + * If we failed, do nothing, as we want the itxs to + * remain here. zil_fail() is under zl_issuer_lock, + * which we hold, so there's no way for us to race them. + */ + mutex_exit(&itxg->itxg_lock); + return; + } /* * If we're adding itx records to the zl_itx_commit_list, @@ -2245,12 +2701,12 @@ zil_prune_commit_list(zilog_t *zilog) IMPLY(itx != NULL, itx->itx_lr.lrc_txtype != TX_COMMIT); } -static void +static int zil_commit_writer_stall(zilog_t *zilog) { /* * When zio_alloc_zil() fails to allocate the next lwb block on - * disk, we must call txg_wait_synced() to ensure all of the + * disk, we must call txg_wait_synced_flags() to ensure all of the * lwbs in the zilog's zl_lwb_list are synced and then freed (in * zil_sync()), such that any subsequent ZIL writer (i.e. a call * to zil_process_commit_list()) will have to call zil_create(), @@ -2266,11 +2722,17 @@ zil_commit_writer_stall(zilog_t *zilog) * We must hold the zilog's zl_issuer_lock while we do this, to * ensure no new threads enter zil_process_commit_list() until * all lwb's in the zl_lwb_list have been synced and freed - * (which is achieved via the txg_wait_synced() call). + * (which is achieved via the txg_wait_synced_flags() call). */ ASSERT(MUTEX_HELD(&zilog->zl_issuer_lock)); - txg_wait_synced(zilog->zl_dmu_pool, 0); + int error = txg_wait_synced_flags(zilog->zl_dmu_pool, 0, + TXG_WAIT_F_NOSUSPEND); + if (error != 0) { + ASSERT3S(error, ==, EAGAIN); + return (EAGAIN); + } ASSERT3P(list_tail(&zilog->zl_lwb_list), ==, NULL); + return (0); } /* @@ -2304,6 +2766,17 @@ zil_process_commit_list(zilog_t *zilog) lwb = list_tail(&zilog->zl_lwb_list); if (lwb == NULL) { lwb = zil_create(zilog); + if (lwb == NULL) { + /* + * Couldn't allocate the first block or write the + * header. Immediately fail the ZIL. The itxs are on the + * commit list, which means not much to do. + */ + mutex_enter(&zilog->zl_lock); + zil_fail(zilog); + mutex_exit(&zilog->zl_lock); + return; + } } else { ASSERT3S(lwb->lwb_state, !=, LWB_STATE_ISSUED); ASSERT3S(lwb->lwb_state, !=, LWB_STATE_WRITE_DONE); @@ -2397,33 +2870,93 @@ zil_process_commit_list(zilog_t *zilog) if (lwb == NULL) { /* - * This indicates zio_alloc_zil() failed to allocate the - * "next" lwb on-disk. When this happens, we must stall - * the ZIL write pipeline; see the comment within + * This indicates either zio_alloc_zil() failed to allocate the + * "next" lwb on-disk, or the pool suspended while trying to + * assign that block to a tx (or, in rare cases, the pool + * suspended while trying to commit a TX_WRITE). + * + * If the lwb was issued, we can try to recover by stalling the + * ZIL write pipeline; see the comment within * zil_commit_writer_stall() for more details. + * + * If the lwb was not issued, stalling is not an option. Almost + * certainly it will fail anyway, but if the pool was restored + * and it succeeded, we'd be left with an unissued lwb carrying + * itxs for data that is already on the main pool, and no one + * has responsibility for calling the waiters. + * + * Regardless, if the stall fails or would be unsafe, we have no + * choice but to fail the ZIL entirely. */ - zil_commit_writer_stall(zilog); - /* - * Additionally, we have to signal and mark the "nolwb" - * waiters as "done" here, since without an lwb, we - * can't do this via zil_lwb_flush_vdevs_done() like - * normal. - */ - zil_commit_waiter_t *zcw; - while ((zcw = list_head(&nolwb_waiters)) != NULL) { - zil_commit_waiter_skip(zcw); - list_remove(&nolwb_waiters, zcw); - } + lwb = list_tail(&zilog->zl_lwb_list); + ASSERT3S(lwb->lwb_state, !=, LWB_STATE_CLOSED); - /* - * And finally, we have to destroy the itx's that - * couldn't be committed to an lwb; this will also call - * the itx's callback if one exists for the itx. - */ - while ((itx = list_head(&nolwb_itxs)) != NULL) { - list_remove(&nolwb_itxs, itx); - zil_itx_destroy(itx); + if (lwb->lwb_state != LWB_STATE_OPENED && + zil_commit_writer_stall(zilog) == 0) { + /* + * Additionally, we have to signal and mark the "nolwb" + * waiters as "done" here, since without an lwb, we + * can't do this via zil_lwb_flush_vdevs_done() like + * normal. + */ + zil_commit_waiter_t *zcw; + while ((zcw = list_head(&nolwb_waiters)) != NULL) { + list_remove(&nolwb_waiters, zcw); + zil_commit_waiter_skip(zcw); + } + + /* + * And finally, we have to destroy the itx's that + * couldn't be committed to an lwb; this will also call + * the itx's callback if one exists for the itx. + */ + while ((itx = list_head(&nolwb_itxs)) != NULL) { + list_remove(&nolwb_itxs, itx); + zil_itx_destroy(itx); + } + } else { + ASSERT(spa_suspended(spa)); + ASSERT(!zil_failed(zilog)); + + /* + * The pool suspended, so all we can do is fail the ZIL + * and keep the itxs alive in case it ever comes back. + */ + + ASSERT(MUTEX_HELD(&zilog->zl_issuer_lock)); + mutex_enter(&zilog->zl_lock); + + if (lwb->lwb_state == LWB_STATE_OPENED) { + /* + * The LWB was never issued, so will never get + * cleaned up in zil_lwb_flush_vdevs_done(). Do + * that now. + */ + ASSERT3P(lwb->lwb_buf, !=, NULL); + ASSERT3P(lwb->lwb_tx, !=, NULL); + ASSERT3P(lwb->lwb_root_zio, ==, NULL); + ASSERT3P(lwb->lwb_write_zio, ==, NULL); + + zio_buf_free(lwb->lwb_buf, lwb->lwb_sz); + dmu_tx_t *tx = lwb->lwb_tx; + lwb->lwb_buf = NULL; + lwb->lwb_tx = NULL; + dmu_tx_commit(tx); + } + + /* + * Attach the nolwb itxs and waiters to the lead lwb. + * It doesn't matter that they're not going to be + * issued; we just need them in a place that zil_fail() + * can get at them. + */ + list_move_tail(&lwb->lwb_itxs, &nolwb_itxs); + list_move_tail(&lwb->lwb_waiters, &nolwb_waiters); + + zil_fail(zilog); + + mutex_exit(&zilog->zl_lock); } } else { ASSERT(list_is_empty(&nolwb_waiters)); @@ -2532,6 +3065,7 @@ zil_commit_writer(zilog_t *zilog, zil_commit_waiter_t *zcw) out: mutex_exit(&zilog->zl_issuer_lock); + ASSERT(!MUTEX_HELD(&zilog->zl_lock)); } static void @@ -2635,10 +3169,15 @@ zil_commit_waiter_timeout(zilog_t *zilog, zil_commit_waiter_t *zcw) if (nlwb == NULL) { /* * When zil_lwb_write_issue() returns NULL, this - * indicates zio_alloc_zil() failed to allocate the - * "next" lwb on-disk. When this occurs, the ZIL write - * pipeline must be stalled; see the comment within the - * zil_commit_writer_stall() function for more details. + * indicates either zio_alloc_zil() failed to allocate + * the "next" lwb on-disk, or the pool suspended while + * trying to assign that block to a tx. + * + * If the lwb was issued, we can try to recover by + * stalling the ZIL write pipeline. If it was not, or + * the stall also fails, we must fail the ZIL instead. + * See the comments in zil_process_commit_list() and + * zil_commit_writer_stall() for more details. * * We must drop the commit waiter's lock prior to * calling zil_commit_writer_stall() or else we can wind @@ -2655,14 +3194,50 @@ zil_commit_waiter_timeout(zilog_t *zilog, zil_commit_waiter_t *zcw) * because it's blocked trying to acquire the waiter's * lock, which occurs prior to calling dmu_tx_commit() */ + + lwb = list_tail(&zilog->zl_lwb_list); + ASSERT3S(lwb->lwb_state, !=, LWB_STATE_CLOSED); + mutex_exit(&zcw->zcw_lock); - zil_commit_writer_stall(zilog); + + if (lwb->lwb_state == LWB_STATE_OPENED || + zil_commit_writer_stall(zilog) != 0) { + ASSERT(spa_suspended(zilog->zl_spa)); + ASSERT(!zil_failed(zilog)); + + ASSERT(MUTEX_HELD(&zilog->zl_issuer_lock)); + mutex_enter(&zilog->zl_lock); + + if (lwb->lwb_state == LWB_STATE_OPENED) { + /* + * The LWB was never issued, so will never get + * cleaned up in zil_lwb_flush_vdevs_done(). Do + * that now. + */ + ASSERT3P(lwb->lwb_buf, !=, NULL); + ASSERT3P(lwb->lwb_tx, !=, NULL); + ASSERT3P(lwb->lwb_root_zio, ==, NULL); + ASSERT3P(lwb->lwb_write_zio, ==, NULL); + + zio_buf_free(lwb->lwb_buf, lwb->lwb_sz); + dmu_tx_t *tx = lwb->lwb_tx; + lwb->lwb_buf = NULL; + lwb->lwb_tx = NULL; + dmu_tx_commit(tx); + } + + zil_fail(zilog); + + mutex_exit(&zilog->zl_lock); + } + mutex_enter(&zcw->zcw_lock); } out: mutex_exit(&zilog->zl_issuer_lock); ASSERT(MUTEX_HELD(&zcw->zcw_lock)); + ASSERT(!MUTEX_HELD(&zilog->zl_lock)); } /* @@ -2817,15 +3392,17 @@ zil_free_commit_waiter(zil_commit_waiter_t *zcw) * then later committed to an lwb (or skipped) when * zil_process_commit_list() is called. */ -static void +static int zil_commit_itx_assign(zilog_t *zilog, zil_commit_waiter_t *zcw) { dmu_tx_t *tx = dmu_tx_create(zilog->zl_os); + int error = 0; - if (dmu_tx_assign(tx, DMU_TX_ASSIGN_WAIT) != 0) { - ASSERT(dmu_objset_exiting(zilog->zl_os)); + error = dmu_tx_assign(tx, DMU_TX_ASSIGN_WAIT | DMU_TX_ASSIGN_NOSUSPEND); + if (error != 0) { + ASSERT3S(error, ==, EAGAIN); dmu_tx_abort(tx); - return; + return (EAGAIN); } itx_t *itx = zil_itx_create(TX_COMMIT, sizeof (lr_t)); @@ -2835,6 +3412,30 @@ zil_commit_itx_assign(zilog_t *zilog, zil_commit_waiter_t *zcw) zil_itx_assign(zilog, itx, tx); dmu_tx_commit(tx); + + return (0); +} + +/* + * This is a helper for when zil_commit() has to fall back to a full txg sync. + * Callers of zil_commit() are expecting it to block or fail according to the + * failure mode, so this is one of the few places where we honor that. It will + * return EIO on suspend and failmode=continue so we don't have to repeat that + * conversion multiple times at the callers. + */ +static int +zil_commit_fallback_sync(zilog_t *zilog) +{ + int err = 0; + const txg_wait_flag_t flags = + spa_get_failmode(zilog->zl_spa) == ZIO_FAILURE_MODE_CONTINUE ? + TXG_WAIT_F_NOSUSPEND : TXG_WAIT_F_NONE; + + err = txg_wait_synced_flags(zilog->zl_dmu_pool, 0, flags); + if (flags & TXG_WAIT_F_NOSUSPEND && err == EAGAIN) + err = SET_ERROR(EIO); + + return (err); } /* @@ -2952,7 +3553,7 @@ zil_commit_itx_assign(zilog_t *zilog, zil_commit_waiter_t *zcw) * but the order in which they complete will be the same order in * which they were created. */ -void +int zil_commit(zilog_t *zilog, uint64_t foid) { /* @@ -2971,7 +3572,7 @@ zil_commit(zilog_t *zilog, uint64_t foid) ASSERT3B(dmu_objset_is_snapshot(zilog->zl_os), ==, B_FALSE); if (zilog->zl_sync == ZFS_SYNC_DISABLED) - return; + return (0); if (!spa_writeable(zilog->zl_spa)) { /* @@ -2985,33 +3586,33 @@ zil_commit(zilog_t *zilog, uint64_t foid) ASSERT3P(zilog->zl_last_lwb_opened, ==, NULL); for (int i = 0; i < TXG_SIZE; i++) ASSERT3P(zilog->zl_itxg[i].itxg_itxs, ==, NULL); - return; + return (SET_ERROR(EROFS)); } /* * If the objset is being forced to exit, there's nothing more to do. */ if (dmu_objset_exiting(zilog->zl_os)) - return; + return (0); /* - * If the ZIL is suspended, we don't want to dirty it by calling - * zil_commit_itx_assign() below, nor can we write out - * lwbs like would be done in zil_commit_write(). Thus, we - * simply rely on txg_wait_synced() to maintain the necessary - * semantics, and avoid calling those functions altogether. + * If the ZIL has failed or is suspended, we don't want to dirty it by + * calling zil_commit_itx_assign() below, nor can we write out lwbs + * like would be done in zil_commit_write(). Thus, we simply rely on + * txg_wait_synced() to maintain the necessary semantics, and avoid + * calling those functions altogether. */ - if (zilog->zl_suspend > 0) { - txg_wait_synced(zilog->zl_dmu_pool, 0); - return; - } + if (zil_failed(zilog) || zilog->zl_suspend > 0) + return (zil_commit_fallback_sync(zilog)); - zil_commit_impl(zilog, foid); + return (zil_commit_impl(zilog, foid)); } -void +int zil_commit_impl(zilog_t *zilog, uint64_t foid) { + ASSERT0(zil_failed(zilog) || zilog->zl_suspend > 0); + ZIL_STAT_BUMP(zil_commit_count); /* @@ -3041,26 +3642,40 @@ zil_commit_impl(zilog_t *zilog, uint64_t foid) * zil_commit_waiter(). */ zil_commit_waiter_t *zcw = zil_alloc_commit_waiter(); - zil_commit_itx_assign(zilog, zcw); + + if (zil_commit_itx_assign(zilog, zcw) != 0) { + /* Pool suspended while committing TX_COMMIT. */ + zil_free_commit_waiter(zcw); + return (SET_ERROR(EIO)); + } zil_commit_writer(zilog, zcw); zil_commit_waiter(zilog, zcw); - if (zcw->zcw_zio_error != 0) { - /* - * If there was an error writing out the ZIL blocks that - * this thread is waiting on, then we fallback to - * relying on spa_sync() to write out the data this - * thread is waiting on. Obviously this has performance - * implications, but the expectation is for this to be - * an exceptional case, and shouldn't occur often. - */ - DTRACE_PROBE2(zil__commit__io__error, - zilog_t *, zilog, zil_commit_waiter_t *, zcw); - txg_wait_synced(zilog->zl_dmu_pool, 0); + ASSERT(zcw->zcw_done); + + if (zcw->zcw_zio_error == 0) { + zil_free_commit_waiter(zcw); + return (0); } + + ASSERT(zil_failed(zilog)); + + /* + * Ideally, we would just return to zil_commit() here and do the + * call to zil_commit_fallback_sync() from one place, but this dtrace + * probe has the live zcw as an argument, so we have to call it before + * we free it. + */ + DTRACE_PROBE2(zil__commit__io__error, + zilog_t *, zilog, zil_commit_waiter_t *, zcw); + + int err = zil_commit_fallback_sync(zilog); + zil_free_commit_waiter(zcw); + + return (err); } /* @@ -3092,6 +3707,46 @@ zil_sync(zilog_t *zilog, dmu_tx_t *tx) *replayed_seq = 0; } + if (zil_failed(zilog)) { + if (zilog->zl_unfail_txg <= txg) { + /* + * Everything in the failed ZIL is now on the main + * pool, so we can return it to service. + */ + + /* + * The live txgs should be either empty or ahead of the + * unfail txg. The fail txg should be empty. + */ + for (int i = 0; i < TXG_SIZE; i++) { + itxg_t *itxg = &zilog->zl_itxg[i]; + mutex_enter(&itxg->itxg_lock); + + ASSERT(zilog->zl_itxg[i].itxg_txg == 0 || + zilog->zl_itxg[i].itxg_txg >= + zilog->zl_unfail_txg); + + itxg->itxg_failed = B_FALSE; + mutex_exit(&itxg->itxg_lock); + } + + zilog->zl_unfail_txg = 0; + + char ds_name[ZFS_MAX_DATASET_NAME_LEN]; + dsl_dataset_name(dmu_objset_ds(zilog->zl_os), ds_name); + cmn_err(CE_WARN, + "ZIL resumed on pool '%s' dataset '%s': txg=%llu", + spa_name(zilog->zl_spa), ds_name, txg); + } else { + /* Can't unfail yet; re-dirty for next txg. */ + dsl_pool_t *dp = zilog->zl_dmu_pool; + dsl_dataset_t *ds = dmu_objset_ds(zilog->zl_os); + dmu_tx_t *next_tx = dmu_tx_create_assigned(dp, txg + 1); + dsl_dataset_dirty(ds, next_tx); + dmu_tx_commit(next_tx); + } + } + if (zilog->zl_destroy_txg == txg) { blkptr_t blk = zh->zh_log; @@ -3194,6 +3849,19 @@ zil_init(void) void zil_fini(void) { + /* + * XXX If the pool crashed and there is still outstanding IO, there + * will still be allocated lwbs on zil_lwb_cache. On Linux this will + * trip a BUG() in the underlyng __kmem_cache_shutdown. There's + * really nothing we can do here unless we decouple lwbs from their + * zios, either by walking all lwb zios and clearing their + * io_private members under lock, and then checking for that in the + * done callbacks, or by making their io_private members an index + * into some global lwb lookup. These are both substantial + * undertakings, so for now the guidance is simply to not unload the + * module after pool failure and forced export -- robn, 2023-05-08 + */ + kmem_cache_destroy(zil_zcw_cache); kmem_cache_destroy(zil_lwb_cache); @@ -3241,6 +3909,7 @@ zil_alloc(objset_t *os, zil_header_t *zh_phys) mutex_init(&zilog->zl_itxg[i].itxg_lock, NULL, MUTEX_DEFAULT, NULL); } + mutex_init(&zilog->zl_fail_itxg.itxg_lock, NULL, MUTEX_DEFAULT, NULL); list_create(&zilog->zl_lwb_list, sizeof (lwb_t), offsetof(lwb_t, lwb_node)); @@ -3282,6 +3951,9 @@ zil_free(zilog_t *zilog) mutex_destroy(&zilog->zl_itxg[i].itxg_lock); } + /* XXX clean out the fail itxg -- robn, 2023-04-27 */ + mutex_destroy(&zilog->zl_fail_itxg.itxg_lock); + mutex_destroy(&zilog->zl_issuer_lock); mutex_destroy(&zilog->zl_lock); @@ -3351,29 +4023,51 @@ zil_close(zilog_t *zilog) zilog->zl_get_data = NULL; /* - * We should have only one lwb left on the list; remove it now. + * XXX if we're exiting, we may have itxgs holding itxs that need to be + * removed to allow their callbacks to fire -- robn, 2023-05-08 + */ + + /* + * Clean up outstanding lwbs. Under normal circumstances there should + * be no more than one, but if the ZIL has failed or the pool is + * exporting, there might be additional ones in flight, so we need to + * take care. */ mutex_enter(&zilog->zl_lock); lwb = list_head(&zilog->zl_lwb_list); - if (lwb != NULL) { - ASSERT3P(lwb, ==, list_tail(&zilog->zl_lwb_list)); - ASSERT3S(lwb->lwb_state, !=, LWB_STATE_ISSUED); + if (lwb == NULL) { + mutex_exit(&zilog->zl_lock); + return; + } + + if (lwb != list_tail(&zilog->zl_lwb_list)) { + /* More than one; assert something is wrong. */ + ASSERT(zil_failed(zilog) || spa_exiting_any(zilog->zl_spa)); + } + + do { + list_remove(&zilog->zl_lwb_list, lwb); + + if (lwb->lwb_state != LWB_STATE_OPENED && + lwb->lwb_state != LWB_STATE_CLOSED) { + /* + * If its been issued, we can't clean it up, as its zios + * may yet come back to us. So we just skip it here and + * let zil_lwb_write_done() deal with it. + */ + continue; + } 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) { + if (lwb->lwb_buf != NULL) zio_buf_free(lwb->lwb_buf, lwb->lwb_sz); - } else { - /* - * Pool is being force exported, while this lwb was - * between zil_lwb_flush_vdevs_done and zil_sync. - */ - ASSERT(spa_exiting(zilog->zl_spa)); - } + + zio_free(zilog->zl_spa, txg, &lwb->lwb_blk); zil_free_lwb(zilog, lwb); - } + } while ((lwb = list_head(&zilog->zl_lwb_list)) != NULL); + mutex_exit(&zilog->zl_lock); } @@ -3503,6 +4197,9 @@ zil_suspend(const char *osname, void **cookiep) * Now that we've ensured all lwb's are LWB_STATE_FLUSH_DONE, we * use txg_wait_synced() to ensure the data from the zilog has * migrated to the main pool before calling zil_destroy(). + * + * Its ok to block here, as this is always arrived at via administrative + * operations, never application IO. */ txg_wait_synced(zilog->zl_dmu_pool, 0);