]> granicus.if.org Git - zfs/blobdiff - module/zfs/arc.c
Add ddt, ddt_entry, and l2arc_hdr caches
[zfs] / module / zfs / arc.c
index 1298c5b91bc6a5157f47c038ef3aa203b8a0b7ef..2459c9326f0d28c16f75cd85145fad531b95e9e5 100644 (file)
@@ -21,7 +21,7 @@
 /*
  * Copyright (c) 2005, 2010, Oracle and/or its affiliates. All rights reserved.
  * Copyright 2011 Nexenta Systems, Inc.  All rights reserved.
- * Copyright (c) 2011 by Delphix. All rights reserved.
+ * Copyright (c) 2013 by Delphix. All rights reserved.
  * Copyright (c) 2013 by Saso Kiselkov. All rights reserved.
  */
 
  * tight.
  *
  * 3. The Megiddo and Modha model assumes a fixed page size. All
- * elements of the cache are therefor exactly the same size.  So
+ * elements of the cache are therefore exactly the same size.  So
  * when adjusting the cache size following a cache miss, its simply
  * a matter of choosing a single page to evict.  In our model, we
  * have variable sized cache blocks (rangeing from 512 bytes to
- * 128K bytes).  We therefor choose a set of blocks to evict to make
+ * 128K bytes).  We therefore choose a set of blocks to evict to make
  * space for a cache miss that approximates as closely as possible
  * the space used by the new block.
  *
@@ -77,7 +77,7 @@
  * ways: 1) via a hash table lookup using the DVA as a key,
  * or 2) via one of the ARC lists.  The arc_read() interface
  * uses method 1, while the internal arc algorithms for
- * adjusting the cache use method 2.  We therefor provide two
+ * adjusting the cache use method 2.  We therefore provide two
  * types of locks: 1) the hash table lock array, and 2) the
  * arc list locks.
  *
 #include <sys/arc.h>
 #include <sys/vdev.h>
 #include <sys/vdev_impl.h>
+#include <sys/dsl_pool.h>
 #ifdef _KERNEL
 #include <sys/vmsystm.h>
 #include <vm/anon.h>
 #include <sys/dmu_tx.h>
 #include <zfs_fletcher.h>
 
+#ifndef _KERNEL
+/* set with ZFS_DEBUG=watch, to enable watchpoints on frozen buffers */
+boolean_t arc_watch = B_FALSE;
+#endif
+
 static kmutex_t                arc_reclaim_thr_lock;
 static kcondvar_t      arc_reclaim_thr_cv;     /* used to signal reclaim thr */
 static uint8_t         arc_thread_exit;
@@ -157,6 +163,12 @@ typedef enum arc_reclaim_strategy {
        ARC_RECLAIM_CONS                /* Conservative reclaim strategy */
 } arc_reclaim_strategy_t;
 
+/*
+ * The number of iterations through arc_evict_*() before we
+ * drop & reacquire the lock.
+ */
+int arc_evict_iterations = 100;
+
 /* number of seconds before growing cache again */
 int zfs_arc_grow_retry = 5;
 
@@ -178,6 +190,11 @@ int zfs_arc_memory_throttle_disable = 1;
 /* disable duplicate buffer eviction */
 int zfs_disable_dup_eviction = 0;
 
+/*
+ * If this percent of memory is free, don't throttle.
+ */
+int arc_lotsfree_percent = 10;
+
 static int arc_dead;
 
 /* expiration time for arc_no_grow */
@@ -232,6 +249,7 @@ typedef struct arc_state {
        uint64_t arcs_lsize[ARC_BUFC_NUMTYPES]; /* amount of evictable data */
        uint64_t arcs_size;     /* total amount of data in this state */
        kmutex_t arcs_mtx;
+       arc_state_type_t arcs_state;
 } arc_state_t;
 
 /* The 6 states: */
