Print this page
NEX-15279 support NFS server in zone
NEX-15520 online NFS shares cause zoneadm halt to hang in nfs_export_zone_fini
Portions contributed by: Dan Kruchinin dan.kruchinin@nexenta.com
Portions contributed by: Stepan Zastupov stepan.zastupov@gmail.com
Reviewed by: Joyce McIntosh <joyce.mcintosh@nexenta.com>
Reviewed by: Rob Gittins <rob.gittins@nexenta.com>
Reviewed by: Gordon Ross <gordon.ross@nexenta.com>
NEX-9275 Got "bad mutex" panic when run IO to nfs share from clients
Reviewed by: Yuri Pankov <yuri.pankov@nexenta.com>
Reviewed by: Sanjay Nadkarni <sanjay.nadkarni@nexenta.com>
Reviewed by: Rob Gittins <rob.gittins@nexenta.com>
NEX-5667 nfssrv_stats_flags does not work for aggregated kstats
NEX-4472 nfsauth_retrieve() flood caused by NFS clients with personal identity problems
Reviewed by: Yuri Pankov <yuri.pankov@nexenta.com>
NEX-2345 nfsauth_cache_get() could spend a lot of time walking exi_cache
Reviewed by: Gordon Ross <gordon.ross@nexenta.com>
NEX-2972 bad free at checkauth+0x1a2()
Reviewed by: Jan Kryl <jan.kryl@nexenta.com>
Reviewed by: Josef 'Jeff' Sipek <josef.sipek@nexenta.com>
NEX-2949 Panic due to bad mutex, from auth_cache being previously freed
NEX-1974 Support for more than 16 groups with AUTH_SYS
Reviewed by: Gordon Ross <gordon.ross@nexenta.com>
Reviewed by: Josef 'Jeff' Sipek <josef.sipek@nexenta.com>
NEX-1128 NFS server: Generic uid and gid remapping for AUTH_SYS
Reviewed by: Jan Kryl <jan.kryl@nexenta.com>
OS-152 NFS extremely slow in nfsauth
Reviewed by: Jan Kryl <jan.kryl@nexenta.com>
Reviewed by: Ilya Usvyatsky <ilya.usvyatsky@nexenta.com>
Reviewed by: Josef Sipek <josef.sipek@nexenta.com>

@@ -18,12 +18,15 @@
  *
  * CDDL HEADER END
  */
 
 /*
- * Copyright 2016 Nexenta Systems, Inc.  All rights reserved.
  * Copyright (c) 1995, 2010, Oracle and/or its affiliates. All rights reserved.
+ */
+
+/*
+ * Copyright 2018 Nexenta Systems, Inc.
  * Copyright (c) 2015 by Delphix. All rights reserved.
  */
 
 #include <sys/param.h>
 #include <sys/errno.h>

@@ -51,13 +54,17 @@
 #include <nfs/auth.h>
 
 static struct kmem_cache *exi_cache_handle;
 static void exi_cache_reclaim(void *);
 static void exi_cache_trim(struct exportinfo *exi);
+static void *nfsauth_zone_init(zoneid_t);
+static void nfsauth_zone_shutdown(zoneid_t zoneid, void *data);
+static void nfsauth_zone_fini(zoneid_t, void *);
 
 extern pri_t minclsyspri;
 
+/* NFS auth cache statistics */
 volatile uint_t nfsauth_cache_hit;
 volatile uint_t nfsauth_cache_miss;
 volatile uint_t nfsauth_cache_refresh;
 volatile uint_t nfsauth_cache_reclaim;
 volatile uint_t exi_cache_auth_reclaim_failed;

@@ -117,141 +124,168 @@
         char                    *ran_netid;
         list_node_t             ran_node;
 } refreshq_auth_node_t;
 
 /*
- * Used to manipulate things on the refreshq_queue.
- * Note that the refresh thread will effectively
- * pop a node off of the queue, at which point it
+ * Used to manipulate things on the refreshq_queue.  Note that the refresh
+ * thread will effectively pop a node off of the queue, at which point it
  * will no longer need to hold the mutex.
  */
 static kmutex_t refreshq_lock;
 static list_t refreshq_queue;
 static kcondvar_t refreshq_cv;
 
 /*
- * If there is ever a problem with loading the
- * module, then nfsauth_fini() needs to be called
- * to remove state. In that event, since the
- * refreshq thread has been started, they need to
- * work together to get rid of state.
+ * If there is ever a problem with loading the module, then nfsauth_fini()
+ * needs to be called to remove state.  In that event, since the refreshq
+ * thread has been started, they need to work together to get rid of state.
  */
 typedef enum nfsauth_refreshq_thread_state {
         REFRESHQ_THREAD_RUNNING,
         REFRESHQ_THREAD_FINI_REQ,
-        REFRESHQ_THREAD_HALTED
+        REFRESHQ_THREAD_HALTED,
+        REFRESHQ_THREAD_NEED_CREATE
 } nfsauth_refreshq_thread_state_t;
 
