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 */