Print this page
OS-1566 filesystem limits for ZFS datasets

@@ -19,10 +19,11 @@
  * CDDL HEADER END
  */
 /*
  * Copyright (c) 2005, 2010, Oracle and/or its affiliates. All rights reserved.
  * Copyright (c) 2012 by Delphix. All rights reserved.
+ * Copyright (c) 2012 Joyent, Inc. All rights reserved.
  */
 
 #include <sys/dmu.h>
 #include <sys/dmu_objset.h>
 #include <sys/dmu_tx.h>

@@ -36,16 +37,93 @@
 #include <sys/zap.h>
 #include <sys/zio.h>
 #include <sys/arc.h>
 #include <sys/sunddi.h>
 #include <sys/zfs_zone.h>
+#include <sys/zfeature.h>
+#include <sys/policy.h>
+#include <sys/zfs_znode.h>
 #include "zfs_namecheck.h"
+#include "zfs_prop.h"
 
+/*
+ * Filesystem and Snapshot Limits
+ * ------------------------------
+ *
+ * These limits are used to restrict the number of filesystems and/or snapshots
+ * that can be created at a given level in the tree or below. A typical
+ * use-case is with a delegated dataset where the administrator wants to ensure
+ * that a user within the zone is not creating too many additional filesystems
+ * or snapshots, even though they're not exceeding their space quota.
+ *
+ * The count of filesystems and snapshots is stored in the dsl_dir_phys_t which
+ * impacts the on-disk format. As such, this capability is controlled by a
+ * feature flag and must be enabled to be used. Once enabled, the feature is
+ * not active until the first limit is set. At that point, future operations to
+ * create/destroy filesystems or snapshots will validate and update the counts.
+ *
+ * Because the on-disk counts will be uninitialized (0) before the feature is
+ * active, the counts are updated when a limit is first set on an uninitialized
+ * node (The filesystem/snapshot counts on a node includes all of the nested
+ * filesystems/snapshots, plus the node itself. Thus, a new leaf node has a
+ * filesystem count of 1 and a snapshot count of 0. A filesystem count of 0 on
+ * a node indicates uninitialized counts on that node.) When setting a limit on
+ * an uninitialized node, the code starts at the filesystem with the new limit
+ * and descends into all sub-filesystems and updates the counts to be accurate.
+ * In practice this is lightweight since a limit is typically set when the
+ * filesystem is created and thus has no children. Once valid, changing the
+ * limit value won't require a re-traversal since the counts are already valid.
+ * When recursively fixing the counts, if a node with a limit is encountered
+ * during the descent, the counts are known to be valid and there is no need to
+ * descend into that filesystem's children. The counts on filesystems above the
+ * one with the new limit will still be uninitialized (0), unless a limit is
+ * eventually set on one of those filesystems. The counts are always recursively
+ * updated when a limit is set on a dataset, unless there is already a limit.
+ * When a new limit value is set on a filesystem with an existing limit, it is
+ * possible for the new limit to be less than the current count at that level
+ * since a user who can change the limit is also allowed to exceed the limit.
+ *
+ * Once the feature is active, then whenever a filesystem or snapshot is
+ * created, the code recurses up the tree, validating the new count against the
+ * limit at each initialized level. In practice, most levels will not have a
+ * limit set. If there is a limit at any initialized level up the tree, the
+ * check must pass or the creation will fail. Likewise, when a filesystem or
+ * snapshot is destroyed, the counts are recursively adjusted all the way up
+ * the initizized nodes in the tree. Renaming a filesystem into different point
+ * in the tree will first validate, then update the counts on each branch up to
+ * the common ancestor. A receive will also validate the counts and then update
+ * them.
+ *
+ * An exception to the above behavior is that the limit is not enforced if the
+ * user has permission to modify the limit. This is primarily so that
+ * recursive snapshots in the global zone always work. We want to prevent a
+ * denial-of-service in which a lower level delegated dataset could max out its
+ * limit and thus block recursive snapshots from being taken in the global zone.
+ * Because of this, it is possible for the snapshot count to be over the limit
+ * and snapshots taken in the global zone could cause a lower level dataset to
+ * hit or exceed its limit. The administrator taking the global zone recursive
+ * snapshot should be aware of this side-effect and behave accordingly.
+ * For consistency, the filesystem limit is also not enforced if the user can
+ * modify the limit.
+ *
+ * The filesystem limit is validated by dsl_dir_fscount_check() and updated by
+ * dsl_dir_fscount_adjust(). The snapshot limit is validated by
+ * dsl_snapcount_check() and updated by dsl_snapcount_adjust().
+ * A new limit value is validated in dsl_dir_validate_fs_ss_limit() and the
+ * filesystem counts are adjusted, if necessary, by dsl_dir_set_fs_ss_count().
+ *
+ * There is a special case when we receive a filesystem that already exists. In
+ * this case a temporary clone name of %X is created (see dmu_recv_begin). We
+ * never update the filesystem counts for temporary clones.
+ */
+
 static uint64_t dsl_dir_space_towrite(dsl_dir_t *dd);
 static void dsl_dir_set_reservation_sync_impl(dsl_dir_t *dd,
     uint64_t value, dmu_tx_t *tx);
 