-nfsauth_refreshq_thread_state_t
-refreshq_thread_state = REFRESHQ_THREAD_HALTED;
+typedef struct nfsauth_globals {
+        kmutex_t        mountd_lock;
+        door_handle_t   mountd_dh;
 
+        /*
+         * Used to manipulate things on the refreshq_queue.  Note that the
+         * refresh thread will effectively pop a node off of the queue,
+         * at which point it will no longer need to hold the mutex.
+         */
+        kmutex_t        refreshq_lock;
+        list_t          refreshq_queue;
+        kcondvar_t      refreshq_cv;
+
+        /*
+         * A list_t would be overkill.  These are auth_cache entries which are
+         * no longer linked to an exi.  It should be the case that all of their
+         * states are NFS_AUTH_INVALID, i.e., the only way to be put on this
+         * list is iff their state indicated that they had been placed on the
+         * refreshq_queue.
+         *
+         * Note that while there is no link from the exi or back to the exi,
+         * the exi can not go away until these entries are harvested.
+         */
+        struct auth_cache               *refreshq_dead_entries;
+        nfsauth_refreshq_thread_state_t refreshq_thread_state;
+
+} nfsauth_globals_t;
+
 static void nfsauth_free_node(struct auth_cache *);
-static void nfsauth_refresh_thread(void);
+static void nfsauth_refresh_thread(nfsauth_globals_t *);
 
 static int nfsauth_cache_compar(const void *, const void *);
 
-/*
- * mountd is a server-side only daemon. This will need to be
- * revisited if the NFS server is ever made zones-aware.
- */
-kmutex_t        mountd_lock;
-door_handle_t   mountd_dh;
+static zone_key_t       nfsauth_zone_key;
 
 void
 mountd_args(uint_t did)
 {
-        mutex_enter(&mountd_lock);
-        if (mountd_dh != NULL)
-                door_ki_rele(mountd_dh);
-        mountd_dh = door_ki_lookup(did);
-        mutex_exit(&mountd_lock);
+        nfsauth_globals_t *nag;
+
+        nag = zone_getspecific(nfsauth_zone_key, curzone);
+        mutex_enter(&nag->mountd_lock);
+        if (nag->mountd_dh != NULL)
+                door_ki_rele(nag->mountd_dh);
+        nag->mountd_dh = door_ki_lookup(did);
+        mutex_exit(&nag->mountd_lock);
 }
 
 void
 nfsauth_init(void)
 {
-        /*
-         * mountd can be restarted by smf(5). We need to make sure
-         * the updated door handle will safely make it to mountd_dh
-         */
-        mutex_init(&mountd_lock, NULL, MUTEX_DEFAULT, NULL);
+        zone_key_create(&nfsauth_zone_key, nfsauth_zone_init,
+            nfsauth_zone_shutdown, nfsauth_zone_fini);
 
-        mutex_init(&refreshq_lock, NULL, MUTEX_DEFAULT, NULL);
-        list_create(&refreshq_queue, sizeof (refreshq_exi_node_t),
-            offsetof(refreshq_exi_node_t, ren_node));
-
-        cv_init(&refreshq_cv, NULL, CV_DEFAULT, NULL);
-
-        /*
-         * Allocate nfsauth cache handle
-         */
         exi_cache_handle = kmem_cache_create("exi_cache_handle",
             sizeof (struct auth_cache), 0, NULL, NULL,
             exi_cache_reclaim, NULL, NULL, 0);
-
-        refreshq_thread_state = REFRESHQ_THREAD_RUNNING;
-        (void) zthread_create(NULL, 0, nfsauth_refresh_thread,
-            NULL, 0, minclsyspri);
 }
 
