Print this page
NEX-13135 Running BDD tests exposes a panic in ZFS TRIM due to a trimset overlap
Reviewed by: Reviewed by: Sanjay Nadkarni <sanjay.nadkarni@nexenta.com>
Reviewed by: Roman Strashkin <roman.strashkin@nexenta.com>
NEX-9554 dsl_scan.c internals contain some confusingly similar function names for handling the dataset and block sorting queues
Reviewed by: Yuri Pankov <yuri.pankov@nexenta.com>
Reviewed by: Roman Strashkin <roman.strashkin@nexenta.com>
NEX-6088 ZFS scrub/resilver take excessively long due to issuing lots of random IO
Reviewed by: Roman Strashkin <roman.strashkin@nexenta.com>
Reviewed by: Sanjay Nadkarni <sanjay.nadkarni@nexenta.com>
NEX-7298 powertop dumps core when -c, -d or -t flags are used
Reviewed by: Gordon Ross <gordon.ross@nexenta.com>
NEX-6725 zpool offlining/onlining first disk in a mirror causes checksum error
Reviewed by: Alek Pinchuk <alek.pinchuk@nexenta.com>
Reviewed by: Roman Strashkin <roman.strashkin@nexenta.com>
Reviewed by: Sanjay Nadkarni <sanjay.nadkarni@nexenta.com>
NEX-5553 ZFS auto-trim, manual-trim and scrub can race and deadlock
Reviewed by: Alek Pinchuk <alek.pinchuk@nexenta.com>
Reviewed by: Rob Gittins <rob.gittins@nexenta.com>
Reviewed by: Sanjay Nadkarni <sanjay.nadkarni@nexenta.com>
NEX-2846 Enable Automatic/Intelligent Hot Sparing capability
Reviewed by: Jeffry Molanus <jeffry.molanus@nexenta.com>
Reviewed by: Roman Strashkin <roman.strashkin@nexenta.com>
Reviewed by: Saso Kiselkov <saso.kiselkov@nexenta.com>
NEX-4940 Special Vdev operation in presence (or absense) of IO Errors
Reviewed by: Alek Pinchuk <alek.pinchuk@nexenta.com>
Reviewed by: Alex Aizman <alex.aizman@nexenta.com>
NEX-4807 writecache load-balancing statistics: several distinct problems, must be revisited and revised
Reviewed by: Roman Strashkin <roman.strashkin@nexenta.com>
Reviewed by: Alek Pinchuk <alek.pinchuk@nexenta.com>
NEX-4620 ZFS autotrim triggering is unreliable
NEX-4622 On-demand TRIM code illogically enumerates metaslabs via mg_ms_tree
Reviewed by: Josef 'Jeff' Sipek <josef.sipek@nexenta.com>
Reviewed by: Hans Rosenfeld <hans.rosenfeld@nexenta.com>
5818 zfs {ref}compressratio is incorrect with 4k sector size
Reviewed by: Alex Reece <alex@delphix.com>
Reviewed by: George Wilson <george@delphix.com>
Reviewed by: Richard Elling <richard.elling@richardelling.com>
Reviewed by: Steven Hartland <killing@multiplay.co.uk>
Reviewed by: Don Brady <dev.fs.zfs@gmail.com>
Approved by: Albert Lee <trisk@omniti.com>
NEX-3984 On-demand TRIM
Reviewed by: Alek Pinchuk <alek@nexenta.com>
Reviewed by: Josef 'Jeff' Sipek <josef.sipek@nexenta.com>
Conflicts:
        usr/src/common/zfs/zpool_prop.c
        usr/src/uts/common/sys/fs/zfs.h
NEX-3558 KRRP Integration
NEX-3508 CLONE - Port NEX-2946 Add UNMAP/TRIM functionality to ZFS and illumos
Reviewed by: Josef Sipek <josef.sipek@nexenta.com>
Reviewed by: Alek Pinchuk <alek.pinchuk@nexenta.com>
Conflicts:
    usr/src/uts/common/io/scsi/targets/sd.c
    usr/src/uts/common/sys/scsi/targets/sddef.h