+extern dsl_syncfunc_t dsl_prop_set_sync;
+
 /* ARGSUSED */
 static void
 dsl_dir_evict(dmu_buf_t *db, void *arg)
 {
         dsl_dir_t *dd = arg;

@@ -405,19 +483,407 @@
 dsl_dir_open(const char *name, void *tag, dsl_dir_t **ddp, const char **tailp)
 {
         return (dsl_dir_open_spa(NULL, name, tag, ddp, tailp));
 }
 
+/*
+ * Check if the counts are already valid for this filesystem and its
+ * descendants. The counts on this filesystem, and those below, may be
+ * uninitialized due to either the use of a pre-existing pool which did not
+ * support the filesystem/snapshot limit feature, or one in which the feature
+ * had not yet been enabled.
+ *
+ * Recursively descend the filesystem tree and update the filesystem/snapshot
+ * counts on each filesystem below, then update the cumulative count on the
+ * current filesystem. If the filesystem already has a limit set on it,
+ * then we know that its counts, and the counts on the filesystems below it,
+ * have been updated to be correct, so we can skip this filesystem.
+ */
+static int
+dsl_dir_set_fs_ss_count(dsl_dir_t *dd, dmu_tx_t *tx, uint64_t *fscnt,
+    uint64_t *sscnt)
+{
+        uint64_t my_fs_cnt = 0;
+        uint64_t my_ss_cnt = 0;
+        uint64_t curr_ss_cnt;
+        objset_t *os = dd->dd_pool->dp_meta_objset;
+        zap_cursor_t *zc;
+        zap_attribute_t *za;
+        int err;
+        int ret = 0;
+        boolean_t limit_set = B_FALSE;
+        uint64_t fslimit, sslimit;
+        dsl_dataset_t *ds;
+
+        ASSERT(RW_LOCK_HELD(&dd->dd_pool->dp_config_rwlock));
+
+        err = dsl_prop_get_dd(dd, zfs_prop_to_name(ZFS_PROP_FILESYSTEM_LIMIT),
+            8, 1, &fslimit, NULL, B_FALSE);
+        if (err == 0 && fslimit != UINT64_MAX)
+                limit_set = B_TRUE;
+
+        if (!limit_set) {
+                err = dsl_prop_get_dd(dd,
+                    zfs_prop_to_name(ZFS_PROP_SNAPSHOT_LIMIT), 8, 1, &sslimit,
+                    NULL, B_FALSE);
+                if (err == 0 && sslimit != UINT64_MAX)
+                        limit_set = B_TRUE;
+        }
+
+        /*
+         * If the dd has a limit, we know its count is already good and we
+         * don't need to recurse down any further.
+         */
+        if (limit_set) {
+                *fscnt = dd->dd_phys->dd_filesystem_count;
+                *sscnt = dd->dd_phys->dd_snapshot_count;
+                return (ret);
+        }
+
+        zc = kmem_alloc(sizeof (zap_cursor_t), KM_SLEEP);
+        za = kmem_alloc(sizeof (zap_attribute_t), KM_SLEEP);
+
+        mutex_enter(&dd->dd_lock);
+
+        /* Iterate datasets */
+        for (zap_cursor_init(zc, os, dd->dd_phys->dd_child_dir_zapobj);
+            zap_cursor_retrieve(zc, za) == 0; zap_cursor_advance(zc)) {
+                dsl_dir_t *chld_dd;
+                uint64_t chld_fs_cnt = 0;
+                uint64_t chld_ss_cnt = 0;
+
+                if (dsl_dir_open_obj(dd->dd_pool,
+                    ZFS_DIRENT_OBJ(za->za_first_integer), NULL, FTAG,
+                    &chld_dd)) {
+                        ret = 1;
+                        break;
+                }
+
+                if (dsl_dir_set_fs_ss_count(chld_dd, tx, &chld_fs_cnt,
+                    &chld_ss_cnt)) {
+                        ret = 1;
+                        break;
+                }
+
+                dsl_dir_close(chld_dd, FTAG);
+
+                my_fs_cnt += chld_fs_cnt;
+                my_ss_cnt += chld_ss_cnt;
+        }
+        zap_cursor_fini(zc);
+        kmem_free(zc, sizeof (zap_cursor_t));
+        kmem_free(za, sizeof (zap_attribute_t));
+
+        /* Count snapshots */
+        if (dsl_dataset_hold_obj(dd->dd_pool, dd->dd_phys->dd_head_dataset_obj,
+            FTAG, &ds) == 0) {
+                if (zap_count(os, ds->ds_phys->ds_snapnames_zapobj,
+                    &curr_ss_cnt) == 0)
+                        my_ss_cnt += curr_ss_cnt;
+                else
+                        ret = 1;
+                dsl_dataset_rele(ds, FTAG);
+        } else {
+                ret = 1;
+        }
+
+        /* Add 1 for self */
+        my_fs_cnt++;
+
+        /* save updated counts */
+        dmu_buf_will_dirty(dd->dd_dbuf, tx);
+        dd->dd_phys->dd_filesystem_count = my_fs_cnt;
+        dd->dd_phys->dd_snapshot_count = my_ss_cnt;
+
+        mutex_exit(&dd->dd_lock);
+
+        /* Return child dataset count plus self */
+        *fscnt = my_fs_cnt;
+        *sscnt = my_ss_cnt;
+        return (ret);
+}
+
+/* ARGSUSED */
+static int
+fs_ss_limit_feat_check(void *arg1, void *arg2, dmu_tx_t *tx)
+{
+        return (0);
+}
+
+/* ARGSUSED */
+static void
+fs_ss_limit_feat_sync(void *arg1, void *arg2, dmu_tx_t *tx)
+{
+        spa_t *spa = arg1;
+        zfeature_info_t *limit_feat =
+            &spa_feature_table[SPA_FEATURE_FS_SS_LIMIT];
+
+        spa_feature_incr(spa, limit_feat, tx);
+}
+
+/*
+ * Make sure the feature is enabled and activate it if necessary.
+ * If setting a limit, ensure the on-disk counts are valid.
+ *
+ * We do not validate the new limit, since users who can change the limit are
+ * also allowed to exceed the limit.
+ *
+ * Return -1 to force the zfs_set_prop_nvlist code down the default path to set
+ * the value in the nvlist.
+ */
+int
+dsl_dir_validate_fs_ss_limit(const char *ddname, uint64_t limit,
+    zfs_prop_t ptype)
+{
+        dsl_dir_t *dd;
+        dsl_dataset_t *ds;
+        int err;
+        dmu_tx_t *tx;
+        uint64_t my_fs_cnt = 0;
+        uint64_t my_ss_cnt = 0;
+        uint64_t curr_limit;
+        spa_t *spa;
+        zfeature_info_t *limit_feat =
+            &spa_feature_table[SPA_FEATURE_FS_SS_LIMIT];
+
+        if ((err = dsl_dataset_hold(ddname, FTAG, &ds)) != 0)
+                return (err);
+
+        spa = dsl_dataset_get_spa(ds);
+        if (!spa_feature_is_enabled(spa,
+            &spa_feature_table[SPA_FEATURE_FS_SS_LIMIT])) {
+                dsl_dataset_rele(ds, FTAG);
+                return (ENOTSUP);
+        }
+
+        dd = ds->ds_dir;
+
+        if ((err = dsl_prop_get_dd(dd, zfs_prop_to_name(ptype), 8, 1,
+            &curr_limit, NULL, B_FALSE)) != 0) {
+                dsl_dataset_rele(ds, FTAG);
+                return (err);
+        }
+
+        if (limit == UINT64_MAX) {
+                /*
+                 * If we had a limit, since we're now removing that limit, this
+                 * is where we could decrement the feature-active counter so
+                 * that the feature becomes inactive (only enabled) if we
+                 * remove the last limit. However, we do not currently support
+                 * deactivating the feature.
+                 */
+                dsl_dataset_rele(ds, FTAG);
+                return (-1);
+        }
+
+        if (!spa_feature_is_active(spa, limit_feat)) {
+                /*
+                 * Since the feature was not active and we're now setting a
+                 * limit, increment the feature-active counter so that the
+                 * feature becomes active for the first time.
+                 *
+                 * We can't update the MOS in open context, so create a sync
+                 * task.
+                 */
+                err = dsl_sync_task_do(dd->dd_pool, fs_ss_limit_feat_check,
+                    fs_ss_limit_feat_sync, spa, (void *)1, 0);
+                if (err != 0)
+                        return (err);
+        }
+
+        tx = dmu_tx_create_dd(dd);
+        if (dmu_tx_assign(tx, TXG_WAIT)) {
+                dmu_tx_abort(tx);
+                dsl_dataset_rele(ds, FTAG);
+                return (ENOSPC);
+        }
+
+        /*
+         * Since we are now setting a non-UINT64_MAX on the filesystem, we need
+         * to ensure the counts are correct. Descend down the tree from this
+         * point and update all of the counts to be accurate.
+         */
+        err = -1;
+        rw_enter(&dd->dd_pool->dp_config_rwlock, RW_READER);
+        if (dsl_dir_set_fs_ss_count(dd, tx, &my_fs_cnt, &my_ss_cnt))
+                err = ENOSPC;
+        rw_exit(&dd->dd_pool->dp_config_rwlock);
+
+        dmu_tx_commit(tx);
+        dsl_dataset_rele(ds, FTAG);
+
+        return (err);
+}
+
+/*
+ * Used to determine if the filesystem_limit or snapshot_limit should be
+ * enforced. We allow the limit to be exceeded if the user has permission to
+ * write the property value. We pass in the creds that we got in the open
+ * context since we will always be the GZ root in syncing context.
+ *
+ * We can never modify these two properties within a non-global zone. In
+ * addition, the other checks are modeled on zfs_secpolicy_write_perms. We
+ * can't use that function since we are already holding the dp_config_rwlock.
+ * In addition, we already have the dd and dealing with snapshots is simplified.
+ */
+int
+dsl_secpolicy_write_prop(dsl_dir_t *dd, zfs_prop_t prop, cred_t *cr)
+{
+        int err = 0;
+        uint64_t obj;
+        dsl_dataset_t *ds;
+        uint64_t zoned;
+
+#ifdef _KERNEL
+        if (crgetzoneid(cr) != GLOBAL_ZONEID)
+                return (EPERM);
+
+        if (secpolicy_zfs(cr) == 0)
+                return (0);
+#endif
+
+        if ((obj = dd->dd_phys->dd_head_dataset_obj) == NULL)
+                return (ENOENT);
+
+        ASSERT(RW_LOCK_HELD(&dd->dd_pool->dp_config_rwlock));
+
+        if ((err = dsl_dataset_hold_obj(dd->dd_pool, obj, FTAG, &ds)) != 0)
+                return (err);
+
+        if (dsl_prop_get_ds(ds, "zoned", 8, 1, &zoned, NULL) || zoned) {
+                /* Only root can access zoned fs's from the GZ */
+                err = EPERM;
+        } else {
+                err = dsl_deleg_access_impl(ds, zfs_prop_to_name(prop), cr,
+                    B_FALSE);
+        }
+
+        dsl_dataset_rele(ds, FTAG);
+        return (err);
+}
+
+/*
+ * Check if adding additional child filesystem(s) would exceed any filesystem
+ * limits. Note that all filesystem limits up to the root (or the highest
+ * initialized) filesystem or the given ancestor must be satisfied.
+ */
+int
+dsl_dir_fscount_check(dsl_dir_t *dd, uint64_t cnt, dsl_dir_t *ancestor,
+    cred_t *cr)
+{
+        uint64_t limit;
+        int err = 0;
+
+        VERIFY(RW_LOCK_HELD(&dd->dd_pool->dp_config_rwlock));
+
+        /* If we're allowed to change the limit, don't enforce the limit. */
+        if (dsl_secpolicy_write_prop(dd, ZFS_PROP_FILESYSTEM_LIMIT, cr) == 0)
+                return (0);
+
+        /*
+         * If an ancestor has been provided, stop checking the limit once we
+         * hit that dir. We need this during rename so that we don't overcount
+         * the check once we recurse up to the common ancestor.
+         */
+        if (ancestor == dd)
+                return (0);
+
+        /*
+         * If we hit an uninitialized node while recursing up the tree, we can
+         * stop since we know the counts are not valid on this node and we
+         * know we won't touch this node's counts.
+         */
+        if (dd->dd_phys->dd_filesystem_count == 0)
+                return (0);
+
+        err = dsl_prop_get_dd(dd, zfs_prop_to_name(ZFS_PROP_FILESYSTEM_LIMIT),
+            8, 1, &limit, NULL, B_FALSE);
+        if (err != 0)
+                return (err);
+
+        /* Is there a fs limit which we've hit? */
+        if ((dd->dd_phys->dd_filesystem_count + cnt) > limit)
+                return (EDQUOT);
+
+        if (dd->dd_parent != NULL)
+                err = dsl_dir_fscount_check(dd->dd_parent, cnt, ancestor, cr);
+
+        return (err);
+}
+
+/*
+ * Adjust the filesystem count for the specified dsl_dir_t and all parent
+ * filesystems. When a new filesystem is created, increment the count on all
+ * parents, and when a filesystem is destroyed, decrement the count.
+ */
+void
+dsl_dir_fscount_adjust(dsl_dir_t *dd, dmu_tx_t *tx, int64_t delta,
+    boolean_t first)
+{
+        if (first) {
+                VERIFY(RW_LOCK_HELD(&dd->dd_pool->dp_config_rwlock));
+                VERIFY(dmu_tx_is_syncing(tx));
+        }
+
+        /*
+         * When we receive an incremental stream into a filesystem that already
+         * exists, a temporary clone is created.  We don't count this temporary
+         * clone, whose name begins with a '%'.
+         */
+        if (dd->dd_myname[0] == '%')
+                return;
+
+        /*
+         * If we hit an uninitialized node while recursing up the tree, we can
+         * stop since we know the counts are not valid on this node and we
+         * know we shouldn't touch this node's counts. An uninitialized count
+         * on the node indicates that either the feature has not yet been
+         * activated or there are no limits on this part of the tree.
+         */
+        if (dd->dd_phys->dd_filesystem_count == 0)
+                return;
+
+        /*
+         * On initial entry we need to check if this feature is active, but
+         * we don't want to re-check this on each recursive call. Note: the
+         * feature cannot be active if its not enabled. If the feature is not
+         * active, don't touch the on-disk count fields.
+         */
+        if (first) {
+                zfeature_info_t *quota_feat =
+                    &spa_feature_table[SPA_FEATURE_FS_SS_LIMIT];
+
+                if (!spa_feature_is_active(dd->dd_pool->dp_spa, quota_feat))
+                        return;
+        }
+
+        dmu_buf_will_dirty(dd->dd_dbuf, tx);
+
+        mutex_enter(&dd->dd_lock);
+
+        dd->dd_phys->dd_filesystem_count += delta;
+        VERIFY(dd->dd_phys->dd_filesystem_count >= 1);  /* ourself is 1 */
+
+        /* Roll up this additional count into our ancestors */
+        if (dd->dd_parent != NULL)
+                dsl_dir_fscount_adjust(dd->dd_parent, tx, delta, B_FALSE);
+
+        mutex_exit(&dd->dd_lock);
+}
+
 uint64_t
 dsl_dir_create_sync(dsl_pool_t *dp, dsl_dir_t *pds, const char *name,
     dmu_tx_t *tx)
 {
         objset_t *mos = dp->dp_meta_objset;
         uint64_t ddobj;
         dsl_dir_phys_t *ddphys;
         dmu_buf_t *dbuf;
+        zfeature_info_t *limit_feat =
+            &spa_feature_table[SPA_FEATURE_FS_SS_LIMIT];
 
+
         ddobj = dmu_object_alloc(mos, DMU_OT_DSL_DIR, 0,
             DMU_OT_DSL_DIR, sizeof (dsl_dir_phys_t), tx);
         if (pds) {
                 VERIFY(0 == zap_add(mos, pds->dd_phys->dd_child_dir_zapobj,
                     name, sizeof (uint64_t), 1, &ddobj, tx));

@@ -429,10 +895,13 @@
         VERIFY(0 == dmu_bonus_hold(mos, ddobj, FTAG, &dbuf));
         dmu_buf_will_dirty(dbuf, tx);
         ddphys = dbuf->db_data;
 
         ddphys->dd_creation_time = gethrestime_sec();
+        /* Only initialize the count if the limit feature is active */
+        if (spa_feature_is_active(dp->dp_spa, limit_feat))
+                ddphys->dd_filesystem_count = 1;
         if (pds)
                 ddphys->dd_parent_obj = pds->dd_object;
         ddphys->dd_props_zapobj = zap_create(mos,
             DMU_OT_DSL_PROPS, DMU_OT_NONE, 0, tx);
         ddphys->dd_child_dir_zapobj = zap_create(mos,

@@ -486,10 +955,20 @@
 
         ASSERT(RW_WRITE_HELD(&dd->dd_pool->dp_config_rwlock));
         ASSERT(dd->dd_phys->dd_head_dataset_obj == 0);
 
         /*
+         * Decrement the filesystem count for all parent filesystems.
+         *
+         * When we receive an incremental stream into a filesystem that already
+         * exists, a temporary clone is created.  We never count this temporary
+         * clone, whose name begins with a '%'.
+         */
+        if (dd->dd_myname[0] != '%' && dd->dd_parent != NULL)
+                dsl_dir_fscount_adjust(dd->dd_parent, tx, -1, B_TRUE);
+
+        /*
          * Remove our reservation. The impl() routine avoids setting the
          * actual property, which would require the (already destroyed) ds.
          */
         dsl_dir_set_reservation_sync_impl(dd, 0, tx);
 

@@ -1034,12 +1513,10 @@
         }
         mutex_exit(&dd->dd_lock);
         return (err);
 }
 
-extern dsl_syncfunc_t dsl_prop_set_sync;
-
 static void
 dsl_dir_set_quota_sync(void *arg1, void *arg2, dmu_tx_t *tx)
 {
         dsl_dataset_t *ds = arg1;
         dsl_dir_t *dd = ds->ds_dir;

@@ -1236,10 +1713,11 @@
 }
 
 struct renamearg {
         dsl_dir_t *newparent;
         const char *mynewname;
+        cred_t *cr;
 };
 
 static int
 dsl_dir_rename_check(void *arg1, void *arg2, dmu_tx_t *tx)
 {

@@ -1276,13 +1754,27 @@
                 /* no rename into our descendant */
                 if (closest_common_ancestor(dd, ra->newparent) == dd)
                         return (EINVAL);
 
                 if (err = dsl_dir_transfer_possible(dd->dd_parent,
-                    ra->newparent, myspace))
+                    ra->newparent, dd, myspace, ra->cr))
                         return (err);
+
+                if (dd->dd_phys->dd_filesystem_count == 0 &&
+                    dmu_tx_is_syncing(tx)) {
+                        uint64_t fs_cnt = 0;
+                        uint64_t ss_cnt = 0;
+
+                        /*
+                         * Ensure this portion of the tree's counts have been
+                         * initialized in case the new parent has limits set.
+                         */
+                        err = dsl_dir_set_fs_ss_count(dd, tx, &fs_cnt, &ss_cnt);
+                        if (err)
+                                return (EIO);
         }
+        }
 
         return (0);
 }
 
 static void

