Temporarily move taskq+util to libzpool until that directory is broken in to lib...
[zfs.git] / zfs / lib / libzpool / spa_misc.c
index 4ec5380..36046e6 100644 (file)
@@ -23,8 +23,6 @@
  * Use is subject to license terms.
  */
 
-#pragma ident  "@(#)spa_misc.c 1.31    08/04/01 SMI"
-
 #include <sys/zfs_context.h>
 #include <sys/spa_impl.h>
 #include <sys/zio.h>
@@ -45,6 +43,8 @@
 #include <sys/dsl_prop.h>
 #include <sys/fs/zfs.h>
 #include <sys/metaslab_impl.h>
+#include <sys/sunddi.h>
+#include <sys/arc.h>
 #include "zfs_prop.h"
 
 /*
  *     This reference count keep track of any active users of the spa_t.  The
  *     spa_t cannot be destroyed or freed while this is non-zero.  Internally,
  *     the refcount is never really 'zero' - opening a pool implicitly keeps
- *     some references in the DMU.  Internally we check against SPA_MINREF, but
+ *     some references in the DMU.  Internally we check against spa_minref, but
  *     present the image of a zero/non-zero value to consumers.
  *
- * spa_config_lock (per-spa read-priority rwlock)
+ * spa_config_lock[] (per-spa array of rwlocks)
  *
  *     This protects the spa_t from config changes, and must be held in
  *     the following circumstances:
  *             - RW_READER to perform I/O to the spa
  *             - RW_WRITER to change the vdev config
  *
- * spa_config_cache_lock (per-spa mutex)
- *
- *     This mutex prevents the spa_config nvlist from being updated.  No
- *      other locks are required to obtain this lock, although implicitly you
- *      must have the namespace lock or non-zero refcount to have any kind
- *      of spa_t pointer at all.
- *
  * The locking order is fairly straightforward:
  *
  *             spa_namespace_lock      ->      spa_refcount
  *     The namespace lock must be acquired to increase the refcount from 0
  *     or to check if it is zero.
  *
- *             spa_refcount            ->      spa_config_lock
+ *             spa_refcount            ->      spa_config_lock[]
  *
  *     There must be at least one valid reference on the spa_t to acquire
  *     the config lock.
  *
- *             spa_namespace_lock      ->      spa_config_lock
+ *             spa_namespace_lock      ->      spa_config_lock[]
  *
  *     The namespace lock must always be taken before the config lock.
  *
  *
- * The spa_namespace_lock and spa_config_cache_lock can be acquired directly and
- * are globally visible.
+ * The spa_namespace_lock can be acquired directly and is globally visible.
  *
- * The namespace is manipulated using the following functions, all which require
- * the spa_namespace_lock to be held.
+ * The namespace is manipulated using the following functions, all of which
+ * require the spa_namespace_lock to be held.
  *
  *     spa_lookup()            Lookup a spa_t by name.
  *
  *                             zero.  Must be called with spa_namespace_lock
  *                             held.
  *
- * The spa_config_lock is a form of rwlock.  It must be held as RW_READER
- * to perform I/O to the pool, and as RW_WRITER to change the vdev config.
- * The spa_config_lock is manipulated with spa_config_{enter,exit,held}().
+ * The spa_config_lock[] is an array of rwlocks, ordered as follows:
+ * SCL_CONFIG > SCL_STATE > SCL_ALLOC > SCL_ZIO > SCL_FREE > SCL_VDEV.
+ * spa_config_lock[] is manipulated with spa_config_{enter,exit,held}().
+ *
+ * To read the configuration, it suffices to hold one of these locks as reader.
+ * To modify the configuration, you must hold all locks as writer.  To modify
+ * vdev state without altering the vdev tree's topology (e.g. online/offline),
+ * you must hold SCL_STATE and SCL_ZIO as writer.
+ *
+ * We use these distinct config locks to avoid recursive lock entry.
+ * For example, spa_sync() (which holds SCL_CONFIG as reader) induces
+ * block allocations (SCL_ALLOC), which may require reading space maps
+ * from disk (dmu_read() -> zio_read() -> SCL_ZIO).
+ *
+ * The spa config locks cannot be normal rwlocks because we need the
+ * ability to hand off ownership.  For example, SCL_ZIO is acquired
+ * by the issuing thread and later released by an interrupt thread.
+ * They do, however, obey the usual write-wanted semantics to prevent
+ * writer (i.e. system administrator) starvation.
+ *
+ * The lock acquisition rules are as follows:
+ *
+ * SCL_CONFIG
+ *     Protects changes to the vdev tree topology, such as vdev
+ *     add/remove/attach/detach.  Protects the dirty config list
+ *     (spa_config_dirty_list) and the set of spares and l2arc devices.
+ *
+ * SCL_STATE
+ *     Protects changes to pool state and vdev state, such as vdev
+ *     online/offline/fault/degrade/clear.  Protects the dirty state list
+ *     (spa_state_dirty_list) and global pool state (spa_state).
+ *
+ * SCL_ALLOC
+ *     Protects changes to metaslab groups and classes.
+ *     Held as reader by metaslab_alloc() and metaslab_claim().
+ *
+ * SCL_ZIO
+ *     Held by bp-level zios (those which have no io_vd upon entry)
+ *     to prevent changes to the vdev tree.  The bp-level zio implicitly
+ *     protects all of its vdev child zios, which do not hold SCL_ZIO.
+ *
+ * SCL_FREE
+ *     Protects changes to metaslab groups and classes.
+ *     Held as reader by metaslab_free().  SCL_FREE is distinct from
+ *     SCL_ALLOC, and lower than SCL_ZIO, so that we can safely free
+ *     blocks in zio_done() while another i/o that holds either
+ *     SCL_ALLOC or SCL_ZIO is waiting for this i/o to complete.
+ *
+ * SCL_VDEV
+ *     Held as reader to prevent changes to the vdev tree during trivial
+ *     inquiries such as bp_get_dasize().  SCL_VDEV is distinct from the
+ *     other locks, and lower than all of them, to ensure that it's safe
+ *     to acquire regardless of caller context.
+ *
+ * In addition, the following rules apply:
+ *
+ * (a) spa_props_lock protects pool properties, spa_config and spa_config_list.
+ *     The lock ordering is SCL_CONFIG > spa_props_lock.
+ *
+ * (b) I/O operations on leaf vdevs.  For any zio operation that takes
+ *     an explicit vdev_t argument -- such as zio_ioctl(), zio_read_phys(),
+ *     or zio_write_phys() -- the caller must ensure that the config cannot
+ *     cannot change in the interim, and that the vdev cannot be reopened.
+ *     SCL_STATE as reader suffices for both.
  *
  * The vdev configuration is protected by spa_vdev_enter() / spa_vdev_exit().
  *
  *                             to complete, sync the updated configs to the
  *                             cache, and release the namespace lock.
  *
- * The spa_name() function also requires either the spa_namespace_lock
- * or the spa_config_lock, as both are needed to do a rename.  spa_rename() is
- * also implemented within this file since is requires manipulation of the
- * namespace.
+ * vdev state is protected by spa_vdev_state_enter() / spa_vdev_state_exit().
+ * Like spa_vdev_enter/exit, these are convenience wrappers -- the actual
+ * locking is, always, based on spa_namespace_lock and spa_config_lock[].
+ *
+ * spa_rename() is also implemented within this file since is requires
+ * manipulation of the namespace.
  */
 
 static avl_tree_t spa_namespace_avl;
