Print this page
4045 zfs write throttle & i/o scheduler performance work
Reviewed by: George Wilson <george.wilson@delphix.com>
Reviewed by: Adam Leventhal <ahl@delphix.com>
Reviewed by: Christopher Siden <christopher.siden@delphix.com>

Split Close
Expand all
Collapse all
          --- old/usr/src/uts/common/fs/zfs/dsl_pool.c
          +++ new/usr/src/uts/common/fs/zfs/dsl_pool.c
↓ open down ↓ 38 lines elided ↑ open up ↑
  39   39  #include <sys/zfs_context.h>
  40   40  #include <sys/fs/zfs.h>
  41   41  #include <sys/zfs_znode.h>
  42   42  #include <sys/spa_impl.h>
  43   43  #include <sys/dsl_deadlist.h>
  44   44  #include <sys/bptree.h>
  45   45  #include <sys/zfeature.h>
  46   46  #include <sys/zil_impl.h>
  47   47  #include <sys/dsl_userhold.h>
  48   48  
  49      -int zfs_no_write_throttle = 0;
  50      -int zfs_write_limit_shift = 3;                  /* 1/8th of physical memory */
  51      -int zfs_txg_synctime_ms = 1000;         /* target millisecs to sync a txg */
       49 +/*
       50 + * ZFS Write Throttle
       51 + * ------------------
       52 + *
       53 + * ZFS must limit the rate of incoming writes to the rate at which it is able
       54 + * to sync data modifications to the backend storage. Throttling by too much
       55 + * creates an artificial limit; throttling by too little can only be sustained
       56 + * for short periods and would lead to highly lumpy performance. On a per-pool
       57 + * basis, ZFS tracks the amount of modified (dirty) data. As operations change
       58 + * data, the amount of dirty data increases; as ZFS syncs out data, the amount
       59 + * of dirty data decreases. When the amount of dirty data exceeds a
       60 + * predetermined threshold further modifications are blocked until the amount
       61 + * of dirty data decreases (as data is synced out).
       62 + *
       63 + * The limit on dirty data is tunable, and should be adjusted according to
       64 + * both the IO capacity and available memory of the system. The larger the
       65 + * window, the more ZFS is able to aggregate and amortize metadata (and data)
       66 + * changes. However, memory is a limited resource, and allowing for more dirty
       67 + * data comes at the cost of keeping other useful data in memory (for example
       68 + * ZFS data cached by the ARC).
       69 + *
       70 + * Implementation
       71 + *
       72 + * As buffers are modified dsl_pool_willuse_space() increments both the per-
       73 + * txg (dp_dirty_pertxg[]) and poolwide (dp_dirty_total) accounting of
       74 + * dirty space used; dsl_pool_dirty_space() decrements those values as data
       75 + * is synced out from dsl_pool_sync(). While only the poolwide value is
       76 + * relevant, the per-txg value is useful for debugging. The tunable
       77 + * zfs_dirty_data_max determines the dirty space limit. Once that value is
       78 + * exceeded, new writes are halted until space frees up.
       79 + *
       80 + * The zfs_dirty_data_sync tunable dictates the threshold at which we
       81 + * ensure that there is a txg syncing (see the comment in txg.c for a full
       82 + * description of transaction group stages).
       83 + *
       84 + * The IO scheduler uses both the dirty space limit and current amount of
       85 + * dirty data as inputs. Those values affect the number of concurrent IOs ZFS
       86 + * issues. See the comment in vdev_queue.c for details of the IO scheduler.
       87 + *
       88 + * The delay is also calculated based on the amount of dirty data.  See the
       89 + * comment above dmu_tx_delay() for details.
       90 + */
  52   91  
  53      -uint64_t zfs_write_limit_min = 32 << 20;        /* min write limit is 32MB */
  54      -uint64_t zfs_write_limit_max = 0;               /* max data payload per txg */
  55      -uint64_t zfs_write_limit_inflated = 0;
  56      -uint64_t zfs_write_limit_override = 0;
       92 +/*
       93 + * zfs_dirty_data_max will be set to zfs_dirty_data_max_percent% of all memory,
       94 + * capped at zfs_dirty_data_max_max.  It can also be overridden in /etc/system.
       95 + */
       96 +uint64_t zfs_dirty_data_max;
       97 +uint64_t zfs_dirty_data_max_max = 4ULL * 1024 * 1024 * 1024;
       98 +int zfs_dirty_data_max_percent = 10;
  57   99  
  58      -kmutex_t zfs_write_limit_lock;
      100 +/*
      101 + * If there is at least this much dirty data, push out a txg.
      102 + */
      103 +uint64_t zfs_dirty_data_sync = 64 * 1024 * 1024;
  59  104  
  60      -static pgcnt_t old_physmem = 0;
      105 +/*
      106 + * Once there is this amount of dirty data, the dmu_tx_delay() will kick in
      107 + * and delay each transaction.
      108 + * This value should be >= zfs_vdev_async_write_active_max_dirty_percent.
      109 + */
      110 +int zfs_delay_min_dirty_percent = 60;
  61  111  
      112 +/*
      113 + * This controls how quickly the delay approaches infinity.
      114 + * Larger values cause it to delay less for a given amount of dirty data.
      115 + * Therefore larger values will cause there to be more dirty data for a
      116 + * given throughput.
      117 + *
      118 + * For the smoothest delay, this value should be about 1 billion divided
      119 + * by the maximum number of operations per second.  This will smoothly
      120 + * handle between 10x and 1/10th this number.
      121 + *
      122 + * Note: zfs_delay_scale * zfs_dirty_data_max must be < 2^64, due to the
      123 + * multiply in dmu_tx_delay().
      124 + */
      125 +uint64_t zfs_delay_scale = 1000 * 1000 * 1000 / 2000;
      126 +
      127 +
      128 +/*
      129 + * XXX someday maybe turn these into #defines, and you have to tune it on a
      130 + * per-pool basis using zfs.conf.
      131 + */
      132 +
      133 +
  62  134  hrtime_t zfs_throttle_delay = MSEC2NSEC(10);
  63  135  hrtime_t zfs_throttle_resolution = MSEC2NSEC(10);
  64  136  
  65  137  int
  66  138  dsl_pool_open_special_dir(dsl_pool_t *dp, const char *name, dsl_dir_t **ddp)
  67  139  {
  68  140          uint64_t obj;
  69  141          int err;
  70  142  
  71  143          err = zap_lookup(dp->dp_meta_objset,
↓ open down ↓ 8 lines elided ↑ open up ↑
  80  152  static dsl_pool_t *
  81  153  dsl_pool_open_impl(spa_t *spa, uint64_t txg)
  82  154  {
  83  155          dsl_pool_t *dp;
  84  156          blkptr_t *bp = spa_get_rootblkptr(spa);
  85  157  
  86  158          dp = kmem_zalloc(sizeof (dsl_pool_t), KM_SLEEP);
  87  159          dp->dp_spa = spa;
  88  160          dp->dp_meta_rootbp = *bp;
  89  161          rrw_init(&dp->dp_config_rwlock, B_TRUE);
  90      -        dp->dp_write_limit = zfs_write_limit_min;
  91  162          txg_init(dp, txg);
  92  163  
  93  164          txg_list_create(&dp->dp_dirty_datasets,
  94  165              offsetof(dsl_dataset_t, ds_dirty_link));
  95  166          txg_list_create(&dp->dp_dirty_zilogs,
  96  167              offsetof(zilog_t, zl_dirty_link));
  97  168          txg_list_create(&dp->dp_dirty_dirs,
  98  169              offsetof(dsl_dir_t, dd_dirty_link));
  99  170          txg_list_create(&dp->dp_sync_tasks,
 100  171              offsetof(dsl_sync_task_t, dst_node));
 101  172  
 102  173          mutex_init(&dp->dp_lock, NULL, MUTEX_DEFAULT, NULL);
      174 +        cv_init(&dp->dp_spaceavail_cv, NULL, CV_DEFAULT, NULL);
 103  175  
 104  176          dp->dp_vnrele_taskq = taskq_create("zfs_vn_rele_taskq", 1, minclsyspri,
 105  177              1, 4, 0);
 106  178  
 107  179          return (dp);
 108  180  }
 109  181  
 110  182  int
 111  183  dsl_pool_init(spa_t *spa, uint64_t txg, dsl_pool_t **dpp)
 112  184  {
↓ open down ↓ 94 lines elided ↑ open up ↑
 207  279          err = dsl_scan_init(dp, dp->dp_tx.tx_open_txg);
 208  280  
 209  281  out:
 210  282          rrw_exit(&dp->dp_config_rwlock, FTAG);
 211  283          return (err);
 212  284  }
 213  285  
 214  286  void
 215  287  dsl_pool_close(dsl_pool_t *dp)
 216  288  {
 217      -        /* drop our references from dsl_pool_open() */
 218      -
 219  289          /*
      290 +         * Drop our references from dsl_pool_open().
      291 +         *
 220  292           * Since we held the origin_snap from "syncing" context (which
 221  293           * includes pool-opening context), it actually only got a "ref"
 222  294           * and not a hold, so just drop that here.
 223  295           */
 224  296          if (dp->dp_origin_snap)
 225  297                  dsl_dataset_rele(dp->dp_origin_snap, dp);
 226  298          if (dp->dp_mos_dir)
 227  299                  dsl_dir_rele(dp->dp_mos_dir, dp);
 228  300          if (dp->dp_free_dir)
 229  301                  dsl_dir_rele(dp->dp_free_dir, dp);
↓ open down ↓ 109 lines elided ↑ open up ↑
 339  411  }
 340  412  
 341  413  static int
 342  414  deadlist_enqueue_cb(void *arg, const blkptr_t *bp, dmu_tx_t *tx)
 343  415  {
 344  416          dsl_deadlist_t *dl = arg;
 345  417          dsl_deadlist_insert(dl, bp, tx);
 346  418          return (0);
 347  419  }
 348  420  
      421 +static void
      422 +dsl_pool_sync_mos(dsl_pool_t *dp, dmu_tx_t *tx)
      423 +{
      424 +        zio_t *zio = zio_root(dp->dp_spa, NULL, NULL, ZIO_FLAG_MUSTSUCCEED);
      425 +        dmu_objset_sync(dp->dp_meta_objset, zio, tx);
      426 +        VERIFY0(zio_wait(zio));
      427 +        dprintf_bp(&dp->dp_meta_rootbp, "meta objset rootbp is %s", "");
      428 +        spa_set_rootblkptr(dp->dp_spa, &dp->dp_meta_rootbp);
      429 +}
      430 +
      431 +static void
      432 +dsl_pool_dirty_delta(dsl_pool_t *dp, int64_t delta)
      433 +{
      434 +        ASSERT(MUTEX_HELD(&dp->dp_lock));
      435 +
      436 +        if (delta < 0)
      437 +                ASSERT3U(-delta, <=, dp->dp_dirty_total);
      438 +
      439 +        dp->dp_dirty_total += delta;
      440 +
      441 +        /*
      442 +         * Note: we signal even when increasing dp_dirty_total.
      443 +         * This ensures forward progress -- each thread wakes the next waiter.
      444 +         */
      445 +        if (dp->dp_dirty_total <= zfs_dirty_data_max)
      446 +                cv_signal(&dp->dp_spaceavail_cv);
      447 +}
      448 +
 349  449  void
 350  450  dsl_pool_sync(dsl_pool_t *dp, uint64_t txg)
 351  451  {
 352  452          zio_t *zio;
 353  453          dmu_tx_t *tx;
 354  454          dsl_dir_t *dd;
 355  455          dsl_dataset_t *ds;
 356  456          objset_t *mos = dp->dp_meta_objset;
 357      -        hrtime_t start, write_time;
 358      -        uint64_t data_written;
 359      -        int err;
 360  457          list_t synced_datasets;
 361  458  
 362  459          list_create(&synced_datasets, sizeof (dsl_dataset_t),
 363  460              offsetof(dsl_dataset_t, ds_synced_link));
 364  461  
 365      -        /*
 366      -         * We need to copy dp_space_towrite() before doing
 367      -         * dsl_sync_task_sync(), because
 368      -         * dsl_dataset_snapshot_reserve_space() will increase
 369      -         * dp_space_towrite but not actually write anything.
 370      -         */
 371      -        data_written = dp->dp_space_towrite[txg & TXG_MASK];
 372      -
 373  462          tx = dmu_tx_create_assigned(dp, txg);
 374  463  
 375      -        dp->dp_read_overhead = 0;
 376      -        start = gethrtime();
 377      -
      464 +        /*
      465 +         * Write out all dirty blocks of dirty datasets.
      466 +         */
 378  467          zio = zio_root(dp->dp_spa, NULL, NULL, ZIO_FLAG_MUSTSUCCEED);
 379      -        while (ds = txg_list_remove(&dp->dp_dirty_datasets, txg)) {
      468 +        while ((ds = txg_list_remove(&dp->dp_dirty_datasets, txg)) != NULL) {
 380  469                  /*
 381  470                   * We must not sync any non-MOS datasets twice, because
 382  471                   * we may have taken a snapshot of them.  However, we
 383  472                   * may sync newly-created datasets on pass 2.
 384  473                   */
 385  474                  ASSERT(!list_link_active(&ds->ds_synced_link));
 386  475                  list_insert_tail(&synced_datasets, ds);
 387  476                  dsl_dataset_sync(ds, zio, tx);
 388  477          }
 389      -        DTRACE_PROBE(pool_sync__1setup);
 390      -        err = zio_wait(zio);
      478 +        VERIFY0(zio_wait(zio));
 391  479  
 392      -        write_time = gethrtime() - start;
 393      -        ASSERT(err == 0);
 394      -        DTRACE_PROBE(pool_sync__2rootzio);
      480 +        /*
      481 +         * We have written all of the accounted dirty data, so our
      482 +         * dp_space_towrite should now be zero.  However, some seldom-used
      483 +         * code paths do not adhere to this (e.g. dbuf_undirty(), also
      484 +         * rounding error in dbuf_write_physdone).
      485 +         * Shore up the accounting of any dirtied space now.
      486 +         */
      487 +        dsl_pool_undirty_space(dp, dp->dp_dirty_pertxg[txg & TXG_MASK], txg);
 395  488  
 396  489          /*
 397  490           * After the data blocks have been written (ensured by the zio_wait()
 398  491           * above), update the user/group space accounting.
 399  492           */
 400      -        for (ds = list_head(&synced_datasets); ds;
 401      -            ds = list_next(&synced_datasets, ds))
      493 +        for (ds = list_head(&synced_datasets); ds != NULL;
      494 +            ds = list_next(&synced_datasets, ds)) {
 402  495                  dmu_objset_do_userquota_updates(ds->ds_objset, tx);
      496 +        }
 403  497  
 404  498          /*
 405  499           * Sync the datasets again to push out the changes due to
 406  500           * userspace updates.  This must be done before we process the
 407  501           * sync tasks, so that any snapshots will have the correct
 408  502           * user accounting information (and we won't get confused
 409  503           * about which blocks are part of the snapshot).
 410  504           */
 411  505          zio = zio_root(dp->dp_spa, NULL, NULL, ZIO_FLAG_MUSTSUCCEED);
 412      -        while (ds = txg_list_remove(&dp->dp_dirty_datasets, txg)) {
      506 +        while ((ds = txg_list_remove(&dp->dp_dirty_datasets, txg)) != NULL) {
 413  507                  ASSERT(list_link_active(&ds->ds_synced_link));
 414  508                  dmu_buf_rele(ds->ds_dbuf, ds);
 415  509                  dsl_dataset_sync(ds, zio, tx);
 416  510          }
 417      -        err = zio_wait(zio);
      511 +        VERIFY0(zio_wait(zio));
 418  512  
 419  513          /*
 420  514           * Now that the datasets have been completely synced, we can
 421  515           * clean up our in-memory structures accumulated while syncing:
 422  516           *
 423  517           *  - move dead blocks from the pending deadlist to the on-disk deadlist
 424  518           *  - release hold from dsl_dataset_dirty()
 425  519           */
 426      -        while (ds = list_remove_head(&synced_datasets)) {
      520 +        while ((ds = list_remove_head(&synced_datasets)) != NULL) {
 427  521                  objset_t *os = ds->ds_objset;
 428  522                  bplist_iterate(&ds->ds_pending_deadlist,
 429  523                      deadlist_enqueue_cb, &ds->ds_deadlist, tx);
 430  524                  ASSERT(!dmu_objset_is_dirty(os, txg));
 431  525                  dmu_buf_rele(ds->ds_dbuf, ds);
 432  526          }
 433      -
 434      -        start = gethrtime();
 435      -        while (dd = txg_list_remove(&dp->dp_dirty_dirs, txg))
      527 +        while ((dd = txg_list_remove(&dp->dp_dirty_dirs, txg)) != NULL) {
 436  528                  dsl_dir_sync(dd, tx);
 437      -        write_time += gethrtime() - start;
      529 +        }
 438  530  
 439  531          /*
 440  532           * The MOS's space is accounted for in the pool/$MOS
 441  533           * (dp_mos_dir).  We can't modify the mos while we're syncing
 442  534           * it, so we remember the deltas and apply them here.
 443  535           */
 444  536          if (dp->dp_mos_used_delta != 0 || dp->dp_mos_compressed_delta != 0 ||
 445  537              dp->dp_mos_uncompressed_delta != 0) {
 446  538                  dsl_dir_diduse_space(dp->dp_mos_dir, DD_USED_HEAD,
 447  539                      dp->dp_mos_used_delta,
 448  540                      dp->dp_mos_compressed_delta,
 449  541                      dp->dp_mos_uncompressed_delta, tx);
 450  542                  dp->dp_mos_used_delta = 0;
 451  543                  dp->dp_mos_compressed_delta = 0;
 452  544                  dp->dp_mos_uncompressed_delta = 0;
 453  545          }
 454  546  
 455      -        start = gethrtime();
 456  547          if (list_head(&mos->os_dirty_dnodes[txg & TXG_MASK]) != NULL ||
 457  548              list_head(&mos->os_free_dnodes[txg & TXG_MASK]) != NULL) {
 458      -                zio = zio_root(dp->dp_spa, NULL, NULL, ZIO_FLAG_MUSTSUCCEED);
 459      -                dmu_objset_sync(mos, zio, tx);
 460      -                err = zio_wait(zio);
 461      -                ASSERT(err == 0);
 462      -                dprintf_bp(&dp->dp_meta_rootbp, "meta objset rootbp is %s", "");
 463      -                spa_set_rootblkptr(dp->dp_spa, &dp->dp_meta_rootbp);
      549 +                dsl_pool_sync_mos(dp, tx);
 464  550          }
 465      -        write_time += gethrtime() - start;
 466      -        DTRACE_PROBE2(pool_sync__4io, hrtime_t, write_time,
 467      -            hrtime_t, dp->dp_read_overhead);
 468      -        write_time -= dp->dp_read_overhead;
 469  551  
 470  552          /*
 471  553           * If we modify a dataset in the same txg that we want to destroy it,
 472  554           * its dsl_dir's dd_dbuf will be dirty, and thus have a hold on it.
 473  555           * dsl_dir_destroy_check() will fail if there are unexpected holds.
 474  556           * Therefore, we want to sync the MOS (thus syncing the dd_dbuf
 475  557           * and clearing the hold on it) before we process the sync_tasks.
 476  558           * The MOS data dirtied by the sync_tasks will be synced on the next
 477  559           * pass.
 478  560           */
 479      -        DTRACE_PROBE(pool_sync__3task);
 480  561          if (!txg_list_empty(&dp->dp_sync_tasks, txg)) {
 481  562                  dsl_sync_task_t *dst;
 482  563                  /*
 483  564                   * No more sync tasks should have been added while we
 484  565                   * were syncing.
 485  566                   */
 486      -                ASSERT(spa_sync_pass(dp->dp_spa) == 1);
 487      -                while (dst = txg_list_remove(&dp->dp_sync_tasks, txg))
      567 +                ASSERT3U(spa_sync_pass(dp->dp_spa), ==, 1);
      568 +                while ((dst = txg_list_remove(&dp->dp_sync_tasks, txg)) != NULL)
 488  569                          dsl_sync_task_sync(dst, tx);
 489  570          }
 490  571  
 491  572          dmu_tx_commit(tx);
 492  573  
 493      -        dp->dp_space_towrite[txg & TXG_MASK] = 0;
 494      -        ASSERT(dp->dp_tempreserved[txg & TXG_MASK] == 0);
 495      -
 496      -        /*
 497      -         * If the write limit max has not been explicitly set, set it
 498      -         * to a fraction of available physical memory (default 1/8th).
 499      -         * Note that we must inflate the limit because the spa
 500      -         * inflates write sizes to account for data replication.
 501      -         * Check this each sync phase to catch changing memory size.
 502      -         */
 503      -        if (physmem != old_physmem && zfs_write_limit_shift) {
 504      -                mutex_enter(&zfs_write_limit_lock);
 505      -                old_physmem = physmem;
 506      -                zfs_write_limit_max = ptob(physmem) >> zfs_write_limit_shift;
 507      -                zfs_write_limit_inflated = MAX(zfs_write_limit_min,
 508      -                    spa_get_asize(dp->dp_spa, zfs_write_limit_max));
 509      -                mutex_exit(&zfs_write_limit_lock);
 510      -        }
 511      -
 512      -        /*
 513      -         * Attempt to keep the sync time consistent by adjusting the
 514      -         * amount of write traffic allowed into each transaction group.
 515      -         * Weight the throughput calculation towards the current value:
 516      -         *      thru = 3/4 old_thru + 1/4 new_thru
 517      -         *
 518      -         * Note: write_time is in nanosecs while dp_throughput is expressed in
 519      -         * bytes per millisecond.
 520      -         */
 521      -        ASSERT(zfs_write_limit_min > 0);
 522      -        if (data_written > zfs_write_limit_min / 8 &&
 523      -            write_time > MSEC2NSEC(1)) {
 524      -                uint64_t throughput = data_written / NSEC2MSEC(write_time);
 525      -
 526      -                if (dp->dp_throughput)
 527      -                        dp->dp_throughput = throughput / 4 +
 528      -                            3 * dp->dp_throughput / 4;
 529      -                else
 530      -                        dp->dp_throughput = throughput;
 531      -                dp->dp_write_limit = MIN(zfs_write_limit_inflated,
 532      -                    MAX(zfs_write_limit_min,
 533      -                    dp->dp_throughput * zfs_txg_synctime_ms));
 534      -        }
      574 +        DTRACE_PROBE2(dsl_pool_sync__done, dsl_pool_t *dp, dp, uint64_t, txg);
 535  575  }
 536  576  
 537  577  void
 538  578  dsl_pool_sync_done(dsl_pool_t *dp, uint64_t txg)
 539  579  {
 540  580          zilog_t *zilog;
 541      -        dsl_dataset_t *ds;
 542  581  
 543  582          while (zilog = txg_list_remove(&dp->dp_dirty_zilogs, txg)) {
 544      -                ds = dmu_objset_ds(zilog->zl_os);
      583 +                dsl_dataset_t *ds = dmu_objset_ds(zilog->zl_os);
 545  584                  zil_clean(zilog, txg);
 546  585                  ASSERT(!dmu_objset_is_dirty(zilog->zl_os, txg));
 547  586                  dmu_buf_rele(ds->ds_dbuf, zilog);
 548  587          }
 549  588          ASSERT(!dmu_objset_is_dirty(dp->dp_meta_objset, txg));
 550  589  }
 551  590  
 552  591  /*
 553  592   * TRUE if the current thread is the tx_sync_thread or if we
 554  593   * are being called from SPA context during pool initialization.
↓ open down ↓ 21 lines elided ↑ open up ↑
 576  615           * (e.g. make it possible to rm(1) files from a full pool).
 577  616           */
 578  617          space = spa_get_dspace(dp->dp_spa);
 579  618          resv = MAX(space >> 6, SPA_MINDEVSIZE >> 1);
 580  619          if (netfree)
 581  620                  resv >>= 1;
 582  621  
 583  622          return (space - resv);
 584  623  }
 585  624  
 586      -int
 587      -dsl_pool_tempreserve_space(dsl_pool_t *dp, uint64_t space, dmu_tx_t *tx)
      625 +boolean_t
      626 +dsl_pool_need_dirty_delay(dsl_pool_t *dp)
 588  627  {
 589      -        uint64_t reserved = 0;
 590      -        uint64_t write_limit = (zfs_write_limit_override ?
 591      -            zfs_write_limit_override : dp->dp_write_limit);
      628 +        uint64_t delay_min_bytes =
      629 +            zfs_dirty_data_max * zfs_delay_min_dirty_percent / 100;
      630 +        boolean_t rv;
 592  631  
 593      -        if (zfs_no_write_throttle) {
 594      -                atomic_add_64(&dp->dp_tempreserved[tx->tx_txg & TXG_MASK],
 595      -                    space);
 596      -                return (0);
 597      -        }
 598      -
 599      -        /*
 600      -         * Check to see if we have exceeded the maximum allowed IO for
 601      -         * this transaction group.  We can do this without locks since
 602      -         * a little slop here is ok.  Note that we do the reserved check
 603      -         * with only half the requested reserve: this is because the
 604      -         * reserve requests are worst-case, and we really don't want to
 605      -         * throttle based off of worst-case estimates.
 606      -         */
 607      -        if (write_limit > 0) {
 608      -                reserved = dp->dp_space_towrite[tx->tx_txg & TXG_MASK]
 609      -                    + dp->dp_tempreserved[tx->tx_txg & TXG_MASK] / 2;
 610      -
 611      -                if (reserved && reserved > write_limit)
 612      -                        return (SET_ERROR(ERESTART));
 613      -        }
 614      -
 615      -        atomic_add_64(&dp->dp_tempreserved[tx->tx_txg & TXG_MASK], space);
 616      -
 617      -        /*
 618      -         * If this transaction group is over 7/8ths capacity, delay
 619      -         * the caller 1 clock tick.  This will slow down the "fill"
 620      -         * rate until the sync process can catch up with us.
 621      -         */
 622      -        if (reserved && reserved > (write_limit - (write_limit >> 3))) {
 623      -                txg_delay(dp, tx->tx_txg, zfs_throttle_delay,
 624      -                    zfs_throttle_resolution);
 625      -        }
 626      -
 627      -        return (0);
      632 +        mutex_enter(&dp->dp_lock);
      633 +        if (dp->dp_dirty_total > zfs_dirty_data_sync)
      634 +                txg_kick(dp);
      635 +        rv = (dp->dp_dirty_total > delay_min_bytes);
      636 +        mutex_exit(&dp->dp_lock);
      637 +        return (rv);
 628  638  }
 629  639  
 630  640  void
 631      -dsl_pool_tempreserve_clear(dsl_pool_t *dp, int64_t space, dmu_tx_t *tx)
      641 +dsl_pool_dirty_space(dsl_pool_t *dp, int64_t space, dmu_tx_t *tx)
 632  642  {
 633      -        ASSERT(dp->dp_tempreserved[tx->tx_txg & TXG_MASK] >= space);
 634      -        atomic_add_64(&dp->dp_tempreserved[tx->tx_txg & TXG_MASK], -space);
      643 +        if (space > 0) {
      644 +                mutex_enter(&dp->dp_lock);
      645 +                dp->dp_dirty_pertxg[tx->tx_txg & TXG_MASK] += space;
      646 +                dsl_pool_dirty_delta(dp, space);
      647 +                mutex_exit(&dp->dp_lock);
      648 +        }
 635  649  }
 636  650  
 637  651  void
 638      -dsl_pool_memory_pressure(dsl_pool_t *dp)
 639      -{
 640      -        uint64_t space_inuse = 0;
 641      -        int i;
 642      -
 643      -        if (dp->dp_write_limit == zfs_write_limit_min)
      652 +dsl_pool_undirty_space(dsl_pool_t *dp, int64_t space, uint64_t txg) {
      653 +        ASSERT3S(space, >=, 0);
      654 +        if (space == 0)
 644  655                  return;
 645      -
 646      -        for (i = 0; i < TXG_SIZE; i++) {
 647      -                space_inuse += dp->dp_space_towrite[i];
 648      -                space_inuse += dp->dp_tempreserved[i];
      656 +        mutex_enter(&dp->dp_lock);
      657 +        if (dp->dp_dirty_pertxg[txg & TXG_MASK] < space) {
      658 +                /* XXX writing something we didn't dirty? */
      659 +                space = dp->dp_dirty_pertxg[txg & TXG_MASK];
 649  660          }
 650      -        dp->dp_write_limit = MAX(zfs_write_limit_min,
 651      -            MIN(dp->dp_write_limit, space_inuse / 4));
      661 +        ASSERT3U(dp->dp_dirty_pertxg[txg & TXG_MASK], >=, space);
      662 +        dp->dp_dirty_pertxg[txg & TXG_MASK] -= space;
      663 +        ASSERT3U(dp->dp_dirty_total, >=, space);
      664 +        dsl_pool_dirty_delta(dp, -space);
      665 +        mutex_exit(&dp->dp_lock);
 652  666  }
 653  667  
 654      -void
 655      -dsl_pool_willuse_space(dsl_pool_t *dp, int64_t space, dmu_tx_t *tx)
 656      -{
 657      -        if (space > 0) {
 658      -                mutex_enter(&dp->dp_lock);
 659      -                dp->dp_space_towrite[tx->tx_txg & TXG_MASK] += space;
 660      -                mutex_exit(&dp->dp_lock);
 661      -        }
 662      -}
 663      -
 664  668  /* ARGSUSED */
 665  669  static int
 666  670  upgrade_clones_cb(dsl_pool_t *dp, dsl_dataset_t *hds, void *arg)
 667  671  {
 668  672          dmu_tx_t *tx = arg;
 669  673          dsl_dataset_t *ds, *prev = NULL;
 670  674          int err;
 671  675  
 672  676          err = dsl_dataset_hold_obj(dp, hds->ds_object, FTAG, &ds);
 673  677          if (err)
↓ open down ↓ 370 lines elided ↑ open up ↑
XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX