Deleted Added
full compact
53a54
> #include <sys/range_tree.h>
57a59,114
> /*
> * Grand theory statement on scan queue sorting
> *
> * Scanning is implemented by recursively traversing all indirection levels
> * in an object and reading all blocks referenced from said objects. This
> * results in us approximately traversing the object from lowest logical
> * offset to the highest. For best performance, we would want the logical
> * blocks to be physically contiguous. However, this is frequently not the
> * case with pools given the allocation patterns of copy-on-write filesystems.
> * So instead, we put the I/Os into a reordering queue and issue them in a
> * way that will most benefit physical disks (LBA-order).
> *
> * Queue management:
> *
> * Ideally, we would want to scan all metadata and queue up all block I/O
> * prior to starting to issue it, because that allows us to do an optimal
> * sorting job. This can however consume large amounts of memory. Therefore
> * we continuously monitor the size of the queues and constrain them to 5%
> * (zfs_scan_mem_lim_fact) of physmem. If the queues grow larger than this
> * limit, we clear out a few of the largest extents at the head of the queues
> * to make room for more scanning. Hopefully, these extents will be fairly
> * large and contiguous, allowing us to approach sequential I/O throughput
> * even without a fully sorted tree.
> *
> * Metadata scanning takes place in dsl_scan_visit(), which is called from
> * dsl_scan_sync() every spa_sync(). If we have either fully scanned all
> * metadata on the pool, or we need to make room in memory because our
> * queues are too large, dsl_scan_visit() is postponed and
> * scan_io_queues_run() is called from dsl_scan_sync() instead. This implies
> * that metadata scanning and queued I/O issuing are mutually exclusive. This
> * allows us to provide maximum sequential I/O throughput for the majority of
> * I/O's issued since sequential I/O performance is significantly negatively
> * impacted if it is interleaved with random I/O.
> *
> * Implementation Notes
> *
> * One side effect of the queued scanning algorithm is that the scanning code
> * needs to be notified whenever a block is freed. This is needed to allow
> * the scanning code to remove these I/Os from the issuing queue. Additionally,
> * we do not attempt to queue gang blocks to be issued sequentially since this
> * is very hard to do and would have an extremely limitted performance benefit.
> * Instead, we simply issue gang I/Os as soon as we find them using the legacy
> * algorithm.
> *
> * Backwards compatibility
> *
> * This new algorithm is backwards compatible with the legacy on-disk data
> * structures (and therefore does not require a new feature flag).
> * Periodically during scanning (see zfs_scan_checkpoint_intval), the scan
> * will stop scanning metadata (in logical order) and wait for all outstanding
> * sorted I/O to complete. Once this is done, we write out a checkpoint
> * bookmark, indicating that we have scanned everything logically before it.
> * If the pool is imported on a machine without the new sorting algorithm,
> * the scan simply resumes from the last checkpoint using the legacy algorithm.
> */
>
62,64d118
< static void dsl_scan_cancel_sync(void *, dmu_tx_t *);
< static void dsl_scan_sync_state(dsl_scan_t *, dmu_tx_t *);
< static boolean_t dsl_scan_restarting(dsl_scan_t *, dmu_tx_t *);
66,68c120,146
< unsigned int zfs_top_maxinflight = 32; /* maximum I/Os per top-level */
< unsigned int zfs_resilver_delay = 2; /* number of ticks to delay resilver */
< unsigned int zfs_scrub_delay = 4; /* number of ticks to delay scrub */
---
> static int scan_ds_queue_compare(const void *a, const void *b);
> static int scan_prefetch_queue_compare(const void *a, const void *b);
> static void scan_ds_queue_clear(dsl_scan_t *scn);
> static boolean_t scan_ds_queue_contains(dsl_scan_t *scn, uint64_t dsobj,
> uint64_t *txg);
> static void scan_ds_queue_insert(dsl_scan_t *scn, uint64_t dsobj, uint64_t txg);
> static void scan_ds_queue_remove(dsl_scan_t *scn, uint64_t dsobj);
> static void scan_ds_queue_sync(dsl_scan_t *scn, dmu_tx_t *tx);
>
> extern int zfs_vdev_async_write_active_min_dirty_percent;
>
> /*
> * By default zfs will check to ensure it is not over the hard memory
> * limit before each txg. If finer-grained control of this is needed
> * this value can be set to 1 to enable checking before scanning each
> * block.
> */
> int zfs_scan_strict_mem_lim = B_FALSE;
>
> /*
> * Maximum number of parallelly executing I/Os per top-level vdev.
> * Tune with care. Very high settings (hundreds) are known to trigger
> * some firmware bugs and resets on certain SSDs.
> */
> int zfs_top_maxinflight = 32; /* maximum I/Os per top-level */
> unsigned int zfs_resilver_delay = 2; /* number of ticks to delay resilver -- 2 is a good number */
> unsigned int zfs_scrub_delay = 4; /* number of ticks to delay scrub -- 4 is a good number */
71c149,180
< unsigned int zfs_scan_min_time_ms = 1000; /* min millisecs to scrub per txg */
---
> /*
> * Maximum number of parallelly executed bytes per leaf vdev. We attempt
> * to strike a balance here between keeping the vdev queues full of I/Os
> * at all times and not overflowing the queues to cause long latency,
> * which would cause long txg sync times. No matter what, we will not
> * overload the drives with I/O, since that is protected by
> * zfs_vdev_scrub_max_active.
> */
> unsigned long zfs_scan_vdev_limit = 4 << 20;
>
> int zfs_scan_issue_strategy = 0;
> int zfs_scan_legacy = B_FALSE; /* don't queue & sort zios, go direct */
> uint64_t zfs_scan_max_ext_gap = 2 << 20; /* in bytes */
>
> unsigned int zfs_scan_checkpoint_intval = 7200; /* seconds */
> #define ZFS_SCAN_CHECKPOINT_INTVAL SEC_TO_TICK(zfs_scan_checkpoint_intval)
>
> /*
> * fill_weight is non-tunable at runtime, so we copy it at module init from
> * zfs_scan_fill_weight. Runtime adjustments to zfs_scan_fill_weight would
> * break queue sorting.
> */
> uint64_t zfs_scan_fill_weight = 3;
> static uint64_t fill_weight;
>
> /* See dsl_scan_should_clear() for details on the memory limit tunables */
> uint64_t zfs_scan_mem_lim_min = 16 << 20; /* bytes */
> uint64_t zfs_scan_mem_lim_soft_max = 128 << 20; /* bytes */
> int zfs_scan_mem_lim_fact = 20; /* fraction of physmem */
> int zfs_scan_mem_lim_soft_fact = 20; /* fraction of mem lim above */
>
> unsigned int zfs_scrub_min_time_ms = 1000; /* min millisecs to scrub per txg */
74,75c183
< unsigned int zfs_resilver_min_time_ms = 3000; /* min millisecs to resilver
< per txg */
---
> unsigned int zfs_resilver_min_time_ms = 3000; /* min millisecs to resilver per txg */
89c197
< &zfs_scan_min_time_ms, 0, "Min millisecs to scrub per txg");
---
> &zfs_scrub_min_time_ms, 0, "Min millisecs to scrub per txg");
97a206,209
> SYSCTL_UINT(_vfs_zfs, OID_AUTO, zfs_scan_legacy, CTLFLAG_RWTUN,
> &zfs_scan_legacy, 0, "Scrub using legacy non-sequential method");
> SYSCTL_UINT(_vfs_zfs, OID_AUTO, zfs_scan_checkpoint_interval, CTLFLAG_RWTUN,
> &zfs_scan_checkpoint_intval, 0, "Scan progress on-disk checkpointing interval");
104a217,227
> /*
> * We wait a few txgs after importing a pool to begin scanning so that
> * the import / mounting code isn't held up by scrub / resilver IO.
> * Unfortunately, it is a bit difficult to determine exactly how long
> * this will take since userspace will trigger fs mounts asynchronously
> * and the kernel will create zvol minors asynchronously. As a result,
> * the value provided here is a bit arbitrary, but represents a
> * reasonable estimate of how many txgs it will take to finish fully
> * importing a pool
> */
> #define SCAN_IMPORT_WAIT_TXGS 5
105a229
>
126a251,421
> /* In core node for the scn->scn_queue. Represents a dataset to be scanned */
> typedef struct {
> uint64_t sds_dsobj;
> uint64_t sds_txg;
> avl_node_t sds_node;
> } scan_ds_t;
>
> /*
> * This controls what conditions are placed on dsl_scan_sync_state():
> * SYNC_OPTIONAL) write out scn_phys iff scn_bytes_pending == 0
> * SYNC_MANDATORY) write out scn_phys always. scn_bytes_pending must be 0.
> * SYNC_CACHED) if scn_bytes_pending == 0, write out scn_phys. Otherwise
> * write out the scn_phys_cached version.
> * See dsl_scan_sync_state for details.
> */
> typedef enum {
> SYNC_OPTIONAL,
> SYNC_MANDATORY,
> SYNC_CACHED
> } state_sync_type_t;
>
> /*
> * This struct represents the minimum information needed to reconstruct a
> * zio for sequential scanning. This is useful because many of these will
> * accumulate in the sequential IO queues before being issued, so saving
> * memory matters here.
> */
> typedef struct scan_io {
> /* fields from blkptr_t */
> uint64_t sio_offset;
> uint64_t sio_blk_prop;
> uint64_t sio_phys_birth;
> uint64_t sio_birth;
> zio_cksum_t sio_cksum;
> uint32_t sio_asize;
>
> /* fields from zio_t */
> int sio_flags;
> zbookmark_phys_t sio_zb;
>
> /* members for queue sorting */
> union {
> avl_node_t sio_addr_node; /* link into issueing queue */
> list_node_t sio_list_node; /* link for issuing to disk */
> } sio_nodes;
> } scan_io_t;
>
> struct dsl_scan_io_queue {
> dsl_scan_t *q_scn; /* associated dsl_scan_t */
> vdev_t *q_vd; /* top-level vdev that this queue represents */
>
> /* trees used for sorting I/Os and extents of I/Os */
> range_tree_t *q_exts_by_addr;
> avl_tree_t q_exts_by_size;
> avl_tree_t q_sios_by_addr;
>
> /* members for zio rate limiting */
> uint64_t q_maxinflight_bytes;
> uint64_t q_inflight_bytes;
> kcondvar_t q_zio_cv; /* used under vd->vdev_scan_io_queue_lock */
>
> /* per txg statistics */
> uint64_t q_total_seg_size_this_txg;
> uint64_t q_segs_this_txg;
> uint64_t q_total_zio_size_this_txg;
> uint64_t q_zios_this_txg;
> };
>
> /* private data for dsl_scan_prefetch_cb() */
> typedef struct scan_prefetch_ctx {
> refcount_t spc_refcnt; /* refcount for memory management */
> dsl_scan_t *spc_scn; /* dsl_scan_t for the pool */
> boolean_t spc_root; /* is this prefetch for an objset? */
> uint8_t spc_indblkshift; /* dn_indblkshift of current dnode */
> uint16_t spc_datablkszsec; /* dn_idatablkszsec of current dnode */
> } scan_prefetch_ctx_t;
>
> /* private data for dsl_scan_prefetch() */
> typedef struct scan_prefetch_issue_ctx {
> avl_node_t spic_avl_node; /* link into scn->scn_prefetch_queue */
> scan_prefetch_ctx_t *spic_spc; /* spc for the callback */
> blkptr_t spic_bp; /* bp to prefetch */
> zbookmark_phys_t spic_zb; /* bookmark to prefetch */
> } scan_prefetch_issue_ctx_t;
>
> static void scan_exec_io(dsl_pool_t *dp, const blkptr_t *bp, int zio_flags,
> const zbookmark_phys_t *zb, dsl_scan_io_queue_t *queue);
> static void scan_io_queue_insert_impl(dsl_scan_io_queue_t *queue,
> scan_io_t *sio);
>
> static dsl_scan_io_queue_t *scan_io_queue_create(vdev_t *vd);
> static void scan_io_queues_destroy(dsl_scan_t *scn);
>
> static kmem_cache_t *sio_cache;
>
> void
> scan_init(void)
> {
> /*
> * This is used in ext_size_compare() to weight segments
> * based on how sparse they are. This cannot be changed
> * mid-scan and the tree comparison functions don't currently
> * have a mechansim for passing additional context to the
> * compare functions. Thus we store this value globally and
> * we only allow it to be set at module intiailization time
> */
> fill_weight = zfs_scan_fill_weight;
>
> sio_cache = kmem_cache_create("sio_cache",
> sizeof (scan_io_t), 0, NULL, NULL, NULL, NULL, NULL, 0);
> }
>
> void
> scan_fini(void)
> {
> kmem_cache_destroy(sio_cache);
> }
>
> static inline boolean_t
> dsl_scan_is_running(const dsl_scan_t *scn)
> {
> return (scn->scn_phys.scn_state == DSS_SCANNING);
> }
>
> boolean_t
> dsl_scan_resilvering(dsl_pool_t *dp)
> {
> return (dsl_scan_is_running(dp->dp_scan) &&
> dp->dp_scan->scn_phys.scn_func == POOL_SCAN_RESILVER);
> }
>
> static inline void
> sio2bp(const scan_io_t *sio, blkptr_t *bp, uint64_t vdev_id)
> {
> bzero(bp, sizeof (*bp));
> DVA_SET_ASIZE(&bp->blk_dva[0], sio->sio_asize);
> DVA_SET_VDEV(&bp->blk_dva[0], vdev_id);
> DVA_SET_OFFSET(&bp->blk_dva[0], sio->sio_offset);
> bp->blk_prop = sio->sio_blk_prop;
> bp->blk_phys_birth = sio->sio_phys_birth;
> bp->blk_birth = sio->sio_birth;
> bp->blk_fill = 1; /* we always only work with data pointers */
> bp->blk_cksum = sio->sio_cksum;
> }
>
> static inline void
> bp2sio(const blkptr_t *bp, scan_io_t *sio, int dva_i)
> {
> /* we discard the vdev id, since we can deduce it from the queue */
> sio->sio_offset = DVA_GET_OFFSET(&bp->blk_dva[dva_i]);
> sio->sio_asize = DVA_GET_ASIZE(&bp->blk_dva[dva_i]);
> sio->sio_blk_prop = bp->blk_prop;
> sio->sio_phys_birth = bp->blk_phys_birth;
> sio->sio_birth = bp->blk_birth;
> sio->sio_cksum = bp->blk_cksum;
> }
>
> void
> dsl_scan_global_init(void)
> {
> /*
> * This is used in ext_size_compare() to weight segments
> * based on how sparse they are. This cannot be changed
> * mid-scan and the tree comparison functions don't currently
> * have a mechansim for passing additional context to the
> * compare functions. Thus we store this value globally and
> * we only allow it to be set at module intiailization time
> */
> fill_weight = zfs_scan_fill_weight;
> }
>
146a442,448
> bcopy(&scn->scn_phys, &scn->scn_phys_cached, sizeof (scn->scn_phys));
> avl_create(&scn->scn_queue, scan_ds_queue_compare, sizeof (scan_ds_t),
> offsetof(scan_ds_t, sds_node));
> avl_create(&scn->scn_prefetch_queue, scan_prefetch_queue_compare,
> sizeof (scan_prefetch_issue_ctx_t),
> offsetof(scan_prefetch_issue_ctx_t, spic_avl_node));
>
157c459
< scn->scn_restart_txg);
---
> (longlong_t)scn->scn_restart_txg);
175c477,484
< if (scn->scn_phys.scn_state == DSS_SCANNING &&
---
> /*
> * We might be restarting after a reboot, so jump the issued
> * counter to how far we've scanned. We know we're consistent
> * up to here.
> */
> scn->scn_issued_before_pass = scn->scn_phys.scn_examined;
>
> if (dsl_scan_is_running(scn) &&
187c496
< scn->scn_restart_txg);
---
> (longlong_t)scn->scn_restart_txg);
190a500,515
> /* reload the queue into the in-core state */
> if (scn->scn_phys.scn_queue_obj != 0) {
> zap_cursor_t zc;
> zap_attribute_t za;
>
> for (zap_cursor_init(&zc, dp->dp_meta_objset,
> scn->scn_phys.scn_queue_obj);
> zap_cursor_retrieve(&zc, &za) == 0;
> (void) zap_cursor_advance(&zc)) {
> scan_ds_queue_insert(scn,
> zfs_strtonum(za.za_name, NULL),
> za.za_first_integer);
> }
> zap_cursor_fini(&zc);
> }
>
198c523,531
< if (dp->dp_scan) {
---
> if (dp->dp_scan != NULL) {
> dsl_scan_t *scn = dp->dp_scan;
>
> if (scn->scn_taskq != NULL)
> taskq_destroy(scn->scn_taskq);
> scan_ds_queue_clear(scn);
> avl_destroy(&scn->scn_queue);
> avl_destroy(&scn->scn_prefetch_queue);
>
203a537,625
> static boolean_t
> dsl_scan_restarting(dsl_scan_t *scn, dmu_tx_t *tx)
> {
> return (scn->scn_restart_txg != 0 &&
> scn->scn_restart_txg <= tx->tx_txg);
> }
>
> boolean_t
> dsl_scan_scrubbing(const dsl_pool_t *dp)
> {
> dsl_scan_phys_t *scn_phys = &dp->dp_scan->scn_phys;
>
> return (scn_phys->scn_state == DSS_SCANNING &&
> scn_phys->scn_func == POOL_SCAN_SCRUB);
> }
>
> boolean_t
> dsl_scan_is_paused_scrub(const dsl_scan_t *scn)
> {
> return (dsl_scan_scrubbing(scn->scn_dp) &&
> scn->scn_phys.scn_flags & DSF_SCRUB_PAUSED);
> }
>
> /*
> * Writes out a persistent dsl_scan_phys_t record to the pool directory.
> * Because we can be running in the block sorting algorithm, we do not always
> * want to write out the record, only when it is "safe" to do so. This safety
> * condition is achieved by making sure that the sorting queues are empty
> * (scn_bytes_pending == 0). When this condition is not true, the sync'd state
> * is inconsistent with how much actual scanning progress has been made. The
> * kind of sync to be performed is specified by the sync_type argument. If the
> * sync is optional, we only sync if the queues are empty. If the sync is
> * mandatory, we do a hard ASSERT to make sure that the queues are empty. The
> * third possible state is a "cached" sync. This is done in response to:
> * 1) The dataset that was in the last sync'd dsl_scan_phys_t having been
> * destroyed, so we wouldn't be able to restart scanning from it.
> * 2) The snapshot that was in the last sync'd dsl_scan_phys_t having been
> * superseded by a newer snapshot.
> * 3) The dataset that was in the last sync'd dsl_scan_phys_t having been
> * swapped with its clone.
> * In all cases, a cached sync simply rewrites the last record we've written,
> * just slightly modified. For the modifications that are performed to the
> * last written dsl_scan_phys_t, see dsl_scan_ds_destroyed,
> * dsl_scan_ds_snapshotted and dsl_scan_ds_clone_swapped.
> */
> static void
> dsl_scan_sync_state(dsl_scan_t *scn, dmu_tx_t *tx, state_sync_type_t sync_type)
> {
> int i;
> spa_t *spa = scn->scn_dp->dp_spa;
>
> ASSERT(sync_type != SYNC_MANDATORY || scn->scn_bytes_pending == 0);
> if (scn->scn_bytes_pending == 0) {
> for (i = 0; i < spa->spa_root_vdev->vdev_children; i++) {
> vdev_t *vd = spa->spa_root_vdev->vdev_child[i];
> dsl_scan_io_queue_t *q = vd->vdev_scan_io_queue;
>
> if (q == NULL)
> continue;
>
> mutex_enter(&vd->vdev_scan_io_queue_lock);
> ASSERT3P(avl_first(&q->q_sios_by_addr), ==, NULL);
> ASSERT3P(avl_first(&q->q_exts_by_size), ==, NULL);
> ASSERT3P(range_tree_first(q->q_exts_by_addr), ==, NULL);
> mutex_exit(&vd->vdev_scan_io_queue_lock);
> }
>
> if (scn->scn_phys.scn_queue_obj != 0)
> scan_ds_queue_sync(scn, tx);
> VERIFY0(zap_update(scn->scn_dp->dp_meta_objset,
> DMU_POOL_DIRECTORY_OBJECT,
> DMU_POOL_SCAN, sizeof (uint64_t), SCAN_PHYS_NUMINTS,
> &scn->scn_phys, tx));
> bcopy(&scn->scn_phys, &scn->scn_phys_cached,
> sizeof (scn->scn_phys));
>
> if (scn->scn_checkpointing)
> zfs_dbgmsg("finish scan checkpoint");
>
> scn->scn_checkpointing = B_FALSE;
> scn->scn_last_checkpoint = ddi_get_lbolt();
> } else if (sync_type == SYNC_CACHED) {
> VERIFY0(zap_update(scn->scn_dp->dp_meta_objset,
> DMU_POOL_DIRECTORY_OBJECT,
> DMU_POOL_SCAN, sizeof (uint64_t), SCAN_PHYS_NUMINTS,
> &scn->scn_phys_cached, tx));
> }
> }
>
210c632
< if (scn->scn_phys.scn_state == DSS_SCANNING)
---
> if (dsl_scan_is_running(scn))
225c647
< ASSERT(scn->scn_phys.scn_state != DSS_SCANNING);
---
> ASSERT(!dsl_scan_is_running(scn));
235a658
> scn->scn_issued_before_pass = 0;
237a661,662
> scn->scn_last_checkpoint = 0;
> scn->scn_checkpointing = B_FALSE;
269a695,696
> mutex_init(&dp->dp_blkstats->zab_lock, NULL,
> MUTEX_DEFAULT, NULL);
271c698
< bzero(dp->dp_blkstats, sizeof (zfs_all_blkstats_t));
---
> bzero(&dp->dp_blkstats->zab_type, sizeof (dp->dp_blkstats->zab_type));
279c706
< dsl_scan_sync_state(scn, tx);
---
> bcopy(&scn->scn_phys, &scn->scn_phys_cached, sizeof (scn->scn_phys));
280a708,709
> dsl_scan_sync_state(scn, tx, SYNC_MANDATORY);
>
285a715,752
> /*
> * Called by the ZFS_IOC_POOL_SCAN ioctl to start a scrub or resilver.
> * Can also be called to resume a paused scrub.
> */
> int
> dsl_scan(dsl_pool_t *dp, pool_scan_func_t func)
> {
> spa_t *spa = dp->dp_spa;
> dsl_scan_t *scn = dp->dp_scan;
>
> /*
> * Purge all vdev caches and probe all devices. We do this here
> * rather than in sync context because this requires a writer lock
> * on the spa_config lock, which we can't do from sync context. The
> * spa_scrub_reopen flag indicates that vdev_open() should not
> * attempt to start another scrub.
> */
> spa_vdev_state_enter(spa, SCL_NONE);
> spa->spa_scrub_reopen = B_TRUE;
> vdev_reopen(spa->spa_root_vdev);
> spa->spa_scrub_reopen = B_FALSE;
> (void) spa_vdev_state_exit(spa, NULL, 0);
>
> if (func == POOL_SCAN_SCRUB && dsl_scan_is_paused_scrub(scn)) {
> /* got scrub start cmd, resume paused scrub */
> int err = dsl_scrub_set_pause_resume(scn->scn_dp,
> POOL_SCRUB_NORMAL);
> if (err == 0) {
> spa_event_notify(spa, NULL, NULL, ESC_ZFS_SCRUB_RESUME);
> return (ECANCELED);
> }
> return (SET_ERROR(err));
> }
>
> return (dsl_sync_task(spa_name(spa), dsl_scan_setup_check,
> dsl_scan_setup_sync, &func, 0, ZFS_SPACE_CHECK_EXTRA_RESERVED));
> }
>
313c780
< VERIFY(0 == dmu_object_free(dp->dp_meta_objset,
---
> VERIFY0(dmu_object_free(dp->dp_meta_objset,
316a784
> scan_ds_queue_clear(scn);
324c792,793
< if (scn->scn_phys.scn_state != DSS_SCANNING)
---
> if (!dsl_scan_is_running(scn)) {
> ASSERT(!scn->scn_is_sorted);
325a795
> }
327,330c797,799
< if (complete)
< scn->scn_phys.scn_state = DSS_FINISHED;
< else
< scn->scn_phys.scn_state = DSS_CANCELED;
---
> if (scn->scn_is_sorted) {
> scan_io_queues_destroy(scn);
> scn->scn_is_sorted = B_FALSE;
331a801,808
> if (scn->scn_taskq != NULL) {
> taskq_destroy(scn->scn_taskq);
> scn->scn_taskq = NULL;
> }
> }
>
> scn->scn_phys.scn_state = complete ? DSS_FINISHED : DSS_CANCELED;
>
343,348d819
< mutex_enter(&spa->spa_scrub_lock);
< while (spa->spa_scrub_inflight > 0) {
< cv_wait(&spa->spa_scrub_io_cv,
< &spa->spa_scrub_lock);
< }
< mutex_exit(&spa->spa_scrub_lock);
383a855,856
>
> ASSERT(!dsl_scan_is_running(scn));
392c865
< if (scn->scn_phys.scn_state != DSS_SCANNING)
---
> if (!dsl_scan_is_running(scn))
404c877
< dsl_scan_sync_state(scn, tx);
---
> dsl_scan_sync_state(scn, tx, SYNC_MANDATORY);
415,424d887
< boolean_t
< dsl_scan_is_paused_scrub(const dsl_scan_t *scn)
< {
< if (dsl_scan_scrubbing(scn->scn_dp) &&
< scn->scn_phys.scn_flags & DSF_SCRUB_PAUSED)
< return (B_TRUE);
<
< return (B_FALSE);
< }
<
459c922
< dsl_scan_sync_state(scn, tx);
---
> dsl_scan_sync_state(scn, tx, SYNC_CACHED);
473c936
< dsl_scan_sync_state(scn, tx);
---
> dsl_scan_sync_state(scn, tx, SYNC_CACHED);
489,490c952,955
< boolean_t
< dsl_scan_scrubbing(const dsl_pool_t *dp)
---
>
> /* start a new scan, or restart an existing one. */
> void
> dsl_resilver_restart(dsl_pool_t *dp, uint64_t txg)
492c957,960
< dsl_scan_t *scn = dp->dp_scan;
---
> if (txg == 0) {
> dmu_tx_t *tx;
> tx = dmu_tx_create_dd(dp->dp_mos_dir);
> VERIFY(0 == dmu_tx_assign(tx, TXG_WAIT));
494,498c962,968
< if (scn->scn_phys.scn_state == DSS_SCANNING &&
< scn->scn_phys.scn_func == POOL_SCAN_SCRUB)
< return (B_TRUE);
<
< return (B_FALSE);
---
> txg = dmu_tx_get_txg(tx);
> dp->dp_scan->scn_restart_txg = txg;
> dmu_tx_commit(tx);
> } else {
> dp->dp_scan->scn_restart_txg = txg;
> }
> zfs_dbgmsg("restarting resilver txg=%llu", txg);
501,507d970
< static void dsl_scan_visitbp(blkptr_t *bp, const zbookmark_phys_t *zb,
< dnode_phys_t *dnp, dsl_dataset_t *ds, dsl_scan_t *scn,
< dmu_objset_type_t ostype, dmu_tx_t *tx);
< static void dsl_scan_visitdnode(dsl_scan_t *, dsl_dataset_t *ds,
< dmu_objset_type_t ostype,
< dnode_phys_t *dnp, uint64_t object, dmu_tx_t *tx);
<
522,523c985,986
< static uint64_t
< dsl_scan_ds_maxtxg(dsl_dataset_t *ds)
---
> static int
> scan_ds_queue_compare(const void *a, const void *b)
525,528c988,994
< uint64_t smt = ds->ds_dir->dd_pool->dp_scan->scn_phys.scn_max_txg;
< if (ds->ds_is_snapshot)
< return (MIN(smt, dsl_dataset_phys(ds)->ds_creation_txg));
< return (smt);
---
> const scan_ds_t *sds_a = a, *sds_b = b;
>
> if (sds_a->sds_dsobj < sds_b->sds_dsobj)
> return (-1);
> if (sds_a->sds_dsobj == sds_b->sds_dsobj)
> return (0);
> return (1);
532c998
< dsl_scan_sync_state(dsl_scan_t *scn, dmu_tx_t *tx)
---
> scan_ds_queue_clear(dsl_scan_t *scn)
534,537c1000,1004
< VERIFY0(zap_update(scn->scn_dp->dp_meta_objset,
< DMU_POOL_DIRECTORY_OBJECT,
< DMU_POOL_SCAN, sizeof (uint64_t), SCAN_PHYS_NUMINTS,
< &scn->scn_phys, tx));
---
> void *cookie = NULL;
> scan_ds_t *sds;
> while ((sds = avl_destroy_nodes(&scn->scn_queue, &cookie)) != NULL) {
> kmem_free(sds, sizeof (*sds));
> }
540c1007,1010
< extern int zfs_vdev_async_write_active_min_dirty_percent;
---
> static boolean_t
> scan_ds_queue_contains(dsl_scan_t *scn, uint64_t dsobj, uint64_t *txg)
> {
> scan_ds_t srch, *sds;
541a1012,1100
> srch.sds_dsobj = dsobj;
> sds = avl_find(&scn->scn_queue, &srch, NULL);
> if (sds != NULL && txg != NULL)
> *txg = sds->sds_txg;
> return (sds != NULL);
> }
>
> static void
> scan_ds_queue_insert(dsl_scan_t *scn, uint64_t dsobj, uint64_t txg)
> {
> scan_ds_t *sds;
> avl_index_t where;
>
> sds = kmem_zalloc(sizeof (*sds), KM_SLEEP);
> sds->sds_dsobj = dsobj;
> sds->sds_txg = txg;
>
> VERIFY3P(avl_find(&scn->scn_queue, sds, &where), ==, NULL);
> avl_insert(&scn->scn_queue, sds, where);
> }
>
> static void
> scan_ds_queue_remove(dsl_scan_t *scn, uint64_t dsobj)
> {
> scan_ds_t srch, *sds;
>
> srch.sds_dsobj = dsobj;
>
> sds = avl_find(&scn->scn_queue, &srch, NULL);
> VERIFY(sds != NULL);
> avl_remove(&scn->scn_queue, sds);
> kmem_free(sds, sizeof (*sds));
> }
>
> static void
> scan_ds_queue_sync(dsl_scan_t *scn, dmu_tx_t *tx)
> {
> dsl_pool_t *dp = scn->scn_dp;
> spa_t *spa = dp->dp_spa;
> dmu_object_type_t ot = (spa_version(spa) >= SPA_VERSION_DSL_SCRUB) ?
> DMU_OT_SCAN_QUEUE : DMU_OT_ZAP_OTHER;
>
> ASSERT0(scn->scn_bytes_pending);
> ASSERT(scn->scn_phys.scn_queue_obj != 0);
>
> VERIFY0(dmu_object_free(dp->dp_meta_objset,
> scn->scn_phys.scn_queue_obj, tx));
> scn->scn_phys.scn_queue_obj = zap_create(dp->dp_meta_objset, ot,
> DMU_OT_NONE, 0, tx);
> for (scan_ds_t *sds = avl_first(&scn->scn_queue);
> sds != NULL; sds = AVL_NEXT(&scn->scn_queue, sds)) {
> VERIFY0(zap_add_int_key(dp->dp_meta_objset,
> scn->scn_phys.scn_queue_obj, sds->sds_dsobj,
> sds->sds_txg, tx));
> }
> }
>
> /*
> * Computes the memory limit state that we're currently in. A sorted scan
> * needs quite a bit of memory to hold the sorting queue, so we need to
> * reasonably constrain the size so it doesn't impact overall system
> * performance. We compute two limits:
> * 1) Hard memory limit: if the amount of memory used by the sorting
> * queues on a pool gets above this value, we stop the metadata
> * scanning portion and start issuing the queued up and sorted
> * I/Os to reduce memory usage.
> * This limit is calculated as a fraction of physmem (by default 5%).
> * We constrain the lower bound of the hard limit to an absolute
> * minimum of zfs_scan_mem_lim_min (default: 16 MiB). We also constrain
> * the upper bound to 5% of the total pool size - no chance we'll
> * ever need that much memory, but just to keep the value in check.
> * 2) Soft memory limit: once we hit the hard memory limit, we start
> * issuing I/O to reduce queue memory usage, but we don't want to
> * completely empty out the queues, since we might be able to find I/Os
> * that will fill in the gaps of our non-sequential IOs at some point
> * in the future. So we stop the issuing of I/Os once the amount of
> * memory used drops below the soft limit (at which point we stop issuing
> * I/O and start scanning metadata again).
> *
> * This limit is calculated by subtracting a fraction of the hard
> * limit from the hard limit. By default this fraction is 5%, so
> * the soft limit is 95% of the hard limit. We cap the size of the
> * difference between the hard and soft limits at an absolute
> * maximum of zfs_scan_mem_lim_soft_max (default: 128 MiB) - this is
> * sufficient to not cause too frequent switching between the
> * metadata scan and I/O issue (even at 2k recordsize, 128 MiB's
> * worth of queues is about 1.2 GiB of on-pool data, so scanning
> * that should take at least a decent fraction of a second).
> */
542a1102,1149
> dsl_scan_should_clear(dsl_scan_t *scn)
> {
> vdev_t *rvd = scn->scn_dp->dp_spa->spa_root_vdev;
> uint64_t mlim_hard, mlim_soft, mused;
> uint64_t alloc = metaslab_class_get_alloc(spa_normal_class(
> scn->scn_dp->dp_spa));
>
> mlim_hard = MAX((physmem / zfs_scan_mem_lim_fact) * PAGESIZE,
> zfs_scan_mem_lim_min);
> mlim_hard = MIN(mlim_hard, alloc / 20);
> mlim_soft = mlim_hard - MIN(mlim_hard / zfs_scan_mem_lim_soft_fact,
> zfs_scan_mem_lim_soft_max);
> mused = 0;
> for (uint64_t i = 0; i < rvd->vdev_children; i++) {
> vdev_t *tvd = rvd->vdev_child[i];
> dsl_scan_io_queue_t *queue;
>
> mutex_enter(&tvd->vdev_scan_io_queue_lock);
> queue = tvd->vdev_scan_io_queue;
> if (queue != NULL) {
> /* #extents in exts_by_size = # in exts_by_addr */
> mused += avl_numnodes(&queue->q_exts_by_size) *
> sizeof (range_seg_t) +
> avl_numnodes(&queue->q_sios_by_addr) *
> sizeof (scan_io_t);
> }
> mutex_exit(&tvd->vdev_scan_io_queue_lock);
> }
>
> dprintf("current scan memory usage: %llu bytes\n", (longlong_t)mused);
>
> if (mused == 0)
> ASSERT0(scn->scn_bytes_pending);
>
> /*
> * If we are above our hard limit, we need to clear out memory.
> * If we are below our soft limit, we need to accumulate sequential IOs.
> * Otherwise, we should keep doing whatever we are currently doing.
> */
> if (mused >= mlim_hard)
> return (B_TRUE);
> else if (mused < mlim_soft)
> return (B_FALSE);
> else
> return (scn->scn_clearing);
> }
>
> static boolean_t
561,563d1167
< * - we have scanned for the maximum time: an entire txg
< * timeout (default 5 sec)
< * or
571a1176,1177
> * or
> * - the scan queue has reached its memory use limit
573,575c1179,1184
< int mintime = (scn->scn_phys.scn_func == POOL_SCAN_RESILVER) ?
< zfs_resilver_min_time_ms : zfs_scan_min_time_ms;
< uint64_t elapsed_nanosecs = gethrtime() - scn->scn_sync_start_time;
---
> uint64_t elapsed_nanosecs = gethrtime();
> uint64_t curr_time_ns = gethrtime();
> uint64_t scan_time_ns = curr_time_ns - scn->scn_sync_start_time;
> uint64_t sync_time_ns = curr_time_ns -
> scn->scn_dp->dp_spa->spa_sync_starttime;
>
577,581c1186,1194
< if (elapsed_nanosecs / NANOSEC >= zfs_txg_timeout ||
< (NSEC2MSEC(elapsed_nanosecs) > mintime &&
< (txg_sync_waiting(scn->scn_dp) ||
< dirty_pct >= zfs_vdev_async_write_active_min_dirty_percent)) ||
< spa_shutting_down(scn->scn_dp->dp_spa)) {
---
> int mintime = (scn->scn_phys.scn_func == POOL_SCAN_RESILVER) ?
> zfs_resilver_min_time_ms : zfs_scrub_min_time_ms;
>
> if ((NSEC2MSEC(scan_time_ns) > mintime &&
> (dirty_pct >= zfs_vdev_async_write_active_min_dirty_percent ||
> txg_sync_waiting(scn->scn_dp) ||
> NSEC2SEC(sync_time_ns) >= zfs_txg_timeout)) ||
> spa_shutting_down(scn->scn_dp->dp_spa) ||
> (zfs_scan_strict_mem_lim && dsl_scan_should_clear(scn))) {
588a1202,1210
> } else {
> dsl_scan_phys_t *scnp = &scn->scn_phys;
>
> dprintf("suspending at at DDT bookmark "
> "%llx/%llx/%llx/%llx\n",
> (longlong_t)scnp->scn_ddt_bookmark.ddb_class,
> (longlong_t)scnp->scn_ddt_bookmark.ddb_type,
> (longlong_t)scnp->scn_ddt_bookmark.ddb_checksum,
> (longlong_t)scnp->scn_ddt_bookmark.ddb_cursor);
590,594d1211
< dprintf("suspending at DDT bookmark %llx/%llx/%llx/%llx\n",
< (longlong_t)scn->scn_phys.scn_ddt_bookmark.ddb_class,
< (longlong_t)scn->scn_phys.scn_ddt_bookmark.ddb_type,
< (longlong_t)scn->scn_phys.scn_ddt_bookmark.ddb_checksum,
< (longlong_t)scn->scn_phys.scn_ddt_bookmark.ddb_cursor);
693c1310,1325
< /* ARGSUSED */
---
> /*
> * We compare scan_prefetch_issue_ctx_t's based on their bookmarks. The idea
> * here is to sort the AVL tree by the order each block will be needed.
> */
> static int
> scan_prefetch_queue_compare(const void *a, const void *b)
> {
> const scan_prefetch_issue_ctx_t *spic_a = a, *spic_b = b;
> const scan_prefetch_ctx_t *spc_a = spic_a->spic_spc;
> const scan_prefetch_ctx_t *spc_b = spic_b->spic_spc;
>
> return (zbookmark_compare(spc_a->spc_datablkszsec,
> spc_a->spc_indblkshift, spc_b->spc_datablkszsec,
> spc_b->spc_indblkshift, &spic_a->spic_zb, &spic_b->spic_zb));
> }
>
695,696c1327
< dsl_scan_prefetch(dsl_scan_t *scn, arc_buf_t *buf, blkptr_t *bp,
< uint64_t objset, uint64_t object, uint64_t blkid)
---
> scan_prefetch_ctx_rele(scan_prefetch_ctx_t *spc, void *tag)
698,699c1329,1333
< zbookmark_phys_t czb;
< arc_flags_t flags = ARC_FLAG_NOWAIT | ARC_FLAG_PREFETCH;
---
> if (refcount_remove(&spc->spc_refcnt, tag) == 0) {
> refcount_destroy(&spc->spc_refcnt);
> kmem_free(spc, sizeof (scan_prefetch_ctx_t));
> }
> }
700a1335,1392
> static scan_prefetch_ctx_t *
> scan_prefetch_ctx_create(dsl_scan_t *scn, dnode_phys_t *dnp, void *tag)
> {
> scan_prefetch_ctx_t *spc;
>
> spc = kmem_alloc(sizeof (scan_prefetch_ctx_t), KM_SLEEP);
> refcount_create(&spc->spc_refcnt);
> refcount_add(&spc->spc_refcnt, tag);
> spc->spc_scn = scn;
> if (dnp != NULL) {
> spc->spc_datablkszsec = dnp->dn_datablkszsec;
> spc->spc_indblkshift = dnp->dn_indblkshift;
> spc->spc_root = B_FALSE;
> } else {
> spc->spc_datablkszsec = 0;
> spc->spc_indblkshift = 0;
> spc->spc_root = B_TRUE;
> }
>
> return (spc);
> }
>
> static void
> scan_prefetch_ctx_add_ref(scan_prefetch_ctx_t *spc, void *tag)
> {
> refcount_add(&spc->spc_refcnt, tag);
> }
>
> static boolean_t
> dsl_scan_check_prefetch_resume(scan_prefetch_ctx_t *spc,
> const zbookmark_phys_t *zb)
> {
> zbookmark_phys_t *last_zb = &spc->spc_scn->scn_prefetch_bookmark;
> dnode_phys_t tmp_dnp;
> dnode_phys_t *dnp = (spc->spc_root) ? NULL : &tmp_dnp;
>
> if (zb->zb_objset != last_zb->zb_objset)
> return (B_TRUE);
> if ((int64_t)zb->zb_object < 0)
> return (B_FALSE);
>
> tmp_dnp.dn_datablkszsec = spc->spc_datablkszsec;
> tmp_dnp.dn_indblkshift = spc->spc_indblkshift;
>
> if (zbookmark_subtree_completed(dnp, zb, last_zb))
> return (B_TRUE);
>
> return (B_FALSE);
> }
>
> static void
> dsl_scan_prefetch(scan_prefetch_ctx_t *spc, blkptr_t *bp, zbookmark_phys_t *zb)
> {
> avl_index_t idx;
> dsl_scan_t *scn = spc->spc_scn;
> spa_t *spa = scn->scn_dp->dp_spa;
> scan_prefetch_issue_ctx_t *spic;
>
704,705c1396,1398
< if (BP_IS_HOLE(bp) || bp->blk_birth <= scn->scn_phys.scn_min_txg ||
< (BP_GET_LEVEL(bp) == 0 && BP_GET_TYPE(bp) != DMU_OT_DNODE))
---
> if (BP_IS_HOLE(bp) || bp->blk_birth <= scn->scn_phys.scn_cur_min_txg ||
> (BP_GET_LEVEL(bp) == 0 && BP_GET_TYPE(bp) != DMU_OT_DNODE &&
> BP_GET_TYPE(bp) != DMU_OT_OBJSET))
708c1401,1402
< SET_BOOKMARK(&czb, objset, object, BP_GET_LEVEL(bp), blkid);
---
> if (dsl_scan_check_prefetch_resume(spc, zb))
> return;
710,712c1404,1426
< (void) arc_read(scn->scn_zio_root, scn->scn_dp->dp_spa, bp,
< NULL, NULL, ZIO_PRIORITY_ASYNC_READ,
< ZIO_FLAG_CANFAIL | ZIO_FLAG_SCAN_THREAD, &flags, &czb);
---
> scan_prefetch_ctx_add_ref(spc, scn);
> spic = kmem_alloc(sizeof (scan_prefetch_issue_ctx_t), KM_SLEEP);
> spic->spic_spc = spc;
> spic->spic_bp = *bp;
> spic->spic_zb = *zb;
>
> /*
> * Add the IO to the queue of blocks to prefetch. This allows us to
> * prioritize blocks that we will need first for the main traversal
> * thread.
> */
> mutex_enter(&spa->spa_scrub_lock);
> if (avl_find(&scn->scn_prefetch_queue, spic, &idx) != NULL) {
> /* this block is already queued for prefetch */
> kmem_free(spic, sizeof (scan_prefetch_issue_ctx_t));
> scan_prefetch_ctx_rele(spc, scn);
> mutex_exit(&spa->spa_scrub_lock);
> return;
> }
>
> avl_insert(&scn->scn_prefetch_queue, spic, idx);
> cv_broadcast(&spa->spa_scrub_io_cv);
> mutex_exit(&spa->spa_scrub_lock);
714a1429,1581
> static void
> dsl_scan_prefetch_dnode(dsl_scan_t *scn, dnode_phys_t *dnp,
> uint64_t objset, uint64_t object)
> {
> int i;
> zbookmark_phys_t zb;
> scan_prefetch_ctx_t *spc;
>
> if (dnp->dn_nblkptr == 0 && !(dnp->dn_flags & DNODE_FLAG_SPILL_BLKPTR))
> return;
>
> SET_BOOKMARK(&zb, objset, object, 0, 0);
>
> spc = scan_prefetch_ctx_create(scn, dnp, FTAG);
>
> for (i = 0; i < dnp->dn_nblkptr; i++) {
> zb.zb_level = BP_GET_LEVEL(&dnp->dn_blkptr[i]);
> zb.zb_blkid = i;
> dsl_scan_prefetch(spc, &dnp->dn_blkptr[i], &zb);
> }
>
> if (dnp->dn_flags & DNODE_FLAG_SPILL_BLKPTR) {
> zb.zb_level = 0;
> zb.zb_blkid = DMU_SPILL_BLKID;
> dsl_scan_prefetch(spc, &dnp->dn_spill, &zb);
> }
>
> scan_prefetch_ctx_rele(spc, FTAG);
> }
>
> void
> dsl_scan_prefetch_cb(zio_t *zio, const zbookmark_phys_t *zb, const blkptr_t *bp,
> arc_buf_t *buf, void *private)
> {
> scan_prefetch_ctx_t *spc = private;
> dsl_scan_t *scn = spc->spc_scn;
> spa_t *spa = scn->scn_dp->dp_spa;
>
> /* broadcast that the IO has completed for rate limitting purposes */
> mutex_enter(&spa->spa_scrub_lock);
> ASSERT3U(spa->spa_scrub_inflight, >=, BP_GET_PSIZE(bp));
> spa->spa_scrub_inflight -= BP_GET_PSIZE(bp);
> cv_broadcast(&spa->spa_scrub_io_cv);
> mutex_exit(&spa->spa_scrub_lock);
>
> /* if there was an error or we are done prefetching, just cleanup */
> if (buf == NULL || scn->scn_suspending)
> goto out;
>
> if (BP_GET_LEVEL(bp) > 0) {
> int i;
> blkptr_t *cbp;
> int epb = BP_GET_LSIZE(bp) >> SPA_BLKPTRSHIFT;
> zbookmark_phys_t czb;
>
> for (i = 0, cbp = buf->b_data; i < epb; i++, cbp++) {
> SET_BOOKMARK(&czb, zb->zb_objset, zb->zb_object,
> zb->zb_level - 1, zb->zb_blkid * epb + i);
> dsl_scan_prefetch(spc, cbp, &czb);
> }
> } else if (BP_GET_TYPE(bp) == DMU_OT_DNODE) {
> dnode_phys_t *cdnp = buf->b_data;
> int i;
> int epb = BP_GET_LSIZE(bp) >> DNODE_SHIFT;
>
> for (i = 0, cdnp = buf->b_data; i < epb; i++, cdnp++) {
> dsl_scan_prefetch_dnode(scn, cdnp,
> zb->zb_objset, zb->zb_blkid * epb + i);
> }
> } else if (BP_GET_TYPE(bp) == DMU_OT_OBJSET) {
> objset_phys_t *osp = buf->b_data;
>
> dsl_scan_prefetch_dnode(scn, &osp->os_meta_dnode,
> zb->zb_objset, DMU_META_DNODE_OBJECT);
>
> if (OBJSET_BUF_HAS_USERUSED(buf)) {
> dsl_scan_prefetch_dnode(scn,
> &osp->os_groupused_dnode, zb->zb_objset,
> DMU_GROUPUSED_OBJECT);
> dsl_scan_prefetch_dnode(scn,
> &osp->os_userused_dnode, zb->zb_objset,
> DMU_USERUSED_OBJECT);
> }
> }
>
> out:
> if (buf != NULL)
> arc_buf_destroy(buf, private);
> scan_prefetch_ctx_rele(spc, scn);
> }
>
> /* ARGSUSED */
> static void
> dsl_scan_prefetch_thread(void *arg)
> {
> dsl_scan_t *scn = arg;
> spa_t *spa = scn->scn_dp->dp_spa;
> vdev_t *rvd = spa->spa_root_vdev;
> uint64_t maxinflight = rvd->vdev_children * zfs_top_maxinflight;
> scan_prefetch_issue_ctx_t *spic;
>
> /* loop until we are told to stop */
> while (!scn->scn_prefetch_stop) {
> arc_flags_t flags = ARC_FLAG_NOWAIT |
> ARC_FLAG_PRESCIENT_PREFETCH | ARC_FLAG_PREFETCH;
> int zio_flags = ZIO_FLAG_CANFAIL | ZIO_FLAG_SCAN_THREAD;
>
> mutex_enter(&spa->spa_scrub_lock);
>
> /*
> * Wait until we have an IO to issue and are not above our
> * maximum in flight limit.
> */
> while (!scn->scn_prefetch_stop &&
> (avl_numnodes(&scn->scn_prefetch_queue) == 0 ||
> spa->spa_scrub_inflight >= scn->scn_maxinflight_bytes)) {
> cv_wait(&spa->spa_scrub_io_cv, &spa->spa_scrub_lock);
> }
>
> /* recheck if we should stop since we waited for the cv */
> if (scn->scn_prefetch_stop) {
> mutex_exit(&spa->spa_scrub_lock);
> break;
> }
>
> /* remove the prefetch IO from the tree */
> spic = avl_first(&scn->scn_prefetch_queue);
> spa->spa_scrub_inflight += BP_GET_PSIZE(&spic->spic_bp);
> avl_remove(&scn->scn_prefetch_queue, spic);
>
> mutex_exit(&spa->spa_scrub_lock);
>
> /* issue the prefetch asynchronously */
> (void) arc_read(scn->scn_zio_root, scn->scn_dp->dp_spa,
> &spic->spic_bp, dsl_scan_prefetch_cb, spic->spic_spc,
> ZIO_PRIORITY_SCRUB, zio_flags, &flags, &spic->spic_zb);
>
> kmem_free(spic, sizeof (scan_prefetch_issue_ctx_t));
> }
>
> ASSERT(scn->scn_prefetch_stop);
>
> /* free any prefetches we didn't get to complete */
> mutex_enter(&spa->spa_scrub_lock);
> while ((spic = avl_first(&scn->scn_prefetch_queue)) != NULL) {
> avl_remove(&scn->scn_prefetch_queue, spic);
> scan_prefetch_ctx_rele(spic->spic_spc, scn);
> kmem_free(spic, sizeof (scan_prefetch_issue_ctx_t));
> }
> ASSERT0(avl_numnodes(&scn->scn_prefetch_queue));
> mutex_exit(&spa->spa_scrub_lock);
> }
>
750a1618,1624
> static void dsl_scan_visitbp(blkptr_t *bp, const zbookmark_phys_t *zb,
> dnode_phys_t *dnp, dsl_dataset_t *ds, dsl_scan_t *scn,
> dmu_objset_type_t ostype, dmu_tx_t *tx);
> static void dsl_scan_visitdnode(
> dsl_scan_t *, dsl_dataset_t *ds, dmu_objset_type_t ostype,
> dnode_phys_t *dnp, uint64_t object, dmu_tx_t *tx);
>
772c1646
< ZIO_PRIORITY_ASYNC_READ, zio_flags, &flags, zb);
---
> ZIO_PRIORITY_SCRUB, zio_flags, &flags, zb);
778,781d1651
< dsl_scan_prefetch(scn, buf, cbp, zb->zb_objset,
< zb->zb_object, zb->zb_blkid * epb + i);
< }
< for (i = 0, cbp = buf->b_data; i < epb; i++, cbp++) {
794c1664
< int i, j;
---
> int i;
799c1669
< ZIO_PRIORITY_ASYNC_READ, zio_flags, &flags, zb);
---
> ZIO_PRIORITY_SCRUB, zio_flags, &flags, zb);
805,811d1674
< for (j = 0; j < cdnp->dn_nblkptr; j++) {
< blkptr_t *cbp = &cdnp->dn_blkptr[j];
< dsl_scan_prefetch(scn, buf, cbp,
< zb->zb_objset, zb->zb_blkid * epb + i, j);
< }
< }
< for (i = 0, cdnp = buf->b_data; i < epb; i++, cdnp++) {
823c1686
< ZIO_PRIORITY_ASYNC_READ, zio_flags, &flags, zb);
---
> ZIO_PRIORITY_SCRUB, zio_flags, &flags, zb);
889,890c1752
< arc_buf_t *buf = NULL;
< blkptr_t bp_toread = *bp;
---
> blkptr_t *bp_toread = NULL;
892,893d1753
< /* ASSERT(pbuf == NULL || arc_released(pbuf)); */
<
900,902d1759
< if (BP_IS_HOLE(bp))
< return;
<
911c1768,1769
< if (bp->blk_birth <= scn->scn_phys.scn_cur_min_txg)
---
> if (BP_IS_HOLE(bp)) {
> scn->scn_holes_this_txg++;
912a1771
> }
914c1773,1774
< if (dsl_scan_recurse(scn, ds, ostype, dnp, &bp_toread, zb, tx) != 0)
---
> if (bp->blk_birth <= scn->scn_phys.scn_cur_min_txg) {
> scn->scn_lt_min_this_txg++;
915a1776
> }
916a1778,1783
> bp_toread = kmem_alloc(sizeof (blkptr_t), KM_SLEEP);
> *bp_toread = *bp;
>
> if (dsl_scan_recurse(scn, ds, ostype, dnp, bp_toread, zb, tx) != 0)
> return;
>
924,925c1791,1792
< ASSERT(buf == NULL);
< return;
---
> scn->scn_ddt_contained_this_txg++;
> goto out;
935,936c1802,1804
< if (BP_PHYSICAL_BIRTH(bp) <= scn->scn_phys.scn_cur_max_txg) {
< scan_funcs[scn->scn_phys.scn_func](dp, bp, zb);
---
> if (BP_PHYSICAL_BIRTH(bp) > scn->scn_phys.scn_cur_max_txg) {
> scn->scn_gt_max_this_txg++;
> goto out;
937a1806,1809
>
> scan_funcs[scn->scn_phys.scn_func](dp, bp, zb);
> out:
> kmem_free(bp_toread, sizeof (blkptr_t));
944a1817
> scan_prefetch_ctx_t *spc;
948,949d1820
< dsl_scan_visitbp(bp, &zb, NULL,
< ds, scn, DMU_OST_NONE, tx);
950a1822,1836
> if (ZB_IS_ZERO(&scn->scn_phys.scn_bookmark)) {
> SET_BOOKMARK(&scn->scn_prefetch_bookmark,
> zb.zb_objset, 0, 0, 0);
> } else {
> scn->scn_prefetch_bookmark = scn->scn_phys.scn_bookmark;
> }
>
> scn->scn_objsets_visited_this_txg++;
>
> spc = scan_prefetch_ctx_create(scn, NULL, FTAG);
> dsl_scan_prefetch(spc, bp, &zb);
> scan_prefetch_ctx_rele(spc, FTAG);
>
> dsl_scan_visitbp(bp, &zb, NULL, ds, scn, DMU_OST_NONE, tx);
>
954,955c1840,1841
< void
< dsl_scan_ds_destroyed(dsl_dataset_t *ds, dmu_tx_t *tx)
---
> static void
> ds_destroyed_scn_phys(dsl_dataset_t *ds, dsl_scan_phys_t *scn_phys)
957,964c1843
< dsl_pool_t *dp = ds->ds_dir->dd_pool;
< dsl_scan_t *scn = dp->dp_scan;
< uint64_t mintxg;
<
< if (scn->scn_phys.scn_state != DSS_SCANNING)
< return;
<
< if (scn->scn_phys.scn_bookmark.zb_objset == ds->ds_object) {
---
> if (scn_phys->scn_bookmark.zb_objset == ds->ds_object) {
976c1855
< scn->scn_phys.scn_bookmark.zb_objset =
---
> scn_phys->scn_bookmark.zb_objset =
983c1862
< scn->scn_phys.scn_flags |= DSF_VISIT_DS_AGAIN;
---
> scn_phys->scn_flags |= DSF_VISIT_DS_AGAIN;
985c1864
< SET_BOOKMARK(&scn->scn_phys.scn_bookmark,
---
> SET_BOOKMARK(&scn_phys->scn_bookmark,
991,992c1870,1905
< } else if (zap_lookup_int_key(dp->dp_meta_objset,
< scn->scn_phys.scn_queue_obj, ds->ds_object, &mintxg) == 0) {
---
> }
> }
>
> /*
> * Invoked when a dataset is destroyed. We need to make sure that:
> *
> * 1) If it is the dataset that was currently being scanned, we write
> * a new dsl_scan_phys_t and marking the objset reference in it
> * as destroyed.
> * 2) Remove it from the work queue, if it was present.
> *
> * If the dataset was actually a snapshot, instead of marking the dataset
> * as destroyed, we instead substitute the next snapshot in line.
> */
> void
> dsl_scan_ds_destroyed(dsl_dataset_t *ds, dmu_tx_t *tx)
> {
> dsl_pool_t *dp = ds->ds_dir->dd_pool;
> dsl_scan_t *scn = dp->dp_scan;
> uint64_t mintxg;
>
> if (!dsl_scan_is_running(scn))
> return;
>
> ds_destroyed_scn_phys(ds, &scn->scn_phys);
> ds_destroyed_scn_phys(ds, &scn->scn_phys_cached);
>
> if (scan_ds_queue_contains(scn, ds->ds_object, &mintxg)) {
> scan_ds_queue_remove(scn, ds->ds_object);
> if (ds->ds_is_snapshot)
> scan_ds_queue_insert(scn,
> dsl_dataset_phys(ds)->ds_next_snap_obj, mintxg);
> }
>
> if (zap_lookup_int_key(dp->dp_meta_objset, scn->scn_phys.scn_queue_obj,
> ds->ds_object, &mintxg) == 0) {
1021c1934
< dsl_scan_sync_state(scn, tx);
---
> dsl_scan_sync_state(scn, tx, SYNC_CACHED);
1023a1937,1955
> static void
> ds_snapshotted_bookmark(dsl_dataset_t *ds, zbookmark_phys_t *scn_bookmark)
> {
> if (scn_bookmark->zb_objset == ds->ds_object) {
> scn_bookmark->zb_objset =
> dsl_dataset_phys(ds)->ds_prev_snap_obj;
> zfs_dbgmsg("snapshotting ds %llu; currently traversing; "
> "reset zb_objset to %llu",
> (u_longlong_t)ds->ds_object,
> (u_longlong_t)dsl_dataset_phys(ds)->ds_prev_snap_obj);
> }
> }
>
> /*
> * Called when a dataset is snapshotted. If we were currently traversing
> * this snapshot, we reset our bookmark to point at the newly created
> * snapshot. We also modify our work queue to remove the old snapshot and
> * replace with the new one.
> */
1031c1963
< if (scn->scn_phys.scn_state != DSS_SCANNING)
---
> if (!dsl_scan_is_running(scn))
1036,1044c1968,1978
< if (scn->scn_phys.scn_bookmark.zb_objset == ds->ds_object) {
< scn->scn_phys.scn_bookmark.zb_objset =
< dsl_dataset_phys(ds)->ds_prev_snap_obj;
< zfs_dbgmsg("snapshotting ds %llu; currently traversing; "
< "reset zb_objset to %llu",
< (u_longlong_t)ds->ds_object,
< (u_longlong_t)dsl_dataset_phys(ds)->ds_prev_snap_obj);
< } else if (zap_lookup_int_key(dp->dp_meta_objset,
< scn->scn_phys.scn_queue_obj, ds->ds_object, &mintxg) == 0) {
---
> ds_snapshotted_bookmark(ds, &scn->scn_phys.scn_bookmark);
> ds_snapshotted_bookmark(ds, &scn->scn_phys_cached.scn_bookmark);
>
> if (scan_ds_queue_contains(scn, ds->ds_object, &mintxg)) {
> scan_ds_queue_remove(scn, ds->ds_object);
> scan_ds_queue_insert(scn,
> dsl_dataset_phys(ds)->ds_prev_snap_obj, mintxg);
> }
>
> if (zap_lookup_int_key(dp->dp_meta_objset, scn->scn_phys.scn_queue_obj,
> ds->ds_object, &mintxg) == 0) {
1055c1989,1990
< dsl_scan_sync_state(scn, tx);
---
>
> dsl_scan_sync_state(scn, tx, SYNC_CACHED);
1058,1059c1993,1995
< void
< dsl_scan_ds_clone_swapped(dsl_dataset_t *ds1, dsl_dataset_t *ds2, dmu_tx_t *tx)
---
> static void
> ds_clone_swapped_bookmark(dsl_dataset_t *ds1, dsl_dataset_t *ds2,
> zbookmark_phys_t *scn_bookmark)
1061,1069c1997,1998
< dsl_pool_t *dp = ds1->ds_dir->dd_pool;
< dsl_scan_t *scn = dp->dp_scan;
< uint64_t mintxg;
<
< if (scn->scn_phys.scn_state != DSS_SCANNING)
< return;
<
< if (scn->scn_phys.scn_bookmark.zb_objset == ds1->ds_object) {
< scn->scn_phys.scn_bookmark.zb_objset = ds2->ds_object;
---
> if (scn_bookmark->zb_objset == ds1->ds_object) {
> scn_bookmark->zb_objset = ds2->ds_object;
1074,1075c2003,2004
< } else if (scn->scn_phys.scn_bookmark.zb_objset == ds2->ds_object) {
< scn->scn_phys.scn_bookmark.zb_objset = ds1->ds_object;
---
> } else if (scn_bookmark->zb_objset == ds2->ds_object) {
> scn_bookmark->zb_objset = ds1->ds_object;
1080a2010
> }
1081a2012,2038
> /*
> * Called when a parent dataset and its clone are swapped. If we were
> * currently traversing the dataset, we need to switch to traversing the
> * newly promoted parent.
> */
> void
> dsl_scan_ds_clone_swapped(dsl_dataset_t *ds1, dsl_dataset_t *ds2, dmu_tx_t *tx)
> {
> dsl_pool_t *dp = ds1->ds_dir->dd_pool;
> dsl_scan_t *scn = dp->dp_scan;
> uint64_t mintxg;
>
> if (!dsl_scan_is_running(scn))
> return;
>
> ds_clone_swapped_bookmark(ds1, ds2, &scn->scn_phys.scn_bookmark);
> ds_clone_swapped_bookmark(ds1, ds2, &scn->scn_phys_cached.scn_bookmark);
>
> if (scan_ds_queue_contains(scn, ds1->ds_object, &mintxg)) {
> scan_ds_queue_remove(scn, ds1->ds_object);
> scan_ds_queue_insert(scn, ds2->ds_object, mintxg);
> }
> if (scan_ds_queue_contains(scn, ds2->ds_object, &mintxg)) {
> scan_ds_queue_remove(scn, ds2->ds_object);
> scan_ds_queue_insert(scn, ds1->ds_object, mintxg);
> }
>
1085d2041
<
1103,1104c2059,2061
< } else if (zap_lookup_int_key(dp->dp_meta_objset,
< scn->scn_phys.scn_queue_obj, ds2->ds_object, &mintxg) == 0) {
---
> }
> if (zap_lookup_int_key(dp->dp_meta_objset, scn->scn_phys.scn_queue_obj,
> ds2->ds_object, &mintxg) == 0) {
1117c2074
< dsl_scan_sync_state(scn, tx);
---
> dsl_scan_sync_state(scn, tx, SYNC_CACHED);
1120,1124d2076
< struct enqueue_clones_arg {
< dmu_tx_t *tx;
< uint64_t originobj;
< };
<
1129c2081
< struct enqueue_clones_arg *eca = arg;
---
> uint64_t originobj = *(uint64_t *)arg;
1134c2086
< if (dsl_dir_phys(hds->ds_dir)->dd_origin_obj != eca->originobj)
---
> if (dsl_dir_phys(hds->ds_dir)->dd_origin_obj != originobj)
1141c2093
< while (dsl_dataset_phys(ds)->ds_prev_snap_obj != eca->originobj) {
---
> while (dsl_dataset_phys(ds)->ds_prev_snap_obj != originobj) {
1151,1153c2103,2104
< VERIFY(zap_add_int_key(dp->dp_meta_objset,
< scn->scn_phys.scn_queue_obj, ds->ds_object,
< dsl_dataset_phys(ds)->ds_prev_snap_txg, eca->tx) == 0);
---
> scan_ds_queue_insert(scn, ds->ds_object,
> dsl_dataset_phys(ds)->ds_prev_snap_txg);
1198,1200c2149,2151
< dsobj, dsname,
< scn->scn_phys.scn_cur_min_txg,
< scn->scn_phys.scn_max_txg);
---
> (longlong_t)dsobj, dsname,
> (longlong_t)scn->scn_phys.scn_cur_min_txg,
> (longlong_t)scn->scn_phys.scn_max_txg);
1255,1257c2206,2207
< VERIFY(zap_add_int_key(dp->dp_meta_objset,
< scn->scn_phys.scn_queue_obj, ds->ds_object,
< scn->scn_phys.scn_cur_max_txg, tx) == 0);
---
> scan_ds_queue_insert(scn, ds->ds_object,
> scn->scn_phys.scn_cur_max_txg);
1265,1266c2215
< VERIFY(zap_add_int_key(dp->dp_meta_objset,
< scn->scn_phys.scn_queue_obj,
---
> scan_ds_queue_insert(scn,
1268c2217
< dsl_dataset_phys(ds)->ds_creation_txg, tx) == 0);
---
> dsl_dataset_phys(ds)->ds_creation_txg);
1289,1292c2238,2248
< VERIFY0(zap_join_key(dp->dp_meta_objset,
< dsl_dataset_phys(ds)->ds_next_clones_obj,
< scn->scn_phys.scn_queue_obj,
< dsl_dataset_phys(ds)->ds_creation_txg, tx));
---
> zap_cursor_t zc;
> zap_attribute_t za;
> for (zap_cursor_init(&zc, dp->dp_meta_objset,
> dsl_dataset_phys(ds)->ds_next_clones_obj);
> zap_cursor_retrieve(&zc, &za) == 0;
> (void) zap_cursor_advance(&zc)) {
> scan_ds_queue_insert(scn,
> zfs_strtonum(za.za_name, NULL),
> dsl_dataset_phys(ds)->ds_creation_txg);
> }
> zap_cursor_fini(&zc);
1294,1297d2249
< struct enqueue_clones_arg eca;
< eca.tx = tx;
< eca.originobj = ds->ds_object;
<
1299c2251,2252
< enqueue_clones_cb, &eca, DS_FIND_CHILDREN));
---
> enqueue_clones_cb, &ds->ds_object,
> DS_FIND_CHILDREN));
1311d2263
< dmu_tx_t *tx = arg;
1341,1342c2293,2294
< VERIFY(zap_add_int_key(dp->dp_meta_objset, scn->scn_phys.scn_queue_obj,
< ds->ds_object, dsl_dataset_phys(ds)->ds_prev_snap_txg, tx) == 0);
---
> scan_ds_queue_insert(scn, ds->ds_object,
> dsl_dataset_phys(ds)->ds_prev_snap_txg);
1346a2299,2323
> /* ARGSUSED */
> void
> dsl_scan_ddt_entry(dsl_scan_t *scn, enum zio_checksum checksum,
> ddt_entry_t *dde, dmu_tx_t *tx)
> {
> const ddt_key_t *ddk = &dde->dde_key;
> ddt_phys_t *ddp = dde->dde_phys;
> blkptr_t bp;
> zbookmark_phys_t zb = { 0 };
> int p;
>
> if (scn->scn_phys.scn_state != DSS_SCANNING)
> return;
>
> for (p = 0; p < DDT_PHYS_TYPES; p++, ddp++) {
> if (ddp->ddp_phys_birth == 0 ||
> ddp->ddp_phys_birth > scn->scn_phys.scn_max_txg)
> continue;
> ddt_bp_create(checksum, ddk, ddp, &bp);
>
> scn->scn_visited_this_txg++;
> scan_funcs[scn->scn_phys.scn_func](scn->scn_dp, &bp, &zb);
> }
> }
>
1419,1422c2396,2397
< /* ARGSUSED */
< void
< dsl_scan_ddt_entry(dsl_scan_t *scn, enum zio_checksum checksum,
< ddt_entry_t *dde, dmu_tx_t *tx)
---
> static uint64_t
> dsl_scan_ds_maxtxg(dsl_dataset_t *ds)
1424,1440c2399,2402
< const ddt_key_t *ddk = &dde->dde_key;
< ddt_phys_t *ddp = dde->dde_phys;
< blkptr_t bp;
< zbookmark_phys_t zb = { 0 };
<
< if (scn->scn_phys.scn_state != DSS_SCANNING)
< return;
<
< for (int p = 0; p < DDT_PHYS_TYPES; p++, ddp++) {
< if (ddp->ddp_phys_birth == 0 ||
< ddp->ddp_phys_birth > scn->scn_phys.scn_max_txg)
< continue;
< ddt_bp_create(checksum, ddk, ddp, &bp);
<
< scn->scn_visited_this_txg++;
< scan_funcs[scn->scn_phys.scn_func](scn->scn_dp, &bp, &zb);
< }
---
> uint64_t smt = ds->ds_dir->dd_pool->dp_scan->scn_phys.scn_max_txg;
> if (ds->ds_is_snapshot)
> return (MIN(smt, dsl_dataset_phys(ds)->ds_creation_txg));
> return (smt);
1445a2408
> scan_ds_t *sds;
1447,1448d2409
< zap_cursor_t zc;
< zap_attribute_t za;
1472c2433
< enqueue_cb, tx, DS_FIND_CHILDREN));
---
> enqueue_cb, NULL, DS_FIND_CHILDREN));
1479a2441
> uint64_t dsobj = scn->scn_phys.scn_bookmark.zb_objset;
1481c2443
< * If we were suspended, continue from here. Note if the
---
> * If we were suspended, continue from here. Note if the
1486c2448
< dsl_scan_visitds(scn, scn->scn_phys.scn_bookmark.zb_objset, tx);
---
> dsl_scan_visitds(scn, dsobj, tx);
1492c2454
< * In case we were suspended right at the end of the ds, zero the
---
> * In case we suspended right at the end of the ds, zero the
1497,1500c2459,2463
< /* keep pulling things out of the zap-object-as-queue */
< while (zap_cursor_init(&zc, dp->dp_meta_objset,
< scn->scn_phys.scn_queue_obj),
< zap_cursor_retrieve(&zc, &za) == 0) {
---
> /*
> * Keep pulling things out of the dataset avl queue. Updates to the
> * persistent zap-object-as-queue happen only at checkpoints.
> */
> while ((sds = avl_first(&scn->scn_queue)) != NULL) {
1502c2465,2466
< uint64_t dsobj;
---
> uint64_t dsobj = sds->sds_dsobj;
> uint64_t txg = sds->sds_txg;
1504,1506c2468,2470
< dsobj = zfs_strtonum(za.za_name, NULL);
< VERIFY3U(0, ==, zap_remove_int(dp->dp_meta_objset,
< scn->scn_phys.scn_queue_obj, dsobj, tx));
---
> /* dequeue and free the ds from the queue */
> scan_ds_queue_remove(scn, dsobj);
> sds = NULL; /* must not be touched after removal */
1508c2472
< /* Set up min/max txg */
---
> /* Set up min / max txg */
1510c2474
< if (za.za_first_integer != 0) {
---
> if (txg != 0) {
1512,1513c2476
< MAX(scn->scn_phys.scn_min_txg,
< za.za_first_integer);
---
> MAX(scn->scn_phys.scn_min_txg, txg);
1523d2485
< zap_cursor_fini(&zc);
1527c2489,2491
< zap_cursor_fini(&zc);
---
> /* No more objsets to fetch, we're done */
> scn->scn_phys.scn_bookmark.zb_objset = ZB_DESTROYED_OBJSET;
> ASSERT0(scn->scn_suspending);
1529a2494,2536
> static uint64_t
> dsl_scan_count_leaves(vdev_t *vd)
> {
> uint64_t i, leaves = 0;
>
> /* we only count leaves that belong to the main pool and are readable */
> if (vd->vdev_islog || vd->vdev_isspare ||
> vd->vdev_isl2cache || !vdev_readable(vd))
> return (0);
>
> if (vd->vdev_ops->vdev_op_leaf)
> return (1);
>
> for (i = 0; i < vd->vdev_children; i++) {
> leaves += dsl_scan_count_leaves(vd->vdev_child[i]);
> }
>
> return (leaves);
> }
>
>
> static void
> scan_io_queues_update_zio_stats(dsl_scan_io_queue_t *q, const blkptr_t *bp)
> {
> int i;
> uint64_t cur_size = 0;
>
> for (i = 0; i < BP_GET_NDVAS(bp); i++) {
> cur_size += DVA_GET_ASIZE(&bp->blk_dva[i]);
> }
>
> q->q_total_zio_size_this_txg += cur_size;
> q->q_zios_this_txg++;
> }
>
> static void
> scan_io_queues_update_seg_stats(dsl_scan_io_queue_t *q, uint64_t start,
> uint64_t end)
> {
> q->q_total_seg_size_this_txg += end - start;
> q->q_segs_this_txg++;
> }
>
1530a2538,2846
> scan_io_queue_check_suspend(dsl_scan_t *scn)
> {
> /* See comment in dsl_scan_check_suspend() */
> uint64_t curr_time_ns = gethrtime();
> uint64_t scan_time_ns = curr_time_ns - scn->scn_sync_start_time;
> uint64_t sync_time_ns = curr_time_ns -
> scn->scn_dp->dp_spa->spa_sync_starttime;
> int dirty_pct = scn->scn_dp->dp_dirty_total * 100 / zfs_dirty_data_max;
> int mintime = (scn->scn_phys.scn_func == POOL_SCAN_RESILVER) ?
> zfs_resilver_min_time_ms : zfs_scrub_min_time_ms;
>
> return ((NSEC2MSEC(scan_time_ns) > mintime &&
> (dirty_pct >= zfs_vdev_async_write_active_min_dirty_percent ||
> txg_sync_waiting(scn->scn_dp) ||
> NSEC2SEC(sync_time_ns) >= zfs_txg_timeout)) ||
> spa_shutting_down(scn->scn_dp->dp_spa));
> }
>
> /*
> * Given a list of scan_io_t's in io_list, this issues the io's out to
> * disk. This consumes the io_list and frees the scan_io_t's. This is
> * called when emptying queues, either when we're up against the memory
> * limit or when we have finished scanning. Returns B_TRUE if we stopped
> * processing the list before we finished. Any zios that were not issued
> * will remain in the io_list.
> */
> static boolean_t
> scan_io_queue_issue(dsl_scan_io_queue_t *queue, list_t *io_list)
> {
> dsl_scan_t *scn = queue->q_scn;
> scan_io_t *sio;
> int64_t bytes_issued = 0;
> boolean_t suspended = B_FALSE;
>
> while ((sio = list_head(io_list)) != NULL) {
> blkptr_t bp;
>
> if (scan_io_queue_check_suspend(scn)) {
> suspended = B_TRUE;
> break;
> }
>
> sio2bp(sio, &bp, queue->q_vd->vdev_id);
> bytes_issued += sio->sio_asize;
> scan_exec_io(scn->scn_dp, &bp, sio->sio_flags,
> &sio->sio_zb, queue);
> (void) list_remove_head(io_list);
> scan_io_queues_update_zio_stats(queue, &bp);
> kmem_free(sio, sizeof (*sio));
> }
>
> atomic_add_64(&scn->scn_bytes_pending, -bytes_issued);
>
> return (suspended);
> }
>
> /*
> * Given a range_seg_t (extent) and a list, this function passes over a
> * scan queue and gathers up the appropriate ios which fit into that
> * scan seg (starting from lowest LBA). At the end, we remove the segment
> * from the q_exts_by_addr range tree.
> */
> static boolean_t
> scan_io_queue_gather(dsl_scan_io_queue_t *queue, range_seg_t *rs, list_t *list)
> {
> scan_io_t srch_sio, *sio, *next_sio;
> avl_index_t idx;
> uint_t num_sios = 0;
> int64_t bytes_issued = 0;
>
> ASSERT(rs != NULL);
> ASSERT(MUTEX_HELD(&queue->q_vd->vdev_scan_io_queue_lock));
>
> srch_sio.sio_offset = rs->rs_start;
>
> /*
> * The exact start of the extent might not contain any matching zios,
> * so if that's the case, examine the next one in the tree.
> */
> sio = avl_find(&queue->q_sios_by_addr, &srch_sio, &idx);
> if (sio == NULL)
> sio = avl_nearest(&queue->q_sios_by_addr, idx, AVL_AFTER);
>
> while (sio != NULL && sio->sio_offset < rs->rs_end && num_sios <= 32) {
> ASSERT3U(sio->sio_offset, >=, rs->rs_start);
> ASSERT3U(sio->sio_offset + sio->sio_asize, <=, rs->rs_end);
>
> next_sio = AVL_NEXT(&queue->q_sios_by_addr, sio);
> avl_remove(&queue->q_sios_by_addr, sio);
>
> bytes_issued += sio->sio_asize;
> num_sios++;
> list_insert_tail(list, sio);
> sio = next_sio;
> }
>
> /*
> * We limit the number of sios we process at once to 32 to avoid
> * biting off more than we can chew. If we didn't take everything
> * in the segment we update it to reflect the work we were able to
> * complete. Otherwise, we remove it from the range tree entirely.
> */
> if (sio != NULL && sio->sio_offset < rs->rs_end) {
> range_tree_adjust_fill(queue->q_exts_by_addr, rs,
> -bytes_issued);
> range_tree_resize_segment(queue->q_exts_by_addr, rs,
> sio->sio_offset, rs->rs_end - sio->sio_offset);
>
> return (B_TRUE);
> } else {
> range_tree_remove(queue->q_exts_by_addr, rs->rs_start,
> rs->rs_end - rs->rs_start);
> return (B_FALSE);
> }
> }
>
>
> /*
> * This is called from the queue emptying thread and selects the next
> * extent from which we are to issue io's. The behavior of this function
> * depends on the state of the scan, the current memory consumption and
> * whether or not we are performing a scan shutdown.
> * 1) We select extents in an elevator algorithm (LBA-order) if the scan
> * needs to perform a checkpoint
> * 2) We select the largest available extent if we are up against the
> * memory limit.
> * 3) Otherwise we don't select any extents.
> */
> static const range_seg_t *
> scan_io_queue_fetch_ext(dsl_scan_io_queue_t *queue)
> {
> dsl_scan_t *scn = queue->q_scn;
>
> ASSERT(MUTEX_HELD(&queue->q_vd->vdev_scan_io_queue_lock));
> ASSERT(scn->scn_is_sorted);
>
> /* handle tunable overrides */
> if (scn->scn_checkpointing || scn->scn_clearing) {
> if (zfs_scan_issue_strategy == 1) {
> return (range_tree_first(queue->q_exts_by_addr));
> } else if (zfs_scan_issue_strategy == 2) {
> return (avl_first(&queue->q_exts_by_size));
> }
> }
>
> /*
> * During normal clearing, we want to issue our largest segments
> * first, keeping IO as sequential as possible, and leaving the
> * smaller extents for later with the hope that they might eventually
> * grow to larger sequential segments. However, when the scan is
> * checkpointing, no new extents will be added to the sorting queue,
> * so the way we are sorted now is as good as it will ever get.
> * In this case, we instead switch to issuing extents in LBA order.
> */
> if (scn->scn_checkpointing) {
> return (range_tree_first(queue->q_exts_by_addr));
> } else if (scn->scn_clearing) {
> return (avl_first(&queue->q_exts_by_size));
> } else {
> return (NULL);
> }
> }
>
> static void
> scan_io_queues_run_one(void *arg)
> {
> dsl_scan_io_queue_t *queue = arg;
> kmutex_t *q_lock = &queue->q_vd->vdev_scan_io_queue_lock;
> boolean_t suspended = B_FALSE;
> range_seg_t *rs = NULL;
> scan_io_t *sio = NULL;
> list_t sio_list;
> uint64_t bytes_per_leaf = zfs_scan_vdev_limit;
> uint64_t nr_leaves = dsl_scan_count_leaves(queue->q_vd);
>
> ASSERT(queue->q_scn->scn_is_sorted);
>
> list_create(&sio_list, sizeof (scan_io_t),
> offsetof(scan_io_t, sio_nodes.sio_list_node));
> mutex_enter(q_lock);
>
> /* calculate maximum in-flight bytes for this txg (min 1MB) */
> queue->q_maxinflight_bytes =
> MAX(nr_leaves * bytes_per_leaf, 1ULL << 20);
>
> /* reset per-queue scan statistics for this txg */
> queue->q_total_seg_size_this_txg = 0;
> queue->q_segs_this_txg = 0;
> queue->q_total_zio_size_this_txg = 0;
> queue->q_zios_this_txg = 0;
>
> /* loop until we have run out of time or sios */
> while ((rs = (range_seg_t*)scan_io_queue_fetch_ext(queue)) != NULL) {
> uint64_t seg_start = 0, seg_end = 0;
> boolean_t more_left = B_TRUE;
>
> ASSERT(list_is_empty(&sio_list));
>
> /* loop while we still have sios left to process in this rs */
> while (more_left) {
> scan_io_t *first_sio, *last_sio;
>
> /*
> * We have selected which extent needs to be
> * processed next. Gather up the corresponding sios.
> */
> more_left = scan_io_queue_gather(queue, rs, &sio_list);
> ASSERT(!list_is_empty(&sio_list));
> first_sio = list_head(&sio_list);
> last_sio = list_tail(&sio_list);
>
> seg_end = last_sio->sio_offset + last_sio->sio_asize;
> if (seg_start == 0)
> seg_start = first_sio->sio_offset;
>
> /*
> * Issuing sios can take a long time so drop the
> * queue lock. The sio queue won't be updated by
> * other threads since we're in syncing context so
> * we can be sure that our trees will remain exactly
> * as we left them.
> */
> mutex_exit(q_lock);
> suspended = scan_io_queue_issue(queue, &sio_list);
> mutex_enter(q_lock);
>
> if (suspended)
> break;
> }
> /* update statistics for debugging purposes */
> scan_io_queues_update_seg_stats(queue, seg_start, seg_end);
>
> if (suspended)
> break;
> }
>
>
> /* If we were suspended in the middle of processing,
> * requeue any unfinished sios and exit.
> */
> while ((sio = list_head(&sio_list)) != NULL) {
> list_remove(&sio_list, sio);
> scan_io_queue_insert_impl(queue, sio);
> }
>
> mutex_exit(q_lock);
> list_destroy(&sio_list);
> }
>
> /*
> * Performs an emptying run on all scan queues in the pool. This just
> * punches out one thread per top-level vdev, each of which processes
> * only that vdev's scan queue. We can parallelize the I/O here because
> * we know that each queue's io's only affect its own top-level vdev.
> *
> * This function waits for the queue runs to complete, and must be
> * called from dsl_scan_sync (or in general, syncing context).
> */
> static void
> scan_io_queues_run(dsl_scan_t *scn)
> {
> spa_t *spa = scn->scn_dp->dp_spa;
>
> ASSERT(scn->scn_is_sorted);
> ASSERT(spa_config_held(spa, SCL_CONFIG, RW_READER));
>
> if (scn->scn_bytes_pending == 0)
> return;
>
> if (scn->scn_taskq == NULL) {
> char *tq_name = kmem_zalloc(ZFS_MAX_DATASET_NAME_LEN + 16,
> KM_SLEEP);
> int nthreads = spa->spa_root_vdev->vdev_children;
>
> /*
> * We need to make this taskq *always* execute as many
> * threads in parallel as we have top-level vdevs and no
> * less, otherwise strange serialization of the calls to
> * scan_io_queues_run_one can occur during spa_sync runs
> * and that significantly impacts performance.
> */
> (void) snprintf(tq_name, ZFS_MAX_DATASET_NAME_LEN + 16,
> "dsl_scan_tq_%s", spa->spa_name);
> scn->scn_taskq = taskq_create(tq_name, nthreads, minclsyspri,
> nthreads, nthreads, TASKQ_PREPOPULATE);
> kmem_free(tq_name, ZFS_MAX_DATASET_NAME_LEN + 16);
> }
>
> for (uint64_t i = 0; i < spa->spa_root_vdev->vdev_children; i++) {
> vdev_t *vd = spa->spa_root_vdev->vdev_child[i];
>
> mutex_enter(&vd->vdev_scan_io_queue_lock);
> if (vd->vdev_scan_io_queue != NULL) {
> VERIFY(taskq_dispatch(scn->scn_taskq,
> scan_io_queues_run_one, vd->vdev_scan_io_queue,
> TQ_SLEEP) != TASKQID_INVALID);
> }
> mutex_exit(&vd->vdev_scan_io_queue_lock);
> }
>
> /*
> * Wait for the queues to finish issuing thir IOs for this run
> * before we return. There may still be IOs in flight at this
> * point.
> */
> taskq_wait(scn->scn_taskq);
> }
>
> static boolean_t
1567a2884,2918
> static void
> dsl_scan_update_stats(dsl_scan_t *scn)
> {
> spa_t *spa = scn->scn_dp->dp_spa;
> uint64_t i;
> uint64_t seg_size_total = 0, zio_size_total = 0;
> uint64_t seg_count_total = 0, zio_count_total = 0;
>
> for (i = 0; i < spa->spa_root_vdev->vdev_children; i++) {
> vdev_t *vd = spa->spa_root_vdev->vdev_child[i];
> dsl_scan_io_queue_t *queue = vd->vdev_scan_io_queue;
>
> if (queue == NULL)
> continue;
>
> seg_size_total += queue->q_total_seg_size_this_txg;
> zio_size_total += queue->q_total_zio_size_this_txg;
> seg_count_total += queue->q_segs_this_txg;
> zio_count_total += queue->q_zios_this_txg;
> }
>
> if (seg_count_total == 0 || zio_count_total == 0) {
> scn->scn_avg_seg_size_this_txg = 0;
> scn->scn_avg_zio_size_this_txg = 0;
> scn->scn_segs_this_txg = 0;
> scn->scn_zios_this_txg = 0;
> return;
> }
>
> scn->scn_avg_seg_size_this_txg = seg_size_total / seg_count_total;
> scn->scn_avg_zio_size_this_txg = zio_size_total / zio_count_total;
> scn->scn_segs_this_txg = seg_count_total;
> scn->scn_zios_this_txg = zio_count_total;
> }
>
1594,1595c2945
< if ((scn->scn_phys.scn_state == DSS_SCANNING &&
< !dsl_scan_is_paused_scrub(scn)) ||
---
> if ((dsl_scan_is_running(scn) && !dsl_scan_is_paused_scrub(scn)) ||
1605a2956,2994
> static boolean_t
> dsl_scan_need_resilver(spa_t *spa, const dva_t *dva, size_t psize,
> uint64_t phys_birth)
> {
> vdev_t *vd;
>
> if (DVA_GET_GANG(dva)) {
> /*
> * Gang members may be spread across multiple
> * vdevs, so the best estimate we have is the
> * scrub range, which has already been checked.
> * XXX -- it would be better to change our
> * allocation policy to ensure that all
> * gang members reside on the same vdev.
> */
> return (B_TRUE);
> }
>
> vd = vdev_lookup_top(spa, DVA_GET_VDEV(dva));
>
> /*
> * Check if the txg falls within the range which must be
> * resilvered. DVAs outside this range can always be skipped.
> */
> if (!vdev_dtl_contains(vd, DTL_PARTIAL, phys_birth, 1))
> return (B_FALSE);
>
> /*
> * Check if the top-level vdev must resilver this offset.
> * When the offset does not intersect with a dirty leaf DTL
> * then it may be possible to skip the resilver IO. The psize
> * is provided instead of asize to simplify the check for RAIDZ.
> */
> if (!vdev_dtl_need_resilver(vd, DVA_GET_OFFSET(dva), psize))
> return (B_FALSE);
>
> return (B_TRUE);
> }
>
1608a2998
> int err = 0;
1611d3000
< int err = 0;
1617c3006
< spa_version(dp->dp_spa) >= SPA_VERSION_DEADLISTS) {
---
> spa_version(spa) >= SPA_VERSION_DEADLISTS) {
1620c3009
< scn->scn_zio_root = zio_root(dp->dp_spa, NULL,
---
> scn->scn_zio_root = zio_root(spa, NULL,
1624c3013,3014
< VERIFY3U(0, ==, zio_wait(scn->scn_zio_root));
---
> VERIFY0(zio_wait(scn->scn_zio_root));
> scn->scn_zio_root = NULL;
1633c3023
< scn->scn_zio_root = zio_root(dp->dp_spa, NULL,
---
> scn->scn_zio_root = zio_root(spa, NULL,
1637a3028
> scn->scn_zio_root = NULL;
1745a3137,3144
> /*
> * This is the primary entry point for scans that is called from syncing
> * context. Scans must happen entirely during syncing context so that we
> * cna guarantee that blocks we are currently scanning will not change out
> * from under us. While a scan is active, this funciton controls how quickly
> * transaction groups proceed, instead of the normal handling provided by
> * txg_sync_thread().
> */
1751a3151
> state_sync_type_t sync_type = SYNC_OPTIONAL;
1764c3164
< func, tx->tx_txg);
---
> func, (longlong_t)tx->tx_txg);
1787a3188
> /* reset scan statistics */
1788a3190,3198
> scn->scn_holes_this_txg = 0;
> scn->scn_lt_min_this_txg = 0;
> scn->scn_gt_max_this_txg = 0;
> scn->scn_ddt_contained_this_txg = 0;
> scn->scn_objsets_visited_this_txg = 0;
> scn->scn_avg_seg_size_this_txg = 0;
> scn->scn_segs_this_txg = 0;
> scn->scn_avg_zio_size_this_txg = 0;
> scn->scn_zios_this_txg = 0;
1804c3214
< if (scn->scn_phys.scn_state != DSS_SCANNING)
---
> if (!dsl_scan_is_running(scn) || dsl_scan_is_paused_scrub(scn))
1807,1813c3217,3221
< if (scn->scn_done_txg == tx->tx_txg) {
< ASSERT(!scn->scn_suspending);
< /* finished with scan. */
< zfs_dbgmsg("txg %llu scan complete", tx->tx_txg);
< dsl_scan_done(scn, B_TRUE, tx);
< ASSERT3U(spa->spa_scrub_inflight, ==, 0);
< dsl_scan_sync_state(scn, tx);
---
> /*
> * Wait a few txgs after importing to begin scanning so that
> * we can get the pool imported quickly.
> */
> if (spa->spa_syncing_txg < spa->spa_first_txg + SCAN_IMPORT_WAIT_TXGS)
1814a3223,3232
>
> /*
> * It is possible to switch from unsorted to sorted at any time,
> * but afterwards the scan will remain sorted unless reloaded from
> * a checkpoint after a reboot.
> */
> if (!zfs_scan_legacy) {
> scn->scn_is_sorted = B_TRUE;
> if (scn->scn_last_checkpoint == 0)
> scn->scn_last_checkpoint = ddi_get_lbolt();
1817,1818c3235,3255
< if (dsl_scan_is_paused_scrub(scn))
< return;
---
> /*
> * For sorted scans, determine what kind of work we will be doing
> * this txg based on our memory limitations and whether or not we
> * need to perform a checkpoint.
> */
> if (scn->scn_is_sorted) {
> /*
> * If we are over our checkpoint interval, set scn_clearing
> * so that we can begin checkpointing immediately. The
> * checkpoint allows us to save a consisent bookmark
> * representing how much data we have scrubbed so far.
> * Otherwise, use the memory limit to determine if we should
> * scan for metadata or start issue scrub IOs. We accumulate
> * metadata until we hit our hard memory limit at which point
> * we issue scrub IOs until we are at our soft memory limit.
> */
> if (scn->scn_checkpointing ||
> ddi_get_lbolt() - scn->scn_last_checkpoint >
> SEC_TO_TICK(zfs_scan_checkpoint_intval)) {
> if (!scn->scn_checkpointing)
> zfs_dbgmsg("begin scan checkpoint");
1820,1832c3257,3268
< if (scn->scn_phys.scn_ddt_bookmark.ddb_class <=
< scn->scn_phys.scn_ddt_class_max) {
< zfs_dbgmsg("doing scan sync txg %llu; "
< "ddt bm=%llu/%llu/%llu/%llx",
< (longlong_t)tx->tx_txg,
< (longlong_t)scn->scn_phys.scn_ddt_bookmark.ddb_class,
< (longlong_t)scn->scn_phys.scn_ddt_bookmark.ddb_type,
< (longlong_t)scn->scn_phys.scn_ddt_bookmark.ddb_checksum,
< (longlong_t)scn->scn_phys.scn_ddt_bookmark.ddb_cursor);
< ASSERT(scn->scn_phys.scn_bookmark.zb_objset == 0);
< ASSERT(scn->scn_phys.scn_bookmark.zb_object == 0);
< ASSERT(scn->scn_phys.scn_bookmark.zb_level == 0);
< ASSERT(scn->scn_phys.scn_bookmark.zb_blkid == 0);
---
> scn->scn_checkpointing = B_TRUE;
> scn->scn_clearing = B_TRUE;
> } else {
> boolean_t should_clear = dsl_scan_should_clear(scn);
> if (should_clear && !scn->scn_clearing) {
> zfs_dbgmsg("begin scan clearing");
> scn->scn_clearing = B_TRUE;
> } else if (!should_clear && scn->scn_clearing) {
> zfs_dbgmsg("finish scan clearing");
> scn->scn_clearing = B_FALSE;
> }
> }
1834,1839c3270,3271
< zfs_dbgmsg("doing scan sync txg %llu; bm=%llu/%llu/%llu/%llu",
< (longlong_t)tx->tx_txg,
< (longlong_t)scn->scn_phys.scn_bookmark.zb_objset,
< (longlong_t)scn->scn_phys.scn_bookmark.zb_object,
< (longlong_t)scn->scn_phys.scn_bookmark.zb_level,
< (longlong_t)scn->scn_phys.scn_bookmark.zb_blkid);
---
> ASSERT0(scn->scn_checkpointing);
> ASSERT0(scn->scn_clearing);
1842,1848c3274,3279
< scn->scn_zio_root = zio_root(dp->dp_spa, NULL,
< NULL, ZIO_FLAG_CANFAIL);
< dsl_pool_config_enter(dp, FTAG);
< dsl_scan_visit(scn, tx);
< dsl_pool_config_exit(dp, FTAG);
< (void) zio_wait(scn->scn_zio_root);
< scn->scn_zio_root = NULL;
---
> if (!scn->scn_clearing && scn->scn_done_txg == 0) {
> /* Need to scan metadata for more blocks to scrub */
> dsl_scan_phys_t *scnp = &scn->scn_phys;
> taskqid_t prefetch_tqid;
> uint64_t bytes_per_leaf = zfs_scan_vdev_limit;
> uint64_t nr_leaves = dsl_scan_count_leaves(spa->spa_root_vdev);
1850,1852c3281,3287
< zfs_dbgmsg("visited %llu blocks in %llums",
< (longlong_t)scn->scn_visited_this_txg,
< (longlong_t)NSEC2MSEC(gethrtime() - scn->scn_sync_start_time));
---
> /*
> * Calculate the max number of in-flight bytes for pool-wide
> * scanning operations (minimum 1MB). Limits for the issuing
> * phase are done per top-level vdev and are handled separately.
> */
> scn->scn_maxinflight_bytes =
> MAX(nr_leaves * bytes_per_leaf, 1ULL << 20);
1854,1864c3289,3306
< if (!scn->scn_suspending) {
< scn->scn_done_txg = tx->tx_txg + 1;
< zfs_dbgmsg("txg %llu traversal complete, waiting till txg %llu",
< tx->tx_txg, scn->scn_done_txg);
< }
<
< if (DSL_SCAN_IS_SCRUB_RESILVER(scn)) {
< mutex_enter(&spa->spa_scrub_lock);
< while (spa->spa_scrub_inflight > 0) {
< cv_wait(&spa->spa_scrub_io_cv,
< &spa->spa_scrub_lock);
---
> if (scnp->scn_ddt_bookmark.ddb_class <=
> scnp->scn_ddt_class_max) {
> ASSERT(ZB_IS_ZERO(&scnp->scn_bookmark));
> zfs_dbgmsg("doing scan sync txg %llu; "
> "ddt bm=%llu/%llu/%llu/%llx",
> (longlong_t)tx->tx_txg,
> (longlong_t)scnp->scn_ddt_bookmark.ddb_class,
> (longlong_t)scnp->scn_ddt_bookmark.ddb_type,
> (longlong_t)scnp->scn_ddt_bookmark.ddb_checksum,
> (longlong_t)scnp->scn_ddt_bookmark.ddb_cursor);
> } else {
> zfs_dbgmsg("doing scan sync txg %llu; "
> "bm=%llu/%llu/%llu/%llu",
> (longlong_t)tx->tx_txg,
> (longlong_t)scnp->scn_bookmark.zb_objset,
> (longlong_t)scnp->scn_bookmark.zb_object,
> (longlong_t)scnp->scn_bookmark.zb_level,
> (longlong_t)scnp->scn_bookmark.zb_blkid);
1866,1867d3307
< mutex_exit(&spa->spa_scrub_lock);
< }
1869,1870c3309,3310
< dsl_scan_sync_state(scn, tx);
< }
---
> scn->scn_zio_root = zio_root(dp->dp_spa, NULL,
> NULL, ZIO_FLAG_CANFAIL);
1872,1881c3312,3315
< /*
< * This will start a new scan, or restart an existing one.
< */
< void
< dsl_resilver_restart(dsl_pool_t *dp, uint64_t txg)
< {
< if (txg == 0) {
< dmu_tx_t *tx;
< tx = dmu_tx_create_dd(dp->dp_mos_dir);
< VERIFY(0 == dmu_tx_assign(tx, TXG_WAIT));
---
> scn->scn_prefetch_stop = B_FALSE;
> prefetch_tqid = taskq_dispatch(dp->dp_sync_taskq,
> dsl_scan_prefetch_thread, scn, TQ_SLEEP);
> ASSERT(prefetch_tqid != TASKQID_INVALID);
1883,1887c3317,3380
< txg = dmu_tx_get_txg(tx);
< dp->dp_scan->scn_restart_txg = txg;
< dmu_tx_commit(tx);
< } else {
< dp->dp_scan->scn_restart_txg = txg;
---
> dsl_pool_config_enter(dp, FTAG);
> dsl_scan_visit(scn, tx);
> dsl_pool_config_exit(dp, FTAG);
>
> mutex_enter(&dp->dp_spa->spa_scrub_lock);
> scn->scn_prefetch_stop = B_TRUE;
> cv_broadcast(&spa->spa_scrub_io_cv);
> mutex_exit(&dp->dp_spa->spa_scrub_lock);
>
> taskq_wait_id(dp->dp_sync_taskq, prefetch_tqid);
> (void) zio_wait(scn->scn_zio_root);
> scn->scn_zio_root = NULL;
>
> zfs_dbgmsg("scan visited %llu blocks in %llums "
> "(%llu os's, %llu holes, %llu < mintxg, "
> "%llu in ddt, %llu > maxtxg)",
> (longlong_t)scn->scn_visited_this_txg,
> (longlong_t)NSEC2MSEC(gethrtime() -
> scn->scn_sync_start_time),
> (longlong_t)scn->scn_objsets_visited_this_txg,
> (longlong_t)scn->scn_holes_this_txg,
> (longlong_t)scn->scn_lt_min_this_txg,
> (longlong_t)scn->scn_ddt_contained_this_txg,
> (longlong_t)scn->scn_gt_max_this_txg);
>
> if (!scn->scn_suspending) {
> ASSERT0(avl_numnodes(&scn->scn_queue));
> scn->scn_done_txg = tx->tx_txg + 1;
> if (scn->scn_is_sorted) {
> scn->scn_checkpointing = B_TRUE;
> scn->scn_clearing = B_TRUE;
> }
> zfs_dbgmsg("scan complete txg %llu",
> (longlong_t)tx->tx_txg);
> }
> } else if (scn->scn_is_sorted && scn->scn_bytes_pending != 0) {
> /* need to issue scrubbing IOs from per-vdev queues */
> scn->scn_zio_root = zio_root(dp->dp_spa, NULL,
> NULL, ZIO_FLAG_CANFAIL);
> scan_io_queues_run(scn);
> (void) zio_wait(scn->scn_zio_root);
> scn->scn_zio_root = NULL;
>
> /* calculate and dprintf the current memory usage */
> (void) dsl_scan_should_clear(scn);
> dsl_scan_update_stats(scn);
>
> zfs_dbgmsg("scrubbed %llu blocks (%llu segs) in %llums "
> "(avg_block_size = %llu, avg_seg_size = %llu)",
> (longlong_t)scn->scn_zios_this_txg,
> (longlong_t)scn->scn_segs_this_txg,
> (longlong_t)NSEC2MSEC(gethrtime() -
> scn->scn_sync_start_time),
> (longlong_t)scn->scn_avg_zio_size_this_txg,
> (longlong_t)scn->scn_avg_seg_size_this_txg);
> } else if (scn->scn_done_txg != 0 && scn->scn_done_txg <= tx->tx_txg) {
> /* Finished with everything. Mark the scrub as complete */
> zfs_dbgmsg("scan issuing complete txg %llu",
> (longlong_t)tx->tx_txg);
> ASSERT3U(scn->scn_done_txg, !=, 0);
> ASSERT0(spa->spa_scrub_inflight);
> ASSERT0(scn->scn_bytes_pending);
> dsl_scan_done(scn, B_TRUE, tx);
> sync_type = SYNC_MANDATORY;
1889,1890d3381
< zfs_dbgmsg("restarting resilver txg=%llu", txg);
< }
1892,1896c3383
< boolean_t
< dsl_scan_resilvering(dsl_pool_t *dp)
< {
< return (dp->dp_scan->scn_phys.scn_state == DSS_SCANNING &&
< dp->dp_scan->scn_phys.scn_func == POOL_SCAN_RESILVER);
---
> dsl_scan_sync_state(scn, tx, sync_type);
1899,1902d3385
< /*
< * scrub consumers
< */
<
1904c3387
< count_block(zfs_all_blkstats_t *zab, const blkptr_t *bp)
---
> count_block(dsl_scan_t *scn, zfs_all_blkstats_t *zab, const blkptr_t *bp)
1907a3391,3396
> /* update the spa's stats on how many bytes we have issued */
> for (i = 0; i < BP_GET_NDVAS(bp); i++) {
> atomic_add_64(&scn->scn_dp->dp_spa->spa_scan_pass_issued,
> DVA_GET_ASIZE(&bp->blk_dva[i]));
> }
>
1914a3404,3405
> mutex_enter(&zab->zab_lock);
>
1948a3440,3441
>
> mutex_exit(&zab->zab_lock);
1952c3445
< dsl_scan_scrub_done(zio_t *zio)
---
> scan_io_queue_insert_impl(dsl_scan_io_queue_t *queue, scan_io_t *sio)
1954c3447,3449
< spa_t *spa = zio->io_spa;
---
> avl_index_t idx;
> int64_t asize = sio->sio_asize;
> dsl_scan_t *scn = queue->q_scn;
1956c3451
< abd_free(zio->io_abd);
---
> ASSERT(MUTEX_HELD(&queue->q_vd->vdev_scan_io_queue_lock));
1958,1960c3453,3461
< mutex_enter(&spa->spa_scrub_lock);
< spa->spa_scrub_inflight--;
< cv_broadcast(&spa->spa_scrub_io_cv);
---
> if (avl_find(&queue->q_sios_by_addr, sio, &idx) != NULL) {
> /* block is already scheduled for reading */
> atomic_add_64(&scn->scn_bytes_pending, -asize);
> kmem_free(sio, sizeof (*sio));
> return;
> }
> avl_insert(&queue->q_sios_by_addr, sio, idx);
> range_tree_add(queue->q_exts_by_addr, sio->sio_offset, asize);
> }
1962,1964c3463,3512
< if (zio->io_error && (zio->io_error != ECKSUM ||
< !(zio->io_flags & ZIO_FLAG_SPECULATIVE))) {
< spa->spa_dsl_pool->dp_scan->scn_phys.scn_errors++;
---
> /*
> * Given all the info we got from our metadata scanning process, we
> * construct a scan_io_t and insert it into the scan sorting queue. The
> * I/O must already be suitable for us to process. This is controlled
> * by dsl_scan_enqueue().
> */
> static void
> scan_io_queue_insert(dsl_scan_io_queue_t *queue, const blkptr_t *bp, int dva_i,
> int zio_flags, const zbookmark_phys_t *zb)
> {
> dsl_scan_t *scn = queue->q_scn;
> scan_io_t *sio = kmem_zalloc(sizeof (*sio), KM_SLEEP);
>
> ASSERT0(BP_IS_GANG(bp));
> ASSERT(MUTEX_HELD(&queue->q_vd->vdev_scan_io_queue_lock));
>
> bp2sio(bp, sio, dva_i);
> sio->sio_flags = zio_flags;
> sio->sio_zb = *zb;
>
> /*
> * Increment the bytes pending counter now so that we can't
> * get an integer underflow in case the worker processes the
> * zio before we get to incrementing this counter.
> */
> atomic_add_64(&scn->scn_bytes_pending, sio->sio_asize);
>
> scan_io_queue_insert_impl(queue, sio);
> }
>
> /*
> * Given a set of I/O parameters as discovered by the metadata traversal
> * process, attempts to place the I/O into the sorted queues (if allowed),
> * or immediately executes the I/O.
> */
> static void
> dsl_scan_enqueue(dsl_pool_t *dp, const blkptr_t *bp, int zio_flags,
> const zbookmark_phys_t *zb)
> {
> spa_t *spa = dp->dp_spa;
>
> ASSERT(!BP_IS_EMBEDDED(bp));
>
> /*
> * Gang blocks are hard to issue sequentially, so we just issue them
> * here immediately instead of queuing them.
> */
> if (!dp->dp_scan->scn_is_sorted || BP_IS_GANG(bp)) {
> scan_exec_io(dp, bp, zio_flags, zb, NULL);
> return;
1966c3514,3529
< mutex_exit(&spa->spa_scrub_lock);
---
> for (int i = 0; i < BP_GET_NDVAS(bp); i++) {
> dva_t dva;
> vdev_t *vdev;
>
> dva = bp->blk_dva[i];
> vdev = vdev_lookup_top(spa, DVA_GET_VDEV(&dva));
> ASSERT(vdev != NULL);
>
> mutex_enter(&vdev->vdev_scan_io_queue_lock);
> if (vdev->vdev_scan_io_queue == NULL)
> vdev->vdev_scan_io_queue = scan_io_queue_create(vdev);
> ASSERT(dp->dp_scan != NULL);
> scan_io_queue_insert(vdev->vdev_scan_io_queue, bp,
> i, zio_flags, zb);
> mutex_exit(&vdev->vdev_scan_io_queue_lock);
> }
1974d3536
< size_t size = BP_GET_PSIZE(bp);
1976a3539
> size_t psize = BP_GET_PSIZE(bp);
1979,1980c3542,3543
< unsigned int scan_delay = 0;
<
---
> int d;
>
1985,1987c3548,3549
< count_block(dp->dp_blkstats, bp);
<
< if (BP_IS_EMBEDDED(bp))
---
> if (BP_IS_EMBEDDED(bp)) {
> count_block(scn, dp->dp_blkstats, bp);
1988a3551
> }
1994d3556
< scan_delay = zfs_scrub_delay;
1999d3560
< scan_delay = zfs_resilver_delay;
2006,2008c3567,3568
< for (int d = 0; d < BP_GET_NDVAS(bp); d++) {
< vdev_t *vd = vdev_lookup_top(spa,
< DVA_GET_VDEV(&bp->blk_dva[d]));
---
> for (d = 0; d < BP_GET_NDVAS(bp); d++) {
> const dva_t *dva = &bp->blk_dva[d];
2014,2015c3574,3575
< scn->scn_phys.scn_examined += DVA_GET_ASIZE(&bp->blk_dva[d]);
< spa->spa_scan_pass_exam += DVA_GET_ASIZE(&bp->blk_dva[d]);
---
> scn->scn_phys.scn_examined += DVA_GET_ASIZE(dva);
> spa->spa_scan_pass_exam += DVA_GET_ASIZE(dva);
2018,2033c3578,3580
< if (!needs_io) {
< if (DVA_GET_GANG(&bp->blk_dva[d])) {
< /*
< * Gang members may be spread across multiple
< * vdevs, so the best estimate we have is the
< * scrub range, which has already been checked.
< * XXX -- it would be better to change our
< * allocation policy to ensure that all
< * gang members reside on the same vdev.
< */
< needs_io = B_TRUE;
< } else {
< needs_io = vdev_dtl_contains(vd, DTL_PARTIAL,
< phys_birth, 1);
< }
< }
---
> if (!needs_io)
> needs_io = dsl_scan_need_resilver(spa, dva, psize,
> phys_birth);
2037,2039c3584,3587
< vdev_t *rvd = spa->spa_root_vdev;
< uint64_t maxinflight = rvd->vdev_children *
< MAX(zfs_top_maxinflight, 1);
---
> dsl_scan_enqueue(dp, bp, zio_flags, zb);
> } else {
> count_block(scn, dp->dp_blkstats, bp);
> }
2040a3589,3602
> /* do not relocate this block */
> return (0);
> }
>
> static void
> dsl_scan_scrub_done(zio_t *zio)
> {
> spa_t *spa = zio->io_spa;
> blkptr_t *bp = zio->io_bp;
> dsl_scan_io_queue_t *queue = zio->io_private;
>
> abd_free(zio->io_abd);
>
> if (queue == NULL) {
2042c3604,3641
< while (spa->spa_scrub_inflight >= maxinflight)
---
> ASSERT3U(spa->spa_scrub_inflight, >=, BP_GET_PSIZE(bp));
> spa->spa_scrub_inflight -= BP_GET_PSIZE(bp);
> cv_broadcast(&spa->spa_scrub_io_cv);
> mutex_exit(&spa->spa_scrub_lock);
> } else {
> mutex_enter(&queue->q_vd->vdev_scan_io_queue_lock);
> ASSERT3U(queue->q_inflight_bytes, >=, BP_GET_PSIZE(bp));
> queue->q_inflight_bytes -= BP_GET_PSIZE(bp);
> cv_broadcast(&queue->q_zio_cv);
> mutex_exit(&queue->q_vd->vdev_scan_io_queue_lock);
> }
>
> if (zio->io_error && (zio->io_error != ECKSUM ||
> !(zio->io_flags & ZIO_FLAG_SPECULATIVE))) {
> atomic_inc_64(&spa->spa_dsl_pool->dp_scan->scn_phys.scn_errors);
> }
> }
>
> /*
> * Given a scanning zio's information, executes the zio. The zio need
> * not necessarily be only sortable, this function simply executes the
> * zio, no matter what it is. The optional queue argument allows the
> * caller to specify that they want per top level vdev IO rate limiting
> * instead of the legacy global limiting.
> */
> static void
> scan_exec_io(dsl_pool_t *dp, const blkptr_t *bp, int zio_flags,
> const zbookmark_phys_t *zb, dsl_scan_io_queue_t *queue)
> {
> spa_t *spa = dp->dp_spa;
> dsl_scan_t *scn = dp->dp_scan;
> size_t size = BP_GET_PSIZE(bp);
> abd_t *data = abd_alloc_for_io(size, B_FALSE);
> unsigned int scan_delay = 0;
>
> if (queue == NULL) {
> mutex_enter(&spa->spa_scrub_lock);
> while (spa->spa_scrub_inflight >= scn->scn_maxinflight_bytes)
2044c3643
< spa->spa_scrub_inflight++;
---
> spa->spa_scrub_inflight += BP_GET_PSIZE(bp);
2045a3645,3646
> } else {
> kmutex_t *q_lock = &queue->q_vd->vdev_scan_io_queue_lock;
2047,2052c3648,3653
< /*
< * If we're seeing recent (zfs_scan_idle) "important" I/Os
< * then throttle our workload to limit the impact of a scan.
< */
< if (ddi_get_lbolt64() - spa->spa_last_io <= zfs_scan_idle)
< delay(MAX((int)scan_delay, 0));
---
> mutex_enter(q_lock);
> while (queue->q_inflight_bytes >= queue->q_maxinflight_bytes)
> cv_wait(&queue->q_zio_cv, q_lock);
> queue->q_inflight_bytes += BP_GET_PSIZE(bp);
> mutex_exit(q_lock);
> }
2054,2056c3655,3659
< zio_nowait(zio_read(NULL, spa, bp,
< abd_alloc_for_io(size, B_FALSE), size, dsl_scan_scrub_done,
< NULL, ZIO_PRIORITY_SCRUB, zio_flags, zb));
---
> if (zio_flags & ZIO_FLAG_RESILVER)
> scan_delay = zfs_resilver_delay;
> else {
> ASSERT(zio_flags & ZIO_FLAG_SCRUB);
> scan_delay = zfs_scrub_delay;
2059,2060c3662,3667
< /* do not relocate this block */
< return (0);
---
> if (scan_delay && (ddi_get_lbolt64() - spa->spa_last_io <= zfs_scan_idle))
> delay(MAX((int)scan_delay, 0));
>
> count_block(dp->dp_scan, dp->dp_blkstats, bp);
> zio_nowait(zio_read(dp->dp_scan->scn_zio_root, spa, bp, data, size,
> dsl_scan_scrub_done, queue, ZIO_PRIORITY_SCRUB, zio_flags, zb));
2064,2065c3671,3700
< * Called by the ZFS_IOC_POOL_SCAN ioctl to start a scrub or resilver.
< * Can also be called to resume a paused scrub.
---
> * This is the primary extent sorting algorithm. We balance two parameters:
> * 1) how many bytes of I/O are in an extent
> * 2) how well the extent is filled with I/O (as a fraction of its total size)
> * Since we allow extents to have gaps between their constituent I/Os, it's
> * possible to have a fairly large extent that contains the same amount of
> * I/O bytes than a much smaller extent, which just packs the I/O more tightly.
> * The algorithm sorts based on a score calculated from the extent's size,
> * the relative fill volume (in %) and a "fill weight" parameter that controls
> * the split between whether we prefer larger extents or more well populated
> * extents:
> *
> * SCORE = FILL_IN_BYTES + (FILL_IN_PERCENT * FILL_IN_BYTES * FILL_WEIGHT)
> *
> * Example:
> * 1) assume extsz = 64 MiB
> * 2) assume fill = 32 MiB (extent is half full)
> * 3) assume fill_weight = 3
> * 4) SCORE = 32M + (((32M * 100) / 64M) * 3 * 32M) / 100
> * SCORE = 32M + (50 * 3 * 32M) / 100
> * SCORE = 32M + (4800M / 100)
> * SCORE = 32M + 48M
> * ^ ^
> * | +--- final total relative fill-based score
> * +--------- final total fill-based score
> * SCORE = 80M
> *
> * As can be seen, at fill_ratio=3, the algorithm is slightly biased towards
> * extents that are more completely filled (in a 3:2 ratio) vs just larger.
> * Note that as an optimization, we replace multiplication and division by
> * 100 with bitshifting by 7 (which effecitvely multiplies and divides by 128).
2067,2068c3702,3703
< int
< dsl_scan(dsl_pool_t *dp, pool_scan_func_t func)
---
> static int
> ext_size_compare(const void *x, const void *y)
2070c3705,3833
< spa_t *spa = dp->dp_spa;
---
> const range_seg_t *rsa = x, *rsb = y;
> uint64_t sa = rsa->rs_end - rsa->rs_start,
> sb = rsb->rs_end - rsb->rs_start;
> uint64_t score_a, score_b;
>
> score_a = rsa->rs_fill + ((((rsa->rs_fill << 7) / sa) *
> fill_weight * rsa->rs_fill) >> 7);
> score_b = rsb->rs_fill + ((((rsb->rs_fill << 7) / sb) *
> fill_weight * rsb->rs_fill) >> 7);
>
> if (score_a > score_b)
> return (-1);
> if (score_a == score_b) {
> if (rsa->rs_start < rsb->rs_start)
> return (-1);
> if (rsa->rs_start == rsb->rs_start)
> return (0);
> return (1);
> }
> return (1);
> }
>
> /*
> * Comparator for the q_sios_by_addr tree. Sorting is simply performed
> * based on LBA-order (from lowest to highest).
> */
> static int
> io_addr_compare(const void *x, const void *y)
> {
> const scan_io_t *a = x, *b = y;
>
> if (a->sio_offset < b->sio_offset)
> return (-1);
> if (a->sio_offset == b->sio_offset)
> return (0);
> return (1);
> }
>
> /* IO queues are created on demand when they are needed. */
> static dsl_scan_io_queue_t *
> scan_io_queue_create(vdev_t *vd)
> {
> dsl_scan_t *scn = vd->vdev_spa->spa_dsl_pool->dp_scan;
> dsl_scan_io_queue_t *q = kmem_zalloc(sizeof (*q), KM_SLEEP);
>
> q->q_scn = scn;
> q->q_vd = vd;
> cv_init(&q->q_zio_cv, NULL, CV_DEFAULT, NULL);
> q->q_exts_by_addr = range_tree_create_impl(&rt_avl_ops,
> &q->q_exts_by_size, ext_size_compare, zfs_scan_max_ext_gap);
> avl_create(&q->q_sios_by_addr, io_addr_compare,
> sizeof (scan_io_t), offsetof(scan_io_t, sio_nodes.sio_addr_node));
>
> return (q);
> }
>
> /*
> * Destroys a scan queue and all segments and scan_io_t's contained in it.
> * No further execution of I/O occurs, anything pending in the queue is
> * simply freed without being executed.
> */
> void
> dsl_scan_io_queue_destroy(dsl_scan_io_queue_t *queue)
> {
> dsl_scan_t *scn = queue->q_scn;
> scan_io_t *sio;
> void *cookie = NULL;
> int64_t bytes_dequeued = 0;
>
> ASSERT(MUTEX_HELD(&queue->q_vd->vdev_scan_io_queue_lock));
>
> while ((sio = avl_destroy_nodes(&queue->q_sios_by_addr, &cookie)) !=
> NULL) {
> ASSERT(range_tree_contains(queue->q_exts_by_addr,
> sio->sio_offset, sio->sio_asize));
> bytes_dequeued += sio->sio_asize;
> kmem_free(sio, sizeof (*sio));
> }
>
> atomic_add_64(&scn->scn_bytes_pending, -bytes_dequeued);
> range_tree_vacate(queue->q_exts_by_addr, NULL, queue);
> range_tree_destroy(queue->q_exts_by_addr);
> avl_destroy(&queue->q_sios_by_addr);
> cv_destroy(&queue->q_zio_cv);
>
> kmem_free(queue, sizeof (*queue));
> }
>
> /*
> * Properly transfers a dsl_scan_queue_t from `svd' to `tvd'. This is
> * called on behalf of vdev_top_transfer when creating or destroying
> * a mirror vdev due to zpool attach/detach.
> */
> void
> dsl_scan_io_queue_vdev_xfer(vdev_t *svd, vdev_t *tvd)
> {
> mutex_enter(&svd->vdev_scan_io_queue_lock);
> mutex_enter(&tvd->vdev_scan_io_queue_lock);
>
> VERIFY3P(tvd->vdev_scan_io_queue, ==, NULL);
> tvd->vdev_scan_io_queue = svd->vdev_scan_io_queue;
> svd->vdev_scan_io_queue = NULL;
> if (tvd->vdev_scan_io_queue != NULL)
> tvd->vdev_scan_io_queue->q_vd = tvd;
>
> mutex_exit(&tvd->vdev_scan_io_queue_lock);
> mutex_exit(&svd->vdev_scan_io_queue_lock);
> }
>
> static void
> scan_io_queues_destroy(dsl_scan_t *scn)
> {
> vdev_t *rvd = scn->scn_dp->dp_spa->spa_root_vdev;
>
> for (uint64_t i = 0; i < rvd->vdev_children; i++) {
> vdev_t *tvd = rvd->vdev_child[i];
>
> mutex_enter(&tvd->vdev_scan_io_queue_lock);
> if (tvd->vdev_scan_io_queue != NULL)
> dsl_scan_io_queue_destroy(tvd->vdev_scan_io_queue);
> tvd->vdev_scan_io_queue = NULL;
> mutex_exit(&tvd->vdev_scan_io_queue_lock);
> }
> }
>
> static void
> dsl_scan_freed_dva(spa_t *spa, const blkptr_t *bp, int dva_i)
> {
> dsl_pool_t *dp = spa->spa_dsl_pool;
2071a3835,3840
> vdev_t *vdev;
> kmutex_t *q_lock;
> dsl_scan_io_queue_t *queue;
> scan_io_t srch, *sio;
> avl_index_t idx;
> uint64_t start, size;
2072a3842,3856
> vdev = vdev_lookup_top(spa, DVA_GET_VDEV(&bp->blk_dva[dva_i]));
> ASSERT(vdev != NULL);
> q_lock = &vdev->vdev_scan_io_queue_lock;
> queue = vdev->vdev_scan_io_queue;
>
> mutex_enter(q_lock);
> if (queue == NULL) {
> mutex_exit(q_lock);
> return;
> }
>
> bp2sio(bp, &srch, dva_i);
> start = srch.sio_offset;
> size = srch.sio_asize;
>
2074,2078c3858,3873
< * Purge all vdev caches and probe all devices. We do this here
< * rather than in sync context because this requires a writer lock
< * on the spa_config lock, which we can't do from sync context. The
< * spa_scrub_reopen flag indicates that vdev_open() should not
< * attempt to start another scrub.
---
> * We can find the zio in two states:
> * 1) Cold, just sitting in the queue of zio's to be issued at
> * some point in the future. In this case, all we do is
> * remove the zio from the q_sios_by_addr tree, decrement
> * its data volume from the containing range_seg_t and
> * resort the q_exts_by_size tree to reflect that the
> * range_seg_t has lost some of its 'fill'. We don't shorten
> * the range_seg_t - this is usually rare enough not to be
> * worth the extra hassle of trying keep track of precise
> * extent boundaries.
> * 2) Hot, where the zio is currently in-flight in
> * dsl_scan_issue_ios. In this case, we can't simply
> * reach in and stop the in-flight zio's, so we instead
> * block the caller. Eventually, dsl_scan_issue_ios will
> * be done with issuing the zio's it gathered and will
> * signal us.
2080,2084c3875,3878
< spa_vdev_state_enter(spa, SCL_NONE);
< spa->spa_scrub_reopen = B_TRUE;
< vdev_reopen(spa->spa_root_vdev);
< spa->spa_scrub_reopen = B_FALSE;
< (void) spa_vdev_state_exit(spa, NULL, 0);
---
> sio = avl_find(&queue->q_sios_by_addr, &srch, &idx);
> if (sio != NULL) {
> int64_t asize = sio->sio_asize;
> blkptr_t tmpbp;
2086,2093c3880,3883
< if (func == POOL_SCAN_SCRUB && dsl_scan_is_paused_scrub(scn)) {
< /* got scrub start cmd, resume paused scrub */
< int err = dsl_scrub_set_pause_resume(scn->scn_dp,
< POOL_SCRUB_NORMAL);
< if (err == 0) {
< spa_event_notify(spa, NULL, NULL, ESC_ZFS_SCRUB_RESUME);
< return (ECANCELED);
< }
---
> /* Got it while it was cold in the queue */
> ASSERT3U(start, ==, sio->sio_offset);
> ASSERT3U(size, ==, asize);
> avl_remove(&queue->q_sios_by_addr, sio);
2095,2096c3885,3886
< return (SET_ERROR(err));
< }
---
> ASSERT(range_tree_contains(queue->q_exts_by_addr, start, size));
> range_tree_remove_fill(queue->q_exts_by_addr, start, size);
2098,2099c3888,3901
< return (dsl_sync_task(spa_name(spa), dsl_scan_setup_check,
< dsl_scan_setup_sync, &func, 0, ZFS_SPACE_CHECK_EXTRA_RESERVED));
---
> /*
> * We only update scn_bytes_pending in the cold path,
> * otherwise it will already have been accounted for as
> * part of the zio's execution.
> */
> atomic_add_64(&scn->scn_bytes_pending, -asize);
>
> /* count the block as though we issued it */
> sio2bp(sio, &tmpbp, dva_i);
> count_block(scn, dp->dp_blkstats, &tmpbp);
>
> kmem_free(sio, sizeof (*sio));
> }
> mutex_exit(q_lock);
2102,2103c3904,3911
< static boolean_t
< dsl_scan_restarting(dsl_scan_t *scn, dmu_tx_t *tx)
---
> /*
> * Callback invoked when a zio_free() zio is executing. This needs to be
> * intercepted to prevent the zio from deallocating a particular portion
> * of disk space and it then getting reallocated and written to, while we
> * still have it queued up for processing.
> */
> void
> dsl_scan_freed(spa_t *spa, const blkptr_t *bp)
2105,2106c3913,3922
< return (scn->scn_restart_txg != 0 &&
< scn->scn_restart_txg <= tx->tx_txg);
---
> dsl_pool_t *dp = spa->spa_dsl_pool;
> dsl_scan_t *scn = dp->dp_scan;
>
> ASSERT(!BP_IS_EMBEDDED(bp));
> ASSERT(scn != NULL);
> if (!dsl_scan_is_running(scn))
> return;
>
> for (int i = 0; i < BP_GET_NDVAS(bp); i++)
> dsl_scan_freed_dva(spa, bp, i);