@@ -191,7 +246,6 @@ int zfs_flags = 0;
  */
 int zfs_recover = 0;
 
-#define        SPA_MINREF      5       /* spa_refcnt for an open-but-idle pool */
 
 /*
  * ==========================================================================
@@ -199,72 +253,120 @@ int zfs_recover = 0;
  * ==========================================================================
  */
 static void
-spa_config_lock_init(spa_config_lock_t *scl)
-{
-       mutex_init(&scl->scl_lock, NULL, MUTEX_DEFAULT, NULL);
-       scl->scl_writer = NULL;
-       cv_init(&scl->scl_cv, NULL, CV_DEFAULT, NULL);
-       refcount_create(&scl->scl_count);
+spa_config_lock_init(spa_t *spa)
+{
+       for (int i = 0; i < SCL_LOCKS; i++) {
+               spa_config_lock_t *scl = &spa->spa_config_lock[i];
+               mutex_init(&scl->scl_lock, NULL, MUTEX_DEFAULT, NULL);
+               cv_init(&scl->scl_cv, NULL, CV_DEFAULT, NULL);
+               refcount_create(&scl->scl_count);
+               scl->scl_writer = NULL;
+               scl->scl_write_wanted = 0;
+       }
 }
 
 static void
-spa_config_lock_destroy(spa_config_lock_t *scl)
+spa_config_lock_destroy(spa_t *spa)
+{
+       for (int i = 0; i < SCL_LOCKS; i++) {
+               spa_config_lock_t *scl = &spa->spa_config_lock[i];
+               mutex_destroy(&scl->scl_lock);
+               cv_destroy(&scl->scl_cv);
+               refcount_destroy(&scl->scl_count);
+               ASSERT(scl->scl_writer == NULL);
+               ASSERT(scl->scl_write_wanted == 0);
+       }
+}
+
+int
+spa_config_tryenter(spa_t *spa, int locks, void *tag, krw_t rw)
 {
-       mutex_destroy(&scl->scl_lock);
-       ASSERT(scl->scl_writer == NULL);
-       cv_destroy(&scl->scl_cv);
-       refcount_destroy(&scl->scl_count);
+       for (int i = 0; i < SCL_LOCKS; i++) {
+               spa_config_lock_t *scl = &spa->spa_config_lock[i];
+               if (!(locks & (1 << i)))
+                       continue;
+               mutex_enter(&scl->scl_lock);
+               if (rw == RW_READER) {
+                       if (scl->scl_writer || scl->scl_write_wanted) {
+                               mutex_exit(&scl->scl_lock);
+                               spa_config_exit(spa, locks ^ (1 << i), tag);
+                               return (0);
+                       }
+               } else {
+                       ASSERT(scl->scl_writer != curthread);
+                       if (!refcount_is_zero(&scl->scl_count)) {
+                               mutex_exit(&scl->scl_lock);
+                               spa_config_exit(spa, locks ^ (1 << i), tag);
+                               return (0);
+                       }
+                       scl->scl_writer = curthread;
+               }
+               (void) refcount_add(&scl->scl_count, tag);
+               mutex_exit(&scl->scl_lock);
+       }
+       return (1);
 }
 
 void
-spa_config_enter(spa_t *spa, krw_t rw, void *tag)
+spa_config_enter(spa_t *spa, int locks, void *tag, krw_t rw)
 {
-       spa_config_lock_t *scl = &spa->spa_config_lock;
-
-       mutex_enter(&scl->scl_lock);
-
-       if (rw == RW_READER) {
-               while (scl->scl_writer != NULL && scl->scl_writer != curthread)
-                       cv_wait(&scl->scl_cv, &scl->scl_lock);
-       } else {
-               while (!refcount_is_zero(&scl->scl_count) &&
-                   scl->scl_writer != curthread)
-                       cv_wait(&scl->scl_cv, &scl->scl_lock);
-               scl->scl_writer = curthread;
+       for (int i = 0; i < SCL_LOCKS; i++) {
+               spa_config_lock_t *scl = &spa->spa_config_lock[i];
+               if (!(locks & (1 << i)))
+                       continue;
+               mutex_enter(&scl->scl_lock);
+               if (rw == RW_READER) {
+                       while (scl->scl_writer || scl->scl_write_wanted) {
+                               cv_wait(&scl->scl_cv, &scl->scl_lock);
+                       }
+               } else {
+                       ASSERT(scl->scl_writer != curthread);
+                       while (!refcount_is_zero(&scl->scl_count)) {
+                               scl->scl_write_wanted++;
+                               cv_wait(&scl->scl_cv, &scl->scl_lock);
+                               scl->scl_write_wanted--;
+                       }
+                       scl->scl_writer = curthread;
+               }
+               (void) refcount_add(&scl->scl_count, tag);
+               mutex_exit(&scl->scl_lock);
        }
-
-       (void) refcount_add(&scl->scl_count, tag);
-
-       mutex_exit(&scl->scl_lock);
 }
 
 void
-spa_config_exit(spa_t *spa, void *tag)
+spa_config_exit(spa_t *spa, int locks, void *tag)
 {
-       spa_config_lock_t *scl = &spa->spa_config_lock;
-
-       mutex_enter(&scl->scl_lock);
-
-       ASSERT(!refcount_is_zero(&scl->scl_count));
-
-       if (refcount_remove(&scl->scl_count, tag) == 0) {
-               cv_broadcast(&scl->scl_cv);
-               ASSERT(scl->scl_writer == NULL || scl->scl_writer == curthread);
-               scl->scl_writer = NULL;  /* OK in either case */
+       for (int i = SCL_LOCKS - 1; i >= 0; i--) {
+               spa_config_lock_t *scl = &spa->spa_config_lock[i];
+               if (!(locks & (1 << i)))
+                       continue;
+               mutex_enter(&scl->scl_lock);
+               ASSERT(!refcount_is_zero(&scl->scl_count));
+               if (refcount_remove(&scl->scl_count, tag) == 0) {
+                       ASSERT(scl->scl_writer == NULL ||
+                           scl->scl_writer == curthread);
+                       scl->scl_writer = NULL; /* OK in either case */
+                       cv_broadcast(&scl->scl_cv);
+               }
+               mutex_exit(&scl->scl_lock);
        }
-
-       mutex_exit(&scl->scl_lock);
 }
 
-boolean_t
-spa_config_held(spa_t *spa, krw_t rw)
+int
+spa_config_held(spa_t *spa, int locks, krw_t rw)
 {
-       spa_config_lock_t *scl = &spa->spa_config_lock;
+       int locks_held = 0;
 
-       if (rw == RW_READER)
-               return (!refcount_is_zero(&scl->scl_count));
-       else
-               return (scl->scl_writer == curthread);
+       for (int i = 0; i < SCL_LOCKS; i++) {
+               spa_config_lock_t *scl = &spa->spa_config_lock[i];
+               if (!(locks & (1 << i)))
+                       continue;
+               if ((rw == RW_READER && !refcount_is_zero(&scl->scl_count)) ||
+                   (rw == RW_WRITER && scl->scl_writer == curthread))
+                       locks_held |= 1 << i;
+       }
+
+       return (locks_held);
 }
 
 /*
@@ -280,7 +382,8 @@ spa_config_held(spa_t *spa, krw_t rw)
 spa_t *
 spa_lookup(const char *name)
 {
-       spa_t search, *spa;
+       static spa_t search;    /* spa_t is large; don't allocate on stack */
+       spa_t *spa;
        avl_index_t where;
        char c;
        char *cp;
@@ -297,7 +400,7 @@ spa_lookup(const char *name)
                *cp = '\0';
        }
 
-       search.spa_name = (char *)name;
+       (void) strlcpy(search.spa_name, name, sizeof (search.spa_name));
        spa = avl_find(&spa_namespace_avl, &search, &where);
 
        if (cp)
@@ -315,16 +418,14 @@ spa_t *
 spa_add(const char *name, const char *altroot)
 {
        spa_t *spa;
+       spa_config_dirent_t *dp;
 
        ASSERT(MUTEX_HELD(&spa_namespace_lock));
 
        spa = kmem_zalloc(sizeof (spa_t), KM_SLEEP);
 
-       rw_init(&spa->spa_traverse_lock, NULL, RW_DEFAULT, NULL);
-
-       mutex_init(&spa->spa_uberblock_lock, NULL, MUTEX_DEFAULT, NULL);
        mutex_init(&spa->spa_async_lock, NULL, MUTEX_DEFAULT, NULL);
-       mutex_init(&spa->spa_config_cache_lock, NULL, MUTEX_DEFAULT, NULL);
+       mutex_init(&spa->spa_async_root_lock, NULL, MUTEX_DEFAULT, NULL);
        mutex_init(&spa->spa_scrub_lock, NULL, MUTEX_DEFAULT, NULL);
        mutex_init(&spa->spa_errlog_lock, NULL, MUTEX_DEFAULT, NULL);
        mutex_init(&spa->spa_errlist_lock, NULL, MUTEX_DEFAULT, NULL);
@@ -333,20 +434,21 @@ spa_add(const char *name, const char *altroot)
        mutex_init(&spa->spa_props_lock, NULL, MUTEX_DEFAULT, NULL);
 
        cv_init(&spa->spa_async_cv, NULL, CV_DEFAULT, NULL);
-       cv_init(&spa->spa_scrub_cv, NULL, CV_DEFAULT, NULL);
+       cv_init(&spa->spa_async_root_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);
 
-       spa->spa_name = spa_strdup(name);
+       (void) strlcpy(spa->spa_name, name, sizeof (spa->spa_name));
        spa->spa_state = POOL_STATE_UNINITIALIZED;
        spa->spa_freeze_txg = UINT64_MAX;
        spa->spa_final_txg = UINT64_MAX;
 
        refcount_create(&spa->spa_refcount);
-       spa_config_lock_init(&spa->spa_config_lock);
+       spa_config_lock_init(spa);
 
        avl_add(&spa_namespace_avl, spa);
 
-       mutex_init(&spa->spa_zio_lock, NULL, MUTEX_DEFAULT, NULL);
+       mutex_init(&spa->spa_suspend_lock, NULL, MUTEX_DEFAULT, NULL);
 
        /*
         * Set the alternate root, if there is one.
@@ -356,6 +458,16 @@ spa_add(const char *name, const char *altroot)
                spa_active_count++;
        }
 
+       /*
+        * Every pool starts with the default cachefile
+        */
+       list_create(&spa->spa_config_list, sizeof (spa_config_dirent_t),
+           offsetof(spa_config_dirent_t, scd_link));
+
+       dp = kmem_zalloc(sizeof (spa_config_dirent_t), KM_SLEEP);
+       dp->scd_path = spa_strdup(spa_config_path);
+       list_insert_head(&spa->spa_config_list, dp);
+
        return (spa);
 }
 
@@ -367,9 +479,10 @@ spa_add(const char *name, const char *altroot)
 void
 spa_remove(spa_t *spa)
 {
+       spa_config_dirent_t *dp;
+
        ASSERT(MUTEX_HELD(&spa_namespace_lock));
        ASSERT(spa->spa_state == POOL_STATE_UNINITIALIZED);
-       ASSERT(spa->spa_scrub_thread == NULL);
 
        avl_remove(&spa_namespace_avl, spa);
        cv_broadcast(&spa_namespace_cv);
@@ -379,36 +492,35 @@ spa_remove(spa_t *spa)
                spa_active_count--;
        }
 
-       if (spa->spa_name)
-               spa_strfree(spa->spa_name);
+       while ((dp = list_head(&spa->spa_config_list)) != NULL) {
+               list_remove(&spa->spa_config_list, dp);
+               if (dp->scd_path != NULL)
+                       spa_strfree(dp->scd_path);
+               kmem_free(dp, sizeof (spa_config_dirent_t));
+       }
 
-       if (spa->spa_config_dir)
-               spa_strfree(spa->spa_config_dir);
-       if (spa->spa_config_file)
-               spa_strfree(spa->spa_config_file);
+       list_destroy(&spa->spa_config_list);
 
        spa_config_set(spa, NULL);
 
        refcount_destroy(&spa->spa_refcount);
 
-       spa_config_lock_destroy(&spa->spa_config_lock);
-
-       rw_destroy(&spa->spa_traverse_lock);
+       spa_config_lock_destroy(spa);
 
        cv_destroy(&spa->spa_async_cv);
-       cv_destroy(&spa->spa_scrub_cv);
+       cv_destroy(&spa->spa_async_root_cv);
        cv_destroy(&spa->spa_scrub_io_cv);
+       cv_destroy(&spa->spa_suspend_cv);
 
-       mutex_destroy(&spa->spa_uberblock_lock);
        mutex_destroy(&spa->spa_async_lock);
-       mutex_destroy(&spa->spa_config_cache_lock);
+       mutex_destroy(&spa->spa_async_root_lock);
        mutex_destroy(&spa->spa_scrub_lock);
        mutex_destroy(&spa->spa_errlog_lock);
        mutex_destroy(&spa->spa_errlist_lock);
        mutex_destroy(&spa->spa_sync_bplist.bpl_lock);
        mutex_destroy(&spa->spa_history_lock);
        mutex_destroy(&spa->spa_props_lock);
-       mutex_destroy(&spa->spa_zio_lock);
+       mutex_destroy(&spa->spa_suspend_lock);
 
        kmem_free(spa, sizeof (spa_t));
 }
@@ -441,9 +553,8 @@ spa_next(spa_t *prev)
 void
 spa_open_ref(spa_t *spa, void *tag)
 {
-       ASSERT(refcount_count(&spa->spa_refcount) > SPA_MINREF ||
+       ASSERT(refcount_count(&spa->spa_refcount) >= spa->spa_minref ||
            MUTEX_HELD(&spa_namespace_lock));
-
        (void) refcount_add(&spa->spa_refcount, tag);
 }
 
@@ -454,15 +565,14 @@ spa_open_ref(spa_t *spa, void *tag)
 void
 spa_close(spa_t *spa, void *tag)
 {
-       ASSERT(refcount_count(&spa->spa_refcount) > SPA_MINREF ||
+       ASSERT(refcount_count(&spa->spa_refcount) > spa->spa_minref ||
            MUTEX_HELD(&spa_namespace_lock));
-
        (void) refcount_remove(&spa->spa_refcount, tag);
 }
 
 /*
  * Check to see if the spa refcount is zero.  Must be called with
- * spa_namespace_lock held.  We really compare against SPA_MINREF, which is the
+ * spa_namespace_lock held.  We really compare against spa_minref, which is the
  * number of references acquired when opening a pool
  */
 boolean_t
@@ -470,7 +580,7 @@ spa_refcount_zero(spa_t *spa)
 {
        ASSERT(MUTEX_HELD(&spa_namespace_lock));
 
-       return (refcount_count(&spa->spa_refcount) == SPA_MINREF);
+       return (refcount_count(&spa->spa_refcount) == spa->spa_minref);
 }
 
 /*
@@ -544,13 +654,12 @@ spa_aux_remove(vdev_t *vd, avl_tree_t *avl)
 }
 
 boolean_t
-spa_aux_exists(uint64_t guid, uint64_t *pool, avl_tree_t *avl)
+spa_aux_exists(uint64_t guid, uint64_t *pool, int *refcnt, avl_tree_t *avl)
 {
        spa_aux_t search, *found;
-       avl_index_t where;
 
        search.aux_guid = guid;
-       found = avl_find(avl, &search, &where);
+       found = avl_find(avl, &search, NULL);
 
        if (pool) {
                if (found)
@@ -559,6 +668,13 @@ spa_aux_exists(uint64_t guid, uint64_t *pool, avl_tree_t *avl)
                        *pool = 0ULL;
        }
 
+       if (refcnt) {
+               if (found)
+                       *refcnt = found->aux_count;
+               else
+                       *refcnt = 0;
+       }
+
        return (found != NULL);
 }
 
@@ -625,12 +741,12 @@ spa_spare_remove(vdev_t *vd)
 }
 
 boolean_t
-spa_spare_exists(uint64_t guid, uint64_t *pool)
+spa_spare_exists(uint64_t guid, uint64_t *pool, int *refcnt)
 {
        boolean_t found;
 
        mutex_enter(&spa_spare_lock);
-       found = spa_aux_exists(guid, pool, &spa_spare_avl);
+       found = spa_aux_exists(guid, pool, refcnt, &spa_spare_avl);
        mutex_exit(&spa_spare_lock);
 
        return (found);
@@ -683,7 +799,7 @@ spa_l2cache_exists(uint64_t guid, uint64_t *pool)
        boolean_t found;
 
        mutex_enter(&spa_l2cache_lock);
-       found = spa_aux_exists(guid, pool, &spa_l2cache_avl);
+       found = spa_aux_exists(guid, pool, NULL, &spa_l2cache_avl);
        mutex_exit(&spa_l2cache_lock);
 
        return (found);
@@ -720,14 +836,7 @@ spa_vdev_enter(spa_t *spa)
 {
        mutex_enter(&spa_namespace_lock);
 
-       /*
-        * Suspend scrub activity while we mess with the config.  We must do
-        * this after acquiring the namespace lock to avoid a 3-way deadlock
-        * with spa_scrub_stop() and the scrub thread.
-        */
-       spa_scrub_suspend(spa);
-
-       spa_config_enter(spa, RW_WRITER, spa);
+       spa_config_enter(spa, SCL_ALL, spa, RW_WRITER);
 
        return (spa_last_synced_txg(spa) + 1);
 }
@@ -745,6 +854,8 @@ spa_vdev_exit(spa_t *spa, vdev_t *vd, uint64_t txg, int error)
 
        ASSERT(txg > spa_last_synced_txg(spa));
 
+       spa->spa_pending_vdev = NULL;
+
        /*
         * Reassess the DTLs.
         */
@@ -753,17 +864,12 @@ spa_vdev_exit(spa_t *spa, vdev_t *vd, uint64_t txg, int error)
        /*
         * If the config changed, notify the scrub thread that it must restart.
         */
-       if (error == 0 && !list_is_empty(&spa->spa_dirty_list)) {
+       if (error == 0 && !list_is_empty(&spa->spa_config_dirty_list)) {
+               dsl_pool_scrub_restart(spa->spa_dsl_pool);
                config_changed = B_TRUE;
-               spa_scrub_restart(spa, txg);
        }
 
-       spa_config_exit(spa, spa);
-
-       /*
-        * Allow scrubbing to resume.
-        */
-       spa_scrub_resume(spa);
+       spa_config_exit(spa, SCL_ALL, spa);
 
        /*
         * Note: this txg_wait_synced() is important because it ensures
@@ -782,7 +888,7 @@ spa_vdev_exit(spa_t *spa, vdev_t *vd, uint64_t txg, int error)
         * If the config changed, update the config cache.
         */
        if (config_changed)
-               spa_config_sync();
+               spa_config_sync(spa, B_FALSE, B_TRUE);
 
        mutex_exit(&spa_namespace_lock);
 
@@ -790,6 +896,26 @@ spa_vdev_exit(spa_t *spa, vdev_t *vd, uint64_t txg, int error)
 }
 
 /*
+ * Lock the given spa_t for the purpose of changing vdev state.
+ */
+void
+spa_vdev_state_enter(spa_t *spa)
+{
+       spa_config_enter(spa, SCL_STATE_ALL, spa, RW_WRITER);
+}
+
+int
+spa_vdev_state_exit(spa_t *spa, vdev_t *vd, int error)
+{
+       if (vd != NULL)
+               vdev_state_dirty(vd->vdev_top);
+
+       spa_config_exit(spa, SCL_STATE_ALL, spa);
+
+       return (error);
+}
+
+/*
  * ==========================================================================
  * Miscellaneous functions
  * ==========================================================================
@@ -816,11 +942,10 @@ spa_rename(const char *name, const char *newname)
                return (err);
        }
 
-       spa_config_enter(spa, RW_WRITER, FTAG);
+       spa_config_enter(spa, SCL_ALL, FTAG, RW_WRITER);
 
        avl_remove(&spa_namespace_avl, spa);
-       spa_strfree(spa->spa_name);
-       spa->spa_name = spa_strdup(newname);
+       (void) strlcpy(spa->spa_name, newname, sizeof (spa->spa_name));
        avl_add(&spa_namespace_avl, spa);
 
        /*
@@ -830,14 +955,14 @@ spa_rename(const char *name, const char *newname)
         */
        vdev_config_dirty(spa->spa_root_vdev);
 
-       spa_config_exit(spa, FTAG);
+       spa_config_exit(spa, SCL_ALL, FTAG);
 
        txg_wait_synced(spa->spa_dsl_pool, 0);
 
        /*
         * Sync the updated config cache.
         */
-       spa_config_sync();
+       spa_config_sync(spa, B_FALSE, B_TRUE);
 
        spa_close(spa, FTAG);
 
@@ -968,12 +1093,12 @@ spa_freeze(spa_t *spa)
 {
        uint64_t freeze_txg = 0;
 
-       spa_config_enter(spa, RW_WRITER, FTAG);
+       spa_config_enter(spa, SCL_ALL, FTAG, RW_WRITER);
        if (spa->spa_freeze_txg == UINT64_MAX) {
                freeze_txg = spa_last_synced_txg(spa) + TXG_SIZE;
                spa->spa_freeze_txg = freeze_txg;
        }
-       spa_config_exit(spa, FTAG);
+       spa_config_exit(spa, SCL_ALL, FTAG);
        if (freeze_txg != 0)
                txg_wait_synced(spa_get_dsl(spa), freeze_txg);
 }
@@ -994,16 +1119,10 @@ zfs_panic_recover(const char *fmt, ...)
  * ==========================================================================
  */
 
-krwlock_t *
-spa_traverse_rwlock(spa_t *spa)
-{
-       return (&spa->spa_traverse_lock);
-}
-
-int
-spa_traverse_wanted(spa_t *spa)
+boolean_t
+spa_shutting_down(spa_t *spa)
 {
-       return (spa->spa_traverse_wanted);
+       return (spa->spa_async_suspended);
 }
 
 dsl_pool_t *
@@ -1042,13 +1161,6 @@ spa_sync_pass(spa_t *spa)
 char *
 spa_name(spa_t *spa)
 {
-       /*
-        * Accessing the name requires holding either the namespace lock or the
-        * config lock, both of which are required to do a rename.
-        */
-       ASSERT(MUTEX_HELD(&spa_namespace_lock) ||
-           spa_config_held(spa, RW_READER));
-
        return (spa->spa_name);
 }
 
@@ -1079,7 +1191,7 @@ spa_first_txg(spa_t *spa)
        return (spa->spa_first_txg);
 }
 
-int
+pool_state_t
 spa_state(spa_t *spa)
 {
        return (spa->spa_state);
@@ -1144,6 +1256,12 @@ spa_get_failmode(spa_t *spa)
        return (spa->spa_failmode);
 }
 
+boolean_t
+spa_suspended(spa_t *spa)
+{
+       return (spa->spa_suspended);
+}
+
 uint64_t
 spa_version(spa_t *spa)
 {
@@ -1171,7 +1289,7 @@ bp_get_dasize(spa_t *spa, const blkptr_t *bp)
        if (!spa->spa_deflate)
                return (BP_GET_ASIZE(bp));
 
-       spa_config_enter(spa, RW_READER, FTAG);
+       spa_config_enter(spa, SCL_VDEV, FTAG, RW_READER);
        for (i = 0; i < SPA_DVAS_PER_BP; i++) {
                vdev_t *vd =
                    vdev_lookup_top(spa, DVA_GET_VDEV(&bp->blk_dva[i]));
@@ -1179,7 +1297,7 @@ bp_get_dasize(spa_t *spa, const blkptr_t *bp)
                        sz += (DVA_GET_ASIZE(&bp->blk_dva[i]) >>
                            SPA_MINBLOCKSHIFT) * vd->vdev_deflate_ratio;
        }
-       spa_config_exit(spa, FTAG);
+       spa_config_exit(spa, SCL_VDEV, FTAG);
        return (sz);
 }
 
@@ -1244,11 +1362,14 @@ spa_init(int mode)
        zfs_prop_init();
        zpool_prop_init();
        spa_config_load();
+       l2arc_start();
 }
 
 void
 spa_fini(void)
 {
+       l2arc_stop();
+
        spa_evict_all();
 
        vdev_cache_stat_fini();
@@ -1278,3 +1399,12 @@ spa_has_slogs(spa_t *spa)
 {
        return (spa->spa_log_class->mc_rotor != NULL);
 }
+
+/*
+ * Return whether this pool is the root pool.
+ */
+boolean_t
+spa_is_root(spa_t *spa)
+{
+       return (spa->spa_is_root);
+}