@@ -259,7 +277,18 @@ typedef struct arc_stats {
        kstat_named_t arcstat_mfu_ghost_hits;
        kstat_named_t arcstat_deleted;
        kstat_named_t arcstat_recycle_miss;
+       /*
+        * Number of buffers that could not be evicted because the hash lock
+        * was held by another thread.  The lock may not necessarily be held
+        * by something using the same buffer, since hash locks are shared
+        * by multiple buffers.
+        */
        kstat_named_t arcstat_mutex_miss;
+       /*
+        * Number of buffers skipped because they have I/O in progress, are
+        * indrect prefetch buffers that have not lived long enough, or are
+        * not from the spa we're trying to evict from.
+        */
        kstat_named_t arcstat_evict_skip;
        kstat_named_t arcstat_evict_l2_cached;
        kstat_named_t arcstat_evict_l2_eligible;
@@ -419,7 +448,7 @@ static arc_stats_t arc_stats = {
 #define        ARCSTAT(stat)   (arc_stats.stat.value.ui64)
 
 #define        ARCSTAT_INCR(stat, val) \
-       atomic_add_64(&arc_stats.stat.value.ui64, (val));
+       atomic_add_64(&arc_stats.stat.value.ui64, (val))
 
 #define        ARCSTAT_BUMP(stat)      ARCSTAT_INCR(stat, 1)
 #define        ARCSTAT_BUMPDOWN(stat)  ARCSTAT_INCR(stat, -1)
@@ -478,9 +507,9 @@ static arc_state_t  *arc_l2c_only;
 #define        arc_no_grow     ARCSTAT(arcstat_no_grow)
 #define        arc_tempreserve ARCSTAT(arcstat_tempreserve)
 #define        arc_loaned_bytes        ARCSTAT(arcstat_loaned_bytes)
-#define        arc_meta_used   ARCSTAT(arcstat_meta_used)
-#define        arc_meta_limit  ARCSTAT(arcstat_meta_limit)
-#define        arc_meta_max    ARCSTAT(arcstat_meta_max)
+#define        arc_meta_limit  ARCSTAT(arcstat_meta_limit) /* max size for metadata */
+#define        arc_meta_used   ARCSTAT(arcstat_meta_used) /* size of metadata */
+#define        arc_meta_max    ARCSTAT(arcstat_meta_max) /* max size of metadata */
 
 #define        L2ARC_IS_VALID_COMPRESS(_c_) \
        ((_c_) == ZIO_COMPRESS_LZ4 || (_c_) == ZIO_COMPRESS_EMPTY)
@@ -502,6 +531,7 @@ typedef struct arc_write_callback arc_write_callback_t;
 struct arc_write_callback {
        void            *awcb_private;
        arc_done_func_t *awcb_ready;
+       arc_done_func_t *awcb_physdone;
        arc_done_func_t *awcb_done;
        arc_buf_t       *awcb_buf;
 };
@@ -534,6 +564,11 @@ struct arc_buf_hdr {
 
        /* updated atomically */
        clock_t                 b_arc_access;
+       uint32_t                b_mru_hits;
+       uint32_t                b_mru_ghost_hits;
+       uint32_t                b_mfu_hits;
+       uint32_t                b_mfu_ghost_hits;
+       uint32_t                b_l2_hits;
 
        /* self protecting */
        refcount_t              b_refcnt;
@@ -552,6 +587,7 @@ static void arc_access(arc_buf_hdr_t *buf, kmutex_t *hash_lock);
 static int arc_evict_needed(arc_buf_contents_t type);
 static void arc_evict_ghost(arc_state_t *state, uint64_t spa, int64_t bytes,
     arc_buf_contents_t type);
+static void arc_buf_watch(arc_buf_t *buf);
 
 static boolean_t l2arc_write_eligible(uint64_t spa_guid, arc_buf_hdr_t *ab);
 
@@ -648,9 +684,7 @@ uint64_t zfs_crc64_table[256];
 #define        l2arc_writes_sent       ARCSTAT(arcstat_l2_writes_sent)
 #define        l2arc_writes_done       ARCSTAT(arcstat_l2_writes_done)
 
-/*
- * L2ARC Performance Tunables
- */
+/* L2ARC Performance Tunables */
 unsigned long l2arc_write_max = L2ARC_WRITE_SIZE;      /* def max write size */
 unsigned long l2arc_write_boost = L2ARC_WRITE_SIZE;    /* extra warmup write */
 unsigned long l2arc_headroom = L2ARC_HEADROOM;         /* # of dev writes */
@@ -709,7 +743,8 @@ struct l2arc_buf_hdr {
        /* compression applied to buffer data */
        enum zio_compress       b_compress;
        /* real alloc'd buffer size depending on b_compress applied */
-       int                     b_asize;
+       uint32_t                b_asize;
+       uint32_t                b_hits;
        /* temporary buffer holder for in-flight compressed data */
        void                    *b_tmp_cdata;
 };
@@ -864,6 +899,7 @@ buf_hash_remove(arc_buf_hdr_t *buf)
  */
 static kmem_cache_t *hdr_cache;
 static kmem_cache_t *buf_cache;
+static kmem_cache_t *l2arc_hdr_cache;
 
 static void
 buf_fini(void)
@@ -871,8 +907,10 @@ buf_fini(void)
        int i;
 
 #if defined(_KERNEL) && defined(HAVE_SPL)
-       /* Large allocations which do not require contiguous pages
-        * should be using vmem_free() in the linux kernel */
+       /*
+        * Large allocations which do not require contiguous pages
+        * should be using vmem_free() in the linux kernel\
+        */
        vmem_free(buf_hash_table.ht_table,
            (buf_hash_table.ht_mask + 1) * sizeof (void *));
 #else
@@ -883,6 +921,7 @@ buf_fini(void)
                mutex_destroy(&buf_hash_table.ht_locks[i].ht_lock);
        kmem_cache_destroy(hdr_cache);
        kmem_cache_destroy(buf_cache);
+       kmem_cache_destroy(l2arc_hdr_cache);
 }
 
 /*
@@ -963,8 +1002,10 @@ buf_init(void)
 retry:
        buf_hash_table.ht_mask = hsize - 1;
 #if defined(_KERNEL) && defined(HAVE_SPL)
-       /* Large allocations which do not require contiguous pages
-        * should be using vmem_alloc() in the linux kernel */
+       /*
+        * Large allocations which do not require contiguous pages
+        * should be using vmem_alloc() in the linux kernel
+        */
        buf_hash_table.ht_table =
            vmem_zalloc(hsize * sizeof (void*), KM_SLEEP);
 #else
@@ -981,6 +1022,8 @@ retry:
            0, hdr_cons, hdr_dest, NULL, NULL, NULL, 0);
        buf_cache = kmem_cache_create("arc_buf_t", sizeof (arc_buf_t),
            0, buf_cons, buf_dest, NULL, NULL, NULL, 0);
+       l2arc_hdr_cache = kmem_cache_create("l2arc_buf_hdr_t", L2HDR_SIZE,
+           0, NULL, NULL, NULL, NULL, NULL, 0);
 
        for (i = 0; i < 256; i++)
                for (ct = zfs_crc64_table + i, *ct = i, j = 8; j > 0; j--)
@@ -1040,10 +1083,41 @@ arc_cksum_compute(arc_buf_t *buf, boolean_t force)
                return;
        }
        buf->b_hdr->b_freeze_cksum = kmem_alloc(sizeof (zio_cksum_t),
-                                               KM_PUSHPAGE);
+           KM_PUSHPAGE);
        fletcher_2_native(buf->b_data, buf->b_hdr->b_size,
            buf->b_hdr->b_freeze_cksum);
        mutex_exit(&buf->b_hdr->b_freeze_lock);
+       arc_buf_watch(buf);
+}
+
+#ifndef _KERNEL
+void
+arc_buf_sigsegv(int sig, siginfo_t *si, void *unused)
+{
+       panic("Got SIGSEGV at address: 0x%lx\n", (long) si->si_addr);
+}
+#endif
+
+/* ARGSUSED */
+static void
+arc_buf_unwatch(arc_buf_t *buf)
+{
+#ifndef _KERNEL
+       if (arc_watch) {
+               ASSERT0(mprotect(buf->b_data, buf->b_hdr->b_size,
+                   PROT_READ | PROT_WRITE));
+       }
+#endif
+}
+
+/* ARGSUSED */
+static void
+arc_buf_watch(arc_buf_t *buf)
+{
+#ifndef _KERNEL
+       if (arc_watch)
+               ASSERT0(mprotect(buf->b_data, buf->b_hdr->b_size, PROT_READ));
+#endif
 }
 
 void
@@ -1064,6 +1138,8 @@ arc_buf_thaw(arc_buf_t *buf)
        }
 
        mutex_exit(&buf->b_hdr->b_freeze_lock);
+
+       arc_buf_unwatch(buf);
 }
 
 void
@@ -1081,6 +1157,7 @@ arc_buf_freeze(arc_buf_t *buf)
            buf->b_hdr->b_state == arc_anon);
        arc_cksum_compute(buf, B_FALSE);
        mutex_exit(hash_lock);
