aboutsummaryrefslogtreecommitdiff
path: root/module
diff options
context:
space:
mode:
authorJohn Gallagher <john.gallagher@delphix.com>2019-09-14 01:09:06 +0000
committerBrian Behlendorf <behlendorf1@llnl.gov>2019-09-14 01:09:06 +0000
commite60e158eff920825311c1e18b3631876eaaacb54 (patch)
tree03b5f6ff4855ae0fdc233d377d3c1939d1223912 /module
parent7238cbd4d3ee7eadb3131c890d0692a49ea844af (diff)
downloadsrc-e60e158eff920825311c1e18b3631876eaaacb54.tar.gz
src-e60e158eff920825311c1e18b3631876eaaacb54.zip
Add subcommand to wait for background zfs activity to complete
Currently the best way to wait for the completion of a long-running operation in a pool, like a scrub or device removal, is to poll 'zpool status' and parse its output, which is neither efficient nor convenient. This change adds a 'wait' subcommand to the zpool command. When invoked, 'zpool wait' will block until a specified type of background activity completes. Currently, this subcommand can wait for any of the following: - Scrubs or resilvers to complete - Devices to initialized - Devices to be replaced - Devices to be removed - Checkpoints to be discarded - Background freeing to complete For example, a scrub that is in progress could be waited for by running zpool wait -t scrub <pool> This also adds a -w flag to the attach, checkpoint, initialize, replace, remove, and scrub subcommands. When used, this flag makes the operations kicked off by these subcommands synchronous instead of asynchronous. This functionality is implemented using a new ioctl. The type of activity to wait for is provided as input to the ioctl, and the ioctl blocks until all activity of that type has completed. An ioctl was used over other methods of kernel-userspace communiction primarily for the sake of portability. Porting Notes: This is ported from Delphix OS change DLPX-44432. The following changes were made while porting: - Added ZoL-style ioctl input declaration. - Reorganized error handling in zpool_initialize in libzfs to integrate better with changes made for TRIM support. - Fixed check for whether a checkpoint discard is in progress. Previously it also waited if the pool had a checkpoint, instead of just if a checkpoint was being discarded. - Exposed zfs_initialize_chunk_size as a ZoL-style tunable. - Updated more existing tests to make use of new 'zpool wait' functionality, tests that don't exist in Delphix OS. - Used existing ZoL tunable zfs_scan_suspend_progress, together with zinject, in place of a new tunable zfs_scan_max_blks_per_txg. - Added support for a non-integral interval argument to zpool wait. Future work: ZoL has support for trimming devices, which Delphix OS does not. In the future, 'zpool wait' could be extended to add the ability to wait for trim operations to complete. Reviewed-by: Matt Ahrens <matt@delphix.com> Reviewed-by: John Kennedy <john.kennedy@delphix.com> Reviewed-by: Brian Behlendorf <behlendorf1@llnl.gov> Signed-off-by: John Gallagher <john.gallagher@delphix.com> Closes #9162
Diffstat (limited to 'module')
-rw-r--r--module/zfs/bpobj.c16
-rw-r--r--module/zfs/dsl_scan.c5
-rw-r--r--module/zfs/spa.c277
-rw-r--r--module/zfs/spa_checkpoint.c1
-rw-r--r--module/zfs/spa_misc.c7
-rw-r--r--module/zfs/vdev.c29
-rw-r--r--module/zfs/vdev_initialize.c8
-rw-r--r--module/zfs/vdev_removal.c1
-rw-r--r--module/zfs/zfs_ioctl.c55
9 files changed, 395 insertions, 4 deletions
diff --git a/module/zfs/bpobj.c b/module/zfs/bpobj.c
index 561d0cf8ae65..00c3d7dfa8b0 100644
--- a/module/zfs/bpobj.c
+++ b/module/zfs/bpobj.c
@@ -203,13 +203,23 @@ bpobj_close(bpobj_t *bpo)
mutex_destroy(&bpo->bpo_lock);
}
-boolean_t
-bpobj_is_empty(bpobj_t *bpo)
+static boolean_t
+bpobj_is_empty_impl(bpobj_t *bpo)
{
+ ASSERT(MUTEX_HELD(&bpo->bpo_lock));
return (bpo->bpo_phys->bpo_num_blkptrs == 0 &&
(!bpo->bpo_havesubobj || bpo->bpo_phys->bpo_num_subobjs == 0));
}
+boolean_t
+bpobj_is_empty(bpobj_t *bpo)
+{
+ mutex_enter(&bpo->bpo_lock);
+ boolean_t is_empty = bpobj_is_empty_impl(bpo);
+ mutex_exit(&bpo->bpo_lock);
+ return (is_empty);
+}
+
/*
* A recursive iteration of the bpobjs would be nice here but we run the risk
* of overflowing function stack space. Instead, find each subobj and add it
@@ -387,7 +397,7 @@ bpobj_iterate_impl(bpobj_t *initial_bpo, bpobj_itor_t func, void *arg,
* If there are no entries, there should
* be no bytes.
*/
- if (bpobj_is_empty(bpo)) {
+ if (bpobj_is_empty_impl(bpo)) {
ASSERT0(bpo->bpo_phys->bpo_bytes);
ASSERT0(bpo->bpo_phys->bpo_comp);
ASSERT0(bpo->bpo_phys->bpo_uncomp);
diff --git a/module/zfs/dsl_scan.c b/module/zfs/dsl_scan.c
index 7845f1de2d27..d88ec18d214d 100644
--- a/module/zfs/dsl_scan.c
+++ b/module/zfs/dsl_scan.c
@@ -899,6 +899,8 @@ dsl_scan_done(dsl_scan_t *scn, boolean_t complete, dmu_tx_t *tx)
scn->scn_phys.scn_state = complete ? DSS_FINISHED : DSS_CANCELED;
+ spa_notify_waiters(spa);
+
if (dsl_scan_restarting(scn, tx))
spa_history_log_internal(spa, "scan aborted, restarting", tx,
"errors=%llu", (u_longlong_t)spa_get_errlog_size(spa));
@@ -1038,6 +1040,7 @@ dsl_scrub_pause_resume_sync(void *arg, dmu_tx_t *tx)
scn->scn_phys_cached.scn_flags |= DSF_SCRUB_PAUSED;
dsl_scan_sync_state(scn, tx, SYNC_CACHED);
spa_event_notify(spa, NULL, NULL, ESC_ZFS_SCRUB_PAUSED);
+ spa_notify_waiters(spa);
} else {
ASSERT3U(*cmd, ==, POOL_SCRUB_NORMAL);
if (dsl_scan_is_paused_scrub(scn)) {
@@ -3361,6 +3364,8 @@ dsl_process_async_destroys(dsl_pool_t *dp, dmu_tx_t *tx)
ASSERT0(dsl_dir_phys(dp->dp_free_dir)->dd_uncompressed_bytes);
}
+ spa_notify_waiters(spa);
+
EQUIV(bpobj_is_open(&dp->dp_obsolete_bpobj),
0 == zap_contains(dp->dp_meta_objset, DMU_POOL_DIRECTORY_OBJECT,
DMU_POOL_OBSOLETE_BPOBJ));
diff --git a/module/zfs/spa.c b/module/zfs/spa.c
index 8330ab1ce8d3..0f1a2a9eb38c 100644
--- a/module/zfs/spa.c
+++ b/module/zfs/spa.c
@@ -1541,6 +1541,8 @@ spa_unload(spa_t *spa)
spa_import_progress_remove(spa_guid(spa));
spa_load_note(spa, "UNLOADING");
+ spa_wake_waiters(spa);
+
/*
* If the log space map feature is enabled and the pool is getting
* exported (but not destroyed), we want to spend some time flushing
@@ -2470,6 +2472,7 @@ livelist_delete_sync(void *arg, dmu_tx_t *tx)
DMU_POOL_DELETED_CLONES, tx));
VERIFY0(zap_destroy(mos, zap_obj, tx));
spa->spa_livelists_to_delete = 0;
+ spa_notify_waiters(spa);
}
}
@@ -6947,6 +6950,7 @@ spa_vdev_detach(spa_t *spa, uint64_t guid, uint64_t pguid, int replace_done)
vdev_dirty(tvd, VDD_DTL, vd, txg);
spa_event_notify(spa, vd, NULL, ESC_ZFS_VDEV_REMOVE);
+ spa_notify_waiters(spa);
/* hang on to the spa before we release the lock */
spa_open_ref(spa, FTAG);
@@ -9228,6 +9232,279 @@ spa_total_metaslabs(spa_t *spa)
return (m);
}
+/*
+ * Notify any waiting threads that some activity has switched from being in-
+ * progress to not-in-progress so that the thread can wake up and determine
+ * whether it is finished waiting.
+ */
+void
+spa_notify_waiters(spa_t *spa)
+{
+ /*
+ * Acquiring spa_activities_lock here prevents the cv_broadcast from
+ * happening between the waiting thread's check and cv_wait.
+ */
+ mutex_enter(&spa->spa_activities_lock);
+ cv_broadcast(&spa->spa_activities_cv);
+ mutex_exit(&spa->spa_activities_lock);
+}
+
+/*
+ * Notify any waiting threads that the pool is exporting, and then block until
+ * they are finished using the spa_t.
+ */
+void
+spa_wake_waiters(spa_t *spa)
+{
+ mutex_enter(&spa->spa_activities_lock);
+ spa->spa_waiters_cancel = B_TRUE;
+ cv_broadcast(&spa->spa_activities_cv);
+ while (spa->spa_waiters != 0)
+ cv_wait(&spa->spa_waiters_cv, &spa->spa_activities_lock);
+ spa->spa_waiters_cancel = B_FALSE;
+ mutex_exit(&spa->spa_activities_lock);
+}
+
+/* Whether the vdev or any of its descendants is initializing. */
+static boolean_t
+spa_vdev_initializing_impl(vdev_t *vd)
+{
+ spa_t *spa = vd->vdev_spa;
+ boolean_t initializing;
+
+ ASSERT(spa_config_held(spa, SCL_CONFIG | SCL_STATE, RW_READER));
+ ASSERT(MUTEX_HELD(&spa->spa_activities_lock));
+
+ mutex_exit(&spa->spa_activities_lock);
+ mutex_enter(&vd->vdev_initialize_lock);
+ mutex_enter(&spa->spa_activities_lock);
+
+ initializing = (vd->vdev_initialize_state == VDEV_INITIALIZE_ACTIVE);
+ mutex_exit(&vd->vdev_initialize_lock);
+
+ if (initializing)
+ return (B_TRUE);
+
+ for (int i = 0; i < vd->vdev_children; i++) {
+ if (spa_vdev_initializing_impl(vd->vdev_child[i]))
+ return (B_TRUE);
+ }
+
+ return (B_FALSE);
+}
+
+/*
+ * If use_guid is true, this checks whether the vdev specified by guid is
+ * being initialized. Otherwise, it checks whether any vdev in the pool is being
+ * initialized. The caller must hold the config lock and spa_activities_lock.
+ */
+static int
+spa_vdev_initializing(spa_t *spa, boolean_t use_guid, uint64_t guid,
+ boolean_t *in_progress)
+{
+ mutex_exit(&spa->spa_activities_lock);
+ spa_config_enter(spa, SCL_CONFIG | SCL_STATE, FTAG, RW_READER);
+ mutex_enter(&spa->spa_activities_lock);
+
+ vdev_t *vd;
+ if (use_guid) {
+ vd = spa_lookup_by_guid(spa, guid, B_FALSE);
+ if (vd == NULL || !vd->vdev_ops->vdev_op_leaf) {
+ spa_config_exit(spa, SCL_CONFIG | SCL_STATE, FTAG);
+ return (EINVAL);
+ }
+ } else {
+ vd = spa->spa_root_vdev;
+ }
+
+ *in_progress = spa_vdev_initializing_impl(vd);
+
+ spa_config_exit(spa, SCL_CONFIG | SCL_STATE, FTAG);
+ return (0);
+}
+
+/*
+ * Locking for waiting threads
+ * ---------------------------
+ *
+ * Waiting threads need a way to check whether a given activity is in progress,
+ * and then, if it is, wait for it to complete. Each activity will have some
+ * in-memory representation of the relevant on-disk state which can be used to
+ * determine whether or not the activity is in progress. The in-memory state and
+ * the locking used to protect it will be different for each activity, and may
+ * not be suitable for use with a cvar (e.g., some state is protected by the
+ * config lock). To allow waiting threads to wait without any races, another
+ * lock, spa_activities_lock, is used.
+ *
+ * When the state is checked, both the activity-specific lock (if there is one)
+ * and spa_activities_lock are held. In some cases, the activity-specific lock
+ * is acquired explicitly (e.g. the config lock). In others, the locking is
+ * internal to some check (e.g. bpobj_is_empty). After checking, the waiting
+ * thread releases the activity-specific lock and, if the activity is in
+ * progress, then cv_waits using spa_activities_lock.
+ *
+ * The waiting thread is woken when another thread, one completing some
+ * activity, updates the state of the activity and then calls
+ * spa_notify_waiters, which will cv_broadcast. This 'completing' thread only
+ * needs to hold its activity-specific lock when updating the state, and this
+ * lock can (but doesn't have to) be dropped before calling spa_notify_waiters.
+ *
+ * Because spa_notify_waiters acquires spa_activities_lock before broadcasting,
+ * and because it is held when the waiting thread checks the state of the
+ * activity, it can never be the case that the completing thread both updates
+ * the activity state and cv_broadcasts in between the waiting thread's check
+ * and cv_wait. Thus, a waiting thread can never miss a wakeup.
+ *
+ * In order to prevent deadlock, when the waiting thread does its check, in some
+ * cases it will temporarily drop spa_activities_lock in order to acquire the
+ * activity-specific lock. The order in which spa_activities_lock and the
+ * activity specific lock are acquired in the waiting thread is determined by
+ * the order in which they are acquired in the completing thread; if the
+ * completing thread calls spa_notify_waiters with the activity-specific lock
+ * held, then the waiting thread must also acquire the activity-specific lock
+ * first.
+ */
+
+static int
+spa_activity_in_progress(spa_t *spa, zpool_wait_activity_t activity,
+ boolean_t use_tag, uint64_t tag, boolean_t *in_progress)
+{
+ int error = 0;
+
+ ASSERT(MUTEX_HELD(&spa->spa_activities_lock));
+
+ switch (activity) {
+ case ZPOOL_WAIT_CKPT_DISCARD:
+ *in_progress =
+ (spa_feature_is_active(spa, SPA_FEATURE_POOL_CHECKPOINT) &&
+ zap_contains(spa_meta_objset(spa),
+ DMU_POOL_DIRECTORY_OBJECT, DMU_POOL_ZPOOL_CHECKPOINT) ==
+ ENOENT);
+ break;
+ case ZPOOL_WAIT_FREE:
+ *in_progress = ((spa_version(spa) >= SPA_VERSION_DEADLISTS &&
+ !bpobj_is_empty(&spa->spa_dsl_pool->dp_free_bpobj)) ||
+ spa_feature_is_active(spa, SPA_FEATURE_ASYNC_DESTROY) ||
+ spa_livelist_delete_check(spa));
+ break;
+ case ZPOOL_WAIT_INITIALIZE:
+ error = spa_vdev_initializing(spa, use_tag, tag, in_progress);
+ break;
+ case ZPOOL_WAIT_REPLACE:
+ mutex_exit(&spa->spa_activities_lock);
+ spa_config_enter(spa, SCL_CONFIG | SCL_STATE, FTAG, RW_READER);
+ mutex_enter(&spa->spa_activities_lock);
+
+ *in_progress = vdev_replace_in_progress(spa->spa_root_vdev);
+ spa_config_exit(spa, SCL_CONFIG | SCL_STATE, FTAG);
+ break;
+ case ZPOOL_WAIT_REMOVE:
+ *in_progress = (spa->spa_removing_phys.sr_state ==
+ DSS_SCANNING);
+ break;
+ case ZPOOL_WAIT_RESILVER:
+ case ZPOOL_WAIT_SCRUB:
+ {
+ boolean_t scanning, paused, is_scrub;
+ dsl_scan_t *scn = spa->spa_dsl_pool->dp_scan;
+
+ is_scrub = (scn->scn_phys.scn_func == POOL_SCAN_SCRUB);
+ scanning = (scn->scn_phys.scn_state == DSS_SCANNING);
+ paused = dsl_scan_is_paused_scrub(scn);
+ *in_progress = (scanning && !paused &&
+ is_scrub == (activity == ZPOOL_WAIT_SCRUB));
+ break;
+ }
+ default:
+ panic("unrecognized value for activity %d", activity);
+ }
+
+ return (error);
+}
+
+static int
+spa_wait_common(const char *pool, zpool_wait_activity_t activity,
+ boolean_t use_tag, uint64_t tag, boolean_t *waited)
+{
+ /*
+ * The tag is used to distinguish between instances of an activity.
+ * 'initialize' is the only activity that we use this for. The other
+ * activities can only have a single instance in progress in a pool at
+ * one time, making the tag unnecessary.
+ *
+ * There can be multiple devices being replaced at once, but since they
+ * all finish once resilvering finishes, we don't bother keeping track
+ * of them individually, we just wait for them all to finish.
+ */
+ if (use_tag && activity != ZPOOL_WAIT_INITIALIZE)
+ return (EINVAL);
+
+ if (activity < 0 || activity >= ZPOOL_WAIT_NUM_ACTIVITIES)
+ return (EINVAL);
+
+ spa_t *spa;
+ int error = spa_open(pool, &spa, FTAG);
+ if (error != 0)
+ return (error);
+
+ /*
+ * Increment the spa's waiter count so that we can call spa_close and
+ * still ensure that the spa_t doesn't get freed before this thread is
+ * finished with it when the pool is exported. We want to call spa_close
+ * before we start waiting because otherwise the additional ref would
+ * prevent the pool from being exported or destroyed throughout the
+ * potentially long wait.
+ */
+ mutex_enter(&spa->spa_activities_lock);
+ spa->spa_waiters++;
+ spa_close(spa, FTAG);
+
+ *waited = B_FALSE;
+ for (;;) {
+ boolean_t in_progress;
+ error = spa_activity_in_progress(spa, activity, use_tag, tag,
+ &in_progress);
+
+ if (!in_progress || spa->spa_waiters_cancel || error)
+ break;
+
+ *waited = B_TRUE;
+
+ if (cv_wait_sig(&spa->spa_activities_cv,
+ &spa->spa_activities_lock) == 0) {
+ error = EINTR;
+ break;
+ }
+ }
+
+ spa->spa_waiters--;
+ cv_signal(&spa->spa_waiters_cv);
+ mutex_exit(&spa->spa_activities_lock);
+
+ return (error);
+}
+
+/*
+ * Wait for a particular instance of the specified activity to complete, where
+ * the instance is identified by 'tag'
+ */
+int
+spa_wait_tag(const char *pool, zpool_wait_activity_t activity, uint64_t tag,
+ boolean_t *waited)
+{
+ return (spa_wait_common(pool, activity, B_TRUE, tag, waited));
+}
+
+/*
+ * Wait for all instances of the specified activity complete
+ */
+int
+spa_wait(const char *pool, zpool_wait_activity_t activity, boolean_t *waited)
+{
+
+ return (spa_wait_common(pool, activity, B_FALSE, 0, waited));
+}
+
sysevent_t *
spa_event_create(spa_t *spa, vdev_t *vd, nvlist_t *hist_nvl, const char *name)
{
diff --git a/module/zfs/spa_checkpoint.c b/module/zfs/spa_checkpoint.c
index f6dfdab9d3e0..7b86bd417cd2 100644
--- a/module/zfs/spa_checkpoint.c
+++ b/module/zfs/spa_checkpoint.c
@@ -191,6 +191,7 @@ spa_checkpoint_discard_complete_sync(void *arg, dmu_tx_t *tx)
spa->spa_checkpoint_info.sci_timestamp = 0;
spa_feature_decr(spa, SPA_FEATURE_POOL_CHECKPOINT, tx);
+ spa_notify_waiters(spa);
spa_history_log_internal(spa, "spa discard checkpoint", tx,
"finished discarding checkpointed state from the pool");
diff --git a/module/zfs/spa_misc.c b/module/zfs/spa_misc.c
index 1ee110b54a9e..ac0dd5ba949e 100644
--- a/module/zfs/spa_misc.c
+++ b/module/zfs/spa_misc.c
@@ -650,12 +650,15 @@ spa_add(const char *name, nvlist_t *config, const char *altroot)
mutex_init(&spa->spa_vdev_top_lock, NULL, MUTEX_DEFAULT, NULL);
mutex_init(&spa->spa_feat_stats_lock, NULL, MUTEX_DEFAULT, NULL);
mutex_init(&spa->spa_flushed_ms_lock, NULL, MUTEX_DEFAULT, NULL);
+ mutex_init(&spa->spa_activities_lock, NULL, MUTEX_DEFAULT, NULL);
cv_init(&spa->spa_async_cv, NULL, CV_DEFAULT, NULL);
cv_init(&spa->spa_evicting_os_cv, NULL, CV_DEFAULT, NULL);
cv_init(&spa->spa_proc_cv, NULL, CV_DEFAULT, NULL);
cv_init(&spa->spa_scrub_io_cv, NULL, CV_DEFAULT, NULL);
cv_init(&spa->spa_suspend_cv, NULL, CV_DEFAULT, NULL);
+ cv_init(&spa->spa_activities_cv, NULL, CV_DEFAULT, NULL);
+ cv_init(&spa->spa_waiters_cv, NULL, CV_DEFAULT, NULL);
for (int t = 0; t < TXG_SIZE; t++)
bplist_create(&spa->spa_free_bplist[t]);
@@ -767,6 +770,7 @@ spa_remove(spa_t *spa)
ASSERT(MUTEX_HELD(&spa_namespace_lock));
ASSERT(spa_state(spa) == POOL_STATE_UNINITIALIZED);
ASSERT3U(zfs_refcount_count(&spa->spa_refcount), ==, 0);
+ ASSERT0(spa->spa_waiters);
nvlist_free(spa->spa_config_splitting);
@@ -818,6 +822,8 @@ spa_remove(spa_t *spa)
cv_destroy(&spa->spa_proc_cv);
cv_destroy(&spa->spa_scrub_io_cv);
cv_destroy(&spa->spa_suspend_cv);
+ cv_destroy(&spa->spa_activities_cv);
+ cv_destroy(&spa->spa_waiters_cv);
mutex_destroy(&spa->spa_flushed_ms_lock);
mutex_destroy(&spa->spa_async_lock);
@@ -832,6 +838,7 @@ spa_remove(spa_t *spa)
mutex_destroy(&spa->spa_suspend_lock);
mutex_destroy(&spa->spa_vdev_top_lock);
mutex_destroy(&spa->spa_feat_stats_lock);
+ mutex_destroy(&spa->spa_activities_lock);
kmem_free(spa, sizeof (spa_t));
}
diff --git a/module/zfs/vdev.c b/module/zfs/vdev.c
index eff1d34f7360..af2d1a25a15b 100644
--- a/module/zfs/vdev.c
+++ b/module/zfs/vdev.c
@@ -4764,6 +4764,35 @@ vdev_xlate(vdev_t *vd, const range_seg_t *logical_rs, range_seg_t *physical_rs)
physical_rs->rs_end = intermediate.rs_end;
}
+/*
+ * Look at the vdev tree and determine whether any devices are currently being
+ * replaced.
+ */
+boolean_t
+vdev_replace_in_progress(vdev_t *vdev)
+{
+ ASSERT(spa_config_held(vdev->vdev_spa, SCL_ALL, RW_READER) != 0);
+
+ if (vdev->vdev_ops == &vdev_replacing_ops)
+ return (B_TRUE);
+
+ /*
+ * A 'spare' vdev indicates that we have a replace in progress, unless
+ * it has exactly two children, and the second, the hot spare, has
+ * finished being resilvered.
+ */
+ if (vdev->vdev_ops == &vdev_spare_ops && (vdev->vdev_children > 2 ||
+ !vdev_dtl_empty(vdev->vdev_child[1], DTL_MISSING)))
+ return (B_TRUE);
+
+ for (int i = 0; i < vdev->vdev_children; i++) {
+ if (vdev_replace_in_progress(vdev->vdev_child[i]))
+ return (B_TRUE);
+ }
+
+ return (B_FALSE);
+}
+
EXPORT_SYMBOL(vdev_fault);
EXPORT_SYMBOL(vdev_degrade);
EXPORT_SYMBOL(vdev_online);
diff --git a/module/zfs/vdev_initialize.c b/module/zfs/vdev_initialize.c
index ca83cabedd1b..169b9282def8 100644
--- a/module/zfs/vdev_initialize.c
+++ b/module/zfs/vdev_initialize.c
@@ -46,7 +46,7 @@ unsigned long zfs_initialize_value = 0xdeadbeefdeadbeeeULL;
int zfs_initialize_limit = 1;
/* size of initializing writes; default 1MiB, see zfs_remove_max_segment */
-uint64_t zfs_initialize_chunk_size = 1024 * 1024;
+unsigned long zfs_initialize_chunk_size = 1024 * 1024;
static boolean_t
vdev_initialize_should_stop(vdev_t *vd)
@@ -150,6 +150,9 @@ vdev_initialize_change_state(vdev_t *vd, vdev_initializing_state_t new_state)
}
dmu_tx_commit(tx);
+
+ if (new_state != VDEV_INITIALIZE_ACTIVE)
+ spa_notify_waiters(spa);
}
static void
@@ -732,4 +735,7 @@ EXPORT_SYMBOL(vdev_initialize_restart);
/* BEGIN CSTYLED */
ZFS_MODULE_PARAM(zfs, zfs_, initialize_value, ULONG, ZMOD_RW,
"Value written during zpool initialize");
+
+ZFS_MODULE_PARAM(zfs, zfs_, initialize_chunk_size, ULONG, ZMOD_RW,
+ "Size in bytes of writes by zpool initialize");
/* END CSTYLED */
diff --git a/module/zfs/vdev_removal.c b/module/zfs/vdev_removal.c
index abec4d50f8c3..4ce7b559e11c 100644
--- a/module/zfs/vdev_removal.c
+++ b/module/zfs/vdev_removal.c
@@ -697,6 +697,7 @@ spa_finish_removal(spa_t *spa, dsl_scan_state_t state, dmu_tx_t *tx)
spa_vdev_removal_destroy(svr);
spa_sync_removing_state(spa, tx);
+ spa_notify_waiters(spa);
vdev_config_dirty(spa->spa_root_vdev);
}
diff --git a/module/zfs/zfs_ioctl.c b/module/zfs/zfs_ioctl.c
index c5093fd447aa..fce074147e6e 100644
--- a/module/zfs/zfs_ioctl.c
+++ b/module/zfs/zfs_ioctl.c
@@ -4070,6 +4070,56 @@ zfs_ioc_pool_trim(const char *poolname, nvlist_t *innvl, nvlist_t *outnvl)
}
/*
+ * This ioctl waits for activity of a particular type to complete. If there is
+ * no activity of that type in progress, it returns immediately, and the
+ * returned value "waited" is false. If there is activity in progress, and no
+ * tag is passed in, the ioctl blocks until all activity of that type is
+ * complete, and then returns with "waited" set to true.
+ *
+ * If a tag is provided, it identifies a particular instance of an activity to
+ * wait for. Currently, this is only valid for use with 'initialize', because
+ * that is the only activity for which there can be multiple instances running
+ * concurrently. In the case of 'initialize', the tag corresponds to the guid of
+ * the vdev on which to wait.
+ *
+ * If a thread waiting in the ioctl receives a signal, the call will return
+ * immediately, and the return value will be EINTR.
+ *
+ * innvl: {
+ * "wait_activity" -> int32_t
+ * (optional) "wait_tag" -> uint64_t
+ * }
+ *
+ * outnvl: "waited" -> boolean_t
+ */
+static const zfs_ioc_key_t zfs_keys_pool_wait[] = {
+ {ZPOOL_WAIT_ACTIVITY, DATA_TYPE_INT32, 0},
+ {ZPOOL_WAIT_TAG, DATA_TYPE_UINT64, ZK_OPTIONAL},
+};
+
+static int
+zfs_ioc_wait(const char *name, nvlist_t *innvl, nvlist_t *outnvl)
+{
+ int32_t activity;
+ uint64_t tag;
+ boolean_t waited;
+ int error;
+
+ if (nvlist_lookup_int32(innvl, ZPOOL_WAIT_ACTIVITY, &activity) != 0)
+ return (EINVAL);
+
+ if (nvlist_lookup_uint64(innvl, ZPOOL_WAIT_TAG, &tag) == 0)
+ error = spa_wait_tag(name, activity, tag, &waited);
+ else
+ error = spa_wait(name, activity, &waited);
+
+ if (error == 0)
+ fnvlist_add_boolean_value(outnvl, ZPOOL_WAIT_WAITED, waited);
+
+ return (error);
+}
+
+/*
* fsname is name of dataset to rollback (to most recent snapshot)
*
* innvl may contain name of expected target snapshot
@@ -6894,6 +6944,11 @@ zfs_ioctl_init(void)
POOL_CHECK_SUSPENDED | POOL_CHECK_READONLY, B_TRUE, B_TRUE,
zfs_keys_pool_trim, ARRAY_SIZE(zfs_keys_pool_trim));
+ zfs_ioctl_register("wait", ZFS_IOC_WAIT,
+ zfs_ioc_wait, zfs_secpolicy_none, POOL_NAME,
+ POOL_CHECK_SUSPENDED | POOL_CHECK_READONLY, B_FALSE, B_FALSE,
+ zfs_keys_pool_wait, ARRAY_SIZE(zfs_keys_pool_wait));
+
/* IOCTLS that use the legacy function signature */
zfs_ioctl_register_legacy(ZFS_IOC_POOL_FREEZE, zfs_ioc_pool_freeze,