@@ -1301,10 +1793,24 @@
         dsl_dir_name(ra->newparent, namebuf);
         spa_history_log_internal_dd(dd, "rename", tx,
             "-> %s/%s", namebuf, ra->mynewname);
 
         if (ra->newparent != dd->dd_parent) {
+                int cnt;
+
+                mutex_enter(&dd->dd_lock);
+
+                cnt = dd->dd_phys->dd_filesystem_count;
+                dsl_dir_fscount_adjust(dd->dd_parent, tx, -cnt, B_TRUE);
+                dsl_dir_fscount_adjust(ra->newparent, tx, cnt, B_TRUE);
+
+                cnt = dd->dd_phys->dd_snapshot_count;
+                dsl_snapcount_adjust(dd->dd_parent, tx, -cnt, B_TRUE);
+                dsl_snapcount_adjust(ra->newparent, tx, cnt, B_TRUE);
+
+                mutex_exit(&dd->dd_lock);
+
                 dsl_dir_diduse_space(dd->dd_parent, DD_USED_CHILD,
                     -dd->dd_phys->dd_used_bytes,
                     -dd->dd_phys->dd_compressed_bytes,
                     -dd->dd_phys->dd_uncompressed_bytes, tx);
                 dsl_dir_diduse_space(ra->newparent, DD_USED_CHILD,

@@ -1364,31 +1870,46 @@
         if (ra.mynewname == NULL) {
                 err = EEXIST;
                 goto out;
         }
 
+        ra.cr = CRED();
+
         err = dsl_sync_task_do(dd->dd_pool,
             dsl_dir_rename_check, dsl_dir_rename_sync, dd, &ra, 3);
 
 out:
         dsl_dir_close(ra.newparent, FTAG);
         return (err);
 }
 
 int
-dsl_dir_transfer_possible(dsl_dir_t *sdd, dsl_dir_t *tdd, uint64_t space)
+dsl_dir_transfer_possible(dsl_dir_t *sdd, dsl_dir_t *tdd, dsl_dir_t *moving_dd,
+    uint64_t space, cred_t *cr)
 {
         dsl_dir_t *ancestor;
         int64_t adelta;
         uint64_t avail;
+        int err;
 
         ancestor = closest_common_ancestor(sdd, tdd);
         adelta = would_change(sdd, -space, ancestor);
         avail = dsl_dir_space_available(tdd, ancestor, adelta, FALSE);
         if (avail < space)
                 return (ENOSPC);
 
+        if (sdd != moving_dd) {
+                err = dsl_dir_fscount_check(tdd,
+                    moving_dd->dd_phys->dd_filesystem_count, ancestor, cr);
+                if (err != 0)
+                        return (err);
+        }
+        err = dsl_snapcount_check(tdd, moving_dd->dd_phys->dd_snapshot_count,
+            ancestor, cr);
+        if (err != 0)
+                return (err);
+
         return (0);
 }
 
 timestruc_t
 dsl_dir_snap_cmtime(dsl_dir_t *dd)