+
 }
 
 static void
@@ -1137,6 +1214,54 @@ remove_reference(arc_buf_hdr_t *ab, kmutex_t *hash_lock, void *tag)
        return (cnt);
 }
 
+/*
+ * Returns detailed information about a specific arc buffer.  When the
+ * state_index argument is set the function will calculate the arc header
+ * list position for its arc state.  Since this requires a linear traversal
+ * callers are strongly encourage not to do this.  However, it can be helpful
+ * for targeted analysis so the functionality is provided.
+ */
+void
+arc_buf_info(arc_buf_t *ab, arc_buf_info_t *abi, int state_index)
+{
+       arc_buf_hdr_t *hdr = ab->b_hdr;
+       arc_state_t *state = hdr->b_state;
+
+       memset(abi, 0, sizeof (arc_buf_info_t));
+       abi->abi_flags = hdr->b_flags;
+       abi->abi_datacnt = hdr->b_datacnt;
+       abi->abi_state_type = state ? state->arcs_state : ARC_STATE_ANON;
+       abi->abi_state_contents = hdr->b_type;
+       abi->abi_state_index = -1;
+       abi->abi_size = hdr->b_size;
+       abi->abi_access = hdr->b_arc_access;
+       abi->abi_mru_hits = hdr->b_mru_hits;
+       abi->abi_mru_ghost_hits = hdr->b_mru_ghost_hits;
+       abi->abi_mfu_hits = hdr->b_mfu_hits;
+       abi->abi_mfu_ghost_hits = hdr->b_mfu_ghost_hits;
+       abi->abi_holds = refcount_count(&hdr->b_refcnt);
+
+       if (hdr->b_l2hdr) {
+               abi->abi_l2arc_dattr = hdr->b_l2hdr->b_daddr;
+               abi->abi_l2arc_asize = hdr->b_l2hdr->b_asize;
+               abi->abi_l2arc_compress = hdr->b_l2hdr->b_compress;
+               abi->abi_l2arc_hits = hdr->b_l2hdr->b_hits;
+       }
+
+       if (state && state_index && list_link_active(&hdr->b_arc_node)) {
+               list_t *list = &state->arcs_list[hdr->b_type];
+               arc_buf_hdr_t *h;
+
+               mutex_enter(&state->arcs_mtx);
+               for (h = list_head(list); h != NULL; h = list_next(list, h)) {
+                       abi->abi_state_index++;
+                       if (h == hdr)
+                               break;
+               }
+               mutex_exit(&state->arcs_mtx);
+       }
+}
+
 /*
  * Move the supplied buffer to the indicated state.  The mutex
  * for the buffer must be held by the caller.
@@ -1149,7 +1274,7 @@ arc_change_state(arc_state_t *new_state, arc_buf_hdr_t *ab, kmutex_t *hash_lock)
        uint64_t from_delta, to_delta;
 
        ASSERT(MUTEX_HELD(hash_lock));
-       ASSERT(new_state != old_state);
+       ASSERT3P(new_state, !=, old_state);
        ASSERT(refcnt == 0 || ab->b_datacnt > 0);
        ASSERT(ab->b_datacnt == 0 || !GHOST_STATE(new_state));
        ASSERT(ab->b_datacnt <= 1 || old_state != arc_anon);
@@ -1250,7 +1375,7 @@ arc_space_consume(uint64_t space, arc_space_type_t type)
                break;
        }
 
-       atomic_add_64(&arc_meta_used, space);
+       ARCSTAT_INCR(arcstat_meta_used, space);
        atomic_add_64(&arc_size, space);
 }
 
@@ -1279,7 +1404,7 @@ arc_space_return(uint64_t space, arc_space_type_t type)
        ASSERT(arc_meta_used >= space);
        if (arc_meta_max < arc_meta_used)
                arc_meta_max = arc_meta_used;
-       atomic_add_64(&arc_meta_used, -space);
+       ARCSTAT_INCR(arcstat_meta_used, -space);
        ASSERT(arc_size >= space);
        atomic_add_64(&arc_size, -space);
 }
@@ -1298,6 +1423,11 @@ arc_buf_alloc(spa_t *spa, int size, void *tag, arc_buf_contents_t type)
        hdr->b_spa = spa_load_guid(spa);
        hdr->b_state = arc_anon;
        hdr->b_arc_access = 0;
+       hdr->b_mru_hits = 0;
+       hdr->b_mru_ghost_hits = 0;
+       hdr->b_mfu_hits = 0;
+       hdr->b_mfu_ghost_hits = 0;
+       hdr->b_l2_hits = 0;
        buf = kmem_cache_alloc(buf_cache, KM_PUSHPAGE);
        buf->b_hdr = hdr;
        buf->b_data = NULL;
@@ -1435,21 +1565,22 @@ arc_buf_add_ref(arc_buf_t *buf, void* tag)
  * the buffer is placed on l2arc_free_on_write to be freed later.
  */
 static void
-arc_buf_data_free(arc_buf_hdr_t *hdr, void (*free_func)(void *, size_t),
-    void *data, size_t size)
+arc_buf_data_free(arc_buf_t *buf, void (*free_func)(void *, size_t))
 {
+       arc_buf_hdr_t *hdr = buf->b_hdr;
+
        if (HDR_L2_WRITING(hdr)) {
                l2arc_data_free_t *df;
                df = kmem_alloc(sizeof (l2arc_data_free_t), KM_PUSHPAGE);
-               df->l2df_data = data;
-               df->l2df_size = size;
+               df->l2df_data = buf->b_data;
+               df->l2df_size = hdr->b_size;
                df->l2df_func = free_func;
                mutex_enter(&l2arc_free_on_write_mtx);
                list_insert_head(l2arc_free_on_write, df);
                mutex_exit(&l2arc_free_on_write_mtx);
                ARCSTAT_BUMP(arcstat_l2_free_on_write);
        } else {
-               free_func(data, size);
+               free_func(buf->b_data, hdr->b_size);
        }
 }
 
@@ -1465,16 +1596,15 @@ arc_buf_destroy(arc_buf_t *buf, boolean_t recycle, boolean_t all)
                arc_buf_contents_t type = buf->b_hdr->b_type;
 
                arc_cksum_verify(buf);