NEX-3212 remove vdev prop object type from dmu.h, p2 Reviewed by: Josef 'Jeff' Sipek <josef.sipek@nexenta.com> Reviewed by: Saso Kiselkov <saso.kiselkov@nexenta.com>
NEX-3165 need some dedup improvements
Reviewed by: Josef 'Jeff' Sipek <josef.sipek@nexenta.com>
Reviewed by: Saso Kiselkov <saso.kiselkov@nexenta.com>
NEX-1142 move rwlock to vdev to protect vdev_tsd
not just ldi handle.
This way we serialize open/close, yet allow parallel I/O.
OS-80 support for vdev and CoS properties for the new I/O scheduler
OS-95 lint warning introduced by OS-61
Issue #40: ZDB shouldn't crash with new code
zfsxx issue #11: support for spare device groups
Fixup merge results
re #12585 rb4049 ZFS++ work port - refactoring to improve separation of open/closed code, bug fixes, performance improvements - open code
Bug 11205: add missing libzfs_closed_stubs.c to fix opensource-only build.
ZFS plus work: special vdevs, cos, cos/vdev properties

@@ -19,27 +19,26 @@
  * CDDL HEADER END
  */
 /*
  * Copyright (c) 2005, 2010, Oracle and/or its affiliates. All rights reserved.
  * Copyright (c) 2011, 2015 by Delphix. All rights reserved.
+ * Copyright 2016 Nexenta Systems, Inc. All rights reserved.
  */
 
 #ifndef _SYS_VDEV_IMPL_H
 #define _SYS_VDEV_IMPL_H
 
 #include <sys/avl.h>
-#include <sys/bpobj.h>
 #include <sys/dmu.h>
 #include <sys/metaslab.h>
 #include <sys/nvpair.h>
 #include <sys/space_map.h>
 #include <sys/vdev.h>
 #include <sys/dkio.h>
 #include <sys/uberblock_impl.h>
-#include <sys/vdev_indirect_mapping.h>
-#include <sys/vdev_indirect_births.h>
-#include <sys/vdev_removal.h>
+#include <sys/fs/zfs.h>
+#include <sys/cos.h>
 
 #ifdef  __cplusplus
 extern "C" {
 #endif
 

@@ -71,26 +70,22 @@
 typedef void    vdev_io_start_func_t(zio_t *zio);
 typedef void    vdev_io_done_func_t(zio_t *zio);
 typedef void    vdev_state_change_func_t(vdev_t *vd, int, int);
 typedef void    vdev_hold_func_t(vdev_t *vd);
 typedef void    vdev_rele_func_t(vdev_t *vd);
+typedef void    vdev_trim_func_t(vdev_t *vd, zio_t *pio, void *trim_exts);
 
-typedef void    vdev_remap_cb_t(uint64_t inner_offset, vdev_t *vd,
-    uint64_t offset, uint64_t size, void *arg);
-typedef void    vdev_remap_func_t(vdev_t *vd, uint64_t offset, uint64_t size,
-    vdev_remap_cb_t callback, void *arg);
-
 typedef struct vdev_ops {
         vdev_open_func_t                *vdev_op_open;
         vdev_close_func_t               *vdev_op_close;
         vdev_asize_func_t               *vdev_op_asize;
         vdev_io_start_func_t            *vdev_op_io_start;
         vdev_io_done_func_t             *vdev_op_io_done;
         vdev_state_change_func_t        *vdev_op_state_change;
         vdev_hold_func_t                *vdev_op_hold;
         vdev_rele_func_t                *vdev_op_rele;
-        vdev_remap_func_t               *vdev_op_remap;
+        vdev_trim_func_t                *vdev_op_trim;
         char                            vdev_op_type[16];
         boolean_t                       vdev_op_leaf;
 } vdev_ops_t;
 
 /*

@@ -111,70 +106,79 @@
         avl_tree_t      vc_offset_tree;
         avl_tree_t      vc_lastused_tree;
         kmutex_t        vc_lock;
 };
 
+/*
+ * Macros for conversion between zio priorities and vdev properties.
+ * These rely on the specific corresponding order of the zio_priority_t
+ * and vdev_prop_t enum definitions to simplify the conversion.
+ */
+#define VDEV_PROP_TO_ZIO_PRIO_MIN(prp)  ((prp) - VDEV_PROP_READ_MINACTIVE)
+#define VDEV_ZIO_PRIO_TO_PROP_MIN(pri)  ((pri) + VDEV_PROP_READ_MINACTIVE)
+#define VDEV_PROP_MIN_VALID(prp)                \
+        (((prp) >= VDEV_PROP_READ_MINACTIVE) && \
+        ((prp) <= VDEV_PROP_SCRUB_MINACTIVE))
+#define VDEV_PROP_TO_ZIO_PRIO_MAX(prp)  ((prp) - VDEV_PROP_READ_MAXACTIVE)
+#define VDEV_ZIO_PRIO_TO_PROP_MAX(pri)  ((pri) + VDEV_PROP_READ_MAXACTIVE)
+#define VDEV_PROP_MAX_VALID(prp)                \
+        (((prp) >= VDEV_PROP_READ_MAXACTIVE) && \
+        ((prp) <= VDEV_PROP_SCRUB_MAXACTIVE))
+
 typedef struct vdev_queue_class {
         uint32_t        vqc_active;
 
         /*
+         * If min/max active values are zero, we fall back on the global
+         * corresponding tunables defined in vdev_queue.c; non-zero values
+         * override the global tunables
+         */
+        uint32_t        vqc_min_active; /* min concurently active IOs */
+        uint32_t        vqc_max_active; /* max concurently active IOs */
+
+        /*
          * Sorted by offset or timestamp, depending on if the queue is
          * LBA-ordered vs FIFO.
          */
         avl_tree_t      vqc_queued_tree;
 } vdev_queue_class_t;
 
 struct vdev_queue {
         vdev_t          *vq_vdev;
+
         vdev_queue_class_t vq_class[ZIO_PRIORITY_NUM_QUEUEABLE];
+        cos_t           *vq_cos;                /* assigned class of storage */
+        uint64_t        vq_preferred_read;      /* property setting */
+
         avl_tree_t      vq_active_tree;
         avl_tree_t      vq_read_offset_tree;
         avl_tree_t      vq_write_offset_tree;
         uint64_t        vq_last_offset;
         hrtime_t        vq_io_complete_ts; /* time last i/o completed */
         kmutex_t        vq_lock;
 };
 
 /*
- * On-disk indirect vdev state.
- *
- * An indirect vdev is described exclusively in the MOS config of a pool.
- * The config for an indirect vdev includes several fields, which are
- * accessed in memory by a vdev_indirect_config_t.
+ * vdev auxiliary kstat I/O statistics:
+ * updates every spa_special_stat_update_ticks interval
+ * it is used for adjust special vs normal data routing
  */
-typedef struct vdev_indirect_config {
-        /*
-         * Object (in MOS) which contains the indirect mapping. This object
-         * contains an array of vdev_indirect_mapping_entry_phys_t ordered by
-         * vimep_src. The bonus buffer for this object is a
-         * vdev_indirect_mapping_phys_t. This object is allocated when a vdev
-         * removal is initiated.
-         *
-         * Note that this object can be empty if none of the data on the vdev
-         * has been copied yet.
-         */
-        uint64_t        vic_mapping_object;
+typedef struct vdev_aux_stat {
+        uint64_t nread;         /* number of bytes read */
+        uint64_t nwritten;      /* number of bytes written */
+        uint64_t reads;         /* number of read operations */
+        uint64_t writes;        /* number of write operations */
+        uint64_t rtime;         /* cumulative run (service) time */
+        uint64_t wtime;         /* cumulative wait (pre-service) time */
+        uint64_t rlentime;      /* cumulative run length*time product */
+        uint64_t wlentime;      /* cumulative wait length*time product */
+        uint64_t rlastupdate;   /* last time run queue changed */
+        uint64_t wlastupdate;   /* last time wait queue changed */
+        uint64_t rcnt;          /* count of elements in run state */
+        uint64_t wcnt;          /* count of elements in wait state */
+} vdev_aux_stat_t;
 
-        /*
-         * Object (in MOS) which contains the birth times for the mapping
-         * entries. This object contains an array of
-         * vdev_indirect_birth_entry_phys_t sorted by vibe_offset. The bonus
-         * buffer for this object is a vdev_indirect_birth_phys_t. This object
-         * is allocated when a vdev removal is initiated.
-         *
-         * Note that this object can be empty if none of the vdev has yet been
-         * copied.
-         */
-        uint64_t        vic_births_object;
-
-        /*
-         * This is the vdev ID which was removed previous to this vdev, or
-         * UINT64_MAX if there are no previously removed vdevs.
-         */
-        uint64_t        vic_prev_indirect_vdev;
-} vdev_indirect_config_t;
-
 /*
  * Virtual device descriptor
  */
 struct vdev {
         /*

@@ -223,48 +227,26 @@
         list_node_t     vdev_state_dirty_node; /* state dirty list      */
         uint64_t        vdev_deflate_ratio; /* deflation ratio (x512)   */
         uint64_t        vdev_islog;     /* is an intent log device      */
         uint64_t        vdev_removing;  /* device is being removed?     */
         boolean_t       vdev_ishole;    /* is a hole in the namespace   */
-        kmutex_t        vdev_queue_lock; /* protects vdev_queue_depth   */
         uint64_t        vdev_top_zap;
+        uint64_t        vdev_isspecial; /* is a special device  */
 
-        /*
-         * Values stored in the config for an indirect or removing vdev.
-         */
-        vdev_indirect_config_t  vdev_indirect_config;
+        boolean_t       vdev_man_trimming; /* manual trim is ongoing    */
+        uint64_t        vdev_trim_prog; /* trim progress in bytes       */
 
         /*
-         * The vdev_indirect_rwlock protects the vdev_indirect_mapping
-         * pointer from changing on indirect vdevs (when it is condensed).
-         * Note that removing (not yet indirect) vdevs have different
-         * access patterns (the mapping is not accessed from open context,
-         * e.g. from zio_read) and locking strategy (e.g. svr_lock).
+         * Protects the vdev_scan_io_queue field itself as well as the
+         * structure's contents (when present). The scn_status_lock in
+         * dsl_scan_t must only ever be acquired stand-alone or inside
+         * of vdev_scan_queue_lock, never in reverse.
          */
-        krwlock_t vdev_indirect_rwlock;
-        vdev_indirect_mapping_t *vdev_indirect_mapping;
-        vdev_indirect_births_t *vdev_indirect_births;
+        kmutex_t                        vdev_scan_io_queue_lock;
+        struct dsl_scan_io_queue        *vdev_scan_io_queue;
 
         /*
-         * In memory data structures used to manage the obsolete sm, for
-         * indirect or removing vdevs.
-         *
-         * The vdev_obsolete_segments is the in-core record of the segments
-         * that are no longer referenced anywhere in the pool (due to
-         * being freed or remapped and not referenced by any snapshots).
-         * During a sync, segments are added to vdev_obsolete_segments
-         * via vdev_indirect_mark_obsolete(); at the end of each sync
-         * pass, this is appended to vdev_obsolete_sm via
-         * vdev_indirect_sync_obsolete().  The vdev_obsolete_lock
-         * protects against concurrent modifications of vdev_obsolete_segments
-         * from multiple zio threads.
-         */
-        kmutex_t        vdev_obsolete_lock;
-        range_tree_t    *vdev_obsolete_segments;
-        space_map_t     *vdev_obsolete_sm;
-
-        /*
          * The queue depth parameters determine how many async writes are
          * still pending (i.e. allocated by net yet issued to disk) per
          * top-level (vdev_async_write_queue_depth) and the maximum allowed
          * (vdev_max_async_write_queue_depth). These values only apply to
          * top-level vdevs.

@@ -285,17 +267,20 @@
         uint64_t        vdev_faulted;   /* persistent faulted state     */
         uint64_t        vdev_degraded;  /* persistent degraded state    */
         uint64_t        vdev_removed;   /* persistent removed state     */
         uint64_t        vdev_resilver_txg; /* persistent resilvering state */
         uint64_t        vdev_nparity;   /* number of parity devices for raidz */
+        uint64_t        vdev_l2ad_ddt;  /* L2ARC vdev is used to cache DDT */
         char            *vdev_path;     /* vdev path (if any)           */
         char            *vdev_devid;    /* vdev devid (if any)          */
         char            *vdev_physpath; /* vdev device path (if any)    */
         char            *vdev_fru;      /* physical FRU location        */
+        int64_t         vdev_weight;    /* dynamic weight */
         uint64_t        vdev_not_present; /* not present during import  */
         uint64_t        vdev_unspare;   /* unspare when resilvering done */
         boolean_t       vdev_nowritecache; /* true if flushwritecache failed */
+        boolean_t       vdev_notrim;    /* true if Unmap/TRIM is unsupported */
         boolean_t       vdev_checkremove; /* temporary online test      */
         boolean_t       vdev_forcefault; /* force online fault          */
         boolean_t       vdev_splitting; /* split or repair in progress  */
         boolean_t       vdev_delayed_close; /* delayed device close?    */
         boolean_t       vdev_tmpoffline; /* device taken offline temporarily? */

@@ -302,17 +287,23 @@
         boolean_t       vdev_detached;  /* device detached?             */
         boolean_t       vdev_cant_read; /* vdev is failing all reads    */
         boolean_t       vdev_cant_write; /* vdev is failing all writes  */
         boolean_t       vdev_isspare;   /* was a hot spare              */
         boolean_t       vdev_isl2cache; /* was a l2cache device         */
+        boolean_t       vdev_isspecial_child; /* a child of top-level special */
         vdev_queue_t    vdev_queue;     /* I/O deadline schedule queue  */
         vdev_cache_t    vdev_cache;     /* physical block cache         */
         spa_aux_vdev_t  *vdev_aux;      /* for l2cache and spares vdevs */
         zio_t           *vdev_probe_zio; /* root of current probe       */
         vdev_aux_t      vdev_label_aux; /* on-disk aux state            */
         uint64_t        vdev_leaf_zap;
+        boolean_t       vdev_is_ssd;    /* is solid state device        */
 
+        char            *vdev_spare_group; /* spare group name */
+
+        struct kstat    *vdev_iokstat; /* vdev kstat I/O statistics */
+        vdev_aux_stat_t vdev_aux_stat; /* auxiliary vdev kstat I/O statistics */
         /*
          * For DTrace to work in userland (libzpool) context, these fields must
          * remain at the end of the structure.  DTrace will use the kernel's
          * CTF definition for 'struct vdev', and since the size of a kmutex_t is
          * larger in userland, the offsets for the rest of the fields would be

@@ -319,10 +310,11 @@
          * incorrect.
          */
         kmutex_t        vdev_dtl_lock;  /* vdev_dtl_{map,resilver}      */
         kmutex_t        vdev_stat_lock; /* vdev_stat                    */
         kmutex_t        vdev_probe_lock; /* protects vdev_probe_zio     */
+        krwlock_t       vdev_tsd_lock;  /* protects vdev_tsd */
 };
 
 #define VDEV_RAIDZ_MAXPARITY    3
 
 #define VDEV_PAD_SIZE           (8 << 10)

@@ -403,12 +395,13 @@
 extern void vdev_remove_parent(vdev_t *cvd);
 
 /*
  * vdev sync load and sync
  */
+extern void vdev_load_log_state(vdev_t *nvd, vdev_t *ovd);
 extern boolean_t vdev_log_state_valid(vdev_t *vd);
-extern int vdev_load(vdev_t *vd);
+extern void vdev_load(vdev_t *vd);
 extern int vdev_dtl_load(vdev_t *vd);
 extern void vdev_sync(vdev_t *vd, uint64_t txg);
 extern void vdev_sync_done(vdev_t *vd, uint64_t txg);
 extern void vdev_dirty(vdev_t *vd, int flags, void *arg, uint64_t txg);
 extern void vdev_dirty_leaves(vdev_t *vd, int flags, uint64_t txg);

@@ -423,35 +416,36 @@
 extern vdev_ops_t vdev_disk_ops;
 extern vdev_ops_t vdev_file_ops;
 extern vdev_ops_t vdev_missing_ops;
 extern vdev_ops_t vdev_hole_ops;
 extern vdev_ops_t vdev_spare_ops;
-extern vdev_ops_t vdev_indirect_ops;
 
+extern uint_t vdev_count_leaf_vdevs(vdev_t *root);
+
 /*
  * Common size functions
  */
 extern uint64_t vdev_default_asize(vdev_t *vd, uint64_t psize);
 extern uint64_t vdev_get_min_asize(vdev_t *vd);
 extern void vdev_set_min_asize(vdev_t *vd);
 
+
 /*
+ * Wrapper for getting vdev-specific properties that enforces proper
+ * overriding: vdev-specific properties override CoS properties
+ *
+ * The value of 0 indicates that the property is not set (default).
+ */
+extern uint64_t vdev_queue_get_prop_uint64(vdev_queue_t *vq, vdev_prop_t prop);
+
+/*
  * Global variables
  */
 /* zdb uses this tunable, so it must be declared here to make lint happy. */
 extern int zfs_vdev_cache_size;
 
 /*
- * Functions from vdev_indirect.c
- */
-extern void vdev_indirect_sync_obsolete(vdev_t *vd, dmu_tx_t *tx);
-extern boolean_t vdev_indirect_should_condense(vdev_t *vd);
-extern void spa_condense_indirect_start_sync(vdev_t *vd, dmu_tx_t *tx);
-extern int vdev_obsolete_sm_object(vdev_t *vd);
-extern boolean_t vdev_obsolete_counts_are_precise(vdev_t *vd);
-
-/*
  * The vdev_buf_t is used to translate between zio_t and buf_t, and back again.
  */
 typedef struct vdev_buf {
         buf_t   vb_buf;         /* buffer that describes the io */
         zio_t   *vb_io;         /* pointer back to the original zio_t */