-/*
- * Finalization routine for nfsauth. It is important to call this routine
- * before destroying the exported_lock.
- */
 void
 nfsauth_fini(void)
 {
-        refreshq_exi_node_t     *ren;
+        kmem_cache_destroy(exi_cache_handle);
+}
 
-        /*
-         * Prevent the nfsauth_refresh_thread from getting new
-         * work.
-         */
-        mutex_enter(&refreshq_lock);
-        if (refreshq_thread_state != REFRESHQ_THREAD_HALTED) {
-                refreshq_thread_state = REFRESHQ_THREAD_FINI_REQ;
-                cv_broadcast(&refreshq_cv);
+/*ARGSUSED*/
+static void *
+nfsauth_zone_init(zoneid_t zoneid)
+{
+        nfsauth_globals_t *nag;
 
+        nag = kmem_zalloc(sizeof (*nag), KM_SLEEP);
+
                 /*
-                 * Also, wait for nfsauth_refresh_thread() to exit.
+         * mountd can be restarted by smf(5).  We need to make sure
+         * the updated door handle will safely make it to mountd_dh.
                  */
-                while (refreshq_thread_state != REFRESHQ_THREAD_HALTED) {
-                        cv_wait(&refreshq_cv, &refreshq_lock);
+        mutex_init(&nag->mountd_lock, NULL, MUTEX_DEFAULT, NULL);
+        mutex_init(&nag->refreshq_lock, NULL, MUTEX_DEFAULT, NULL);
+        list_create(&nag->refreshq_queue, sizeof (refreshq_exi_node_t),
+            offsetof(refreshq_exi_node_t, ren_node));
+        cv_init(&nag->refreshq_cv, NULL, CV_DEFAULT, NULL);
+        nag->refreshq_thread_state = REFRESHQ_THREAD_NEED_CREATE;
+
+        return (nag);
+}
+
+/*ARGSUSED*/
+static void
+nfsauth_zone_shutdown(zoneid_t zoneid, void *data)
+{
+        refreshq_exi_node_t     *ren;
+        nfsauth_globals_t       *nag = data;
+
+        /* Prevent the nfsauth_refresh_thread from getting new work */
+        mutex_enter(&nag->refreshq_lock);
+        if (nag->refreshq_thread_state == REFRESHQ_THREAD_RUNNING) {
+                nag->refreshq_thread_state = REFRESHQ_THREAD_FINI_REQ;
+                cv_broadcast(&nag->refreshq_cv);
+
+                /* Wait for nfsauth_refresh_thread() to exit */
+                while (nag->refreshq_thread_state != REFRESHQ_THREAD_HALTED)
+                        cv_wait(&nag->refreshq_cv, &nag->refreshq_lock);
                 }
-        }
-        mutex_exit(&refreshq_lock);
+        mutex_exit(&nag->refreshq_lock);
 
         /*
          * Walk the exi_list and in turn, walk the auth_lists and free all
          * lists.  In addition, free INVALID auth_cache entries.
          */
-        while ((ren = list_remove_head(&refreshq_queue))) {
+        while ((ren = list_remove_head(&nag->refreshq_queue))) {
                 refreshq_auth_node_t *ran;
 
                 while ((ran = list_remove_head(&ren->ren_authlist)) != NULL) {
                         struct auth_cache *p = ran->ran_auth;
                         if (p->auth_state == NFS_AUTH_INVALID)
                                 nfsauth_free_node(p);
                         strfree(ran->ran_netid);
-                        kmem_free(ran, sizeof (refreshq_auth_node_t));
+                        kmem_free(ran, sizeof (*ran));
                 }
 
                 list_destroy(&ren->ren_authlist);
-                exi_rele(ren->ren_exi);
-                kmem_free(ren, sizeof (refreshq_exi_node_t));
+                exi_rele(&ren->ren_exi);
+                kmem_free(ren, sizeof (*ren));
         }
-        list_destroy(&refreshq_queue);
+}
 
-        cv_destroy(&refreshq_cv);
-        mutex_destroy(&refreshq_lock);
+/*ARGSUSED*/
+static void
+nfsauth_zone_fini(zoneid_t zoneid, void *data)
+{
+        nfsauth_globals_t *nag = data;
 
-        mutex_destroy(&mountd_lock);
-
-        /*
-         * Deallocate nfsauth cache handle
-         */
-        kmem_cache_destroy(exi_cache_handle);
+        list_destroy(&nag->refreshq_queue);
+        cv_destroy(&nag->refreshq_cv);
+        mutex_destroy(&nag->refreshq_lock);
+        mutex_destroy(&nag->mountd_lock);
+        kmem_free(nag, sizeof (*nag));
 }
 
 /*
  * Convert the address in a netbuf to
  * a hash index for the auth_cache table.

@@ -339,13 +373,14 @@
 
 /*
  * Callup to the mountd to get access information in the kernel.
  */
 static bool_t
-nfsauth_retrieve(struct exportinfo *exi, char *req_netid, int flavor,
-    struct netbuf *addr, int *access, cred_t *clnt_cred, uid_t *srv_uid,
-    gid_t *srv_gid, uint_t *srv_gids_cnt, gid_t **srv_gids)
+nfsauth_retrieve(nfsauth_globals_t *nag, struct exportinfo *exi,
+    char *req_netid, int flavor, struct netbuf *addr, int *access,
+    cred_t *clnt_cred, uid_t *srv_uid, gid_t *srv_gid, uint_t *srv_gids_cnt,
+    gid_t **srv_gids)
 {
         varg_t                    varg = {0};
         nfsauth_res_t             res = {0};
         XDR                       xdrs;
         size_t                    absz;

@@ -414,15 +449,15 @@
         da.desc_num = 0;
         da.rbuf = NULL;
         da.rsize = 1;
 
 retry:
-        mutex_enter(&mountd_lock);
-        dh = mountd_dh;
+        mutex_enter(&nag->mountd_lock);
+        dh = nag->mountd_dh;
         if (dh != NULL)
                 door_ki_hold(dh);
-        mutex_exit(&mountd_lock);
+        mutex_exit(&nag->mountd_lock);
 
         if (dh == NULL) {
                 /*
                  * The rendezvous point has not been established yet!
                  * This could mean that either mountd(1m) has not yet

@@ -488,16 +523,16 @@
                                  * the (existing) door on us; we
                                  * want to wait to give smf(5) a
                                  * chance to restart mountd(1m)
                                  * and establish a new door handle.
                                  */
-                                mutex_enter(&mountd_lock);
-                                if (dh == mountd_dh) {
-                                        door_ki_rele(mountd_dh);
-                                        mountd_dh = NULL;
+                                mutex_enter(&nag->mountd_lock);
+                                if (dh == nag->mountd_dh) {
+                                        door_ki_rele(nag->mountd_dh);
+                                        nag->mountd_dh = NULL;
                                 }
-                                mutex_exit(&mountd_lock);
+                                mutex_exit(&nag->mountd_lock);
                                 delay(hz);
                                 goto retry;
                         }
                         /*
                          * If the door was _not_ revoked on us,

@@ -585,11 +620,11 @@
 
         return (TRUE);
 }
 
 static void
-nfsauth_refresh_thread(void)
+nfsauth_refresh_thread(nfsauth_globals_t *nag)
 {
         refreshq_exi_node_t     *ren;
         refreshq_auth_node_t    *ran;
 
         struct exportinfo       *exi;

@@ -597,29 +632,29 @@
         int                     access;
         bool_t                  retrieval;
 
         callb_cpr_t             cprinfo;
 
-        CALLB_CPR_INIT(&cprinfo, &refreshq_lock, callb_generic_cpr,
+        CALLB_CPR_INIT(&cprinfo, &nag->refreshq_lock, callb_generic_cpr,
             "nfsauth_refresh");
 
         for (;;) {
-                mutex_enter(&refreshq_lock);
-                if (refreshq_thread_state != REFRESHQ_THREAD_RUNNING) {
+                mutex_enter(&nag->refreshq_lock);
+                if (nag->refreshq_thread_state != REFRESHQ_THREAD_RUNNING) {
                         /* Keep the hold on the lock! */
                         break;
                 }
 
-                ren = list_remove_head(&refreshq_queue);
+                ren = list_remove_head(&nag->refreshq_queue);
                 if (ren == NULL) {
                         CALLB_CPR_SAFE_BEGIN(&cprinfo);
-                        cv_wait(&refreshq_cv, &refreshq_lock);
-                        CALLB_CPR_SAFE_END(&cprinfo, &refreshq_lock);
-                        mutex_exit(&refreshq_lock);
+                        cv_wait(&nag->refreshq_cv, &nag->refreshq_lock);
+                        CALLB_CPR_SAFE_END(&cprinfo, &nag->refreshq_lock);
+                        mutex_exit(&nag->refreshq_lock);
                         continue;
                 }
-                mutex_exit(&refreshq_lock);
+                mutex_exit(&nag->refreshq_lock);
 
                 exi = ren->ren_exi;
                 ASSERT(exi != NULL);
 
                 /*

@@ -662,11 +697,12 @@
                          * will miss such advisory, nothing catastrophic
                          * happens: we will just spin longer here before the
                          * shutdown.
                          */
                         if (p->auth_state == NFS_AUTH_INVALID ||
-                            refreshq_thread_state != REFRESHQ_THREAD_RUNNING) {
+                            nag->refreshq_thread_state !=
+                            REFRESHQ_THREAD_RUNNING) {
                                 mutex_exit(&p->auth_lock);
 
                                 if (p->auth_state == NFS_AUTH_INVALID)
                                         nfsauth_free_node(p);
 

@@ -697,11 +733,11 @@
                          * the same netid. It doesn't matter. So
                          * when we refresh, we simply use the netid
                          * of the request which triggered the
                          * refresh attempt.
                          */
-                        retrieval = nfsauth_retrieve(exi, netid,
+                        retrieval = nfsauth_retrieve(nag, exi, netid,
                             p->auth_flavor, &p->auth_clnt->authc_addr, &access,
                             p->auth_clnt_cred, &uid, &gid, &ngids, &gids);
 
                         /*
                          * This can only be set in one other place

@@ -740,17 +776,18 @@
                                 mutex_exit(&p->auth_lock);
                         }
                 }
 
                 list_destroy(&ren->ren_authlist);
-                exi_rele(ren->ren_exi);
+                exi_rele(&ren->ren_exi);
                 kmem_free(ren, sizeof (refreshq_exi_node_t));
         }
 
-        refreshq_thread_state = REFRESHQ_THREAD_HALTED;
-        cv_broadcast(&refreshq_cv);
+        nag->refreshq_thread_state = REFRESHQ_THREAD_HALTED;
+        cv_broadcast(&nag->refreshq_cv);
         CALLB_CPR_EXIT(&cprinfo);
+        DTRACE_PROBE(nfsauth__nfsauth__refresh__thread__exit);
         zthread_exit();
 }
 
 int
 nfsauth_cache_clnt_compar(const void *v1, const void *v2)

@@ -818,10 +855,11 @@
  */
 static int
 nfsauth_cache_get(struct exportinfo *exi, struct svc_req *req, int flavor,
     cred_t *cr, uid_t *uid, gid_t *gid, uint_t *ngids, gid_t **gids)
 {
+        nfsauth_globals_t       *nag;
         struct netbuf           *taddrmask;
         struct netbuf           addr;   /* temporary copy of client's address */
         const struct netbuf     *claddr;
         avl_tree_t              *tree;
         struct auth_cache       ac;     /* used as a template for avl_find() */

@@ -837,10 +875,12 @@
 
         avl_index_t             where;  /* used for avl_find()/avl_insert() */
 
         ASSERT(cr != NULL);
 
+        nag = zone_getspecific(nfsauth_zone_key, curzone);
+
         /*
          * Now check whether this client already
          * has an entry for this flavor in the cache
          * for this export.
          * Get the caller's address, mask off the

@@ -996,12 +1036,13 @@
                 kmem_free(addr.buf, addr.maxlen);
                 addr = p->auth_clnt->authc_addr;
 
                 atomic_inc_uint(&nfsauth_cache_miss);
 
-                res = nfsauth_retrieve(exi, svc_getnetid(req->rq_xprt), flavor,
-                    &addr, &access, cr, &tmpuid, &tmpgid, &tmpngids, &tmpgids);
+                res = nfsauth_retrieve(nag, exi, svc_getnetid(req->rq_xprt),
+                    flavor, &addr, &access, cr, &tmpuid, &tmpgid, &tmpngids,
+                    &tmpgids);
 
                 p->auth_access = access;
                 p->auth_time = p->auth_freshness = gethrestime_sec();
 
                 if (res == TRUE) {

@@ -1078,25 +1119,37 @@
                         ran = kmem_alloc(sizeof (refreshq_auth_node_t),
                             KM_SLEEP);
                         ran->ran_auth = p;
                         ran->ran_netid = strdup(svc_getnetid(req->rq_xprt));
 
-                        mutex_enter(&refreshq_lock);
+                        mutex_enter(&nag->refreshq_lock);
+
+                        if (nag->refreshq_thread_state ==
+                            REFRESHQ_THREAD_NEED_CREATE) {
+                                /* Launch nfsauth refresh thread */
+                                nag->refreshq_thread_state =
+                                    REFRESHQ_THREAD_RUNNING;
+                                (void) zthread_create(NULL, 0,
+                                    nfsauth_refresh_thread, nag, 0,
+                                    minclsyspri);
+                        }
+
                         /*
-                         * We should not add a work queue
-                         * item if the thread is not
-                         * accepting them.
+                         * We should not add a work queue item if the thread
+                         * is not accepting them.
                          */
-                        if (refreshq_thread_state == REFRESHQ_THREAD_RUNNING) {
+                        if (nag->refreshq_thread_state ==
+                            REFRESHQ_THREAD_RUNNING) {
                                 refreshq_exi_node_t *ren;
 
                                 /*
                                  * Is there an existing exi_list?
                                  */
-                                for (ren = list_head(&refreshq_queue);
+                                for (ren = list_head(&nag->refreshq_queue);
                                     ren != NULL;
-                                    ren = list_next(&refreshq_queue, ren)) {
+                                    ren = list_next(&nag->refreshq_queue,
+                                    ren)) {
                                         if (ren->ren_exi == exi) {
                                                 list_insert_tail(
                                                     &ren->ren_authlist, ran);
                                                 break;
                                         }

@@ -1115,20 +1168,21 @@
                                             offsetof(refreshq_auth_node_t,
                                             ran_node));
 
                                         list_insert_tail(&ren->ren_authlist,
                                             ran);
-                                        list_insert_tail(&refreshq_queue, ren);
+                                        list_insert_tail(&nag->refreshq_queue,
+                                            ren);
                                 }
 
-                                cv_broadcast(&refreshq_cv);
+                                cv_broadcast(&nag->refreshq_cv);
                         } else {
                                 strfree(ran->ran_netid);
                                 kmem_free(ran, sizeof (refreshq_auth_node_t));
                         }
 
-                        mutex_exit(&refreshq_lock);
+                        mutex_exit(&nag->refreshq_lock);
                 } else {
                         mutex_exit(&p->auth_lock);
                 }
 
                 nach = atomic_inc_uint_nv(&nfsauth_cache_hit);

@@ -1150,12 +1204,12 @@
 
         ASSERT(p == NULL);
 
         atomic_inc_uint(&nfsauth_cache_miss);
 
-        if (nfsauth_retrieve(exi, svc_getnetid(req->rq_xprt), flavor, &addr,
-            &access, cr, &tmpuid, &tmpgid, &tmpngids, &tmpgids)) {
+        if (nfsauth_retrieve(nag, exi, svc_getnetid(req->rq_xprt), flavor,
+            &addr, &access, cr, &tmpuid, &tmpgid, &tmpngids, &tmpgids)) {
                 if (uid != NULL)
                         *uid = tmpuid;
                 if (gid != NULL)
                         *gid = tmpgid;
                 if (ngids != NULL && gids != NULL) {

@@ -1409,20 +1463,20 @@
 void
 exi_cache_reclaim(void *cdrarg)
 {
         int i;
         struct exportinfo *exi;
+        nfs_export_t *ne = nfs_get_export();
 
-        rw_enter(&exported_lock, RW_READER);
+        rw_enter(&ne->exported_lock, RW_READER);
 
         for (i = 0; i < EXPTABLESIZE; i++) {
-                for (exi = exptable[i]; exi; exi = exi->fid_hash.next) {
+                for (exi = ne->exptable[i]; exi; exi = exi->fid_hash.next)
                         exi_cache_trim(exi);
                 }
-        }
 
-        rw_exit(&exported_lock);
+        rw_exit(&ne->exported_lock);
 
         atomic_inc_uint(&nfsauth_cache_reclaim);
 }
 
 void