+               arc_buf_unwatch(buf);
 
                if (!recycle) {
                        if (type == ARC_BUFC_METADATA) {
-                               arc_buf_data_free(buf->b_hdr, zio_buf_free,
-                                   buf->b_data, size);
+                               arc_buf_data_free(buf, zio_buf_free);
                                arc_space_return(size, ARC_SPACE_DATA);
                        } else {
                                ASSERT(type == ARC_BUFC_DATA);
-                               arc_buf_data_free(buf->b_hdr,
-                                   zio_data_buf_free, buf->b_data, size);
+                               arc_buf_data_free(buf, zio_data_buf_free);
                                ARCSTAT_INCR(arcstat_data_size, -size);
                                atomic_add_64(&arc_size, -size);
                        }
@@ -1552,7 +1682,7 @@ arc_hdr_destroy(arc_buf_hdr_t *hdr)
                        list_remove(l2hdr->b_dev->l2ad_buflist, hdr);
                        ARCSTAT_INCR(arcstat_l2_size, -hdr->b_size);
                        ARCSTAT_INCR(arcstat_l2_asize, -l2hdr->b_asize);
-                       kmem_free(l2hdr, sizeof (l2arc_buf_hdr_t));
+                       kmem_cache_free(l2arc_hdr_cache, l2hdr);
                        arc_space_return(L2HDR_SIZE, ARC_SPACE_L2HDRS);
                        if (hdr->b_state == arc_l2c_only)
                                l2arc_hdr_stat_remove();
@@ -1750,6 +1880,8 @@ arc_evict(arc_state_t *state, uint64_t spa, int64_t bytes, boolean_t recycle,
        kmutex_t *hash_lock;
        boolean_t have_lock;
        void *stolen = NULL;
+       arc_buf_hdr_t marker = {{{ 0 }}};
+       int count = 0;
 
        ASSERT(state == arc_mru || state == arc_mfu);
 
@@ -1773,6 +1905,33 @@ arc_evict(arc_state_t *state, uint64_t spa, int64_t bytes, boolean_t recycle,
                if (recycle && ab->b_size != bytes &&
                    ab_prev && ab_prev->b_size == bytes)
                        continue;
+
+               /* ignore markers */
+               if (ab->b_spa == 0)
+                       continue;
+
+               /*
+                * It may take a long time to evict all the bufs requested.
+                * To avoid blocking all arc activity, periodically drop
+                * the arcs_mtx and give other threads a chance to run
+                * before reacquiring the lock.
+                *
+                * If we are looking for a buffer to recycle, we are in
+                * the hot code path, so don't sleep.
+                */
+               if (!recycle && count++ > arc_evict_iterations) {
+                       list_insert_after(list, ab, &marker);
+                       mutex_exit(&evicted_state->arcs_mtx);
+                       mutex_exit(&state->arcs_mtx);
+                       kpreempt(KPREEMPT_SYNC);
+                       mutex_enter(&state->arcs_mtx);
+                       mutex_enter(&evicted_state->arcs_mtx);
+                       ab_prev = list_prev(list, &marker);
+                       list_remove(list, &marker);
+                       count = 0;
+                       continue;
+               }
+
                hash_lock = HDR_LOCK(ab);
                have_lock = MUTEX_HELD(hash_lock);
                if (have_lock || mutex_tryenter(hash_lock)) {
@@ -1854,27 +2013,11 @@ arc_evict(arc_state_t *state, uint64_t spa, int64_t bytes, boolean_t recycle,
                ARCSTAT_INCR(arcstat_mutex_miss, missed);
 
        /*
-        * We have just evicted some data into the ghost state, make
-        * sure we also adjust the ghost state size if necessary.
+        * Note: we have just evicted some data into the ghost state,
+        * potentially putting the ghost size over the desired size.  Rather
+        * that evicting from the ghost list in this hot code path, leave
+        * this chore to the arc_reclaim_thread().
         */
-       if (arc_no_grow &&
-           arc_mru_ghost->arcs_size + arc_mfu_ghost->arcs_size > arc_c) {
-               int64_t mru_over = arc_anon->arcs_size + arc_mru->arcs_size +
-                   arc_mru_ghost->arcs_size - arc_c;
-
-               if (mru_over > 0 && arc_mru_ghost->arcs_lsize[type] > 0) {
-                       int64_t todelete =
-                           MIN(arc_mru_ghost->arcs_lsize[type], mru_over);
-                       arc_evict_ghost(arc_mru_ghost, 0, todelete,
-                           ARC_BUFC_DATA);
-               } else if (arc_mfu_ghost->arcs_lsize[type] > 0) {
-                       int64_t todelete = MIN(arc_mfu_ghost->arcs_lsize[type],
-                           arc_mru_ghost->arcs_size +
-                           arc_mfu_ghost->arcs_size - arc_c);
-                       arc_evict_ghost(arc_mfu_ghost, 0, todelete,
-                           ARC_BUFC_DATA);
-               }
-       }
 
        return (stolen);
 }
@@ -1893,13 +2036,16 @@ arc_evict_ghost(arc_state_t *state, uint64_t spa, int64_t bytes,
        kmutex_t *hash_lock;
        uint64_t bytes_deleted = 0;
        uint64_t bufs_skipped = 0;
+       int count = 0;
 
        ASSERT(GHOST_STATE(state));
-       bzero(&marker, sizeof(marker));
+       bzero(&marker, sizeof (marker));
 top:
        mutex_enter(&state->arcs_mtx);
        for (ab = list_tail(list); ab; ab = ab_prev) {
                ab_prev = list_prev(list, ab);
+               if (ab->b_type > ARC_BUFC_NUMTYPES)
+                       panic("invalid ab=%p", (void *)ab);
                if (spa && ab->b_spa != spa)
                        continue;
 
@@ -1911,6 +2057,23 @@ top:
                /* caller may be trying to modify this buffer, skip it */
                if (MUTEX_HELD(hash_lock))
                        continue;
+
+               /*
+                * It may take a long time to evict all the bufs requested.
+                * To avoid blocking all arc activity, periodically drop
+                * the arcs_mtx and give other threads a chance to run
+                * before reacquiring the lock.
+                */
+               if (count++ > arc_evict_iterations) {
+                       list_insert_after(list, ab, &marker);
+                       mutex_exit(&state->arcs_mtx);
+                       kpreempt(KPREEMPT_SYNC);
+                       mutex_enter(&state->arcs_mtx);
+                       ab_prev = list_prev(list, &marker);
+                       list_remove(list, &marker);
+                       count = 0;
+                       continue;
+               }
                if (mutex_tryenter(hash_lock)) {
                        ASSERT(!HDR_IO_IN_PROGRESS(ab));
                        ASSERT(ab->b_buf == NULL);
@@ -1946,8 +2109,9 @@ top:
                        mutex_enter(&state->arcs_mtx);
                        ab_prev = list_prev(list, &marker);
                        list_remove(list, &marker);
-               } else
+               } else {
                        bufs_skipped += 1;
+               }
        }
        mutex_exit(&state->arcs_mtx);
 
@@ -2256,7 +2420,8 @@ arc_adapt_thread(void)
                        }
 
                        /* reset the growth delay for every reclaim */
-                       arc_grow_time = ddi_get_lbolt()+(zfs_arc_grow_retry * hz);
+                       arc_grow_time = ddi_get_lbolt() +
+                           (zfs_arc_grow_retry * hz);
 
                        arc_kmem_reap_now(last_reclaim, 0);
                        arc_warm = B_TRUE;
@@ -2670,6 +2835,7 @@ arc_access(arc_buf_hdr_t *buf, kmutex_t *hash_lock)
                                ASSERT(list_link_active(&buf->b_arc_node));
                        } else {
                                buf->b_flags &= ~ARC_PREFETCH;
+                               atomic_inc_32(&buf->b_mru_hits);
                                ARCSTAT_BUMP(arcstat_mru_hits);
                        }
                        buf->b_arc_access = now;
@@ -2691,6 +2857,7 @@ arc_access(arc_buf_hdr_t *buf, kmutex_t *hash_lock)
                        DTRACE_PROBE1(new_state__mfu, arc_buf_hdr_t *, buf);
                        arc_change_state(arc_mfu, buf, hash_lock);
                }
+               atomic_inc_32(&buf->b_mru_hits);
                ARCSTAT_BUMP(arcstat_mru_hits);
        } else if (buf->b_state == arc_mru_ghost) {
                arc_state_t     *new_state;
@@ -2713,6 +2880,7 @@ arc_access(arc_buf_hdr_t *buf, kmutex_t *hash_lock)
                buf->b_arc_access = ddi_get_lbolt();
                arc_change_state(new_state, buf, hash_lock);
 
+               atomic_inc_32(&buf->b_mru_ghost_hits);
                ARCSTAT_BUMP(arcstat_mru_ghost_hits);
        } else if (buf->b_state == arc_mfu) {
                /*
@@ -2728,6 +2896,7 @@ arc_access(arc_buf_hdr_t *buf, kmutex_t *hash_lock)
                        ASSERT(refcount_count(&buf->b_refcnt) == 0);
                        ASSERT(list_link_active(&buf->b_arc_node));
                }
+               atomic_inc_32(&buf->b_mfu_hits);
                ARCSTAT_BUMP(arcstat_mfu_hits);
                buf->b_arc_access = ddi_get_lbolt();
        } else if (buf->b_state == arc_mfu_ghost) {
@@ -2751,6 +2920,7 @@ arc_access(arc_buf_hdr_t *buf, kmutex_t *hash_lock)
                DTRACE_PROBE1(new_state__mfu, arc_buf_hdr_t *, buf);
                arc_change_state(new_state, buf, hash_lock);
 
+               atomic_inc_32(&buf->b_mfu_ghost_hits);
                ARCSTAT_BUMP(arcstat_mfu_ghost_hits);
        } else if (buf->b_state == arc_l2c_only) {
                /*
@@ -2834,6 +3004,7 @@ arc_read_done(zio_t *zio)
        }
 
        arc_cksum_compute(buf, B_FALSE);
+       arc_buf_watch(buf);
 
        if (hash_lock && zio->io_error == 0 && hdr->b_state == arc_anon) {
                /*
@@ -2935,7 +3106,7 @@ arc_read_done(zio_t *zio)
  */
 int
 arc_read(zio_t *pio, spa_t *spa, const blkptr_t *bp, arc_done_func_t *done,
-    void *private, int priority, int zio_flags, uint32_t *arc_flags,
+    void *private, zio_priority_t priority, int zio_flags, uint32_t *arc_flags,
     const zbookmark_t *zb)
 {
        arc_buf_hdr_t *hdr;
@@ -2943,6 +3114,7 @@ arc_read(zio_t *pio, spa_t *spa, const blkptr_t *bp, arc_done_func_t *done,
        kmutex_t *hash_lock;
        zio_t *rzio;
        uint64_t guid = spa_load_guid(spa);
+       int rc = 0;
 
 top:
        hdr = buf_hash_find(guid, BP_IDENTITY(bp), BP_PHYSICAL_BIRTH(bp),
@@ -2976,10 +3148,10 @@ top:
                                hdr->b_acb = acb;
                                add_reference(hdr, hash_lock, private);
                                mutex_exit(hash_lock);
-                               return (0);
+                               goto out;
                        }
                        mutex_exit(hash_lock);
-                       return (0);
+                       goto out;
                }
 
                ASSERT(hdr->b_state == arc_mru || hdr->b_state == arc_mfu);
@@ -3023,7 +3195,7 @@ top:
                uint64_t size = BP_GET_LSIZE(bp);
                arc_callback_t  *acb;
                vdev_t *vd = NULL;
-               uint64_t addr = -1;
+               uint64_t addr = 0;
                boolean_t devw = B_FALSE;
 
                if (hdr == NULL) {
@@ -3108,6 +3280,10 @@ top:
 
                mutex_exit(hash_lock);
 
+               /*
+                * At this point, we have a level 1 cache miss.  Try again in
+                * L2ARC if possible.
+                */
                ASSERT3U(hdr->b_size, ==, size);
                DTRACE_PROBE4(arc__miss, arc_buf_hdr_t *, hdr, blkptr_t *, bp,
                    uint64_t, size, zbookmark_t *, zb);
@@ -3133,6 +3309,7 @@ top:
 
                                DTRACE_PROBE1(l2arc__hit, arc_buf_hdr_t *, hdr);
                                ARCSTAT_BUMP(arcstat_l2_hits);
+                               atomic_inc_32(&hdr->b_l2hdr->b_hits);
 
                                cb = kmem_zalloc(sizeof (l2arc_read_callback_t),
                                    KM_PUSHPAGE);
@@ -3143,6 +3320,10 @@ top:
                                cb->l2rcb_flags = zio_flags;
                                cb->l2rcb_compress = hdr->b_l2hdr->b_compress;
 
+                               ASSERT(addr >= VDEV_LABEL_START_SIZE &&
+                                   addr + size < vd->vdev_psize -
+                                   VDEV_LABEL_END_SIZE);
+
                                /*
                                 * l2arc read.  The SCL_L2ARC lock will be
                                 * released by l2arc_read_done().
@@ -3174,12 +3355,12 @@ top:
 
                                if (*arc_flags & ARC_NOWAIT) {
                                        zio_nowait(rzio);
-                                       return (0);
+                                       goto out;
                                }
 
                                ASSERT(*arc_flags & ARC_WAIT);
                                if (zio_wait(rzio) == 0)
-                                       return (0);
+                                       goto out;
 
                                /* l2arc read error; goto zio_read() */
                        } else {
@@ -3203,13 +3384,18 @@ top:
                rzio = zio_read(pio, spa, bp, buf->b_data, size,
                    arc_read_done, buf, priority, zio_flags, zb);
 
-               if (*arc_flags & ARC_WAIT)
-                       return (zio_wait(rzio));
+               if (*arc_flags & ARC_WAIT) {
+                       rc = zio_wait(rzio);
+                       goto out;
+               }
 
                ASSERT(*arc_flags & ARC_NOWAIT);
                zio_nowait(rzio);
        }
-       return (0);
+
+out:
+       spa_read_history_add(spa, zb, *arc_flags);
+       return (rc);
 }
 
 arc_prune_t *
@@ -3217,7 +3403,7 @@ arc_add_prune_callback(arc_prune_func_t *func, void *private)
 {
        arc_prune_t *p;
 
-       p = kmem_alloc(sizeof(*p), KM_SLEEP);
+       p = kmem_alloc(sizeof (*p), KM_SLEEP);
        p->p_pfunc = func;
        p->p_private = private;
        list_link_init(&p->p_node);
@@ -3369,8 +3555,8 @@ arc_buf_evict(arc_buf_t *buf)
 }
 
 /*
- * Release this buffer from the cache.  This must be done
- * after a read and prior to modifying the buffer contents.
+ * Release this buffer from the cache, making it an anonymous buffer.  This
+ * must be done after a read and prior to modifying the buffer contents.
  * If the buffer has more than one reference, we must make
  * a new hdr for the buffer.
  */
@@ -3408,8 +3594,8 @@ arc_release(arc_buf_t *buf, void *tag)
        if (l2hdr) {
                mutex_enter(&l2arc_buflist_mtx);
                hdr->b_l2hdr = NULL;
-               buf_size = hdr->b_size;
        }
+       buf_size = hdr->b_size;
 
        /*
         * Do we have more than one buf?
@@ -3453,6 +3639,7 @@ arc_release(arc_buf_t *buf, void *tag)
                }
                hdr->b_datacnt -= 1;
                arc_cksum_verify(buf);
+               arc_buf_unwatch(buf);
 
                mutex_exit(hash_lock);
 
@@ -3463,6 +3650,11 @@ arc_release(arc_buf_t *buf, void *tag)
                nhdr->b_buf = buf;
                nhdr->b_state = arc_anon;
                nhdr->b_arc_access = 0;
+               nhdr->b_mru_hits = 0;
+               nhdr->b_mru_ghost_hits = 0;
+               nhdr->b_mfu_hits = 0;
+               nhdr->b_mfu_ghost_hits = 0;
+               nhdr->b_l2_hits = 0;
                nhdr->b_flags = flags & ARC_L2_WRITING;
                nhdr->b_l2hdr = NULL;
                nhdr->b_datacnt = 1;
@@ -3479,6 +3671,11 @@ arc_release(arc_buf_t *buf, void *tag)
                if (hdr->b_state != arc_anon)
                        arc_change_state(arc_anon, hdr, hash_lock);
                hdr->b_arc_access = 0;
+               hdr->b_mru_hits = 0;
+               hdr->b_mru_ghost_hits = 0;
+               hdr->b_mfu_hits = 0;
+               hdr->b_mfu_ghost_hits = 0;
+               hdr->b_l2_hits = 0;
                if (hash_lock)
                        mutex_exit(hash_lock);
 
@@ -3491,7 +3688,7 @@ arc_release(arc_buf_t *buf, void *tag)
        if (l2hdr) {
                ARCSTAT_INCR(arcstat_l2_asize, -l2hdr->b_asize);
                list_remove(l2hdr->b_dev->l2ad_buflist, hdr);
-               kmem_free(l2hdr, sizeof (l2arc_buf_hdr_t));
+               kmem_cache_free(l2arc_hdr_cache, l2hdr);
                arc_space_return(L2HDR_SIZE, ARC_SPACE_L2HDRS);
                ARCSTAT_INCR(arcstat_l2_size, -buf_size);
                mutex_exit(&l2arc_buflist_mtx);
@@ -3561,6 +3758,18 @@ arc_write_ready(zio_t *zio)
        hdr->b_flags |= ARC_IO_IN_PROGRESS;
 }
 
+/*
+ * The SPA calls this callback for each physical write that happens on behalf
+ * of a logical write.  See the comment in dbuf_write_physdone() for details.
+ */
+static void
+arc_write_physdone(zio_t *zio)
+{
+       arc_write_callback_t *cb = zio->io_private;
+       if (cb->awcb_physdone != NULL)
+               cb->awcb_physdone(zio, cb->awcb_buf, cb->awcb_private);
+}
+
 static void
 arc_write_done(zio_t *zio)
 {
@@ -3609,6 +3818,12 @@ arc_write_done(zio_t *zio)
                                arc_hdr_destroy(exists);
                                exists = buf_hash_insert(hdr, &hash_lock);
                                ASSERT3P(exists, ==, NULL);
+                       } else if (zio->io_flags & ZIO_FLAG_NOPWRITE) {
+                               /* nopwrite */
+                               ASSERT(zio->io_prop.zp_nopwrite);
+                               if (!BP_EQUAL(&zio->io_bp_orig, zio->io_bp))
+                                       panic("bad nopwrite, hdr=%p exists=%p",
+                                           (void *)hdr, (void *)exists);
                        } else {
                                /* Dedup */
                                ASSERT(hdr->b_datacnt == 1);
@@ -3635,8 +3850,9 @@ arc_write_done(zio_t *zio)
 zio_t *
 arc_write(zio_t *pio, spa_t *spa, uint64_t txg,
     blkptr_t *bp, arc_buf_t *buf, boolean_t l2arc, boolean_t l2arc_compress,
-    const zio_prop_t *zp, arc_done_func_t *ready, arc_done_func_t *done,
-    void *private, int priority, int zio_flags, const zbookmark_t *zb)
+    const zio_prop_t *zp, arc_done_func_t *ready, arc_done_func_t *physdone,
+    arc_done_func_t *done, void *private, zio_priority_t priority,
+    int zio_flags, const zbookmark_t *zb)
 {
        arc_buf_hdr_t *hdr = buf->b_hdr;
        arc_write_callback_t *callback;
@@ -3653,38 +3869,29 @@ arc_write(zio_t *pio, spa_t *spa, uint64_t txg,
                hdr->b_flags |= ARC_L2COMPRESS;
        callback = kmem_zalloc(sizeof (arc_write_callback_t), KM_PUSHPAGE);
        callback->awcb_ready = ready;
+       callback->awcb_physdone = physdone;
        callback->awcb_done = done;
        callback->awcb_private = private;
        callback->awcb_buf = buf;
 
        zio = zio_write(pio, spa, txg, bp, buf->b_data, hdr->b_size, zp,
-           arc_write_ready, arc_write_done, callback, priority, zio_flags, zb);
+           arc_write_ready, arc_write_physdone, arc_write_done, callback,
+           priority, zio_flags, zb);
 
        return (zio);
 }
 
 static int
-arc_memory_throttle(uint64_t reserve, uint64_t inflight_data, uint64_t txg)
+arc_memory_throttle(uint64_t reserve, uint64_t txg)
 {
 #ifdef _KERNEL
-       uint64_t available_memory;
-
        if (zfs_arc_memory_throttle_disable)
                return (0);
 
-       /* Easily reclaimable memory (free + inactive + arc-evictable) */
-       available_memory = ptob(spl_kmem_availrmem()) + arc_evictable_memory();
-
-       if (available_memory <= zfs_write_limit_max) {
+       if (freemem <= physmem * arc_lotsfree_percent / 100) {
                ARCSTAT_INCR(arcstat_memory_throttle_count, 1);
                DMU_TX_STAT_BUMP(dmu_tx_memory_reclaim);
-               return (EAGAIN);
-       }
-
-       if (inflight_data > available_memory / 4) {
-               ARCSTAT_INCR(arcstat_memory_throttle_count, 1);
-               DMU_TX_STAT_BUMP(dmu_tx_memory_inflight);
-               return (ERESTART);
+               return (SET_ERROR(EAGAIN));
        }
 #endif
        return (0);
@@ -3703,20 +3910,11 @@ arc_tempreserve_space(uint64_t reserve, uint64_t txg)
        int error;
        uint64_t anon_size;
 
-#ifdef ZFS_DEBUG
-       /*
-        * Once in a while, fail for no reason.  Everything should cope.
-        */
-       if (spa_get_random(10000) == 0) {
-               dprintf("forcing random failure\n");
-               return (ERESTART);
-       }
-#endif
        if (reserve > arc_c/4 && !arc_no_grow)
                arc_c = MIN(arc_c_max, reserve * 4);
        if (reserve > arc_c) {
                DMU_TX_STAT_BUMP(dmu_tx_memory_reserve);
-               return (ENOMEM);
+               return (SET_ERROR(ENOMEM));
        }
 
        /*
@@ -3728,10 +3926,11 @@ arc_tempreserve_space(uint64_t reserve, uint64_t txg)
 
        /*
         * Writes will, almost always, require additional memory allocations
-        * in order to compress/encrypt/etc the data.  We therefor need to
+        * in order to compress/encrypt/etc the data.  We therefore need to
         * make sure that there is sufficient available memory for this.
         */
-       if ((error = arc_memory_throttle(reserve, anon_size, txg)))
+       error = arc_memory_throttle(reserve, txg);
+       if (error != 0)
                return (error);
 
        /*
@@ -3751,7 +3950,7 @@ arc_tempreserve_space(uint64_t reserve, uint64_t txg)
                    arc_anon->arcs_lsize[ARC_BUFC_DATA]>>10,
                    reserve>>10, arc_c>>10);
                DMU_TX_STAT_BUMP(dmu_tx_dirty_throttle);
-               return (ERESTART);
+               return (SET_ERROR(ERESTART));
        }
        atomic_add_64(&arc_tempreserve, reserve);
        return (0);
@@ -3772,7 +3971,7 @@ arc_kstat_update(kstat_t *ksp, int rw)
        arc_stats_t *as = ksp->ks_data;
 
        if (rw == KSTAT_WRITE) {
-               return (EACCES);
+               return (SET_ERROR(EACCES));
        } else {
                arc_kstat_update_state(arc_anon,
                    &as->arcstat_anon_size,
@@ -3829,7 +4028,7 @@ arc_init(void)
        /* set min cache to 1/32 of all memory, or 64MB, whichever is more */
        arc_c_min = MAX(arc_c / 4, 64<<20);
        /* set max to 1/2 of all memory */
-       arc_c_max = MAX(arc_c * 4, arc_c_max);
+       arc_c_max = arc_c * 4;
 
        /*
         * Allow the tunables to override our calculations if they are
@@ -3896,6 +4095,13 @@ arc_init(void)
        list_create(&arc_l2c_only->arcs_list[ARC_BUFC_DATA],
            sizeof (arc_buf_hdr_t), offsetof(arc_buf_hdr_t, b_arc_node));
 
+       arc_anon->arcs_state = ARC_STATE_ANON;
+       arc_mru->arcs_state = ARC_STATE_MRU;
+       arc_mru_ghost->arcs_state = ARC_STATE_MRU_GHOST;
+       arc_mfu->arcs_state = ARC_STATE_MFU;
+       arc_mfu_ghost->arcs_state = ARC_STATE_MFU_GHOST;
+       arc_l2c_only->arcs_state = ARC_STATE_L2C_ONLY;
+
        buf_init();
 
        arc_thread_exit = 0;
@@ -3921,11 +4127,24 @@ arc_init(void)
        arc_dead = FALSE;
        arc_warm = B_FALSE;
 
-       if (zfs_write_limit_max == 0)
-               zfs_write_limit_max = ptob(physmem) >> zfs_write_limit_shift;
-       else
-               zfs_write_limit_shift = 0;
-       mutex_init(&zfs_write_limit_lock, NULL, MUTEX_DEFAULT, NULL);
+       /*
+        * Calculate maximum amount of dirty data per pool.
+        *
+        * If it has been set by a module parameter, take that.
+        * Otherwise, use a percentage of physical memory defined by
+        * zfs_dirty_data_max_percent (default 10%) with a cap at
+        * zfs_dirty_data_max_max (default 25% of physical memory).
+        */
+       if (zfs_dirty_data_max_max == 0)
+               zfs_dirty_data_max_max = physmem * PAGESIZE *
+                   zfs_dirty_data_max_max_percent / 100;
+
+       if (zfs_dirty_data_max == 0) {
+               zfs_dirty_data_max = physmem * PAGESIZE *
+                   zfs_dirty_data_max_percent / 100;
+               zfs_dirty_data_max = MIN(zfs_dirty_data_max,
+                   zfs_dirty_data_max_max);
+       }
 }
 
 void
@@ -3983,8 +4202,6 @@ arc_fini(void)
        mutex_destroy(&arc_mfu_ghost->arcs_mtx);
        mutex_destroy(&arc_l2c_only->arcs_mtx);
 
-       mutex_destroy(&zfs_write_limit_lock);
-
        buf_fini();
 
        ASSERT(arc_loaned_bytes == 0);
@@ -4333,6 +4550,13 @@ l2arc_write_done(zio_t *zio)
         */
        for (ab = list_prev(buflist, head); ab; ab = ab_prev) {
                ab_prev = list_prev(buflist, ab);
+               abl2 = ab->b_l2hdr;
+
+               /*
+                * Release the temporary compressed buffer as soon as possible.
+                */
+               if (abl2->b_compress != ZIO_COMPRESS_OFF)
+                       l2arc_release_cdata_buf(ab);
 
                hash_lock = HDR_LOCK(ab);
                if (!mutex_tryenter(hash_lock)) {
@@ -4345,14 +4569,6 @@ l2arc_write_done(zio_t *zio)
                        continue;
                }
 
-               abl2 = ab->b_l2hdr;
-
-               /*
-                * Release the temporary compressed buffer as soon as possible.
-                */
-               if (abl2->b_compress != ZIO_COMPRESS_OFF)
-                       l2arc_release_cdata_buf(ab);
-
                if (zio->io_error != 0) {
                        /*
                         * Error - drop L2ARC entry.
@@ -4360,7 +4576,7 @@ l2arc_write_done(zio_t *zio)
                        list_remove(buflist, ab);
                        ARCSTAT_INCR(arcstat_l2_asize, -abl2->b_asize);
                        ab->b_l2hdr = NULL;
-                       kmem_free(abl2, sizeof (l2arc_buf_hdr_t));
+                       kmem_cache_free(l2arc_hdr_cache, abl2);
                        arc_space_return(L2HDR_SIZE, ARC_SPACE_L2HDRS);
                        ARCSTAT_INCR(arcstat_l2_size, -ab->b_size);
                }
@@ -4437,7 +4653,7 @@ l2arc_read_done(zio_t *zio)
                if (zio->io_error != 0) {
                        ARCSTAT_BUMP(arcstat_l2_io_error);
                } else {
-                       zio->io_error = EIO;
+                       zio->io_error = SET_ERROR(EIO);
                }
                if (!equal)
                        ARCSTAT_BUMP(arcstat_l2_cksum_bad);
@@ -4615,7 +4831,7 @@ top:
                                abl2 = ab->b_l2hdr;
                                ARCSTAT_INCR(arcstat_l2_asize, -abl2->b_asize);
                                ab->b_l2hdr = NULL;
-                               kmem_free(abl2, sizeof (l2arc_buf_hdr_t));
+                               kmem_cache_free(l2arc_hdr_cache, abl2);
                                arc_space_return(L2HDR_SIZE, ARC_SPACE_L2HDRS);
                                ARCSTAT_INCR(arcstat_l2_size, -ab->b_size);
                        }
@@ -4751,7 +4967,7 @@ l2arc_write_buffers(spa_t *spa, l2arc_dev_t *dev, uint64_t target_sz,
                                list_insert_head(dev->l2ad_buflist, head);
 
                                cb = kmem_alloc(sizeof (l2arc_write_callback_t),
-                                               KM_PUSHPAGE);
+                                   KM_PUSHPAGE);
                                cb->l2wcb_dev = dev;
                                cb->l2wcb_head = head;
                                pio = zio_root(spa, l2arc_write_done, cb,
@@ -4761,9 +4977,9 @@ l2arc_write_buffers(spa_t *spa, l2arc_dev_t *dev, uint64_t target_sz,
                        /*
                         * Create and add a new L2ARC header.
                         */
-                       l2hdr = kmem_zalloc(sizeof (l2arc_buf_hdr_t),
-                           KM_PUSHPAGE);
+                       l2hdr = kmem_cache_alloc(l2arc_hdr_cache, KM_PUSHPAGE);
                        l2hdr->b_dev = dev;
+                       l2hdr->b_daddr = 0;
                        arc_space_consume(L2HDR_SIZE, ARC_SPACE_L2HDRS);
 
                        ab->b_flags |= ARC_L2_WRITING;
@@ -4779,6 +4995,7 @@ l2arc_write_buffers(spa_t *spa, l2arc_dev_t *dev, uint64_t target_sz,
                        l2hdr->b_compress = ZIO_COMPRESS_OFF;
                        l2hdr->b_asize = ab->b_size;
                        l2hdr->b_tmp_cdata = ab->b_buf->b_data;
+                       l2hdr->b_hits = 0;
 
                        buf_sz = ab->b_size;
                        ab->b_l2hdr = l2hdr;
@@ -5015,7 +5232,7 @@ l2arc_decompress_zio(zio_t *zio, arc_buf_hdr_t *hdr, enum zio_compress c)
                bcopy(zio->io_data, cdata, csize);
                if (zio_decompress_data(c, cdata, zio->io_data, csize,
                    hdr->b_size) != 0)
-                       zio->io_error = EIO;
+                       zio->io_error = SET_ERROR(EIO);
                zio_data_buf_free(cdata, csize);
        }
 
@@ -5311,6 +5528,7 @@ l2arc_stop(void)
 #if defined(_KERNEL) && defined(HAVE_SPL)
 EXPORT_SYMBOL(arc_read);
 EXPORT_SYMBOL(arc_buf_remove_ref);
+EXPORT_SYMBOL(arc_buf_info);
 EXPORT_SYMBOL(arc_getbuf_func);
 EXPORT_SYMBOL(arc_add_prune_callback);
 EXPORT_SYMBOL(arc_remove_prune_callback);