2015-08-20 11:17:27

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 00/20] nfsd: open file caching

v3:
- open files are now hashed on inode pointer instead of fh
- eliminate the recurring workqueue job in favor of shrinker/LRU and
notifier from lease setting code
- have nfsv4 use the cache as well
- removal of raparms cache

v2:
- changelog cleanups and clarifications
- allow COMMIT to use cached open files
- tracepoints for nfsd_file cache
- proactively close open files prior to REMOVE, or a RENAME over a
positive dentry

This is the third iteration of the open file cache for knfsd. This one
has some major changes from the last revision. The files are now hashed
on inode pointer instead of the filehandle. An inode can have several
filehandles, and we really only do want to open it once. I've dropped a
lot of the filehandle manipulation patches from the last set since they
are no longer needed here.

I've also removed the recurring workqueue job that cleans out the cache
in favor of a scheme that uses a LRU list and shrinker, plus a new
notifier chain in the lease setting code. With this, knfsd will
basically keep files open indefinitely as long as memory is available
and no one wants to set a lease on the file, or until the exports cache
is flushed. I've dropped most of the changes to the laundry_wq, but
I did leave in the patch that changes it to allow multiple jobs to
run in parallel.

Finally, the other big change is that I've gone ahead and hooked up NFSv4
to use this cache as well. This allows us to finally rip out the raparms
cache which is done in the last patch.

Original cover letter follows:

---------------------[snip]------------------------

Hi Bruce!

This patchset adds a new open file cache for knfsd. As you well know,
nfsd basically does an open() - read/write() - close() cycle for every
nfsv3 READ or WRITE. It's also common for clients to "spray" several
read and write requests in parallel or in quick succession, so we could
skip a lot of that by simply caching these open filps.

The idea here is to cache them in a hashtable for a little while (1s by
default) in the expectation that clients may try to issue more reads or
writes in quick succession. When there are any entries in the hashtable,
there is a recurring workqueue job that will clean the cache.

I've also added some hooks into sunrpc cache code that should allow us
to purge the cache on an unexport event, so this shouldn't cause any
problems with unmounting once you've unexported the fs.

I did a little testing with it, but my test rig is pretty slow, and I
couldn't measure much of a performance difference on a bog standard
local fs.

We do have some patches that allow the reexporting of NFSv4.1 via knfsd.
Since NFS has a relatively slow open routine, this provides a rather
large speedup.

Without these patches:
$ dd if=/dev/urandom of=/mnt/dp01/ddfile bs=4k count=256 oflag=direct
256+0 records in
256+0 records out
1048576 bytes (1.0 MB) copied, 54.3109 s, 19.3 kB/s

With these patches:
$ dd if=/dev/urandom of=/mnt/dp01/ddfile bs=4k count=256 oflag=direct
256+0 records in
256+0 records out
1048576 bytes (1.0 MB) copied, 1.05437 s, 995 kB/s

It should also be possible to hook this code up to the nfs4_file too,
but I haven't done that in this set. I'd like to get this in and settled
before we start looking at that, since it'll mean a bit of reengineering
of the NFSv4 code not to pass around struct file pointers.

I'd like to have these considered for the v4.3 merge window if they look
reasonable.

Jeff Layton (20):
nfsd: allow more than one laundry job to run at a time
nfsd: add a new struct file caching facility to nfsd
list_lru: add list_lru_rotate
nfsd: add a LRU list for nfsd_files
nfsd: add a shrinker to the nfsd_file cache
locks/nfsd: create a new notifier chain for lease attempts
nfsd: hook up nfsd_write to the new nfsd_file cache
nfsd: hook up nfsd_read to the nfsd_file cache
sunrpc: add a new cache_detail operation for when a cache is flushed
nfsd: handle NFSD_MAY_NOT_BREAK_LEASE in open file cache
nfsd: hook nfsd_commit up to the nfsd_file cache
nfsd: move include of state.h from trace.c to trace.h
nfsd: add new tracepoints for nfsd_file cache
nfsd: close cached files prior to a REMOVE or RENAME that would
replace target
nfsd: call flush_delayed_fput from nfsd_file_close_fh
nfsd: convert nfs4_file->fi_fds array to use nfsd_files
nfsd: have nfsd_test_lock use the nfsd_file cache
nfsd: convert fi_deleg_file and ls_file fields to nfsd_file
nfsd: hook up nfs4_preprocess_stateid_op to the nfsd_file cache
nfsd: rip out the raparms cache

fs/file_table.c | 1 +
fs/locks.c | 15 ++
fs/nfsd/Makefile | 3 +-
fs/nfsd/filecache.c | 465 +++++++++++++++++++++++++++++++++++++++++++
fs/nfsd/filecache.h | 33 +++
fs/nfsd/nfs3proc.c | 2 +-
fs/nfsd/nfs4layouts.c | 12 +-
fs/nfsd/nfs4proc.c | 32 +--
fs/nfsd/nfs4state.c | 178 ++++++++---------
fs/nfsd/nfs4xdr.c | 16 +-
fs/nfsd/nfsproc.c | 2 +-
fs/nfsd/nfssvc.c | 16 +-
fs/nfsd/state.h | 10 +-
fs/nfsd/trace.c | 2 -
fs/nfsd/trace.h | 118 +++++++++++
fs/nfsd/vfs.c | 282 ++++++++------------------
fs/nfsd/vfs.h | 8 +-
fs/nfsd/xdr4.h | 15 +-
include/linux/fs.h | 1 +
include/linux/list_lru.h | 13 ++
include/linux/sunrpc/cache.h | 1 +
mm/list_lru.c | 15 ++
net/sunrpc/cache.c | 3 +
23 files changed, 886 insertions(+), 357 deletions(-)
create mode 100644 fs/nfsd/filecache.c
create mode 100644 fs/nfsd/filecache.h

--
2.4.3



2015-08-20 11:17:28

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 01/20] nfsd: allow more than one laundry job to run at a time

We can potentially have several nfs4_laundromat jobs running if there
are multiple namespaces running nfsd on the box. Those are effectively
separated from one another though, so I don't see any reason to
serialize them.

Also, create_singlethread_workqueue automatically adds the
WQ_MEM_RECLAIM flag. Since we run this job on a timer, it's not really
involved in any reclaim paths. I see no need for a rescuer thread.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/nfs4state.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fs/nfsd/nfs4state.c b/fs/nfsd/nfs4state.c
index c0c47a878cc6..af88d1d7ccae 100644
--- a/fs/nfsd/nfs4state.c
+++ b/fs/nfsd/nfs4state.c
@@ -6598,7 +6598,7 @@ nfs4_state_start(void)
ret = set_callback_cred();
if (ret)
return -ENOMEM;
- laundry_wq = create_singlethread_workqueue("nfsd4");
+ laundry_wq = alloc_workqueue("%s", WQ_UNBOUND, 0, "nfsd4");
if (laundry_wq == NULL) {
ret = -ENOMEM;
goto out_recovery;
--
2.4.3


2015-08-20 11:17:29

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 02/20] nfsd: add a new struct file caching facility to nfsd

Currently, NFSv2/3 reads and writes have to open a file, do the read or
write and then close it again for each RPC. This is highly inefficient,
especially when the underlying filesystem has a relatively slow open
routine.

This patch adds a new open file cache to knfsd. Rather than doing an
open for each RPC, the read/write handlers can call into this cache to
see if there is one already there for the correct filehandle and
NFS_MAY_READ/WRITE flags.

If there isn't an entry, then we create a new one and attempt to
perform the open. If there is, then we wait until the entry is fully
instantiated and return it if it is at the end of the wait. If it's
not, then we attempt to take over construction.

Since the main goal is to speed up NFSv2/3 I/O, we don't want to
close these files on last put of these objects. We need to keep them
around for a little while since we never know when the next READ/WRITE
will come in.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/Makefile | 3 +-
fs/nfsd/filecache.c | 273 ++++++++++++++++++++++++++++++++++++++++++++++++++++
fs/nfsd/filecache.h | 29 ++++++
fs/nfsd/nfssvc.c | 10 +-
4 files changed, 313 insertions(+), 2 deletions(-)
create mode 100644 fs/nfsd/filecache.c
create mode 100644 fs/nfsd/filecache.h

diff --git a/fs/nfsd/Makefile b/fs/nfsd/Makefile
index 9a6028e120c6..8908bb467727 100644
--- a/fs/nfsd/Makefile
+++ b/fs/nfsd/Makefile
@@ -10,7 +10,8 @@ obj-$(CONFIG_NFSD) += nfsd.o
nfsd-y += trace.o

nfsd-y += nfssvc.o nfsctl.o nfsproc.o nfsfh.o vfs.o \
- export.o auth.o lockd.o nfscache.o nfsxdr.o stats.o
+ export.o auth.o lockd.o nfscache.o nfsxdr.o \
+ stats.o filecache.o
nfsd-$(CONFIG_NFSD_FAULT_INJECTION) += fault_inject.o
nfsd-$(CONFIG_NFSD_V2_ACL) += nfs2acl.o
nfsd-$(CONFIG_NFSD_V3) += nfs3proc.o nfs3xdr.o
diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
new file mode 100644
index 000000000000..5bb56fa9002f
--- /dev/null
+++ b/fs/nfsd/filecache.c
@@ -0,0 +1,273 @@
+/*
+ * Open file cache.
+ *
+ * (c) 2015 - Jeff Layton <[email protected]>
+ */
+
+#include <linux/hash.h>
+#include <linux/slab.h>
+#include <linux/hash.h>
+#include <linux/file.h>
+#include <linux/sched.h>
+
+#include "vfs.h"
+#include "nfsd.h"
+#include "nfsfh.h"
+#include "filecache.h"
+
+#define NFSDDBG_FACILITY NFSDDBG_FH
+
+/* hash table for nfs4_file */
+#define NFSD_FILE_HASH_BITS 8
+#define NFSD_FILE_HASH_SIZE (1 << NFSD_FILE_HASH_BITS)
+
+/* We only care about NFSD_MAY_READ/WRITE for this cache */
+#define NFSD_FILE_MAY_MASK (NFSD_MAY_READ|NFSD_MAY_WRITE)
+
+struct nfsd_fcache_bucket {
+ struct hlist_head nfb_head;
+ spinlock_t nfb_lock;
+};
+
+static struct nfsd_fcache_bucket *nfsd_file_hashtbl;
+
+static struct nfsd_file *
+nfsd_file_alloc(struct inode *inode, unsigned int may, unsigned int hashval)
+{
+ struct nfsd_file *nf;
+
+ /* FIXME: create a new slabcache for these? */
+ nf = kzalloc(sizeof(*nf), GFP_KERNEL);
+ if (nf) {
+ INIT_HLIST_NODE(&nf->nf_node);
+ INIT_LIST_HEAD(&nf->nf_dispose);
+ nf->nf_inode = inode;
+ nf->nf_hashval = hashval;
+ atomic_set(&nf->nf_ref, 1);
+ nf->nf_may = NFSD_FILE_MAY_MASK & may;
+ }
+ return nf;
+}
+
+static void
+nfsd_file_put_final(struct nfsd_file *nf)
+{
+ if (nf->nf_file)
+ fput(nf->nf_file);
+ kfree_rcu(nf, nf_rcu);
+}
+
+static bool
+nfsd_file_unhash(struct nfsd_file *nf)
+{
+ lockdep_assert_held(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);
+
+ if (test_bit(NFSD_FILE_HASHED, &nf->nf_flags)) {
+ clear_bit(NFSD_FILE_HASHED, &nf->nf_flags);
+ hlist_del_rcu(&nf->nf_node);
+ return true;
+ }
+ return false;
+}
+
+static void
+nfsd_file_unhash_and_release_locked(struct nfsd_file *nf, struct list_head *dispose)
+{
+ lockdep_assert_held(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);
+
+ if (!nfsd_file_unhash(nf))
+ return;
+ if (!atomic_dec_and_test(&nf->nf_ref))
+ return;
+
+ list_add(&nf->nf_dispose, dispose);
+}
+
+void
+nfsd_file_put(struct nfsd_file *nf)
+{
+ if (!atomic_dec_and_test(&nf->nf_ref))
+ return;
+
+ WARN_ON(test_bit(NFSD_FILE_HASHED, &nf->nf_flags));
+ nfsd_file_put_final(nf);
+}
+
+struct nfsd_file *
+nfsd_file_get(struct nfsd_file *nf)
+{
+ if (likely(atomic_inc_not_zero(&nf->nf_ref)))
+ return nf;
+ return NULL;
+}
+
+static void
+nfsd_file_dispose_list(struct list_head *dispose)
+{
+ struct nfsd_file *nf;
+
+ while(!list_empty(dispose)) {
+ nf = list_first_entry(dispose, struct nfsd_file, nf_dispose);
+ list_del(&nf->nf_dispose);
+ nfsd_file_put_final(nf);
+ }
+}
+
+int
+nfsd_file_cache_init(void)
+{
+ unsigned int i;
+
+ if (nfsd_file_hashtbl)
+ return 0;
+
+ nfsd_file_hashtbl = kcalloc(NFSD_FILE_HASH_SIZE,
+ sizeof(*nfsd_file_hashtbl), GFP_KERNEL);
+ if (!nfsd_file_hashtbl)
+ goto out_nomem;
+
+ for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
+ INIT_HLIST_HEAD(&nfsd_file_hashtbl[i].nfb_head);
+ spin_lock_init(&nfsd_file_hashtbl[i].nfb_lock);
+ }
+
+ return 0;
+out_nomem:
+ printk(KERN_ERR "nfsd: failed to init nfsd file cache\n");
+ return -ENOMEM;
+}
+
+void
+nfsd_file_cache_shutdown(void)
+{
+ unsigned int i;
+ struct nfsd_file *nf;
+ LIST_HEAD(dispose);
+
+ for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
+ spin_lock(&nfsd_file_hashtbl[i].nfb_lock);
+ while(!hlist_empty(&nfsd_file_hashtbl[i].nfb_head)) {
+ nf = hlist_entry(nfsd_file_hashtbl[i].nfb_head.first,
+ struct nfsd_file, nf_node);
+ nfsd_file_unhash_and_release_locked(nf, &dispose);
+ }
+ spin_unlock(&nfsd_file_hashtbl[i].nfb_lock);
+ nfsd_file_dispose_list(&dispose);
+ }
+ kfree(nfsd_file_hashtbl);
+ nfsd_file_hashtbl = NULL;
+}
+
+/*
+ * Search nfsd_file_hashtbl[] for file. We hash on the filehandle and also on
+ * the NFSD_MAY_READ/WRITE flags. If the file is open for r/w, then it's usable
+ * for either.
+ */
+static struct nfsd_file *
+nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
+ unsigned int hashval)
+{
+ struct nfsd_file *nf;
+ unsigned char need = may_flags & NFSD_FILE_MAY_MASK;
+
+ hlist_for_each_entry_rcu(nf, &nfsd_file_hashtbl[hashval].nfb_head,
+ nf_node) {
+ if ((need & nf->nf_may) != need)
+ continue;
+ if (nf->nf_inode == inode)
+ return nfsd_file_get(nf);
+ }
+ return NULL;
+}
+
+__be32
+nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
+ unsigned int may_flags, struct nfsd_file **pnf)
+{
+ __be32 status = nfs_ok;
+ struct nfsd_file *nf, *new = NULL;
+ struct inode *inode;
+ unsigned int hashval;
+
+ /* FIXME: skip this if fh_dentry is already set? */
+ status = fh_verify(rqstp, fhp, S_IFREG, may_flags);
+ if (status != nfs_ok)
+ return status;
+
+ /* Mask off any extraneous bits */
+ may_flags &= NFSD_FILE_MAY_MASK;
+
+ inode = d_inode(fhp->fh_dentry);
+ hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
+retry:
+ rcu_read_lock();
+ nf = nfsd_file_find_locked(inode, may_flags, hashval);
+ rcu_read_unlock();
+ if (nf)
+ goto wait_for_construction;
+
+ if (!new) {
+ new = nfsd_file_alloc(inode, may_flags, hashval);
+ if (!new)
+ return nfserr_jukebox;
+ }
+
+ spin_lock(&nfsd_file_hashtbl[hashval].nfb_lock);
+ nf = nfsd_file_find_locked(inode, may_flags, hashval);
+ if (likely(nf == NULL)) {
+ /* Take reference for the hashtable */
+ atomic_inc(&new->nf_ref);
+ __set_bit(NFSD_FILE_HASHED, &new->nf_flags);
+ __set_bit(NFSD_FILE_PENDING, &new->nf_flags);
+ hlist_add_head_rcu(&new->nf_node,
+ &nfsd_file_hashtbl[hashval].nfb_head);
+ spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
+ nf = new;
+ new = NULL;
+ goto open_file;
+ }
+ spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
+
+wait_for_construction:
+ wait_on_bit(&nf->nf_flags, NFSD_FILE_PENDING, TASK_UNINTERRUPTIBLE);
+
+ /* Did construction of this file fail? */
+ if (!nf->nf_file) {
+ /*
+ * We can only take over construction for this nfsd_file if the
+ * MAY flags are equal. Otherwise, we put the reference and try
+ * again.
+ */
+ if (may_flags != nf->nf_may) {
+ nfsd_file_put(nf);
+ goto retry;
+ }
+
+ /* try to take over construction for this file */
+ if (test_and_set_bit(NFSD_FILE_PENDING, &nf->nf_flags))
+ goto wait_for_construction;
+ goto open_file;
+ }
+
+ /*
+ * We have a file that was opened in the context of another rqst. We
+ * must check permissions. Since we're dealing with open files here,
+ * we always want to set the OWNER_OVERRIDE bit.
+ */
+ status = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
+ may_flags|NFSD_MAY_OWNER_OVERRIDE);
+out:
+ if (status == nfs_ok)
+ *pnf = nf;
+ else
+ nfsd_file_put(nf);
+
+ if (new)
+ nfsd_file_put(new);
+ return status;
+open_file:
+ status = nfsd_open(rqstp, fhp, S_IFREG, may_flags, &nf->nf_file);
+ clear_bit(NFSD_FILE_PENDING, &nf->nf_flags);
+ wake_up_bit(&nf->nf_flags, NFSD_FILE_PENDING);
+ goto out;
+}
diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
new file mode 100644
index 000000000000..b0f500353ed4
--- /dev/null
+++ b/fs/nfsd/filecache.h
@@ -0,0 +1,29 @@
+#ifndef _FS_NFSD_FILECACHE_H
+#define _FS_NFSD_FILECACHE_H
+/*
+ * A representation of a file that has been opened by knfsd. These are hashed
+ * in the hashtable by inode pointer value. Note that this object doesn't
+ * hold a reference to the inode by itself, so the nf_inode pointer should
+ * never be dereferenced, only be used for comparison.
+ */
+struct nfsd_file {
+ struct hlist_node nf_node;
+ struct list_head nf_dispose;
+ struct rcu_head nf_rcu;
+ struct file *nf_file;
+#define NFSD_FILE_HASHED (0)
+#define NFSD_FILE_PENDING (1)
+ unsigned long nf_flags;
+ struct inode *nf_inode;
+ unsigned int nf_hashval;
+ atomic_t nf_ref;
+ unsigned char nf_may;
+};
+
+int nfsd_file_cache_init(void);
+void nfsd_file_cache_shutdown(void);
+void nfsd_file_put(struct nfsd_file *nf);
+struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
+__be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
+ unsigned int may_flags, struct nfsd_file **nfp);
+#endif /* _FS_NFSD_FILECACHE_H */
diff --git a/fs/nfsd/nfssvc.c b/fs/nfsd/nfssvc.c
index ad4e2377dd63..d816bb3faa6e 100644
--- a/fs/nfsd/nfssvc.c
+++ b/fs/nfsd/nfssvc.c
@@ -22,6 +22,7 @@
#include "cache.h"
#include "vfs.h"
#include "netns.h"
+#include "filecache.h"

#define NFSDDBG_FACILITY NFSDDBG_SVC

@@ -224,11 +225,17 @@ static int nfsd_startup_generic(int nrservs)
if (ret)
goto dec_users;

- ret = nfs4_state_start();
+ ret = nfsd_file_cache_init();
if (ret)
goto out_racache;
+
+ ret = nfs4_state_start();
+ if (ret)
+ goto out_file_cache;
return 0;

+out_file_cache:
+ nfsd_file_cache_shutdown();
out_racache:
nfsd_racache_shutdown();
dec_users:
@@ -242,6 +249,7 @@ static void nfsd_shutdown_generic(void)
return;

nfs4_state_shutdown();
+ nfsd_file_cache_shutdown();
nfsd_racache_shutdown();
}

--
2.4.3


2015-08-20 11:17:30

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 03/20] list_lru: add list_lru_rotate

Add a function that can move an entry to the MRU end of the list.

Cc: Andrew Morton <[email protected]>
Cc: Vladimir Davydov <[email protected]>
Cc: [email protected]
Signed-off-by: Jeff Layton <[email protected]>
---
include/linux/list_lru.h | 13 +++++++++++++
mm/list_lru.c | 15 +++++++++++++++
2 files changed, 28 insertions(+)

diff --git a/include/linux/list_lru.h b/include/linux/list_lru.h
index 2a6b9947aaa3..4534b1b34d2d 100644
--- a/include/linux/list_lru.h
+++ b/include/linux/list_lru.h
@@ -96,6 +96,19 @@ bool list_lru_add(struct list_lru *lru, struct list_head *item);
bool list_lru_del(struct list_lru *lru, struct list_head *item);

/**
+ * list_lru_rotate: rotate an element to the end of an lru list
+ * @list_lru: the lru pointer
+ * @item: the item to be rotated
+ *
+ * This function moves an entry to the end of an LRU list. Should be used when
+ * an entry that is on the LRU is used, and should be moved to the MRU end of
+ * the list. If the item is not on a list, then this function has no effect.
+ * The comments about an element already pertaining to a list are also valid
+ * for list_lru_rotate.
+ */
+void list_lru_rotate(struct list_lru *lru, struct list_head *item);
+
+/**
* list_lru_count_one: return the number of objects currently held by @lru
* @lru: the lru pointer.
* @nid: the node id to count from.
diff --git a/mm/list_lru.c b/mm/list_lru.c
index e1da19fac1b3..66718c2a9a7b 100644
--- a/mm/list_lru.c
+++ b/mm/list_lru.c
@@ -130,6 +130,21 @@ bool list_lru_del(struct list_lru *lru, struct list_head *item)
}
EXPORT_SYMBOL_GPL(list_lru_del);

+void list_lru_rotate(struct list_lru *lru, struct list_head *item)
+{
+ int nid = page_to_nid(virt_to_page(item));
+ struct list_lru_node *nlru = &lru->node[nid];
+ struct list_lru_one *l;
+
+ spin_lock(&nlru->lock);
+ if (!list_empty(item)) {
+ l = list_lru_from_kmem(nlru, item);
+ list_move_tail(item, &l->list);
+ }
+ spin_unlock(&nlru->lock);
+}
+EXPORT_SYMBOL_GPL(list_lru_rotate);
+
void list_lru_isolate(struct list_lru_one *list, struct list_head *item)
{
list_del_init(item);
--
2.4.3


2015-08-20 11:17:31

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 04/20] nfsd: add a LRU list for nfsd_files

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/filecache.c | 41 +++++++++++++++++++++++++++++------------
fs/nfsd/filecache.h | 2 +-
2 files changed, 30 insertions(+), 13 deletions(-)

diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
index 5bb56fa9002f..63b24419c674 100644
--- a/fs/nfsd/filecache.c
+++ b/fs/nfsd/filecache.c
@@ -9,6 +9,7 @@
#include <linux/hash.h>
#include <linux/file.h>
#include <linux/sched.h>
+#include <linux/list_lru.h>

#include "vfs.h"
#include "nfsd.h"
@@ -30,6 +31,7 @@ struct nfsd_fcache_bucket {
};

static struct nfsd_fcache_bucket *nfsd_file_hashtbl;
+static struct list_lru nfsd_file_lru;

static struct nfsd_file *
nfsd_file_alloc(struct inode *inode, unsigned int may, unsigned int hashval)
@@ -40,7 +42,7 @@ nfsd_file_alloc(struct inode *inode, unsigned int may, unsigned int hashval)
nf = kzalloc(sizeof(*nf), GFP_KERNEL);
if (nf) {
INIT_HLIST_NODE(&nf->nf_node);
- INIT_LIST_HEAD(&nf->nf_dispose);
+ INIT_LIST_HEAD(&nf->nf_lru);
nf->nf_inode = inode;
nf->nf_hashval = hashval;
atomic_set(&nf->nf_ref, 1);
@@ -65,6 +67,7 @@ nfsd_file_unhash(struct nfsd_file *nf)
if (test_bit(NFSD_FILE_HASHED, &nf->nf_flags)) {
clear_bit(NFSD_FILE_HASHED, &nf->nf_flags);
hlist_del_rcu(&nf->nf_node);
+ list_lru_del(&nfsd_file_lru, &nf->nf_lru);
return true;
}
return false;
@@ -80,12 +83,14 @@ nfsd_file_unhash_and_release_locked(struct nfsd_file *nf, struct list_head *disp
if (!atomic_dec_and_test(&nf->nf_ref))
return;

- list_add(&nf->nf_dispose, dispose);
+ list_add(&nf->nf_lru, dispose);
}

void
nfsd_file_put(struct nfsd_file *nf)
{
+ list_lru_rotate(&nfsd_file_lru, &nf->nf_lru);
+
if (!atomic_dec_and_test(&nf->nf_ref))
return;

@@ -107,8 +112,8 @@ nfsd_file_dispose_list(struct list_head *dispose)
struct nfsd_file *nf;

while(!list_empty(dispose)) {
- nf = list_first_entry(dispose, struct nfsd_file, nf_dispose);
- list_del(&nf->nf_dispose);
+ nf = list_first_entry(dispose, struct nfsd_file, nf_lru);
+ list_del(&nf->nf_lru);
nfsd_file_put_final(nf);
}
}
@@ -116,25 +121,35 @@ nfsd_file_dispose_list(struct list_head *dispose)
int
nfsd_file_cache_init(void)
{
- unsigned int i;
+ int ret = -ENOMEM;
+ unsigned int i;

if (nfsd_file_hashtbl)
return 0;

nfsd_file_hashtbl = kcalloc(NFSD_FILE_HASH_SIZE,
sizeof(*nfsd_file_hashtbl), GFP_KERNEL);
- if (!nfsd_file_hashtbl)
- goto out_nomem;
+ if (!nfsd_file_hashtbl) {
+ pr_err("nfsd: unable to allocate nfsd_file_hashtbl\n");
+ goto out_err;
+ }
+
+ ret = list_lru_init(&nfsd_file_lru);
+ if (ret) {
+ pr_err("nfsd: failed to init nfsd_file_lru: %d\n", ret);
+ goto out_err;
+ }

for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
INIT_HLIST_HEAD(&nfsd_file_hashtbl[i].nfb_head);
spin_lock_init(&nfsd_file_hashtbl[i].nfb_lock);
}
-
- return 0;
-out_nomem:
- printk(KERN_ERR "nfsd: failed to init nfsd file cache\n");
- return -ENOMEM;
+out:
+ return ret;
+out_err:
+ kfree(nfsd_file_hashtbl);
+ nfsd_file_hashtbl = NULL;
+ goto out;
}

void
@@ -154,6 +169,7 @@ nfsd_file_cache_shutdown(void)
spin_unlock(&nfsd_file_hashtbl[i].nfb_lock);
nfsd_file_dispose_list(&dispose);
}
+ list_lru_destroy(&nfsd_file_lru);
kfree(nfsd_file_hashtbl);
nfsd_file_hashtbl = NULL;
}
@@ -219,6 +235,7 @@ retry:
atomic_inc(&new->nf_ref);
__set_bit(NFSD_FILE_HASHED, &new->nf_flags);
__set_bit(NFSD_FILE_PENDING, &new->nf_flags);
+ list_lru_add(&nfsd_file_lru, &new->nf_lru);
hlist_add_head_rcu(&new->nf_node,
&nfsd_file_hashtbl[hashval].nfb_head);
spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
index b0f500353ed4..e7da463446dd 100644
--- a/fs/nfsd/filecache.h
+++ b/fs/nfsd/filecache.h
@@ -8,7 +8,7 @@
*/
struct nfsd_file {
struct hlist_node nf_node;
- struct list_head nf_dispose;
+ struct list_head nf_lru;
struct rcu_head nf_rcu;
struct file *nf_file;
#define NFSD_FILE_HASHED (0)
--
2.4.3


2015-08-20 11:17:32

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 05/20] nfsd: add a shrinker to the nfsd_file cache

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/filecache.c | 49 +++++++++++++++++++++++++++++++++++++++++++++++++
1 file changed, 49 insertions(+)

diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
index 63b24419c674..669e62f6f4f6 100644
--- a/fs/nfsd/filecache.c
+++ b/fs/nfsd/filecache.c
@@ -118,6 +118,46 @@ nfsd_file_dispose_list(struct list_head *dispose)
}
}

+static enum lru_status
+nfsd_file_lru_cb(struct list_head *item, struct list_lru_one *lru,
+ spinlock_t *lock, void *arg)
+ __releases(lock)
+ __acquires(lock)
+{
+ struct nfsd_file *nf = list_entry(item, struct nfsd_file, nf_lru);
+ bool unhashed;
+
+ if (atomic_read(&nf->nf_ref) > 1)
+ return LRU_SKIP;
+
+ spin_unlock(lock);
+ spin_lock(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);
+ unhashed = nfsd_file_unhash(nf);
+ spin_unlock(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);
+ if (unhashed)
+ nfsd_file_put(nf);
+ spin_lock(lock);
+ return unhashed ? LRU_REMOVED_RETRY : LRU_RETRY;
+}
+
+static unsigned long
+nfsd_file_lru_count(struct shrinker *s, struct shrink_control *sc)
+{
+ return list_lru_count(&nfsd_file_lru);
+}
+
+static unsigned long
+nfsd_file_lru_scan(struct shrinker *s, struct shrink_control *sc)
+{
+ return list_lru_shrink_walk(&nfsd_file_lru, sc, nfsd_file_lru_cb, NULL);
+}
+
+static struct shrinker nfsd_file_shrinker = {
+ .scan_objects = nfsd_file_lru_scan,
+ .count_objects = nfsd_file_lru_count,
+ .seeks = 1,
+};
+
int
nfsd_file_cache_init(void)
{
@@ -140,12 +180,20 @@ nfsd_file_cache_init(void)
goto out_err;
}

+ ret = register_shrinker(&nfsd_file_shrinker);
+ if (ret) {
+ pr_err("nfsd: failed to register nfsd_file_shrinker: %d\n", ret);
+ goto out_lru;
+ }
+
for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
INIT_HLIST_HEAD(&nfsd_file_hashtbl[i].nfb_head);
spin_lock_init(&nfsd_file_hashtbl[i].nfb_lock);
}
out:
return ret;
+out_lru:
+ list_lru_destroy(&nfsd_file_lru);
out_err:
kfree(nfsd_file_hashtbl);
nfsd_file_hashtbl = NULL;
@@ -159,6 +207,7 @@ nfsd_file_cache_shutdown(void)
struct nfsd_file *nf;
LIST_HEAD(dispose);

+ unregister_shrinker(&nfsd_file_shrinker);
for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
spin_lock(&nfsd_file_hashtbl[i].nfb_lock);
while(!hlist_empty(&nfsd_file_hashtbl[i].nfb_head)) {
--
2.4.3


2015-08-20 11:17:33

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 06/20] locks/nfsd: create a new notifier chain for lease attempts

With the new file caching infrastructure in nfsd, we can end up holding
files open for an indefinite period of time, even when they are still
idle. This may prevent the kernel from handing out leases on the file,
which we don't really want to block.

Fix this by running a blocking notifier call chain whenever on any
lease attempt. nfsd can then purge the cache for that inode before
returning.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/locks.c | 15 +++++++++++++++
fs/nfsd/filecache.c | 27 +++++++++++++++++++++++++++
include/linux/fs.h | 1 +
3 files changed, 43 insertions(+)

diff --git a/fs/locks.c b/fs/locks.c
index d3d558ba4da7..c81b96159e5c 100644
--- a/fs/locks.c
+++ b/fs/locks.c
@@ -167,6 +167,13 @@ DEFINE_STATIC_LGLOCK(file_lock_lglock);
static DEFINE_PER_CPU(struct hlist_head, file_lock_list);

/*
+ * Some subsystems would like to be notified if someone attempts to set a
+ * lease on a file. This notifier chain will be called whenever this occurs.
+ */
+BLOCKING_NOTIFIER_HEAD(lease_notifier_chain);
+EXPORT_SYMBOL_GPL(lease_notifier_chain);
+
+/*
* The blocked_hash is used to find POSIX lock loops for deadlock detection.
* It is protected by blocked_lock_lock.
*
@@ -1795,10 +1802,18 @@ EXPORT_SYMBOL(generic_setlease);
*
* The "priv" pointer is passed directly to the lm_setup function as-is. It
* may be NULL if the lm_setup operation doesn't require it.
+ *
+ * Kernel subsystems can also register to be notified on any attempt to set
+ * a new lease with the lease_notifier_chain. This is used by (e.g.) nfsd
+ * to close files that it may have cached when there is an attempt to set a
+ * conflicting lease.
*/
int
vfs_setlease(struct file *filp, long arg, struct file_lock **lease, void **priv)
{
+ if (arg != F_UNLCK)
+ blocking_notifier_call_chain(&lease_notifier_chain, arg, *lease);
+
if (filp->f_op->setlease)
return filp->f_op->setlease(filp, arg, lease, priv);
else
diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
index 669e62f6f4f6..77041967d8ff 100644
--- a/fs/nfsd/filecache.c
+++ b/fs/nfsd/filecache.c
@@ -158,6 +158,22 @@ static struct shrinker nfsd_file_shrinker = {
.seeks = 1,
};

+static int
+nfsd_file_lease_notifier_call(struct notifier_block *nb, unsigned long arg,
+ void *data)
+{
+ struct file_lock *fl = data;
+
+ /* Don't close files if we're the one trying to set the lease */
+ if (fl->fl_type == FL_LEASE)
+ nfsd_file_close_inode(file_inode(fl->fl_file));
+ return 0;
+}
+
+static struct notifier_block nfsd_file_lease_notifier = {
+ .notifier_call = nfsd_file_lease_notifier_call,
+};
+
int
nfsd_file_cache_init(void)
{
@@ -186,12 +202,21 @@ nfsd_file_cache_init(void)
goto out_lru;
}

+ ret = blocking_notifier_chain_register(&lease_notifier_chain,
+ &nfsd_file_lease_notifier);
+ if (ret) {
+ pr_err("nfsd: unable to register lease notifier: %d\n", ret);
+ goto out_shrinker;
+ }
+
for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
INIT_HLIST_HEAD(&nfsd_file_hashtbl[i].nfb_head);
spin_lock_init(&nfsd_file_hashtbl[i].nfb_lock);
}
out:
return ret;
+out_shrinker:
+ unregister_shrinker(&nfsd_file_shrinker);
out_lru:
list_lru_destroy(&nfsd_file_lru);
out_err:
@@ -207,6 +232,8 @@ nfsd_file_cache_shutdown(void)
struct nfsd_file *nf;
LIST_HEAD(dispose);

+ blocking_notifier_chain_unregister(&lease_notifier_chain,
+ &nfsd_file_lease_notifier);
unregister_shrinker(&nfsd_file_shrinker);
for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
spin_lock(&nfsd_file_hashtbl[i].nfb_lock);
diff --git a/include/linux/fs.h b/include/linux/fs.h
index 9a9d314f7b27..01bb82eae684 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -1041,6 +1041,7 @@ extern int fcntl_setlease(unsigned int fd, struct file *filp, long arg);
extern int fcntl_getlease(struct file *filp);

/* fs/locks.c */
+extern struct blocking_notifier_head lease_notifier_chain;
void locks_free_lock_context(struct file_lock_context *ctx);
void locks_free_lock(struct file_lock *fl);
extern void locks_init_lock(struct file_lock *);
--
2.4.3


2015-08-20 11:17:34

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 07/20] nfsd: hook up nfsd_write to the new nfsd_file cache

Note that all callers currently pass in NULL for "file" anyway, so
there was already some dead code in here. Just eliminate that parm
and have it use the file cache instead of dealing directly with a
filp.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/nfs3proc.c | 2 +-
fs/nfsd/nfsproc.c | 2 +-
fs/nfsd/vfs.c | 33 +++++++++++----------------------
fs/nfsd/vfs.h | 2 +-
4 files changed, 14 insertions(+), 25 deletions(-)

diff --git a/fs/nfsd/nfs3proc.c b/fs/nfsd/nfs3proc.c
index 7b755b7f785c..4e46ac511479 100644
--- a/fs/nfsd/nfs3proc.c
+++ b/fs/nfsd/nfs3proc.c
@@ -192,7 +192,7 @@ nfsd3_proc_write(struct svc_rqst *rqstp, struct nfsd3_writeargs *argp,

fh_copy(&resp->fh, &argp->fh);
resp->committed = argp->stable;
- nfserr = nfsd_write(rqstp, &resp->fh, NULL,
+ nfserr = nfsd_write(rqstp, &resp->fh,
argp->offset,
rqstp->rq_vec, argp->vlen,
&cnt,
diff --git a/fs/nfsd/nfsproc.c b/fs/nfsd/nfsproc.c
index 4cd78ef4c95c..9893095cbee1 100644
--- a/fs/nfsd/nfsproc.c
+++ b/fs/nfsd/nfsproc.c
@@ -213,7 +213,7 @@ nfsd_proc_write(struct svc_rqst *rqstp, struct nfsd_writeargs *argp,
SVCFH_fmt(&argp->fh),
argp->len, argp->offset);

- nfserr = nfsd_write(rqstp, fh_copy(&resp->fh, &argp->fh), NULL,
+ nfserr = nfsd_write(rqstp, fh_copy(&resp->fh, &argp->fh),
argp->offset,
rqstp->rq_vec, argp->vlen,
&cnt,
diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
index b5e077a6e7d4..7c1e7866fec9 100644
--- a/fs/nfsd/vfs.c
+++ b/fs/nfsd/vfs.c
@@ -42,6 +42,7 @@

#include "nfsd.h"
#include "vfs.h"
+#include "filecache.h"

#define NFSDDBG_FACILITY NFSDDBG_FILEOP

@@ -1002,30 +1003,18 @@ __be32 nfsd_read(struct svc_rqst *rqstp, struct svc_fh *fhp,
* N.B. After this call fhp needs an fh_put
*/
__be32
-nfsd_write(struct svc_rqst *rqstp, struct svc_fh *fhp, struct file *file,
- loff_t offset, struct kvec *vec, int vlen, unsigned long *cnt,
- int *stablep)
+nfsd_write(struct svc_rqst *rqstp, struct svc_fh *fhp, loff_t offset,
+ struct kvec *vec, int vlen, unsigned long *cnt, int *stablep)
{
- __be32 err = 0;
-
- if (file) {
- err = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
- NFSD_MAY_WRITE|NFSD_MAY_OWNER_OVERRIDE);
- if (err)
- goto out;
- err = nfsd_vfs_write(rqstp, fhp, file, offset, vec, vlen, cnt,
- stablep);
- } else {
- err = nfsd_open(rqstp, fhp, S_IFREG, NFSD_MAY_WRITE, &file);
- if (err)
- goto out;
-
- if (cnt)
- err = nfsd_vfs_write(rqstp, fhp, file, offset, vec, vlen,
- cnt, stablep);
- fput(file);
+ __be32 err;
+ struct nfsd_file *nf;
+
+ err = nfsd_file_acquire(rqstp, fhp, NFSD_MAY_WRITE, &nf);
+ if (err == nfs_ok) {
+ err = nfsd_vfs_write(rqstp, fhp, nf->nf_file, offset, vec,
+ vlen, cnt, stablep);
+ nfsd_file_put(nf);
}
-out:
return err;
}

diff --git a/fs/nfsd/vfs.h b/fs/nfsd/vfs.h
index 5be875e3e638..78b5527cba93 100644
--- a/fs/nfsd/vfs.h
+++ b/fs/nfsd/vfs.h
@@ -78,7 +78,7 @@ __be32 nfsd_readv(struct file *, loff_t, struct kvec *, int,
unsigned long *);
__be32 nfsd_read(struct svc_rqst *, struct svc_fh *,
loff_t, struct kvec *, int, unsigned long *);
-__be32 nfsd_write(struct svc_rqst *, struct svc_fh *,struct file *,
+__be32 nfsd_write(struct svc_rqst *, struct svc_fh *,
loff_t, struct kvec *,int, unsigned long *, int *);
__be32 nfsd_vfs_write(struct svc_rqst *rqstp, struct svc_fh *fhp,
struct file *file, loff_t offset,
--
2.4.3


2015-08-20 11:17:35

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 08/20] nfsd: hook up nfsd_read to the nfsd_file cache

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/vfs.c | 21 ++++++++-------------
1 file changed, 8 insertions(+), 13 deletions(-)

diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
index 7c1e7866fec9..eb64597f3bde 100644
--- a/fs/nfsd/vfs.c
+++ b/fs/nfsd/vfs.c
@@ -980,20 +980,15 @@ out_nfserr:
__be32 nfsd_read(struct svc_rqst *rqstp, struct svc_fh *fhp,
loff_t offset, struct kvec *vec, int vlen, unsigned long *count)
{
- struct file *file;
- struct raparms *ra;
- __be32 err;
-
- err = nfsd_open(rqstp, fhp, S_IFREG, NFSD_MAY_READ, &file);
- if (err)
- return err;
-
- ra = nfsd_init_raparms(file);
- err = nfsd_vfs_read(rqstp, file, offset, vec, vlen, count);
- if (ra)
- nfsd_put_raparams(file, ra);
- fput(file);
+ __be32 err;
+ struct nfsd_file *nf;

+ err = nfsd_file_acquire(rqstp, fhp, NFSD_MAY_READ, &nf);
+ if (err == nfs_ok) {
+ err = nfsd_vfs_read(rqstp, nf->nf_file, offset, vec, vlen,
+ count);
+ nfsd_file_put(nf);
+ }
return err;
}

--
2.4.3


2015-08-20 11:17:36

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 09/20] sunrpc: add a new cache_detail operation for when a cache is flushed

When the exports table is changed, exportfs will usually write a new
time to the "flush" file in the nfsd.export cache procfile. This tells
the kernel to flush any entries that are older than that value.

This gives us a mechanism to tell whether an unexport might have
occurred. Add a new ->flush cache_detail operation that is called after
flushing the cache whenever someone writes to a "flush" file.

Signed-off-by: Jeff Layton <[email protected]>
---
include/linux/sunrpc/cache.h | 1 +
net/sunrpc/cache.c | 3 +++
2 files changed, 4 insertions(+)

diff --git a/include/linux/sunrpc/cache.h b/include/linux/sunrpc/cache.h
index 03d3b4c92d9f..d1c10a978bb2 100644
--- a/include/linux/sunrpc/cache.h
+++ b/include/linux/sunrpc/cache.h
@@ -98,6 +98,7 @@ struct cache_detail {
int has_died);

struct cache_head * (*alloc)(void);
+ void (*flush)(void);
int (*match)(struct cache_head *orig, struct cache_head *new);
void (*init)(struct cache_head *orig, struct cache_head *new);
void (*update)(struct cache_head *orig, struct cache_head *new);
diff --git a/net/sunrpc/cache.c b/net/sunrpc/cache.c
index 4a2340a54401..60da9aa2bdc5 100644
--- a/net/sunrpc/cache.c
+++ b/net/sunrpc/cache.c
@@ -1451,6 +1451,9 @@ static ssize_t write_flush(struct file *file, const char __user *buf,
cd->nextcheck = seconds_since_boot();
cache_flush();

+ if (cd->flush)
+ cd->flush();
+
*ppos += count;
return count;
}
--
2.4.3


2015-08-20 11:17:37

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 11/20] nfsd: hook nfsd_commit up to the nfsd_file cache

Use cached filps if possible instead of opening a new one every time.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/vfs.c | 14 +++++++-------
1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
index 81600bd06fef..6cfd96adcc71 100644
--- a/fs/nfsd/vfs.c
+++ b/fs/nfsd/vfs.c
@@ -1028,9 +1028,9 @@ __be32
nfsd_commit(struct svc_rqst *rqstp, struct svc_fh *fhp,
loff_t offset, unsigned long count)
{
- struct file *file;
- loff_t end = LLONG_MAX;
- __be32 err = nfserr_inval;
+ struct nfsd_file *nf;
+ loff_t end = LLONG_MAX;
+ __be32 err = nfserr_inval;

if (offset < 0)
goto out;
@@ -1040,12 +1040,12 @@ nfsd_commit(struct svc_rqst *rqstp, struct svc_fh *fhp,
goto out;
}

- err = nfsd_open(rqstp, fhp, S_IFREG,
- NFSD_MAY_WRITE|NFSD_MAY_NOT_BREAK_LEASE, &file);
+ err = nfsd_file_acquire(rqstp, fhp,
+ NFSD_MAY_WRITE|NFSD_MAY_NOT_BREAK_LEASE, &nf);
if (err)
goto out;
if (EX_ISSYNC(fhp->fh_export)) {
- int err2 = vfs_fsync_range(file, offset, end, 0);
+ int err2 = vfs_fsync_range(nf->nf_file, offset, end, 0);

if (err2 != -EINVAL)
err = nfserrno(err2);
@@ -1053,7 +1053,7 @@ nfsd_commit(struct svc_rqst *rqstp, struct svc_fh *fhp,
err = nfserr_notsupp;
}

- fput(file);
+ nfsd_file_put(nf);
out:
return err;
}
--
2.4.3


2015-08-20 11:17:37

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 10/20] nfsd: handle NFSD_MAY_NOT_BREAK_LEASE in open file cache

The NFSD_MAY_NOT_BREAK_LEASE flag needs special handling. If we open a
file in order to do (e.g.) a COMMIT then we don't want to break any
leases, but subsequent READ/WRITE operations must break the leases.

If we construct a new cache entry with a set of may flags that have
NFSD_MAY_NOT_BREAK_LEASE set, then set flags in the cache entry that
indicate that subsequent users of this file must break leases before
using it if they do not have NFSD_MAY_NOT_BREAK_LEASE set.

Note that because NFSD_MAY_READ opens do not break read leases, we
must track what sort of lease breaks have been done. If we're breaking
leases for read, then we still need to do a lease break for write if
it's a R/W open and a writer comes along. Lease breaks for write
however imply a read lease break so we can clear both flags in that
event.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/filecache.c | 39 +++++++++++++++++++++++++++++++++++----
fs/nfsd/filecache.h | 2 ++
fs/nfsd/vfs.c | 3 ++-
fs/nfsd/vfs.h | 1 +
4 files changed, 40 insertions(+), 5 deletions(-)

diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
index 77041967d8ff..10aa698b6057 100644
--- a/fs/nfsd/filecache.c
+++ b/fs/nfsd/filecache.c
@@ -47,6 +47,12 @@ nfsd_file_alloc(struct inode *inode, unsigned int may, unsigned int hashval)
nf->nf_hashval = hashval;
atomic_set(&nf->nf_ref, 1);
nf->nf_may = NFSD_FILE_MAY_MASK & may;
+ if (may & NFSD_MAY_NOT_BREAK_LEASE) {
+ if (may & NFSD_MAY_WRITE)
+ __set_bit(NFSD_FILE_BREAK_WRITE, &nf->nf_flags);
+ if (may & NFSD_MAY_READ)
+ __set_bit(NFSD_FILE_BREAK_READ, &nf->nf_flags);
+ }
}
return nf;
}
@@ -286,9 +292,6 @@ nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
if (status != nfs_ok)
return status;

- /* Mask off any extraneous bits */
- may_flags &= NFSD_FILE_MAY_MASK;
-
inode = d_inode(fhp->fh_dentry);
hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
retry:
@@ -331,7 +334,7 @@ wait_for_construction:
* MAY flags are equal. Otherwise, we put the reference and try
* again.
*/
- if (may_flags != nf->nf_may) {
+ if ((may_flags & NFSD_FILE_MAY_MASK) != nf->nf_may) {
nfsd_file_put(nf);
goto retry;
}
@@ -339,6 +342,18 @@ wait_for_construction:
/* try to take over construction for this file */
if (test_and_set_bit(NFSD_FILE_PENDING, &nf->nf_flags))
goto wait_for_construction;
+
+ /* sync up the BREAK_* flags with our may_flags */
+ if (may_flags & NFSD_MAY_NOT_BREAK_LEASE) {
+ if (may_flags & NFSD_MAY_WRITE)
+ set_bit(NFSD_FILE_BREAK_WRITE, &nf->nf_flags);
+ if (may_flags & NFSD_MAY_READ)
+ set_bit(NFSD_FILE_BREAK_READ, &nf->nf_flags);
+ } else {
+ clear_bit(NFSD_FILE_BREAK_WRITE, &nf->nf_flags);
+ clear_bit(NFSD_FILE_BREAK_READ, &nf->nf_flags);
+ }
+
goto open_file;
}

@@ -349,6 +364,22 @@ wait_for_construction:
*/
status = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
may_flags|NFSD_MAY_OWNER_OVERRIDE);
+
+ if (status == nfs_ok && !(may_flags & NFSD_MAY_NOT_BREAK_LEASE)) {
+ bool write = (may_flags & NFSD_MAY_WRITE);
+
+ if (test_bit(NFSD_FILE_BREAK_READ, &nf->nf_flags) ||
+ (test_bit(NFSD_FILE_BREAK_WRITE, &nf->nf_flags) && write)) {
+ status = nfserrno(nfsd_open_break_lease(
+ file_inode(nf->nf_file), may_flags));
+ if (status == nfs_ok) {
+ clear_bit(NFSD_FILE_BREAK_READ, &nf->nf_flags);
+ if (write)
+ clear_bit(NFSD_FILE_BREAK_WRITE,
+ &nf->nf_flags);
+ }
+ }
+ }
out:
if (status == nfs_ok)
*pnf = nf;
diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
index e7da463446dd..debd558ef786 100644
--- a/fs/nfsd/filecache.h
+++ b/fs/nfsd/filecache.h
@@ -13,6 +13,8 @@ struct nfsd_file {
struct file *nf_file;
#define NFSD_FILE_HASHED (0)
#define NFSD_FILE_PENDING (1)
+#define NFSD_FILE_BREAK_READ (2)
+#define NFSD_FILE_BREAK_WRITE (3)
unsigned long nf_flags;
struct inode *nf_inode;
unsigned int nf_hashval;
diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
index eb64597f3bde..81600bd06fef 100644
--- a/fs/nfsd/vfs.c
+++ b/fs/nfsd/vfs.c
@@ -618,7 +618,8 @@ nfsd_access(struct svc_rqst *rqstp, struct svc_fh *fhp, u32 *access, u32 *suppor
}
#endif /* CONFIG_NFSD_V3 */

-static int nfsd_open_break_lease(struct inode *inode, int access)
+int
+nfsd_open_break_lease(struct inode *inode, int access)
{
unsigned int mode;

diff --git a/fs/nfsd/vfs.h b/fs/nfsd/vfs.h
index 78b5527cba93..a3ec59830297 100644
--- a/fs/nfsd/vfs.h
+++ b/fs/nfsd/vfs.h
@@ -69,6 +69,7 @@ __be32 do_nfsd_create(struct svc_rqst *, struct svc_fh *,
__be32 nfsd_commit(struct svc_rqst *, struct svc_fh *,
loff_t, unsigned long);
#endif /* CONFIG_NFSD_V3 */
+int nfsd_open_break_lease(struct inode *, int);
__be32 nfsd_open(struct svc_rqst *, struct svc_fh *, umode_t,
int, struct file **);
struct raparms;
--
2.4.3


2015-08-20 11:17:38

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 12/20] nfsd: move include of state.h from trace.c to trace.h

Any file which includes trace.h will need to include state.h, even if
they aren't using any state tracepoints. Ensure that we include any
headers that might be needed in trace.h instead of relying on the
*.c files to have the right ones.

Signed-off-by: Jeff Layton <[email protected]>
Reviewed-by: Christoph Hellwig <[email protected]>
---
fs/nfsd/trace.c | 2 --
fs/nfsd/trace.h | 2 ++
2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/fs/nfsd/trace.c b/fs/nfsd/trace.c
index 82f89070594c..90967466a1e5 100644
--- a/fs/nfsd/trace.c
+++ b/fs/nfsd/trace.c
@@ -1,5 +1,3 @@

-#include "state.h"
-
#define CREATE_TRACE_POINTS
#include "trace.h"
diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
index c668520c344b..0befe762762b 100644
--- a/fs/nfsd/trace.h
+++ b/fs/nfsd/trace.h
@@ -9,6 +9,8 @@

#include <linux/tracepoint.h>

+#include "state.h"
+
DECLARE_EVENT_CLASS(nfsd_stateid_class,
TP_PROTO(stateid_t *stp),
TP_ARGS(stp),
--
2.4.3


2015-08-20 11:17:40

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 14/20] nfsd: close cached files prior to a REMOVE or RENAME that would replace target

It's not uncommon for some workloads to do a bunch of I/O to a file and
delete it just afterward. If knfsd has a cached open file however, then
the file may still be open when the dentry is unlinked. If the
underlying filesystem is nfs, then that could trigger it to do a
sillyrename.

On a REMOVE or RENAME scan the nfsd_file cache for open files that
correspond to the inode, and proactively unhash and put their
references. This should prevent any delete-on-last-close activity from
occurring, solely due to knfsd's open file cache.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/filecache.c | 25 +++++++++++++++++++++++++
fs/nfsd/filecache.h | 1 +
fs/nfsd/trace.h | 17 +++++++++++++++++
fs/nfsd/vfs.c | 17 +++++++++++++++--
4 files changed, 58 insertions(+), 2 deletions(-)

diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
index e48b536762aa..4bd683f03b6e 100644
--- a/fs/nfsd/filecache.c
+++ b/fs/nfsd/filecache.c
@@ -283,6 +283,31 @@ nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
return NULL;
}

+/**
+ * nfsd_file_close_inode - attempt to forcibly close a nfsd_file
+ * @inode: inode of the file to attempt to remove
+ *
+ * Walk the whole hash bucket, looking for any files that correspond to "inode".
+ * If any do, then unhash them and put the hashtable reference to them.
+ */
+void
+nfsd_file_close_inode(struct inode *inode)
+{
+ struct nfsd_file *nf;
+ struct hlist_node *tmp;
+ unsigned int hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
+ LIST_HEAD(dispose);
+
+ spin_lock(&nfsd_file_hashtbl[hashval].nfb_lock);
+ hlist_for_each_entry_safe(nf, tmp, &nfsd_file_hashtbl[hashval].nfb_head, nf_node) {
+ if (inode == nf->nf_inode)
+ nfsd_file_unhash_and_release_locked(nf, &dispose);
+ }
+ spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
+ trace_nfsd_file_close_inode(hashval, inode, !list_empty(&dispose));
+ nfsd_file_dispose_list(&dispose);
+}
+
__be32
nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
unsigned int may_flags, struct nfsd_file **pnf)
diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
index debd558ef786..191cdb25aa66 100644
--- a/fs/nfsd/filecache.h
+++ b/fs/nfsd/filecache.h
@@ -26,6 +26,7 @@ int nfsd_file_cache_init(void);
void nfsd_file_cache_shutdown(void);
void nfsd_file_put(struct nfsd_file *nf);
struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
+void nfsd_file_close_inode(struct inode *inode);
__be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
unsigned int may_flags, struct nfsd_file **nfp);
#endif /* _FS_NFSD_FILECACHE_H */
diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
index 2dac872d31e8..95af3b9c7b66 100644
--- a/fs/nfsd/trace.h
+++ b/fs/nfsd/trace.h
@@ -139,6 +139,23 @@ TRACE_EVENT(nfsd_file_acquire,
show_nf_may(__entry->nf_may), __entry->nf_file,
be32_to_cpu(__entry->status))
);
+
+TRACE_EVENT(nfsd_file_close_inode,
+ TP_PROTO(unsigned int hash, struct inode *inode, int found),
+ TP_ARGS(hash, inode, found),
+ TP_STRUCT__entry(
+ __field(unsigned int, hash)
+ __field(struct inode *, inode)
+ __field(int, found)
+ ),
+ TP_fast_assign(
+ __entry->hash = hash;
+ __entry->inode = inode;
+ __entry->found = found;
+ ),
+ TP_printk("hash=0x%x inode=0x%p found=%d", __entry->hash,
+ __entry->inode, __entry->found)
+);
#endif /* _NFSD_TRACE_H */

#undef TRACE_INCLUDE_PATH
diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
index 6cfd96adcc71..98d3b9d96480 100644
--- a/fs/nfsd/vfs.c
+++ b/fs/nfsd/vfs.c
@@ -1583,6 +1583,15 @@ out_nfserr:
goto out_unlock;
}

+static void
+nfsd_close_cached_files(struct dentry *dentry)
+{
+ struct inode *inode = d_inode(dentry);
+
+ if (inode && S_ISREG(inode->i_mode))
+ nfsd_file_close_inode(inode);
+}
+
/*
* Rename a file
* N.B. After this call _both_ ffhp and tfhp need an fh_put
@@ -1652,6 +1661,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
goto out_dput_new;

+ nfsd_close_cached_files(ndentry);
host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
if (!host_err) {
host_err = commit_metadata(tfhp);
@@ -1721,10 +1731,13 @@ nfsd_unlink(struct svc_rqst *rqstp, struct svc_fh *fhp, int type,
if (!type)
type = d_inode(rdentry)->i_mode & S_IFMT;

- if (type != S_IFDIR)
+ if (type != S_IFDIR) {
+ nfsd_close_cached_files(rdentry);
host_err = vfs_unlink(dirp, rdentry, NULL);
- else
+ } else {
host_err = vfs_rmdir(dirp, rdentry);
+ }
+
if (!host_err)
host_err = commit_metadata(fhp);
dput(rdentry);
--
2.4.3


2015-08-20 11:17:39

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 13/20] nfsd: add new tracepoints for nfsd_file cache

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/filecache.c | 20 +++++++++---
fs/nfsd/trace.h | 91 +++++++++++++++++++++++++++++++++++++++++++++++++++++
2 files changed, 107 insertions(+), 4 deletions(-)

diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
index 10aa698b6057..e48b536762aa 100644
--- a/fs/nfsd/filecache.c
+++ b/fs/nfsd/filecache.c
@@ -15,6 +15,7 @@
#include "nfsd.h"
#include "nfsfh.h"
#include "filecache.h"
+#include "trace.h"

#define NFSDDBG_FACILITY NFSDDBG_FH

@@ -53,6 +54,7 @@ nfsd_file_alloc(struct inode *inode, unsigned int may, unsigned int hashval)
if (may & NFSD_MAY_READ)
__set_bit(NFSD_FILE_BREAK_READ, &nf->nf_flags);
}
+ trace_nfsd_file_alloc(nf);
}
return nf;
}
@@ -60,6 +62,7 @@ nfsd_file_alloc(struct inode *inode, unsigned int may, unsigned int hashval)
static void
nfsd_file_put_final(struct nfsd_file *nf)
{
+ trace_nfsd_file_put_final(nf);
if (nf->nf_file)
fput(nf->nf_file);
kfree_rcu(nf, nf_rcu);
@@ -70,6 +73,7 @@ nfsd_file_unhash(struct nfsd_file *nf)
{
lockdep_assert_held(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);

+ trace_nfsd_file_unhash(nf);
if (test_bit(NFSD_FILE_HASHED, &nf->nf_flags)) {
clear_bit(NFSD_FILE_HASHED, &nf->nf_flags);
hlist_del_rcu(&nf->nf_node);
@@ -84,6 +88,7 @@ nfsd_file_unhash_and_release_locked(struct nfsd_file *nf, struct list_head *disp
{
lockdep_assert_held(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);

+ trace_nfsd_file_unhash_and_release_locked(nf);
if (!nfsd_file_unhash(nf))
return;
if (!atomic_dec_and_test(&nf->nf_ref))
@@ -95,8 +100,8 @@ nfsd_file_unhash_and_release_locked(struct nfsd_file *nf, struct list_head *disp
void
nfsd_file_put(struct nfsd_file *nf)
{
+ trace_nfsd_file_put(nf);
list_lru_rotate(&nfsd_file_lru, &nf->nf_lru);
-
if (!atomic_dec_and_test(&nf->nf_ref))
return;

@@ -303,8 +308,11 @@ retry:

if (!new) {
new = nfsd_file_alloc(inode, may_flags, hashval);
- if (!new)
+ if (!new) {
+ trace_nfsd_file_acquire(hashval, inode, may_flags, NULL,
+ nfserr_jukebox);
return nfserr_jukebox;
+ }
}

spin_lock(&nfsd_file_hashtbl[hashval].nfb_lock);
@@ -381,13 +389,17 @@ wait_for_construction:
}
}
out:
- if (status == nfs_ok)
+ if (status == nfs_ok) {
*pnf = nf;
- else
+ } else {
nfsd_file_put(nf);
+ nf = NULL;
+ }

if (new)
nfsd_file_put(new);
+
+ trace_nfsd_file_acquire(hashval, inode, may_flags, nf, status);
return status;
open_file:
status = nfsd_open(rqstp, fhp, S_IFREG, may_flags, &nf->nf_file);
diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
index 0befe762762b..2dac872d31e8 100644
--- a/fs/nfsd/trace.h
+++ b/fs/nfsd/trace.h
@@ -10,6 +10,8 @@
#include <linux/tracepoint.h>

#include "state.h"
+#include "filecache.h"
+#include "vfs.h"

DECLARE_EVENT_CLASS(nfsd_stateid_class,
TP_PROTO(stateid_t *stp),
@@ -48,6 +50,95 @@ DEFINE_STATEID_EVENT(layout_recall_done);
DEFINE_STATEID_EVENT(layout_recall_fail);
DEFINE_STATEID_EVENT(layout_recall_release);

+#define show_nf_flags(val) \
+ __print_flags(val, "|", \
+ { 1 << NFSD_FILE_HASHED, "HASHED" }, \
+ { 1 << NFSD_FILE_PENDING, "PENDING" }, \
+ { 1 << NFSD_FILE_BREAK_READ, "BREAK_READ" }, \
+ { 1 << NFSD_FILE_BREAK_WRITE, "BREAK_WRITE" })
+
+/* FIXME: This should probably be fleshed out in the future. */
+#define show_nf_may(val) \
+ __print_flags(val, "|", \
+ { NFSD_MAY_READ, "READ" }, \
+ { NFSD_MAY_WRITE, "WRITE" }, \
+ { NFSD_MAY_NOT_BREAK_LEASE, "NOT_BREAK_LEASE" })
+
+DECLARE_EVENT_CLASS(nfsd_file_class,
+ TP_PROTO(struct nfsd_file *nf),
+ TP_ARGS(nf),
+ TP_STRUCT__entry(
+ __field(unsigned int, nf_hashval)
+ __field(void *, nf_inode)
+ __field(int, nf_ref)
+ __field(unsigned long, nf_flags)
+ __field(unsigned char, nf_may)
+ __field(struct file *, nf_file)
+ ),
+ TP_fast_assign(
+ __entry->nf_hashval = nf->nf_hashval;
+ __entry->nf_inode = nf->nf_inode;
+ __entry->nf_ref = atomic_read(&nf->nf_ref);
+ __entry->nf_flags = nf->nf_flags;
+ __entry->nf_may = nf->nf_may;
+ __entry->nf_file = nf->nf_file;
+ ),
+ TP_printk("hash=0x%x inode=0x%p ref=%d flags=%s may=%s file=%p",
+ __entry->nf_hashval,
+ __entry->nf_inode,
+ __entry->nf_ref,
+ show_nf_flags(__entry->nf_flags),
+ show_nf_may(__entry->nf_may),
+ __entry->nf_file)
+)
+
+#define DEFINE_NFSD_FILE_EVENT(name) \
+DEFINE_EVENT(nfsd_file_class, name, \
+ TP_PROTO(struct nfsd_file *nf), \
+ TP_ARGS(nf))
+
+DEFINE_NFSD_FILE_EVENT(nfsd_file_alloc);
+DEFINE_NFSD_FILE_EVENT(nfsd_file_put_final);
+DEFINE_NFSD_FILE_EVENT(nfsd_file_unhash);
+DEFINE_NFSD_FILE_EVENT(nfsd_file_put);
+DEFINE_NFSD_FILE_EVENT(nfsd_file_unhash_and_release_locked);
+
+TRACE_EVENT(nfsd_file_acquire,
+ TP_PROTO(unsigned int hash, struct inode *inode,
+ unsigned int may_flags, struct nfsd_file *nf,
+ __be32 status),
+
+ TP_ARGS(hash, inode, may_flags, nf, status),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, hash)
+ __field(void *, inode)
+ __field(unsigned int, may_flags)
+ __field(int, nf_ref)
+ __field(unsigned long, nf_flags)
+ __field(unsigned char, nf_may)
+ __field(struct file *, nf_file)
+ __field(__be32, status)
+ ),
+
+ TP_fast_assign(
+ __entry->hash = hash;
+ __entry->inode = inode;
+ __entry->may_flags = may_flags;
+ __entry->nf_ref = nf ? atomic_read(&nf->nf_ref) : 0;
+ __entry->nf_flags = nf ? nf->nf_flags : 0;
+ __entry->nf_may = nf ? nf->nf_may : 0;
+ __entry->nf_file = nf ? nf->nf_file : NULL;
+ __entry->status = status;
+ ),
+
+ TP_printk("hash=0x%x inode=0x%p may_flags=%s ref=%d nf_flags=%s nf_may=%s nf_file=0x%p status=%u",
+ __entry->hash, __entry->inode,
+ show_nf_may(__entry->may_flags), __entry->nf_ref,
+ show_nf_flags(__entry->nf_flags),
+ show_nf_may(__entry->nf_may), __entry->nf_file,
+ be32_to_cpu(__entry->status))
+);
#endif /* _NFSD_TRACE_H */

#undef TRACE_INCLUDE_PATH
--
2.4.3


2015-08-20 11:17:42

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 16/20] nfsd: convert nfs4_file->fi_fds array to use nfsd_files

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/nfs4state.c | 23 ++++++++++++-----------
fs/nfsd/state.h | 2 +-
2 files changed, 13 insertions(+), 12 deletions(-)

diff --git a/fs/nfsd/nfs4state.c b/fs/nfsd/nfs4state.c
index af88d1d7ccae..9e390b9fb33e 100644
--- a/fs/nfsd/nfs4state.c
+++ b/fs/nfsd/nfs4state.c
@@ -49,6 +49,7 @@

#include "netns.h"
#include "pnfs.h"
+#include "filecache.h"

#define NFSDDBG_FACILITY NFSDDBG_PROC

@@ -292,7 +293,7 @@ static struct file *
__nfs4_get_fd(struct nfs4_file *f, int oflag)
{
if (f->fi_fds[oflag])
- return get_file(f->fi_fds[oflag]);
+ return get_file(f->fi_fds[oflag]->nf_file);
return NULL;
}

@@ -449,17 +450,17 @@ static void __nfs4_file_put_access(struct nfs4_file *fp, int oflag)
might_lock(&fp->fi_lock);

if (atomic_dec_and_lock(&fp->fi_access[oflag], &fp->fi_lock)) {
- struct file *f1 = NULL;
- struct file *f2 = NULL;
+ struct nfsd_file *f1 = NULL;
+ struct nfsd_file *f2 = NULL;

swap(f1, fp->fi_fds[oflag]);
if (atomic_read(&fp->fi_access[1 - oflag]) == 0)
swap(f2, fp->fi_fds[O_RDWR]);
spin_unlock(&fp->fi_lock);
if (f1)
- fput(f1);
+ nfsd_file_put(f1);
if (f2)
- fput(f2);
+ nfsd_file_put(f2);
}
}

@@ -3804,7 +3805,7 @@ static __be32 nfs4_get_vfs_file(struct svc_rqst *rqstp, struct nfs4_file *fp,
struct svc_fh *cur_fh, struct nfs4_ol_stateid *stp,
struct nfsd4_open *open)
{
- struct file *filp = NULL;
+ struct nfsd_file *nf = NULL;
__be32 status;
int oflag = nfs4_access_to_omode(open->op_share_access);
int access = nfs4_access_to_access(open->op_share_access);
@@ -3840,18 +3841,18 @@ static __be32 nfs4_get_vfs_file(struct svc_rqst *rqstp, struct nfs4_file *fp,

if (!fp->fi_fds[oflag]) {
spin_unlock(&fp->fi_lock);
- status = nfsd_open(rqstp, cur_fh, S_IFREG, access, &filp);
+ status = nfsd_file_acquire(rqstp, cur_fh, access, &nf);
if (status)
goto out_put_access;
spin_lock(&fp->fi_lock);
if (!fp->fi_fds[oflag]) {
- fp->fi_fds[oflag] = filp;
- filp = NULL;
+ fp->fi_fds[oflag] = nf;
+ nf = NULL;
}
}
spin_unlock(&fp->fi_lock);
- if (filp)
- fput(filp);
+ if (nf)
+ nfsd_file_put(nf);

status = nfsd4_truncate(rqstp, cur_fh, open);
if (status)
diff --git a/fs/nfsd/state.h b/fs/nfsd/state.h
index 583ffc13cae2..70b3e51ba089 100644
--- a/fs/nfsd/state.h
+++ b/fs/nfsd/state.h
@@ -499,7 +499,7 @@ struct nfs4_file {
};
struct list_head fi_clnt_odstate;
/* One each for O_RDONLY, O_WRONLY, O_RDWR: */
- struct file * fi_fds[3];
+ struct nfsd_file *fi_fds[3];
/*
* Each open or lock stateid contributes 0-4 to the counts
* below depending on which bits are set in st_access_bitmap:
--
2.4.3


2015-08-20 11:17:42

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 15/20] nfsd: call flush_delayed_fput from nfsd_file_close_fh

...when there are open files to be closed.

When knfsd does an fput(), it gets queued to a list and a workqueue job
is then scheduled to do the actual __fput work. In the case of knfsd
closing down the file prior to a REMOVE or RENAME, we really want to
ensure that those files are closed prior to returning. When there are
files to be closed, call flush_delayed_fput to ensure this.

There are deadlock possibilities if you call flush_delayed_fput while
holding locks, however. In the case of nfsd_rename, we don't even do the
lookups of the dentries to be renamed until we've locked for rename.

Once we've figured out what the target dentry is for a rename, check to
see whether there are cached open files associated with it. If there
are, then unwind all of the locking, close them all, and then reattempt
the rename.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/file_table.c | 1 +
fs/nfsd/filecache.c | 33 ++++++++++++++++++++++++++++++++-
fs/nfsd/filecache.h | 1 +
fs/nfsd/trace.h | 10 +++++++++-
fs/nfsd/vfs.c | 47 +++++++++++++++++++++++++++++++++++++++--------
5 files changed, 82 insertions(+), 10 deletions(-)

diff --git a/fs/file_table.c b/fs/file_table.c
index 7f9d407c7595..33898e72618c 100644
--- a/fs/file_table.c
+++ b/fs/file_table.c
@@ -257,6 +257,7 @@ void flush_delayed_fput(void)
{
delayed_fput(NULL);
}
+EXPORT_SYMBOL_GPL(flush_delayed_fput);

static DECLARE_DELAYED_WORK(delayed_fput_work, delayed_fput);

diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
index 4bd683f03b6e..b62942ba6e7b 100644
--- a/fs/nfsd/filecache.c
+++ b/fs/nfsd/filecache.c
@@ -284,6 +284,34 @@ nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
}

/**
+ * nfsd_file_is_cached - are there any cached open files for this fh?
+ * @inode: inode of the file to check
+ *
+ * Scan the hashtable for open files that match this fh. Returns true if there
+ * are any, and false if not.
+ */
+bool
+nfsd_file_is_cached(struct inode *inode)
+{
+ bool ret = false;
+ struct nfsd_file *nf;
+ unsigned int hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
+
+ rcu_read_lock();
+ hlist_for_each_entry_rcu(nf, &nfsd_file_hashtbl[hashval].nfb_head,
+ nf_node) {
+ if (inode == nf->nf_inode) {
+ ret = true;
+ break;
+ }
+ }
+ rcu_read_unlock();
+ trace_nfsd_file_is_cached(hashval, inode, (int)ret);
+ return ret;
+}
+
+
+/**
* nfsd_file_close_inode - attempt to forcibly close a nfsd_file
* @inode: inode of the file to attempt to remove
*
@@ -305,7 +333,10 @@ nfsd_file_close_inode(struct inode *inode)
}
spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
trace_nfsd_file_close_inode(hashval, inode, !list_empty(&dispose));
- nfsd_file_dispose_list(&dispose);
+ if (!list_empty(&dispose)) {
+ nfsd_file_dispose_list(&dispose);
+ flush_delayed_fput();
+ }
}

__be32
diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
index 191cdb25aa66..4a873efb7953 100644
--- a/fs/nfsd/filecache.h
+++ b/fs/nfsd/filecache.h
@@ -27,6 +27,7 @@ void nfsd_file_cache_shutdown(void);
void nfsd_file_put(struct nfsd_file *nf);
struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
void nfsd_file_close_inode(struct inode *inode);
+bool nfsd_file_is_cached(struct inode *inode);
__be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
unsigned int may_flags, struct nfsd_file **nfp);
#endif /* _FS_NFSD_FILECACHE_H */
diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
index 95af3b9c7b66..fc6d8ee51a00 100644
--- a/fs/nfsd/trace.h
+++ b/fs/nfsd/trace.h
@@ -140,7 +140,7 @@ TRACE_EVENT(nfsd_file_acquire,
be32_to_cpu(__entry->status))
);

-TRACE_EVENT(nfsd_file_close_inode,
+DECLARE_EVENT_CLASS(nfsd_file_search_class,
TP_PROTO(unsigned int hash, struct inode *inode, int found),
TP_ARGS(hash, inode, found),
TP_STRUCT__entry(
@@ -156,6 +156,14 @@ TRACE_EVENT(nfsd_file_close_inode,
TP_printk("hash=0x%x inode=0x%p found=%d", __entry->hash,
__entry->inode, __entry->found)
);
+
+#define DEFINE_NFSD_FILE_SEARCH_EVENT(name) \
+DEFINE_EVENT(nfsd_file_search_class, name, \
+ TP_PROTO(unsigned int hash, struct inode *inode, int found), \
+ TP_ARGS(hash, inode, found))
+
+DEFINE_NFSD_FILE_SEARCH_EVENT(nfsd_file_close_inode);
+DEFINE_NFSD_FILE_SEARCH_EVENT(nfsd_file_is_cached);
#endif /* _NFSD_TRACE_H */

#undef TRACE_INCLUDE_PATH
diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
index 98d3b9d96480..4cc78a4ec694 100644
--- a/fs/nfsd/vfs.c
+++ b/fs/nfsd/vfs.c
@@ -1592,6 +1592,17 @@ nfsd_close_cached_files(struct dentry *dentry)
nfsd_file_close_inode(inode);
}

+static bool
+nfsd_has_cached_files(struct dentry *dentry)
+{
+ bool ret = false;
+ struct inode *inode = d_inode(dentry);
+
+ if (inode && S_ISREG(inode->i_mode))
+ ret = nfsd_file_is_cached(inode);
+ return ret;
+}
+
/*
* Rename a file
* N.B. After this call _both_ ffhp and tfhp need an fh_put
@@ -1604,6 +1615,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
struct inode *fdir, *tdir;
__be32 err;
int host_err;
+ bool has_cached = false;

err = fh_verify(rqstp, ffhp, S_IFDIR, NFSD_MAY_REMOVE);
if (err)
@@ -1622,6 +1634,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
if (!flen || isdotent(fname, flen) || !tlen || isdotent(tname, tlen))
goto out;

+retry:
host_err = fh_want_write(ffhp);
if (host_err) {
err = nfserrno(host_err);
@@ -1661,12 +1674,16 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
goto out_dput_new;

- nfsd_close_cached_files(ndentry);
- host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
- if (!host_err) {
- host_err = commit_metadata(tfhp);
- if (!host_err)
- host_err = commit_metadata(ffhp);
+ if (nfsd_has_cached_files(ndentry)) {
+ has_cached = true;
+ goto out_dput_old;
+ } else {
+ host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
+ if (!host_err) {
+ host_err = commit_metadata(tfhp);
+ if (!host_err)
+ host_err = commit_metadata(ffhp);
+ }
}
out_dput_new:
dput(ndentry);
@@ -1679,12 +1696,26 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
* as that would do the wrong thing if the two directories
* were the same, so again we do it by hand.
*/
- fill_post_wcc(ffhp);
- fill_post_wcc(tfhp);
+ if (!has_cached) {
+ fill_post_wcc(ffhp);
+ fill_post_wcc(tfhp);
+ }
unlock_rename(tdentry, fdentry);
ffhp->fh_locked = tfhp->fh_locked = 0;
fh_drop_write(ffhp);

+ /*
+ * If the target dentry has cached open files, then we need to try to
+ * close them prior to doing the rename. Flushing delayed fput
+ * shouldn't be done with locks held however, so we delay it until this
+ * point and then reattempt the whole shebang.
+ */
+ if (has_cached) {
+ has_cached = false;
+ nfsd_close_cached_files(ndentry);
+ dput(ndentry);
+ goto retry;
+ }
out:
return err;
}
--
2.4.3


2015-08-20 11:17:43

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 17/20] nfsd: have nfsd_test_lock use the nfsd_file cache

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/nfs4state.c | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/fs/nfsd/nfs4state.c b/fs/nfsd/nfs4state.c
index 9e390b9fb33e..a92faba3a847 100644
--- a/fs/nfsd/nfs4state.c
+++ b/fs/nfsd/nfs4state.c
@@ -5542,11 +5542,11 @@ out:
*/
static __be32 nfsd_test_lock(struct svc_rqst *rqstp, struct svc_fh *fhp, struct file_lock *lock)
{
- struct file *file;
- __be32 err = nfsd_open(rqstp, fhp, S_IFREG, NFSD_MAY_READ, &file);
+ struct nfsd_file *nf;
+ __be32 err = nfsd_file_acquire(rqstp, fhp, NFSD_MAY_READ, &nf);
if (!err) {
- err = nfserrno(vfs_test_lock(file, lock));
- fput(file);
+ err = nfserrno(vfs_test_lock(nf->nf_file, lock));
+ nfsd_file_put(nf);
}
return err;
}
--
2.4.3


2015-08-20 11:17:44

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 18/20] nfsd: convert fi_deleg_file and ls_file fields to nfsd_file

Have them keep an nfsd_file reference instead of a struct file.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/nfs4layouts.c | 12 ++---
fs/nfsd/nfs4state.c | 131 ++++++++++++++++++++++++++------------------------
fs/nfsd/state.h | 6 +--
3 files changed, 76 insertions(+), 73 deletions(-)

diff --git a/fs/nfsd/nfs4layouts.c b/fs/nfsd/nfs4layouts.c
index ebf90e487c75..9b49876a9cc1 100644
--- a/fs/nfsd/nfs4layouts.c
+++ b/fs/nfsd/nfs4layouts.c
@@ -144,8 +144,8 @@ nfsd4_free_layout_stateid(struct nfs4_stid *stid)
list_del_init(&ls->ls_perfile);
spin_unlock(&fp->fi_lock);

- vfs_setlease(ls->ls_file, F_UNLCK, NULL, (void **)&ls);
- fput(ls->ls_file);
+ vfs_setlease(ls->ls_file->nf_file, F_UNLCK, NULL, (void **)&ls);
+ nfsd_file_put(ls->ls_file);

if (ls->ls_recalled)
atomic_dec(&ls->ls_stid.sc_file->fi_lo_recalls);
@@ -169,7 +169,7 @@ nfsd4_layout_setlease(struct nfs4_layout_stateid *ls)
fl->fl_end = OFFSET_MAX;
fl->fl_owner = ls;
fl->fl_pid = current->tgid;
- fl->fl_file = ls->ls_file;
+ fl->fl_file = ls->ls_file->nf_file;

status = vfs_setlease(fl->fl_file, fl->fl_type, &fl, NULL);
if (status) {
@@ -206,13 +206,13 @@ nfsd4_alloc_layout_stateid(struct nfsd4_compound_state *cstate,
NFSPROC4_CLNT_CB_LAYOUT);

if (parent->sc_type == NFS4_DELEG_STID)
- ls->ls_file = get_file(fp->fi_deleg_file);
+ ls->ls_file = nfsd_file_get(fp->fi_deleg_file);
else
ls->ls_file = find_any_file(fp);
BUG_ON(!ls->ls_file);

if (nfsd4_layout_setlease(ls)) {
- fput(ls->ls_file);
+ nfsd_file_put(ls->ls_file);
put_nfs4_file(fp);
kmem_cache_free(nfs4_layout_stateid_cache, ls);
return NULL;
@@ -598,7 +598,7 @@ nfsd4_cb_layout_fail(struct nfs4_layout_stateid *ls)

argv[0] = "/sbin/nfsd-recall-failed";
argv[1] = addr_str;
- argv[2] = ls->ls_file->f_path.mnt->mnt_sb->s_id;
+ argv[2] = ls->ls_file->nf_file->f_path.mnt->mnt_sb->s_id;
argv[3] = NULL;

error = call_usermodehelper(argv[0], argv, envp, UMH_WAIT_PROC);
diff --git a/fs/nfsd/nfs4state.c b/fs/nfsd/nfs4state.c
index a92faba3a847..f8394a4cd126 100644
--- a/fs/nfsd/nfs4state.c
+++ b/fs/nfsd/nfs4state.c
@@ -289,18 +289,18 @@ put_nfs4_file(struct nfs4_file *fi)
}
}

-static struct file *
+static struct nfsd_file *
__nfs4_get_fd(struct nfs4_file *f, int oflag)
{
if (f->fi_fds[oflag])
- return get_file(f->fi_fds[oflag]->nf_file);
+ return nfsd_file_get(f->fi_fds[oflag]);
return NULL;
}

-static struct file *
+static struct nfsd_file *
find_writeable_file_locked(struct nfs4_file *f)
{
- struct file *ret;
+ struct nfsd_file *ret;

lockdep_assert_held(&f->fi_lock);

@@ -310,10 +310,10 @@ find_writeable_file_locked(struct nfs4_file *f)
return ret;
}

-static struct file *
+static struct nfsd_file *
find_writeable_file(struct nfs4_file *f)
{
- struct file *ret;
+ struct nfsd_file *ret;

spin_lock(&f->fi_lock);
ret = find_writeable_file_locked(f);
@@ -322,9 +322,10 @@ find_writeable_file(struct nfs4_file *f)
return ret;
}

-static struct file *find_readable_file_locked(struct nfs4_file *f)
+static struct nfsd_file *
+find_readable_file_locked(struct nfs4_file *f)
{
- struct file *ret;
+ struct nfsd_file *ret;

lockdep_assert_held(&f->fi_lock);

@@ -334,10 +335,10 @@ static struct file *find_readable_file_locked(struct nfs4_file *f)
return ret;
}

-static struct file *
+static struct nfsd_file *
find_readable_file(struct nfs4_file *f)
{
- struct file *ret;
+ struct nfsd_file *ret;

spin_lock(&f->fi_lock);
ret = find_readable_file_locked(f);
@@ -346,10 +347,10 @@ find_readable_file(struct nfs4_file *f)
return ret;
}

-struct file *
+struct nfsd_file *
find_any_file(struct nfs4_file *f)
{
- struct file *ret;
+ struct nfsd_file *ret;

spin_lock(&f->fi_lock);
ret = __nfs4_get_fd(f, O_RDWR);
@@ -748,16 +749,16 @@ nfs4_put_stid(struct nfs4_stid *s)

static void nfs4_put_deleg_lease(struct nfs4_file *fp)
{
- struct file *filp = NULL;
+ struct nfsd_file *nf = NULL;

spin_lock(&fp->fi_lock);
if (fp->fi_deleg_file && --fp->fi_delegees == 0)
- swap(filp, fp->fi_deleg_file);
+ swap(nf, fp->fi_deleg_file);
spin_unlock(&fp->fi_lock);

- if (filp) {
- vfs_setlease(filp, F_UNLCK, NULL, (void **)&fp);
- fput(filp);
+ if (nf) {
+ vfs_setlease(nf->nf_file, F_UNLCK, NULL, (void **)&fp);
+ nfsd_file_put(nf);
}
}

@@ -1037,11 +1038,14 @@ static void nfs4_free_lock_stateid(struct nfs4_stid *stid)
{
struct nfs4_ol_stateid *stp = openlockstateid(stid);
struct nfs4_lockowner *lo = lockowner(stp->st_stateowner);
- struct file *file;
+ struct nfsd_file *nf;

- file = find_any_file(stp->st_stid.sc_file);
- if (file)
- filp_close(file, (fl_owner_t)lo);
+ nf = find_any_file(stp->st_stid.sc_file);
+ if (nf) {
+ get_file(nf->nf_file);
+ filp_close(nf->nf_file, (fl_owner_t)lo);
+ nfsd_file_put(nf);
+ }
nfs4_free_ol_stateid(stid);
}

@@ -3927,21 +3931,21 @@ static int nfs4_setlease(struct nfs4_delegation *dp)
{
struct nfs4_file *fp = dp->dl_stid.sc_file;
struct file_lock *fl;
- struct file *filp;
+ struct nfsd_file *nf;
int status = 0;

fl = nfs4_alloc_init_lease(fp, NFS4_OPEN_DELEGATE_READ);
if (!fl)
return -ENOMEM;
- filp = find_readable_file(fp);
- if (!filp) {
+ nf = find_readable_file(fp);
+ if (!nf) {
/* We should always have a readable file here */
WARN_ON_ONCE(1);
locks_free_lock(fl);
return -EBADF;
}
- fl->fl_file = filp;
- status = vfs_setlease(filp, fl->fl_type, &fl, NULL);
+ fl->fl_file = nf->nf_file;
+ status = vfs_setlease(nf->nf_file, fl->fl_type, &fl, NULL);
if (fl)
locks_free_lock(fl);
if (status)
@@ -3959,7 +3963,7 @@ static int nfs4_setlease(struct nfs4_delegation *dp)
hash_delegation_locked(dp, fp);
goto out_unlock;
}
- fp->fi_deleg_file = filp;
+ fp->fi_deleg_file = nf;
fp->fi_delegees = 1;
hash_delegation_locked(dp, fp);
spin_unlock(&fp->fi_lock);
@@ -3969,7 +3973,7 @@ out_unlock:
spin_unlock(&fp->fi_lock);
spin_unlock(&state_lock);
out_fput:
- fput(filp);
+ nfsd_file_put(nf);
return status;
}

@@ -4575,7 +4579,7 @@ nfsd4_lookup_stateid(struct nfsd4_compound_state *cstate,
return nfs_ok;
}

-static struct file *
+static struct nfsd_file *
nfs4_find_file(struct nfs4_stid *s, int flags)
{
if (!s)
@@ -4585,7 +4589,7 @@ nfs4_find_file(struct nfs4_stid *s, int flags)
case NFS4_DELEG_STID:
if (WARN_ON_ONCE(!s->sc_file->fi_deleg_file))
return NULL;
- return get_file(s->sc_file->fi_deleg_file);
+ return nfsd_file_get(s->sc_file->fi_deleg_file);
case NFS4_OPEN_STID:
case NFS4_LOCK_STID:
if (flags & RD_STATE)
@@ -4614,21 +4618,17 @@ nfs4_check_file(struct svc_rqst *rqstp, struct svc_fh *fhp, struct nfs4_stid *s,
struct file **filpp, bool *tmp_file, int flags)
{
int acc = (flags & RD_STATE) ? NFSD_MAY_READ : NFSD_MAY_WRITE;
- struct file *file;
+ struct nfsd_file *nf;
__be32 status;

- file = nfs4_find_file(s, flags);
- if (file) {
+ nf = nfs4_find_file(s, flags);
+ if (nf) {
status = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
acc | NFSD_MAY_OWNER_OVERRIDE);
- if (status) {
- fput(file);
- return status;
- }
-
- *filpp = file;
+ if (status)
+ goto out;
} else {
- status = nfsd_open(rqstp, fhp, S_IFREG, acc, filpp);
+ status = nfsd_file_acquire(rqstp, fhp, acc, &nf);
if (status)
return status;

@@ -4636,7 +4636,10 @@ nfs4_check_file(struct svc_rqst *rqstp, struct svc_fh *fhp, struct nfs4_stid *s,
*tmp_file = true;
}

- return 0;
+ *filpp = get_file(nf->nf_file);
+out:
+ nfsd_file_put(nf);
+ return status;
}

/*
@@ -5362,7 +5365,7 @@ nfsd4_lock(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
struct nfs4_ol_stateid *lock_stp = NULL;
struct nfs4_ol_stateid *open_stp = NULL;
struct nfs4_file *fp;
- struct file *filp = NULL;
+ struct nfsd_file *nf = NULL;
struct file_lock *file_lock = NULL;
struct file_lock *conflock = NULL;
__be32 status = 0;
@@ -5444,8 +5447,8 @@ nfsd4_lock(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
case NFS4_READ_LT:
case NFS4_READW_LT:
spin_lock(&fp->fi_lock);
- filp = find_readable_file_locked(fp);
- if (filp)
+ nf = find_readable_file_locked(fp);
+ if (nf)
get_lock_access(lock_stp, NFS4_SHARE_ACCESS_READ);
spin_unlock(&fp->fi_lock);
file_lock->fl_type = F_RDLCK;
@@ -5453,8 +5456,8 @@ nfsd4_lock(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
case NFS4_WRITE_LT:
case NFS4_WRITEW_LT:
spin_lock(&fp->fi_lock);
- filp = find_writeable_file_locked(fp);
- if (filp)
+ nf = find_writeable_file_locked(fp);
+ if (nf)
get_lock_access(lock_stp, NFS4_SHARE_ACCESS_WRITE);
spin_unlock(&fp->fi_lock);
file_lock->fl_type = F_WRLCK;
@@ -5463,14 +5466,14 @@ nfsd4_lock(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
status = nfserr_inval;
goto out;
}
- if (!filp) {
+ if (!nf) {
status = nfserr_openmode;
goto out;
}

file_lock->fl_owner = (fl_owner_t)lockowner(nfs4_get_stateowner(&lock_sop->lo_owner));
file_lock->fl_pid = current->tgid;
- file_lock->fl_file = filp;
+ file_lock->fl_file = nf->nf_file;
file_lock->fl_flags = FL_POSIX;
file_lock->fl_lmops = &nfsd_posix_mng_ops;
file_lock->fl_start = lock->lk_offset;
@@ -5484,7 +5487,7 @@ nfsd4_lock(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
goto out;
}

- err = vfs_lock_file(filp, F_SETLK, file_lock, conflock);
+ err = vfs_lock_file(nf->nf_file, F_SETLK, file_lock, conflock);
switch (-err) {
case 0: /* success! */
update_stateid(&lock_stp->st_stid.sc_stateid);
@@ -5506,8 +5509,8 @@ nfsd4_lock(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
break;
}
out:
- if (filp)
- fput(filp);
+ if (nf)
+ nfsd_file_put(nf);
if (lock_stp) {
/* Bump seqid manually if the 4.0 replay owner is openowner */
if (cstate->replay_owner &&
@@ -5632,7 +5635,7 @@ nfsd4_locku(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
struct nfsd4_locku *locku)
{
struct nfs4_ol_stateid *stp;
- struct file *filp = NULL;
+ struct nfsd_file *nf;
struct file_lock *file_lock = NULL;
__be32 status;
int err;
@@ -5650,8 +5653,8 @@ nfsd4_locku(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
&stp, nn);
if (status)
goto out;
- filp = find_any_file(stp->st_stid.sc_file);
- if (!filp) {
+ nf = find_any_file(stp->st_stid.sc_file);
+ if (!nf) {
status = nfserr_lock_range;
goto put_stateid;
}
@@ -5659,13 +5662,13 @@ nfsd4_locku(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
if (!file_lock) {
dprintk("NFSD: %s: unable to allocate lock!\n", __func__);
status = nfserr_jukebox;
- goto fput;
+ goto put_file;
}

file_lock->fl_type = F_UNLCK;
file_lock->fl_owner = (fl_owner_t)lockowner(nfs4_get_stateowner(stp->st_stateowner));
file_lock->fl_pid = current->tgid;
- file_lock->fl_file = filp;
+ file_lock->fl_file = nf->nf_file;
file_lock->fl_flags = FL_POSIX;
file_lock->fl_lmops = &nfsd_posix_mng_ops;
file_lock->fl_start = locku->lu_offset;
@@ -5674,15 +5677,15 @@ nfsd4_locku(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
locku->lu_length);
nfs4_transform_lock_offset(file_lock);

- err = vfs_lock_file(filp, F_SETLK, file_lock, NULL);
+ err = vfs_lock_file(nf->nf_file, F_SETLK, file_lock, NULL);
if (err) {
dprintk("NFSD: nfs4_locku: vfs_lock_file failed!\n");
goto out_nfserr;
}
update_stateid(&stp->st_stid.sc_stateid);
memcpy(&locku->lu_stateid, &stp->st_stid.sc_stateid, sizeof(stateid_t));
-fput:
- fput(filp);
+put_file:
+ nfsd_file_put(nf);
put_stateid:
nfs4_put_stid(&stp->st_stid);
out:
@@ -5693,7 +5696,7 @@ out:

out_nfserr:
status = nfserrno(err);
- goto fput;
+ goto put_file;
}

/*
@@ -5706,17 +5709,17 @@ check_for_locks(struct nfs4_file *fp, struct nfs4_lockowner *lowner)
{
struct file_lock *fl;
int status = false;
- struct file *filp = find_any_file(fp);
+ struct nfsd_file *nf = find_any_file(fp);
struct inode *inode;
struct file_lock_context *flctx;

- if (!filp) {
+ if (!nf) {
/* Any valid lock stateid should have some sort of access */
WARN_ON_ONCE(1);
return status;
}

- inode = file_inode(filp);
+ inode = file_inode(nf->nf_file);
flctx = inode->i_flctx;

if (flctx && !list_empty_careful(&flctx->flc_posix)) {
@@ -5729,7 +5732,7 @@ check_for_locks(struct nfs4_file *fp, struct nfs4_lockowner *lowner)
}
spin_unlock(&flctx->flc_lock);
}
- fput(filp);
+ nfsd_file_put(nf);
return status;
}

diff --git a/fs/nfsd/state.h b/fs/nfsd/state.h
index 70b3e51ba089..8a317de773b9 100644
--- a/fs/nfsd/state.h
+++ b/fs/nfsd/state.h
@@ -509,7 +509,7 @@ struct nfs4_file {
*/
atomic_t fi_access[2];
u32 fi_share_deny;
- struct file *fi_deleg_file;
+ struct nfsd_file *fi_deleg_file;
int fi_delegees;
struct knfsd_fh fi_fhandle;
bool fi_had_conflict;
@@ -557,7 +557,7 @@ struct nfs4_layout_stateid {
spinlock_t ls_lock;
struct list_head ls_layouts;
u32 ls_layout_type;
- struct file *ls_file;
+ struct nfsd_file *ls_file;
struct nfsd4_callback ls_recall;
stateid_t ls_recall_sid;
bool ls_recalled;
@@ -620,7 +620,7 @@ static inline void get_nfs4_file(struct nfs4_file *fi)
{
atomic_inc(&fi->fi_ref);
}
-struct file *find_any_file(struct nfs4_file *f);
+struct nfsd_file *find_any_file(struct nfs4_file *f);

/* grace period management */
void nfsd4_end_grace(struct nfsd_net *nn);
--
2.4.3


2015-08-20 11:17:45

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 19/20] nfsd: hook up nfs4_preprocess_stateid_op to the nfsd_file cache

Have nfs4_preprocess_stateid_op pass back a nfsd_file instead of a filp.
Since we now presume that the struct file will be persistent in most
cases, we can stop fiddling with the raparms in the read code. This
also means that we don't really care about the rd_tmp_file field
anymore.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/nfs4proc.c | 32 ++++++++++++++++----------------
fs/nfsd/nfs4state.c | 20 +++++++-------------
fs/nfsd/nfs4xdr.c | 16 +++++-----------
fs/nfsd/state.h | 2 +-
fs/nfsd/xdr4.h | 15 +++++++--------
5 files changed, 36 insertions(+), 49 deletions(-)

diff --git a/fs/nfsd/nfs4proc.c b/fs/nfsd/nfs4proc.c
index b9681ee0ed19..42a3f8b50814 100644
--- a/fs/nfsd/nfs4proc.c
+++ b/fs/nfsd/nfs4proc.c
@@ -758,7 +758,7 @@ nfsd4_read(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
{
__be32 status;

- read->rd_filp = NULL;
+ read->rd_nf = NULL;
if (read->rd_offset >= OFFSET_MAX)
return nfserr_inval;

@@ -775,7 +775,7 @@ nfsd4_read(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,

/* check stateid */
status = nfs4_preprocess_stateid_op(rqstp, cstate, &read->rd_stateid,
- RD_STATE, &read->rd_filp, &read->rd_tmp_file);
+ RD_STATE, &read->rd_nf);
if (status) {
dprintk("NFSD: nfsd4_read: couldn't process stateid!\n");
goto out;
@@ -921,7 +921,7 @@ nfsd4_setattr(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,

if (setattr->sa_iattr.ia_valid & ATTR_SIZE) {
status = nfs4_preprocess_stateid_op(rqstp, cstate,
- &setattr->sa_stateid, WR_STATE, NULL, NULL);
+ &setattr->sa_stateid, WR_STATE, NULL);
if (status) {
dprintk("NFSD: nfsd4_setattr: couldn't process stateid!\n");
return status;
@@ -977,7 +977,7 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
struct nfsd4_write *write)
{
stateid_t *stateid = &write->wr_stateid;
- struct file *filp = NULL;
+ struct nfsd_file *nf = NULL;
__be32 status = nfs_ok;
unsigned long cnt;
int nvecs;
@@ -986,7 +986,7 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
return nfserr_inval;

status = nfs4_preprocess_stateid_op(rqstp, cstate, stateid, WR_STATE,
- &filp, NULL);
+ &nf);
if (status) {
dprintk("NFSD: nfsd4_write: couldn't process stateid!\n");
return status;
@@ -999,10 +999,10 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
nvecs = fill_in_write_vector(rqstp->rq_vec, write);
WARN_ON_ONCE(nvecs > ARRAY_SIZE(rqstp->rq_vec));

- status = nfsd_vfs_write(rqstp, &cstate->current_fh, filp,
+ status = nfsd_vfs_write(rqstp, &cstate->current_fh, nf->nf_file,
write->wr_offset, rqstp->rq_vec, nvecs, &cnt,
&write->wr_how_written);
- fput(filp);
+ nfsd_file_put(nf);

write->wr_bytes_written = cnt;

@@ -1014,21 +1014,21 @@ nfsd4_fallocate(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
struct nfsd4_fallocate *fallocate, int flags)
{
__be32 status = nfserr_notsupp;
- struct file *file;
+ struct nfsd_file *nf;

status = nfs4_preprocess_stateid_op(rqstp, cstate,
&fallocate->falloc_stateid,
- WR_STATE, &file, NULL);
+ WR_STATE, &nf);
if (status != nfs_ok) {
dprintk("NFSD: nfsd4_fallocate: couldn't process stateid!\n");
return status;
}

- status = nfsd4_vfs_fallocate(rqstp, &cstate->current_fh, file,
+ status = nfsd4_vfs_fallocate(rqstp, &cstate->current_fh, nf->nf_file,
fallocate->falloc_offset,
fallocate->falloc_length,
flags);
- fput(file);
+ nfsd_file_put(nf);
return status;
}

@@ -1053,11 +1053,11 @@ nfsd4_seek(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
{
int whence;
__be32 status;
- struct file *file;
+ struct nfsd_file *nf;

status = nfs4_preprocess_stateid_op(rqstp, cstate,
&seek->seek_stateid,
- RD_STATE, &file, NULL);
+ RD_STATE, &nf);
if (status) {
dprintk("NFSD: nfsd4_seek: couldn't process stateid!\n");
return status;
@@ -1079,14 +1079,14 @@ nfsd4_seek(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
* Note: This call does change file->f_pos, but nothing in NFSD
* should ever file->f_pos.
*/
- seek->seek_pos = vfs_llseek(file, seek->seek_offset, whence);
+ seek->seek_pos = vfs_llseek(nf->nf_file, seek->seek_offset, whence);
if (seek->seek_pos < 0)
status = nfserrno(seek->seek_pos);
- else if (seek->seek_pos >= i_size_read(file_inode(file)))
+ else if (seek->seek_pos >= i_size_read(file_inode(nf->nf_file)))
seek->seek_eof = true;

out:
- fput(file);
+ nfsd_file_put(nf);
return status;
}

diff --git a/fs/nfsd/nfs4state.c b/fs/nfsd/nfs4state.c
index f8394a4cd126..c626358c2bad 100644
--- a/fs/nfsd/nfs4state.c
+++ b/fs/nfsd/nfs4state.c
@@ -4615,7 +4615,7 @@ nfs4_check_olstateid(struct svc_fh *fhp, struct nfs4_ol_stateid *ols, int flags)

static __be32
nfs4_check_file(struct svc_rqst *rqstp, struct svc_fh *fhp, struct nfs4_stid *s,
- struct file **filpp, bool *tmp_file, int flags)
+ struct nfsd_file **nfp, int flags)
{
int acc = (flags & RD_STATE) ? NFSD_MAY_READ : NFSD_MAY_WRITE;
struct nfsd_file *nf;
@@ -4631,14 +4631,10 @@ nfs4_check_file(struct svc_rqst *rqstp, struct svc_fh *fhp, struct nfs4_stid *s,
status = nfsd_file_acquire(rqstp, fhp, acc, &nf);
if (status)
return status;
-
- if (tmp_file)
- *tmp_file = true;
}

- *filpp = get_file(nf->nf_file);
+ *nfp = nf;
out:
- nfsd_file_put(nf);
return status;
}

@@ -4648,7 +4644,7 @@ out:
__be32
nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
struct nfsd4_compound_state *cstate, stateid_t *stateid,
- int flags, struct file **filpp, bool *tmp_file)
+ int flags, struct nfsd_file **nfp)
{
struct svc_fh *fhp = &cstate->current_fh;
struct inode *ino = d_inode(fhp->fh_dentry);
@@ -4657,10 +4653,8 @@ nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
struct nfs4_stid *s = NULL;
__be32 status;

- if (filpp)
- *filpp = NULL;
- if (tmp_file)
- *tmp_file = false;
+ if (nfp)
+ *nfp = NULL;

if (grace_disallows_io(net, ino))
return nfserr_grace;
@@ -4697,8 +4691,8 @@ nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
status = nfs4_check_fh(fhp, s);

done:
- if (!status && filpp)
- status = nfs4_check_file(rqstp, fhp, s, filpp, tmp_file, flags);
+ if (status == nfs_ok && nfp)
+ status = nfs4_check_file(rqstp, fhp, s, nfp, flags);
out:
if (s)
nfs4_put_stid(s);
diff --git a/fs/nfsd/nfs4xdr.c b/fs/nfsd/nfs4xdr.c
index 75e0563c09d1..7e25a31f8e60 100644
--- a/fs/nfsd/nfs4xdr.c
+++ b/fs/nfsd/nfs4xdr.c
@@ -49,6 +49,7 @@
#include "cache.h"
#include "netns.h"
#include "pnfs.h"
+#include "filecache.h"

#ifdef CONFIG_NFSD_V4_SECURITY_LABEL
#include <linux/security.h>
@@ -3418,14 +3419,14 @@ nfsd4_encode_read(struct nfsd4_compoundres *resp, __be32 nfserr,
{
unsigned long maxcount;
struct xdr_stream *xdr = &resp->xdr;
- struct file *file = read->rd_filp;
+ struct file *file;
int starting_len = xdr->buf->len;
- struct raparms *ra = NULL;
__be32 *p;

if (nfserr)
goto out;

+ file = read->rd_nf->nf_file;
p = xdr_reserve_space(xdr, 8); /* eof flag and byte count */
if (!p) {
WARN_ON_ONCE(test_bit(RQ_SPLICE_OK, &resp->rqstp->rq_flags));
@@ -3445,24 +3446,17 @@ nfsd4_encode_read(struct nfsd4_compoundres *resp, __be32 nfserr,
(xdr->buf->buflen - xdr->buf->len));
maxcount = min_t(unsigned long, maxcount, read->rd_length);

- if (read->rd_tmp_file)
- ra = nfsd_init_raparms(file);
-
if (file->f_op->splice_read &&
test_bit(RQ_SPLICE_OK, &resp->rqstp->rq_flags))
nfserr = nfsd4_encode_splice_read(resp, read, file, maxcount);
else
nfserr = nfsd4_encode_readv(resp, read, file, maxcount);

- if (ra)
- nfsd_put_raparams(file, ra);
-
if (nfserr)
xdr_truncate_encode(xdr, starting_len);
-
out:
- if (file)
- fput(file);
+ if (read->rd_nf)
+ nfsd_file_put(read->rd_nf);
return nfserr;
}

diff --git a/fs/nfsd/state.h b/fs/nfsd/state.h
index 8a317de773b9..cf7e27199507 100644
--- a/fs/nfsd/state.h
+++ b/fs/nfsd/state.h
@@ -585,7 +585,7 @@ struct nfsd_net;

extern __be32 nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
struct nfsd4_compound_state *cstate, stateid_t *stateid,
- int flags, struct file **filp, bool *tmp_file);
+ int flags, struct nfsd_file **filp);
__be32 nfsd4_lookup_stateid(struct nfsd4_compound_state *cstate,
stateid_t *stateid, unsigned char typemask,
struct nfs4_stid **s, struct nfsd_net *nn);
diff --git a/fs/nfsd/xdr4.h b/fs/nfsd/xdr4.h
index 9f991007a578..ea016fb24675 100644
--- a/fs/nfsd/xdr4.h
+++ b/fs/nfsd/xdr4.h
@@ -268,15 +268,14 @@ struct nfsd4_open_downgrade {


struct nfsd4_read {
- stateid_t rd_stateid; /* request */
- u64 rd_offset; /* request */
- u32 rd_length; /* request */
- int rd_vlen;
- struct file *rd_filp;
- bool rd_tmp_file;
+ stateid_t rd_stateid; /* request */
+ u64 rd_offset; /* request */
+ u32 rd_length; /* request */
+ int rd_vlen;
+ struct nfsd_file *rd_nf;

- struct svc_rqst *rd_rqstp; /* response */
- struct svc_fh * rd_fhp; /* response */
+ struct svc_rqst *rd_rqstp; /* response */
+ struct svc_fh *rd_fhp; /* response */
};

struct nfsd4_readdir {
--
2.4.3


2015-08-20 11:17:45

by Jeff Layton

[permalink] [raw]
Subject: [PATCH v3 20/20] nfsd: rip out the raparms cache

Nothing uses it anymore.

Signed-off-by: Jeff Layton <[email protected]>
---
fs/nfsd/nfssvc.c | 14 +-----
fs/nfsd/vfs.c | 147 -------------------------------------------------------
fs/nfsd/vfs.h | 5 --
3 files changed, 1 insertion(+), 165 deletions(-)

diff --git a/fs/nfsd/nfssvc.c b/fs/nfsd/nfssvc.c
index d816bb3faa6e..d1034d119afb 100644
--- a/fs/nfsd/nfssvc.c
+++ b/fs/nfsd/nfssvc.c
@@ -216,18 +216,9 @@ static int nfsd_startup_generic(int nrservs)
if (nfsd_users++)
return 0;

- /*
- * Readahead param cache - will no-op if it already exists.
- * (Note therefore results will be suboptimal if number of
- * threads is modified after nfsd start.)
- */
- ret = nfsd_racache_init(2*nrservs);
- if (ret)
- goto dec_users;
-
ret = nfsd_file_cache_init();
if (ret)
- goto out_racache;
+ goto dec_users;

ret = nfs4_state_start();
if (ret)
@@ -236,8 +227,6 @@ static int nfsd_startup_generic(int nrservs)

out_file_cache:
nfsd_file_cache_shutdown();
-out_racache:
- nfsd_racache_shutdown();
dec_users:
nfsd_users--;
return ret;
@@ -250,7 +239,6 @@ static void nfsd_shutdown_generic(void)

nfs4_state_shutdown();
nfsd_file_cache_shutdown();
- nfsd_racache_shutdown();
}

static bool nfsd_needs_lockd(void)
diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
index 4cc78a4ec694..9d9e734d1176 100644
--- a/fs/nfsd/vfs.c
+++ b/fs/nfsd/vfs.c
@@ -46,34 +46,6 @@

#define NFSDDBG_FACILITY NFSDDBG_FILEOP

-
-/*
- * This is a cache of readahead params that help us choose the proper
- * readahead strategy. Initially, we set all readahead parameters to 0
- * and let the VFS handle things.
- * If you increase the number of cached files very much, you'll need to
- * add a hash table here.
- */
-struct raparms {
- struct raparms *p_next;
- unsigned int p_count;
- ino_t p_ino;
- dev_t p_dev;
- int p_set;
- struct file_ra_state p_ra;
- unsigned int p_hindex;
-};
-
-struct raparm_hbucket {
- struct raparms *pb_head;
- spinlock_t pb_lock;
-} ____cacheline_aligned_in_smp;
-
-#define RAPARM_HASH_BITS 4
-#define RAPARM_HASH_SIZE (1<<RAPARM_HASH_BITS)
-#define RAPARM_HASH_MASK (RAPARM_HASH_SIZE-1)
-static struct raparm_hbucket raparm_hash[RAPARM_HASH_SIZE];
-
/*
* Called from nfsd_lookup and encode_dirent. Check if we have crossed
* a mount point.
@@ -722,65 +694,6 @@ out:
return err;
}

-struct raparms *
-nfsd_init_raparms(struct file *file)
-{
- struct inode *inode = file_inode(file);
- dev_t dev = inode->i_sb->s_dev;
- ino_t ino = inode->i_ino;
- struct raparms *ra, **rap, **frap = NULL;
- int depth = 0;
- unsigned int hash;
- struct raparm_hbucket *rab;
-
- hash = jhash_2words(dev, ino, 0xfeedbeef) & RAPARM_HASH_MASK;
- rab = &raparm_hash[hash];
-
- spin_lock(&rab->pb_lock);
- for (rap = &rab->pb_head; (ra = *rap); rap = &ra->p_next) {
- if (ra->p_ino == ino && ra->p_dev == dev)
- goto found;
- depth++;
- if (ra->p_count == 0)
- frap = rap;
- }
- depth = nfsdstats.ra_size;
- if (!frap) {
- spin_unlock(&rab->pb_lock);
- return NULL;
- }
- rap = frap;
- ra = *frap;
- ra->p_dev = dev;
- ra->p_ino = ino;
- ra->p_set = 0;
- ra->p_hindex = hash;
-found:
- if (rap != &rab->pb_head) {
- *rap = ra->p_next;
- ra->p_next = rab->pb_head;
- rab->pb_head = ra;
- }
- ra->p_count++;
- nfsdstats.ra_depth[depth*10/nfsdstats.ra_size]++;
- spin_unlock(&rab->pb_lock);
-
- if (ra->p_set)
- file->f_ra = ra->p_ra;
- return ra;
-}
-
-void nfsd_put_raparams(struct file *file, struct raparms *ra)
-{
- struct raparm_hbucket *rab = &raparm_hash[ra->p_hindex];
-
- spin_lock(&rab->pb_lock);
- ra->p_ra = file->f_ra;
- ra->p_set = 1;
- ra->p_count--;
- spin_unlock(&rab->pb_lock);
-}
-
/*
* Grab and keep cached pages associated with a file in the svc_rqst
* so that they can be passed to the network sendmsg/sendpage routines
@@ -2051,63 +1964,3 @@ nfsd_permission(struct svc_rqst *rqstp, struct svc_export *exp,

return err? nfserrno(err) : 0;
}
-
-void
-nfsd_racache_shutdown(void)
-{
- struct raparms *raparm, *last_raparm;
- unsigned int i;
-
- dprintk("nfsd: freeing readahead buffers.\n");
-
- for (i = 0; i < RAPARM_HASH_SIZE; i++) {
- raparm = raparm_hash[i].pb_head;
- while(raparm) {
- last_raparm = raparm;
- raparm = raparm->p_next;
- kfree(last_raparm);
- }
- raparm_hash[i].pb_head = NULL;
- }
-}
-/*
- * Initialize readahead param cache
- */
-int
-nfsd_racache_init(int cache_size)
-{
- int i;
- int j = 0;
- int nperbucket;
- struct raparms **raparm = NULL;
-
-
- if (raparm_hash[0].pb_head)
- return 0;
- nperbucket = DIV_ROUND_UP(cache_size, RAPARM_HASH_SIZE);
- nperbucket = max(2, nperbucket);
- cache_size = nperbucket * RAPARM_HASH_SIZE;
-
- dprintk("nfsd: allocating %d readahead buffers.\n", cache_size);
-
- for (i = 0; i < RAPARM_HASH_SIZE; i++) {
- spin_lock_init(&raparm_hash[i].pb_lock);
-
- raparm = &raparm_hash[i].pb_head;
- for (j = 0; j < nperbucket; j++) {
- *raparm = kzalloc(sizeof(struct raparms), GFP_KERNEL);
- if (!*raparm)
- goto out_nomem;
- raparm = &(*raparm)->p_next;
- }
- *raparm = NULL;
- }
-
- nfsdstats.ra_size = cache_size;
- return 0;
-
-out_nomem:
- dprintk("nfsd: kmalloc failed, freeing readahead buffers\n");
- nfsd_racache_shutdown();
- return -ENOMEM;
-}
diff --git a/fs/nfsd/vfs.h b/fs/nfsd/vfs.h
index a3ec59830297..8b4deac64011 100644
--- a/fs/nfsd/vfs.h
+++ b/fs/nfsd/vfs.h
@@ -39,8 +39,6 @@
typedef int (*nfsd_filldir_t)(void *, const char *, int, loff_t, u64, unsigned);

/* nfsd/vfs.c */
-int nfsd_racache_init(int);
-void nfsd_racache_shutdown(void);
int nfsd_cross_mnt(struct svc_rqst *rqstp, struct dentry **dpp,
struct svc_export **expp);
__be32 nfsd_lookup(struct svc_rqst *, struct svc_fh *,
@@ -105,9 +103,6 @@ __be32 nfsd_statfs(struct svc_rqst *, struct svc_fh *,
__be32 nfsd_permission(struct svc_rqst *, struct svc_export *,
struct dentry *, int);

-struct raparms *nfsd_init_raparms(struct file *file);
-void nfsd_put_raparams(struct file *file, struct raparms *ra);
-
static inline int fh_want_write(struct svc_fh *fh)
{
int ret = mnt_want_write(fh->fh_export->ex_path.mnt);
--
2.4.3


2015-08-20 23:11:40

by Peng Tao

[permalink] [raw]
Subject: Re: [PATCH v3 02/20] nfsd: add a new struct file caching facility to nfsd

On Thu, Aug 20, 2015 at 4:17 AM, Jeff Layton <[email protected]> wrote:
> Currently, NFSv2/3 reads and writes have to open a file, do the read or
> write and then close it again for each RPC. This is highly inefficient,
> especially when the underlying filesystem has a relatively slow open
> routine.
>
> This patch adds a new open file cache to knfsd. Rather than doing an
> open for each RPC, the read/write handlers can call into this cache to
> see if there is one already there for the correct filehandle and
> NFS_MAY_READ/WRITE flags.
>
> If there isn't an entry, then we create a new one and attempt to
> perform the open. If there is, then we wait until the entry is fully
> instantiated and return it if it is at the end of the wait. If it's
> not, then we attempt to take over construction.
>
> Since the main goal is to speed up NFSv2/3 I/O, we don't want to
> close these files on last put of these objects. We need to keep them
> around for a little while since we never know when the next READ/WRITE
> will come in.
>
> Signed-off-by: Jeff Layton <[email protected]>
> ---
> fs/nfsd/Makefile | 3 +-
> fs/nfsd/filecache.c | 273 ++++++++++++++++++++++++++++++++++++++++++++++++++++
> fs/nfsd/filecache.h | 29 ++++++
> fs/nfsd/nfssvc.c | 10 +-
> 4 files changed, 313 insertions(+), 2 deletions(-)
> create mode 100644 fs/nfsd/filecache.c
> create mode 100644 fs/nfsd/filecache.h
>
> diff --git a/fs/nfsd/Makefile b/fs/nfsd/Makefile
> index 9a6028e120c6..8908bb467727 100644
> --- a/fs/nfsd/Makefile
> +++ b/fs/nfsd/Makefile
> @@ -10,7 +10,8 @@ obj-$(CONFIG_NFSD) += nfsd.o
> nfsd-y += trace.o
>
> nfsd-y += nfssvc.o nfsctl.o nfsproc.o nfsfh.o vfs.o \
> - export.o auth.o lockd.o nfscache.o nfsxdr.o stats.o
> + export.o auth.o lockd.o nfscache.o nfsxdr.o \
> + stats.o filecache.o
> nfsd-$(CONFIG_NFSD_FAULT_INJECTION) += fault_inject.o
> nfsd-$(CONFIG_NFSD_V2_ACL) += nfs2acl.o
> nfsd-$(CONFIG_NFSD_V3) += nfs3proc.o nfs3xdr.o
> diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> new file mode 100644
> index 000000000000..5bb56fa9002f
> --- /dev/null
> +++ b/fs/nfsd/filecache.c
> @@ -0,0 +1,273 @@
> +/*
> + * Open file cache.
> + *
> + * (c) 2015 - Jeff Layton <[email protected]>
> + */
> +
> +#include <linux/hash.h>
> +#include <linux/slab.h>
> +#include <linux/hash.h>
> +#include <linux/file.h>
> +#include <linux/sched.h>
> +
> +#include "vfs.h"
> +#include "nfsd.h"
> +#include "nfsfh.h"
> +#include "filecache.h"
> +
> +#define NFSDDBG_FACILITY NFSDDBG_FH
> +
> +/* hash table for nfs4_file */
> +#define NFSD_FILE_HASH_BITS 8
> +#define NFSD_FILE_HASH_SIZE (1 << NFSD_FILE_HASH_BITS)
> +
> +/* We only care about NFSD_MAY_READ/WRITE for this cache */
> +#define NFSD_FILE_MAY_MASK (NFSD_MAY_READ|NFSD_MAY_WRITE)
> +
> +struct nfsd_fcache_bucket {
> + struct hlist_head nfb_head;
> + spinlock_t nfb_lock;
> +};
> +
> +static struct nfsd_fcache_bucket *nfsd_file_hashtbl;
> +
> +static struct nfsd_file *
> +nfsd_file_alloc(struct inode *inode, unsigned int may, unsigned int hashval)
> +{
> + struct nfsd_file *nf;
> +
> + /* FIXME: create a new slabcache for these? */
> + nf = kzalloc(sizeof(*nf), GFP_KERNEL);
> + if (nf) {
> + INIT_HLIST_NODE(&nf->nf_node);
> + INIT_LIST_HEAD(&nf->nf_dispose);
> + nf->nf_inode = inode;
> + nf->nf_hashval = hashval;
> + atomic_set(&nf->nf_ref, 1);
> + nf->nf_may = NFSD_FILE_MAY_MASK & may;
> + }
> + return nf;
> +}
> +
> +static void
> +nfsd_file_put_final(struct nfsd_file *nf)
> +{
> + if (nf->nf_file)
> + fput(nf->nf_file);
> + kfree_rcu(nf, nf_rcu);
> +}
> +
> +static bool
> +nfsd_file_unhash(struct nfsd_file *nf)
> +{
> + lockdep_assert_held(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);
> +
> + if (test_bit(NFSD_FILE_HASHED, &nf->nf_flags)) {
> + clear_bit(NFSD_FILE_HASHED, &nf->nf_flags);
nit: why not test_and_clear_bit()?

> + hlist_del_rcu(&nf->nf_node);
> + return true;
> + }
> + return false;
> +}
> +
> +static void
> +nfsd_file_unhash_and_release_locked(struct nfsd_file *nf, struct list_head *dispose)
> +{
> + lockdep_assert_held(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);
> +
> + if (!nfsd_file_unhash(nf))
> + return;
> + if (!atomic_dec_and_test(&nf->nf_ref))
> + return;
> +
> + list_add(&nf->nf_dispose, dispose);
> +}
> +
> +void
> +nfsd_file_put(struct nfsd_file *nf)
> +{
> + if (!atomic_dec_and_test(&nf->nf_ref))
> + return;
> +
> + WARN_ON(test_bit(NFSD_FILE_HASHED, &nf->nf_flags));
> + nfsd_file_put_final(nf);
> +}
> +
> +struct nfsd_file *
> +nfsd_file_get(struct nfsd_file *nf)
> +{
> + if (likely(atomic_inc_not_zero(&nf->nf_ref)))
> + return nf;
> + return NULL;
> +}
> +
> +static void
> +nfsd_file_dispose_list(struct list_head *dispose)
> +{
> + struct nfsd_file *nf;
> +
> + while(!list_empty(dispose)) {
> + nf = list_first_entry(dispose, struct nfsd_file, nf_dispose);
> + list_del(&nf->nf_dispose);
> + nfsd_file_put_final(nf);
> + }
> +}
> +
> +int
> +nfsd_file_cache_init(void)
> +{
> + unsigned int i;
> +
> + if (nfsd_file_hashtbl)
> + return 0;
> +
> + nfsd_file_hashtbl = kcalloc(NFSD_FILE_HASH_SIZE,
> + sizeof(*nfsd_file_hashtbl), GFP_KERNEL);
> + if (!nfsd_file_hashtbl)
> + goto out_nomem;
> +
> + for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
> + INIT_HLIST_HEAD(&nfsd_file_hashtbl[i].nfb_head);
> + spin_lock_init(&nfsd_file_hashtbl[i].nfb_lock);
> + }
> +
> + return 0;
> +out_nomem:
> + printk(KERN_ERR "nfsd: failed to init nfsd file cache\n");
> + return -ENOMEM;
> +}
> +
> +void
> +nfsd_file_cache_shutdown(void)
> +{
> + unsigned int i;
> + struct nfsd_file *nf;
> + LIST_HEAD(dispose);
> +
> + for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
> + spin_lock(&nfsd_file_hashtbl[i].nfb_lock);
> + while(!hlist_empty(&nfsd_file_hashtbl[i].nfb_head)) {
> + nf = hlist_entry(nfsd_file_hashtbl[i].nfb_head.first,
> + struct nfsd_file, nf_node);
> + nfsd_file_unhash_and_release_locked(nf, &dispose);
> + }
> + spin_unlock(&nfsd_file_hashtbl[i].nfb_lock);
> + nfsd_file_dispose_list(&dispose);
> + }
> + kfree(nfsd_file_hashtbl);
> + nfsd_file_hashtbl = NULL;
> +}
> +
> +/*
> + * Search nfsd_file_hashtbl[] for file. We hash on the filehandle and also on
> + * the NFSD_MAY_READ/WRITE flags. If the file is open for r/w, then it's usable
> + * for either.
> + */
> +static struct nfsd_file *
> +nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
> + unsigned int hashval)
> +{
> + struct nfsd_file *nf;
> + unsigned char need = may_flags & NFSD_FILE_MAY_MASK;
> +
> + hlist_for_each_entry_rcu(nf, &nfsd_file_hashtbl[hashval].nfb_head,
> + nf_node) {
> + if ((need & nf->nf_may) != need)
> + continue;
> + if (nf->nf_inode == inode)
> + return nfsd_file_get(nf);
> + }
> + return NULL;
> +}
> +
> +__be32
> +nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> + unsigned int may_flags, struct nfsd_file **pnf)
> +{
> + __be32 status = nfs_ok;
> + struct nfsd_file *nf, *new = NULL;
> + struct inode *inode;
> + unsigned int hashval;
> +
> + /* FIXME: skip this if fh_dentry is already set? */
> + status = fh_verify(rqstp, fhp, S_IFREG, may_flags);
> + if (status != nfs_ok)
> + return status;
> +
> + /* Mask off any extraneous bits */
> + may_flags &= NFSD_FILE_MAY_MASK;
> +
> + inode = d_inode(fhp->fh_dentry);
> + hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
> +retry:
> + rcu_read_lock();
> + nf = nfsd_file_find_locked(inode, may_flags, hashval);
> + rcu_read_unlock();
> + if (nf)
> + goto wait_for_construction;
> +
> + if (!new) {
> + new = nfsd_file_alloc(inode, may_flags, hashval);
> + if (!new)
> + return nfserr_jukebox;
> + }
> +
> + spin_lock(&nfsd_file_hashtbl[hashval].nfb_lock);
> + nf = nfsd_file_find_locked(inode, may_flags, hashval);
> + if (likely(nf == NULL)) {
> + /* Take reference for the hashtable */
> + atomic_inc(&new->nf_ref);
> + __set_bit(NFSD_FILE_HASHED, &new->nf_flags);
> + __set_bit(NFSD_FILE_PENDING, &new->nf_flags);
> + hlist_add_head_rcu(&new->nf_node,
> + &nfsd_file_hashtbl[hashval].nfb_head);
> + spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> + nf = new;
> + new = NULL;
> + goto open_file;
> + }
> + spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> +
> +wait_for_construction:
> + wait_on_bit(&nf->nf_flags, NFSD_FILE_PENDING, TASK_UNINTERRUPTIBLE);
> +
> + /* Did construction of this file fail? */
> + if (!nf->nf_file) {
> + /*
> + * We can only take over construction for this nfsd_file if the
> + * MAY flags are equal. Otherwise, we put the reference and try
> + * again.
> + */
> + if (may_flags != nf->nf_may) {
> + nfsd_file_put(nf);
> + goto retry;
> + }
> +
> + /* try to take over construction for this file */
> + if (test_and_set_bit(NFSD_FILE_PENDING, &nf->nf_flags))
> + goto wait_for_construction;
> + goto open_file;
> + }
> +
> + /*
> + * We have a file that was opened in the context of another rqst. We
> + * must check permissions. Since we're dealing with open files here,
> + * we always want to set the OWNER_OVERRIDE bit.
> + */
> + status = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
> + may_flags|NFSD_MAY_OWNER_OVERRIDE);
> +out:
> + if (status == nfs_ok)
> + *pnf = nf;
> + else
> + nfsd_file_put(nf);
> +
> + if (new)
> + nfsd_file_put(new);
> + return status;
> +open_file:
> + status = nfsd_open(rqstp, fhp, S_IFREG, may_flags, &nf->nf_file);
> + clear_bit(NFSD_FILE_PENDING, &nf->nf_flags);
DO you need clear_bit_unlock() and smp_mb__after_atomic() to make sure
waiters can see the flag change when they are waken up?

Cheers,
Tao
> + wake_up_bit(&nf->nf_flags, NFSD_FILE_PENDING);
> + goto out;
> +}
> diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
> new file mode 100644
> index 000000000000..b0f500353ed4
> --- /dev/null
> +++ b/fs/nfsd/filecache.h
> @@ -0,0 +1,29 @@
> +#ifndef _FS_NFSD_FILECACHE_H
> +#define _FS_NFSD_FILECACHE_H
> +/*
> + * A representation of a file that has been opened by knfsd. These are hashed
> + * in the hashtable by inode pointer value. Note that this object doesn't
> + * hold a reference to the inode by itself, so the nf_inode pointer should
> + * never be dereferenced, only be used for comparison.
> + */
> +struct nfsd_file {
> + struct hlist_node nf_node;
> + struct list_head nf_dispose;
> + struct rcu_head nf_rcu;
> + struct file *nf_file;
> +#define NFSD_FILE_HASHED (0)
> +#define NFSD_FILE_PENDING (1)
> + unsigned long nf_flags;
> + struct inode *nf_inode;
> + unsigned int nf_hashval;
> + atomic_t nf_ref;
> + unsigned char nf_may;
> +};
> +
> +int nfsd_file_cache_init(void);
> +void nfsd_file_cache_shutdown(void);
> +void nfsd_file_put(struct nfsd_file *nf);
> +struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
> +__be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> + unsigned int may_flags, struct nfsd_file **nfp);
> +#endif /* _FS_NFSD_FILECACHE_H */
> diff --git a/fs/nfsd/nfssvc.c b/fs/nfsd/nfssvc.c
> index ad4e2377dd63..d816bb3faa6e 100644
> --- a/fs/nfsd/nfssvc.c
> +++ b/fs/nfsd/nfssvc.c
> @@ -22,6 +22,7 @@
> #include "cache.h"
> #include "vfs.h"
> #include "netns.h"
> +#include "filecache.h"
>
> #define NFSDDBG_FACILITY NFSDDBG_SVC
>
> @@ -224,11 +225,17 @@ static int nfsd_startup_generic(int nrservs)
> if (ret)
> goto dec_users;
>
> - ret = nfs4_state_start();
> + ret = nfsd_file_cache_init();
> if (ret)
> goto out_racache;
> +
> + ret = nfs4_state_start();
> + if (ret)
> + goto out_file_cache;
> return 0;
>
> +out_file_cache:
> + nfsd_file_cache_shutdown();
> out_racache:
> nfsd_racache_shutdown();
> dec_users:
> @@ -242,6 +249,7 @@ static void nfsd_shutdown_generic(void)
> return;
>
> nfs4_state_shutdown();
> + nfsd_file_cache_shutdown();
> nfsd_racache_shutdown();
> }
>
> --
> 2.4.3
>
> --
> To unsubscribe from this list: send the line "unsubscribe linux-nfs" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html

2015-08-20 23:43:05

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH v3 02/20] nfsd: add a new struct file caching facility to nfsd

On Thu, 20 Aug 2015 16:11:20 -0700
Peng Tao <[email protected]> wrote:

> On Thu, Aug 20, 2015 at 4:17 AM, Jeff Layton <[email protected]> wrote:
> > Currently, NFSv2/3 reads and writes have to open a file, do the read or
> > write and then close it again for each RPC. This is highly inefficient,
> > especially when the underlying filesystem has a relatively slow open
> > routine.
> >
> > This patch adds a new open file cache to knfsd. Rather than doing an
> > open for each RPC, the read/write handlers can call into this cache to
> > see if there is one already there for the correct filehandle and
> > NFS_MAY_READ/WRITE flags.
> >
> > If there isn't an entry, then we create a new one and attempt to
> > perform the open. If there is, then we wait until the entry is fully
> > instantiated and return it if it is at the end of the wait. If it's
> > not, then we attempt to take over construction.
> >
> > Since the main goal is to speed up NFSv2/3 I/O, we don't want to
> > close these files on last put of these objects. We need to keep them
> > around for a little while since we never know when the next READ/WRITE
> > will come in.
> >
> > Signed-off-by: Jeff Layton <[email protected]>
> > ---
> > fs/nfsd/Makefile | 3 +-
> > fs/nfsd/filecache.c | 273 ++++++++++++++++++++++++++++++++++++++++++++++++++++
> > fs/nfsd/filecache.h | 29 ++++++
> > fs/nfsd/nfssvc.c | 10 +-
> > 4 files changed, 313 insertions(+), 2 deletions(-)
> > create mode 100644 fs/nfsd/filecache.c
> > create mode 100644 fs/nfsd/filecache.h
> >
> > diff --git a/fs/nfsd/Makefile b/fs/nfsd/Makefile
> > index 9a6028e120c6..8908bb467727 100644
> > --- a/fs/nfsd/Makefile
> > +++ b/fs/nfsd/Makefile
> > @@ -10,7 +10,8 @@ obj-$(CONFIG_NFSD) += nfsd.o
> > nfsd-y += trace.o
> >
> > nfsd-y += nfssvc.o nfsctl.o nfsproc.o nfsfh.o vfs.o \
> > - export.o auth.o lockd.o nfscache.o nfsxdr.o stats.o
> > + export.o auth.o lockd.o nfscache.o nfsxdr.o \
> > + stats.o filecache.o
> > nfsd-$(CONFIG_NFSD_FAULT_INJECTION) += fault_inject.o
> > nfsd-$(CONFIG_NFSD_V2_ACL) += nfs2acl.o
> > nfsd-$(CONFIG_NFSD_V3) += nfs3proc.o nfs3xdr.o
> > diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> > new file mode 100644
> > index 000000000000..5bb56fa9002f
> > --- /dev/null
> > +++ b/fs/nfsd/filecache.c
> > @@ -0,0 +1,273 @@
> > +/*
> > + * Open file cache.
> > + *
> > + * (c) 2015 - Jeff Layton <[email protected]>
> > + */
> > +
> > +#include <linux/hash.h>
> > +#include <linux/slab.h>
> > +#include <linux/hash.h>
> > +#include <linux/file.h>
> > +#include <linux/sched.h>
> > +
> > +#include "vfs.h"
> > +#include "nfsd.h"
> > +#include "nfsfh.h"
> > +#include "filecache.h"
> > +
> > +#define NFSDDBG_FACILITY NFSDDBG_FH
> > +
> > +/* hash table for nfs4_file */
> > +#define NFSD_FILE_HASH_BITS 8
> > +#define NFSD_FILE_HASH_SIZE (1 << NFSD_FILE_HASH_BITS)
> > +
> > +/* We only care about NFSD_MAY_READ/WRITE for this cache */
> > +#define NFSD_FILE_MAY_MASK (NFSD_MAY_READ|NFSD_MAY_WRITE)
> > +
> > +struct nfsd_fcache_bucket {
> > + struct hlist_head nfb_head;
> > + spinlock_t nfb_lock;
> > +};
> > +
> > +static struct nfsd_fcache_bucket *nfsd_file_hashtbl;
> > +
> > +static struct nfsd_file *
> > +nfsd_file_alloc(struct inode *inode, unsigned int may, unsigned int hashval)
> > +{
> > + struct nfsd_file *nf;
> > +
> > + /* FIXME: create a new slabcache for these? */
> > + nf = kzalloc(sizeof(*nf), GFP_KERNEL);
> > + if (nf) {
> > + INIT_HLIST_NODE(&nf->nf_node);
> > + INIT_LIST_HEAD(&nf->nf_dispose);
> > + nf->nf_inode = inode;
> > + nf->nf_hashval = hashval;
> > + atomic_set(&nf->nf_ref, 1);
> > + nf->nf_may = NFSD_FILE_MAY_MASK & may;
> > + }
> > + return nf;
> > +}
> > +
> > +static void
> > +nfsd_file_put_final(struct nfsd_file *nf)
> > +{
> > + if (nf->nf_file)
> > + fput(nf->nf_file);
> > + kfree_rcu(nf, nf_rcu);
> > +}
> > +
> > +static bool
> > +nfsd_file_unhash(struct nfsd_file *nf)
> > +{
> > + lockdep_assert_held(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);
> > +
> > + if (test_bit(NFSD_FILE_HASHED, &nf->nf_flags)) {
> > + clear_bit(NFSD_FILE_HASHED, &nf->nf_flags);
> nit: why not test_and_clear_bit()?
>
> > + hlist_del_rcu(&nf->nf_node);
> > + return true;
> > + }
> > + return false;
> > +}
> > +
> > +static void
> > +nfsd_file_unhash_and_release_locked(struct nfsd_file *nf, struct list_head *dispose)
> > +{
> > + lockdep_assert_held(&nfsd_file_hashtbl[nf->nf_hashval].nfb_lock);
> > +
> > + if (!nfsd_file_unhash(nf))
> > + return;
> > + if (!atomic_dec_and_test(&nf->nf_ref))
> > + return;
> > +
> > + list_add(&nf->nf_dispose, dispose);
> > +}
> > +
> > +void
> > +nfsd_file_put(struct nfsd_file *nf)
> > +{
> > + if (!atomic_dec_and_test(&nf->nf_ref))
> > + return;
> > +
> > + WARN_ON(test_bit(NFSD_FILE_HASHED, &nf->nf_flags));
> > + nfsd_file_put_final(nf);
> > +}
> > +
> > +struct nfsd_file *
> > +nfsd_file_get(struct nfsd_file *nf)
> > +{
> > + if (likely(atomic_inc_not_zero(&nf->nf_ref)))
> > + return nf;
> > + return NULL;
> > +}
> > +
> > +static void
> > +nfsd_file_dispose_list(struct list_head *dispose)
> > +{
> > + struct nfsd_file *nf;
> > +
> > + while(!list_empty(dispose)) {
> > + nf = list_first_entry(dispose, struct nfsd_file, nf_dispose);
> > + list_del(&nf->nf_dispose);
> > + nfsd_file_put_final(nf);
> > + }
> > +}
> > +
> > +int
> > +nfsd_file_cache_init(void)
> > +{
> > + unsigned int i;
> > +
> > + if (nfsd_file_hashtbl)
> > + return 0;
> > +
> > + nfsd_file_hashtbl = kcalloc(NFSD_FILE_HASH_SIZE,
> > + sizeof(*nfsd_file_hashtbl), GFP_KERNEL);
> > + if (!nfsd_file_hashtbl)
> > + goto out_nomem;
> > +
> > + for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
> > + INIT_HLIST_HEAD(&nfsd_file_hashtbl[i].nfb_head);
> > + spin_lock_init(&nfsd_file_hashtbl[i].nfb_lock);
> > + }
> > +
> > + return 0;
> > +out_nomem:
> > + printk(KERN_ERR "nfsd: failed to init nfsd file cache\n");
> > + return -ENOMEM;
> > +}
> > +
> > +void
> > +nfsd_file_cache_shutdown(void)
> > +{
> > + unsigned int i;
> > + struct nfsd_file *nf;
> > + LIST_HEAD(dispose);
> > +
> > + for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
> > + spin_lock(&nfsd_file_hashtbl[i].nfb_lock);
> > + while(!hlist_empty(&nfsd_file_hashtbl[i].nfb_head)) {
> > + nf = hlist_entry(nfsd_file_hashtbl[i].nfb_head.first,
> > + struct nfsd_file, nf_node);
> > + nfsd_file_unhash_and_release_locked(nf, &dispose);
> > + }
> > + spin_unlock(&nfsd_file_hashtbl[i].nfb_lock);
> > + nfsd_file_dispose_list(&dispose);
> > + }
> > + kfree(nfsd_file_hashtbl);
> > + nfsd_file_hashtbl = NULL;
> > +}
> > +
> > +/*
> > + * Search nfsd_file_hashtbl[] for file. We hash on the filehandle and also on
> > + * the NFSD_MAY_READ/WRITE flags. If the file is open for r/w, then it's usable
> > + * for either.
> > + */
> > +static struct nfsd_file *
> > +nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
> > + unsigned int hashval)
> > +{
> > + struct nfsd_file *nf;
> > + unsigned char need = may_flags & NFSD_FILE_MAY_MASK;
> > +
> > + hlist_for_each_entry_rcu(nf, &nfsd_file_hashtbl[hashval].nfb_head,
> > + nf_node) {
> > + if ((need & nf->nf_may) != need)
> > + continue;
> > + if (nf->nf_inode == inode)
> > + return nfsd_file_get(nf);
> > + }
> > + return NULL;
> > +}
> > +
> > +__be32
> > +nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> > + unsigned int may_flags, struct nfsd_file **pnf)
> > +{
> > + __be32 status = nfs_ok;
> > + struct nfsd_file *nf, *new = NULL;
> > + struct inode *inode;
> > + unsigned int hashval;
> > +
> > + /* FIXME: skip this if fh_dentry is already set? */
> > + status = fh_verify(rqstp, fhp, S_IFREG, may_flags);
> > + if (status != nfs_ok)
> > + return status;
> > +
> > + /* Mask off any extraneous bits */
> > + may_flags &= NFSD_FILE_MAY_MASK;
> > +
> > + inode = d_inode(fhp->fh_dentry);
> > + hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
> > +retry:
> > + rcu_read_lock();
> > + nf = nfsd_file_find_locked(inode, may_flags, hashval);
> > + rcu_read_unlock();
> > + if (nf)
> > + goto wait_for_construction;
> > +
> > + if (!new) {
> > + new = nfsd_file_alloc(inode, may_flags, hashval);
> > + if (!new)
> > + return nfserr_jukebox;
> > + }
> > +
> > + spin_lock(&nfsd_file_hashtbl[hashval].nfb_lock);
> > + nf = nfsd_file_find_locked(inode, may_flags, hashval);
> > + if (likely(nf == NULL)) {
> > + /* Take reference for the hashtable */
> > + atomic_inc(&new->nf_ref);
> > + __set_bit(NFSD_FILE_HASHED, &new->nf_flags);
> > + __set_bit(NFSD_FILE_PENDING, &new->nf_flags);
> > + hlist_add_head_rcu(&new->nf_node,
> > + &nfsd_file_hashtbl[hashval].nfb_head);
> > + spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> > + nf = new;
> > + new = NULL;
> > + goto open_file;
> > + }
> > + spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> > +
> > +wait_for_construction:
> > + wait_on_bit(&nf->nf_flags, NFSD_FILE_PENDING, TASK_UNINTERRUPTIBLE);
> > +
> > + /* Did construction of this file fail? */
> > + if (!nf->nf_file) {
> > + /*
> > + * We can only take over construction for this nfsd_file if the
> > + * MAY flags are equal. Otherwise, we put the reference and try
> > + * again.
> > + */
> > + if (may_flags != nf->nf_may) {
> > + nfsd_file_put(nf);
> > + goto retry;
> > + }
> > +
> > + /* try to take over construction for this file */
> > + if (test_and_set_bit(NFSD_FILE_PENDING, &nf->nf_flags))
> > + goto wait_for_construction;
> > + goto open_file;
> > + }
> > +
> > + /*
> > + * We have a file that was opened in the context of another rqst. We
> > + * must check permissions. Since we're dealing with open files here,
> > + * we always want to set the OWNER_OVERRIDE bit.
> > + */
> > + status = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
> > + may_flags|NFSD_MAY_OWNER_OVERRIDE);
> > +out:
> > + if (status == nfs_ok)
> > + *pnf = nf;
> > + else
> > + nfsd_file_put(nf);
> > +
> > + if (new)
> > + nfsd_file_put(new);
> > + return status;
> > +open_file:
> > + status = nfsd_open(rqstp, fhp, S_IFREG, may_flags, &nf->nf_file);
> > + clear_bit(NFSD_FILE_PENDING, &nf->nf_flags);
> DO you need clear_bit_unlock() and smp_mb__after_atomic() to make sure
> waiters can see the flag change when they are waken up?
>

Good point -- yes, I think we need some barriers here. I'll fix in the next respin.

Thanks!

> Cheers,
> Tao
> > + wake_up_bit(&nf->nf_flags, NFSD_FILE_PENDING);
> > + goto out;
> > +}
> > diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
> > new file mode 100644
> > index 000000000000..b0f500353ed4
> > --- /dev/null
> > +++ b/fs/nfsd/filecache.h
> > @@ -0,0 +1,29 @@
> > +#ifndef _FS_NFSD_FILECACHE_H
> > +#define _FS_NFSD_FILECACHE_H
> > +/*
> > + * A representation of a file that has been opened by knfsd. These are hashed
> > + * in the hashtable by inode pointer value. Note that this object doesn't
> > + * hold a reference to the inode by itself, so the nf_inode pointer should
> > + * never be dereferenced, only be used for comparison.
> > + */
> > +struct nfsd_file {
> > + struct hlist_node nf_node;
> > + struct list_head nf_dispose;
> > + struct rcu_head nf_rcu;
> > + struct file *nf_file;
> > +#define NFSD_FILE_HASHED (0)
> > +#define NFSD_FILE_PENDING (1)
> > + unsigned long nf_flags;
> > + struct inode *nf_inode;
> > + unsigned int nf_hashval;
> > + atomic_t nf_ref;
> > + unsigned char nf_may;
> > +};
> > +
> > +int nfsd_file_cache_init(void);
> > +void nfsd_file_cache_shutdown(void);
> > +void nfsd_file_put(struct nfsd_file *nf);
> > +struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
> > +__be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> > + unsigned int may_flags, struct nfsd_file **nfp);
> > +#endif /* _FS_NFSD_FILECACHE_H */
> > diff --git a/fs/nfsd/nfssvc.c b/fs/nfsd/nfssvc.c
> > index ad4e2377dd63..d816bb3faa6e 100644
> > --- a/fs/nfsd/nfssvc.c
> > +++ b/fs/nfsd/nfssvc.c
> > @@ -22,6 +22,7 @@
> > #include "cache.h"
> > #include "vfs.h"
> > #include "netns.h"
> > +#include "filecache.h"
> >
> > #define NFSDDBG_FACILITY NFSDDBG_SVC
> >
> > @@ -224,11 +225,17 @@ static int nfsd_startup_generic(int nrservs)
> > if (ret)
> > goto dec_users;
> >
> > - ret = nfs4_state_start();
> > + ret = nfsd_file_cache_init();
> > if (ret)
> > goto out_racache;
> > +
> > + ret = nfs4_state_start();
> > + if (ret)
> > + goto out_file_cache;
> > return 0;
> >
> > +out_file_cache:
> > + nfsd_file_cache_shutdown();
> > out_racache:
> > nfsd_racache_shutdown();
> > dec_users:
> > @@ -242,6 +249,7 @@ static void nfsd_shutdown_generic(void)
> > return;
> >
> > nfs4_state_shutdown();
> > + nfsd_file_cache_shutdown();
> > nfsd_racache_shutdown();
> > }
> >
> > --
> > 2.4.3
> >
> > --
> > To unsubscribe from this list: send the line "unsubscribe linux-nfs" in
> > the body of a message to [email protected]
> > More majordomo info at http://vger.kernel.org/majordomo-info.html


--
Jeff Layton <[email protected]>

2015-08-21 01:02:18

by Peng Tao

[permalink] [raw]
Subject: Re: [PATCH v3 15/20] nfsd: call flush_delayed_fput from nfsd_file_close_fh

On Thu, Aug 20, 2015 at 4:17 AM, Jeff Layton <[email protected]> wrote:
> ...when there are open files to be closed.
>
> When knfsd does an fput(), it gets queued to a list and a workqueue job
> is then scheduled to do the actual __fput work. In the case of knfsd
> closing down the file prior to a REMOVE or RENAME, we really want to
> ensure that those files are closed prior to returning. When there are
> files to be closed, call flush_delayed_fput to ensure this.
>
> There are deadlock possibilities if you call flush_delayed_fput while
> holding locks, however. In the case of nfsd_rename, we don't even do the
> lookups of the dentries to be renamed until we've locked for rename.
>
> Once we've figured out what the target dentry is for a rename, check to
> see whether there are cached open files associated with it. If there
> are, then unwind all of the locking, close them all, and then reattempt
> the rename.
>
> Signed-off-by: Jeff Layton <[email protected]>
> ---
> fs/file_table.c | 1 +
> fs/nfsd/filecache.c | 33 ++++++++++++++++++++++++++++++++-
> fs/nfsd/filecache.h | 1 +
> fs/nfsd/trace.h | 10 +++++++++-
> fs/nfsd/vfs.c | 47 +++++++++++++++++++++++++++++++++++++++--------
> 5 files changed, 82 insertions(+), 10 deletions(-)
>
> diff --git a/fs/file_table.c b/fs/file_table.c
> index 7f9d407c7595..33898e72618c 100644
> --- a/fs/file_table.c
> +++ b/fs/file_table.c
> @@ -257,6 +257,7 @@ void flush_delayed_fput(void)
> {
> delayed_fput(NULL);
> }
> +EXPORT_SYMBOL_GPL(flush_delayed_fput);
>
> static DECLARE_DELAYED_WORK(delayed_fput_work, delayed_fput);
>
> diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> index 4bd683f03b6e..b62942ba6e7b 100644
> --- a/fs/nfsd/filecache.c
> +++ b/fs/nfsd/filecache.c
> @@ -284,6 +284,34 @@ nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
> }
>
> /**
> + * nfsd_file_is_cached - are there any cached open files for this fh?
> + * @inode: inode of the file to check
> + *
> + * Scan the hashtable for open files that match this fh. Returns true if there
> + * are any, and false if not.
> + */
> +bool
> +nfsd_file_is_cached(struct inode *inode)
> +{
> + bool ret = false;
> + struct nfsd_file *nf;
> + unsigned int hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
> +
> + rcu_read_lock();
> + hlist_for_each_entry_rcu(nf, &nfsd_file_hashtbl[hashval].nfb_head,
> + nf_node) {
> + if (inode == nf->nf_inode) {
> + ret = true;
> + break;
> + }
> + }
> + rcu_read_unlock();
> + trace_nfsd_file_is_cached(hashval, inode, (int)ret);
> + return ret;
> +}
> +
> +
> +/**
> * nfsd_file_close_inode - attempt to forcibly close a nfsd_file
> * @inode: inode of the file to attempt to remove
> *
> @@ -305,7 +333,10 @@ nfsd_file_close_inode(struct inode *inode)
> }
> spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> trace_nfsd_file_close_inode(hashval, inode, !list_empty(&dispose));
> - nfsd_file_dispose_list(&dispose);
> + if (!list_empty(&dispose)) {
> + nfsd_file_dispose_list(&dispose);
> + flush_delayed_fput();
It looks like flush_delayed_fput() is not exported symbol?

And if flush_delayed_fput() is acceptable, it looks like __fput_sync()
is a better fit, because knfsd would not try to do all the delayed
fput() work, just the dispose list...

Cheers,
Tao

> + }
> }
>
> __be32
> diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
> index 191cdb25aa66..4a873efb7953 100644
> --- a/fs/nfsd/filecache.h
> +++ b/fs/nfsd/filecache.h
> @@ -27,6 +27,7 @@ void nfsd_file_cache_shutdown(void);
> void nfsd_file_put(struct nfsd_file *nf);
> struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
> void nfsd_file_close_inode(struct inode *inode);
> +bool nfsd_file_is_cached(struct inode *inode);
> __be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> unsigned int may_flags, struct nfsd_file **nfp);
> #endif /* _FS_NFSD_FILECACHE_H */
> diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
> index 95af3b9c7b66..fc6d8ee51a00 100644
> --- a/fs/nfsd/trace.h
> +++ b/fs/nfsd/trace.h
> @@ -140,7 +140,7 @@ TRACE_EVENT(nfsd_file_acquire,
> be32_to_cpu(__entry->status))
> );
>
> -TRACE_EVENT(nfsd_file_close_inode,
> +DECLARE_EVENT_CLASS(nfsd_file_search_class,
> TP_PROTO(unsigned int hash, struct inode *inode, int found),
> TP_ARGS(hash, inode, found),
> TP_STRUCT__entry(
> @@ -156,6 +156,14 @@ TRACE_EVENT(nfsd_file_close_inode,
> TP_printk("hash=0x%x inode=0x%p found=%d", __entry->hash,
> __entry->inode, __entry->found)
> );
> +
> +#define DEFINE_NFSD_FILE_SEARCH_EVENT(name) \
> +DEFINE_EVENT(nfsd_file_search_class, name, \
> + TP_PROTO(unsigned int hash, struct inode *inode, int found), \
> + TP_ARGS(hash, inode, found))
> +
> +DEFINE_NFSD_FILE_SEARCH_EVENT(nfsd_file_close_inode);
> +DEFINE_NFSD_FILE_SEARCH_EVENT(nfsd_file_is_cached);
> #endif /* _NFSD_TRACE_H */
>
> #undef TRACE_INCLUDE_PATH
> diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
> index 98d3b9d96480..4cc78a4ec694 100644
> --- a/fs/nfsd/vfs.c
> +++ b/fs/nfsd/vfs.c
> @@ -1592,6 +1592,17 @@ nfsd_close_cached_files(struct dentry *dentry)
> nfsd_file_close_inode(inode);
> }
>
> +static bool
> +nfsd_has_cached_files(struct dentry *dentry)
> +{
> + bool ret = false;
> + struct inode *inode = d_inode(dentry);
> +
> + if (inode && S_ISREG(inode->i_mode))
> + ret = nfsd_file_is_cached(inode);
> + return ret;
> +}
> +
> /*
> * Rename a file
> * N.B. After this call _both_ ffhp and tfhp need an fh_put
> @@ -1604,6 +1615,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> struct inode *fdir, *tdir;
> __be32 err;
> int host_err;
> + bool has_cached = false;
>
> err = fh_verify(rqstp, ffhp, S_IFDIR, NFSD_MAY_REMOVE);
> if (err)
> @@ -1622,6 +1634,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> if (!flen || isdotent(fname, flen) || !tlen || isdotent(tname, tlen))
> goto out;
>
> +retry:
> host_err = fh_want_write(ffhp);
> if (host_err) {
> err = nfserrno(host_err);
> @@ -1661,12 +1674,16 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
> goto out_dput_new;
>
> - nfsd_close_cached_files(ndentry);
> - host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
> - if (!host_err) {
> - host_err = commit_metadata(tfhp);
> - if (!host_err)
> - host_err = commit_metadata(ffhp);
> + if (nfsd_has_cached_files(ndentry)) {
> + has_cached = true;
> + goto out_dput_old;
> + } else {
> + host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
> + if (!host_err) {
> + host_err = commit_metadata(tfhp);
> + if (!host_err)
> + host_err = commit_metadata(ffhp);
> + }
> }
> out_dput_new:
> dput(ndentry);
> @@ -1679,12 +1696,26 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> * as that would do the wrong thing if the two directories
> * were the same, so again we do it by hand.
> */
> - fill_post_wcc(ffhp);
> - fill_post_wcc(tfhp);
> + if (!has_cached) {
> + fill_post_wcc(ffhp);
> + fill_post_wcc(tfhp);
> + }
> unlock_rename(tdentry, fdentry);
> ffhp->fh_locked = tfhp->fh_locked = 0;
> fh_drop_write(ffhp);
>
> + /*
> + * If the target dentry has cached open files, then we need to try to
> + * close them prior to doing the rename. Flushing delayed fput
> + * shouldn't be done with locks held however, so we delay it until this
> + * point and then reattempt the whole shebang.
> + */
> + if (has_cached) {
> + has_cached = false;
> + nfsd_close_cached_files(ndentry);
> + dput(ndentry);
> + goto retry;
> + }
> out:
> return err;
> }
> --
> 2.4.3
>
> --
> To unsubscribe from this list: send the line "unsubscribe linux-nfs" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html

2015-08-21 01:28:25

by Peng Tao

[permalink] [raw]
Subject: Re: [PATCH v3 19/20] nfsd: hook up nfs4_preprocess_stateid_op to the nfsd_file cache

On Thu, Aug 20, 2015 at 4:17 AM, Jeff Layton <[email protected]> wrote:
> Have nfs4_preprocess_stateid_op pass back a nfsd_file instead of a filp.
> Since we now presume that the struct file will be persistent in most
> cases, we can stop fiddling with the raparms in the read code. This
> also means that we don't really care about the rd_tmp_file field
> anymore.
>
> Signed-off-by: Jeff Layton <[email protected]>
> ---
> fs/nfsd/nfs4proc.c | 32 ++++++++++++++++----------------
> fs/nfsd/nfs4state.c | 20 +++++++-------------
> fs/nfsd/nfs4xdr.c | 16 +++++-----------
> fs/nfsd/state.h | 2 +-
> fs/nfsd/xdr4.h | 15 +++++++--------
> 5 files changed, 36 insertions(+), 49 deletions(-)
>
> diff --git a/fs/nfsd/nfs4proc.c b/fs/nfsd/nfs4proc.c
> index b9681ee0ed19..42a3f8b50814 100644
> --- a/fs/nfsd/nfs4proc.c
> +++ b/fs/nfsd/nfs4proc.c
> @@ -758,7 +758,7 @@ nfsd4_read(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> {
> __be32 status;
>
> - read->rd_filp = NULL;
> + read->rd_nf = NULL;
> if (read->rd_offset >= OFFSET_MAX)
> return nfserr_inval;
>
> @@ -775,7 +775,7 @@ nfsd4_read(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
>
> /* check stateid */
> status = nfs4_preprocess_stateid_op(rqstp, cstate, &read->rd_stateid,
> - RD_STATE, &read->rd_filp, &read->rd_tmp_file);
> + RD_STATE, &read->rd_nf);
> if (status) {
> dprintk("NFSD: nfsd4_read: couldn't process stateid!\n");
> goto out;
> @@ -921,7 +921,7 @@ nfsd4_setattr(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
>
> if (setattr->sa_iattr.ia_valid & ATTR_SIZE) {
> status = nfs4_preprocess_stateid_op(rqstp, cstate,
> - &setattr->sa_stateid, WR_STATE, NULL, NULL);
> + &setattr->sa_stateid, WR_STATE, NULL);
> if (status) {
> dprintk("NFSD: nfsd4_setattr: couldn't process stateid!\n");
> return status;
> @@ -977,7 +977,7 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> struct nfsd4_write *write)
> {
> stateid_t *stateid = &write->wr_stateid;
> - struct file *filp = NULL;
> + struct nfsd_file *nf = NULL;
> __be32 status = nfs_ok;
> unsigned long cnt;
> int nvecs;
> @@ -986,7 +986,7 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> return nfserr_inval;
>
> status = nfs4_preprocess_stateid_op(rqstp, cstate, stateid, WR_STATE,
> - &filp, NULL);
> + &nf);
> if (status) {
> dprintk("NFSD: nfsd4_write: couldn't process stateid!\n");
> return status;
> @@ -999,10 +999,10 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> nvecs = fill_in_write_vector(rqstp->rq_vec, write);
> WARN_ON_ONCE(nvecs > ARRAY_SIZE(rqstp->rq_vec));
>
> - status = nfsd_vfs_write(rqstp, &cstate->current_fh, filp,
> + status = nfsd_vfs_write(rqstp, &cstate->current_fh, nf->nf_file,
> write->wr_offset, rqstp->rq_vec, nvecs, &cnt,
> &write->wr_how_written);
> - fput(filp);
> + nfsd_file_put(nf);
>
> write->wr_bytes_written = cnt;
>
> @@ -1014,21 +1014,21 @@ nfsd4_fallocate(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> struct nfsd4_fallocate *fallocate, int flags)
> {
> __be32 status = nfserr_notsupp;
> - struct file *file;
> + struct nfsd_file *nf;
>
> status = nfs4_preprocess_stateid_op(rqstp, cstate,
> &fallocate->falloc_stateid,
> - WR_STATE, &file, NULL);
> + WR_STATE, &nf);
> if (status != nfs_ok) {
> dprintk("NFSD: nfsd4_fallocate: couldn't process stateid!\n");
> return status;
> }
>
> - status = nfsd4_vfs_fallocate(rqstp, &cstate->current_fh, file,
> + status = nfsd4_vfs_fallocate(rqstp, &cstate->current_fh, nf->nf_file,
> fallocate->falloc_offset,
> fallocate->falloc_length,
> flags);
> - fput(file);
> + nfsd_file_put(nf);
> return status;
> }
>
> @@ -1053,11 +1053,11 @@ nfsd4_seek(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> {
> int whence;
> __be32 status;
> - struct file *file;
> + struct nfsd_file *nf;
>
> status = nfs4_preprocess_stateid_op(rqstp, cstate,
> &seek->seek_stateid,
> - RD_STATE, &file, NULL);
> + RD_STATE, &nf);
> if (status) {
> dprintk("NFSD: nfsd4_seek: couldn't process stateid!\n");
> return status;
> @@ -1079,14 +1079,14 @@ nfsd4_seek(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> * Note: This call does change file->f_pos, but nothing in NFSD
> * should ever file->f_pos.
> */
> - seek->seek_pos = vfs_llseek(file, seek->seek_offset, whence);
> + seek->seek_pos = vfs_llseek(nf->nf_file, seek->seek_offset, whence);
> if (seek->seek_pos < 0)
> status = nfserrno(seek->seek_pos);
> - else if (seek->seek_pos >= i_size_read(file_inode(file)))
> + else if (seek->seek_pos >= i_size_read(file_inode(nf->nf_file)))
> seek->seek_eof = true;
>
> out:
> - fput(file);
> + nfsd_file_put(nf);
> return status;
> }
>
> diff --git a/fs/nfsd/nfs4state.c b/fs/nfsd/nfs4state.c
> index f8394a4cd126..c626358c2bad 100644
> --- a/fs/nfsd/nfs4state.c
> +++ b/fs/nfsd/nfs4state.c
> @@ -4615,7 +4615,7 @@ nfs4_check_olstateid(struct svc_fh *fhp, struct nfs4_ol_stateid *ols, int flags)
>
> static __be32
> nfs4_check_file(struct svc_rqst *rqstp, struct svc_fh *fhp, struct nfs4_stid *s,
> - struct file **filpp, bool *tmp_file, int flags)
> + struct nfsd_file **nfp, int flags)
> {
> int acc = (flags & RD_STATE) ? NFSD_MAY_READ : NFSD_MAY_WRITE;
> struct nfsd_file *nf;
> @@ -4631,14 +4631,10 @@ nfs4_check_file(struct svc_rqst *rqstp, struct svc_fh *fhp, struct nfs4_stid *s,
> status = nfsd_file_acquire(rqstp, fhp, acc, &nf);
> if (status)
> return status;
> -
> - if (tmp_file)
> - *tmp_file = true;
> }
>
> - *filpp = get_file(nf->nf_file);
> + *nfp = nf;
> out:
> - nfsd_file_put(nf);
If nfsd_permission() fails, nf is leaked. Previous patch has:

@@ -4614,21 +4618,17 @@ nfs4_check_file(struct svc_rqst *rqstp, struct
svc_fh *fhp, struct nfs4_stid *s,
struct file **filpp, bool *tmp_file, int flags)
{
int acc = (flags & RD_STATE) ? NFSD_MAY_READ : NFSD_MAY_WRITE;
- struct file *file;
+ struct nfsd_file *nf;
__be32 status;

- file = nfs4_find_file(s, flags);
- if (file) {
+ nf = nfs4_find_file(s, flags);
+ if (nf) {
status = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
acc | NFSD_MAY_OWNER_OVERRIDE);
- if (status) {
- fput(file);
- return status;
- }
-
- *filpp = file;
+ if (status)
+ goto out;
} else {
- status = nfsd_open(rqstp, fhp, S_IFREG, acc, filpp);
+ status = nfsd_file_acquire(rqstp, fhp, acc, &nf);
if (status)
return status;

Cheers,
Tao

> return status;
> }
>
> @@ -4648,7 +4644,7 @@ out:
> __be32
> nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
> struct nfsd4_compound_state *cstate, stateid_t *stateid,
> - int flags, struct file **filpp, bool *tmp_file)
> + int flags, struct nfsd_file **nfp)
> {
> struct svc_fh *fhp = &cstate->current_fh;
> struct inode *ino = d_inode(fhp->fh_dentry);
> @@ -4657,10 +4653,8 @@ nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
> struct nfs4_stid *s = NULL;
> __be32 status;
>
> - if (filpp)
> - *filpp = NULL;
> - if (tmp_file)
> - *tmp_file = false;
> + if (nfp)
> + *nfp = NULL;
>
> if (grace_disallows_io(net, ino))
> return nfserr_grace;
> @@ -4697,8 +4691,8 @@ nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
> status = nfs4_check_fh(fhp, s);
>
> done:
> - if (!status && filpp)
> - status = nfs4_check_file(rqstp, fhp, s, filpp, tmp_file, flags);
> + if (status == nfs_ok && nfp)
> + status = nfs4_check_file(rqstp, fhp, s, nfp, flags);
> out:
> if (s)
> nfs4_put_stid(s);
> diff --git a/fs/nfsd/nfs4xdr.c b/fs/nfsd/nfs4xdr.c
> index 75e0563c09d1..7e25a31f8e60 100644
> --- a/fs/nfsd/nfs4xdr.c
> +++ b/fs/nfsd/nfs4xdr.c
> @@ -49,6 +49,7 @@
> #include "cache.h"
> #include "netns.h"
> #include "pnfs.h"
> +#include "filecache.h"
>
> #ifdef CONFIG_NFSD_V4_SECURITY_LABEL
> #include <linux/security.h>
> @@ -3418,14 +3419,14 @@ nfsd4_encode_read(struct nfsd4_compoundres *resp, __be32 nfserr,
> {
> unsigned long maxcount;
> struct xdr_stream *xdr = &resp->xdr;
> - struct file *file = read->rd_filp;
> + struct file *file;
> int starting_len = xdr->buf->len;
> - struct raparms *ra = NULL;
> __be32 *p;
>
> if (nfserr)
> goto out;
>
> + file = read->rd_nf->nf_file;
> p = xdr_reserve_space(xdr, 8); /* eof flag and byte count */
> if (!p) {
> WARN_ON_ONCE(test_bit(RQ_SPLICE_OK, &resp->rqstp->rq_flags));
> @@ -3445,24 +3446,17 @@ nfsd4_encode_read(struct nfsd4_compoundres *resp, __be32 nfserr,
> (xdr->buf->buflen - xdr->buf->len));
> maxcount = min_t(unsigned long, maxcount, read->rd_length);
>
> - if (read->rd_tmp_file)
> - ra = nfsd_init_raparms(file);
> -
> if (file->f_op->splice_read &&
> test_bit(RQ_SPLICE_OK, &resp->rqstp->rq_flags))
> nfserr = nfsd4_encode_splice_read(resp, read, file, maxcount);
> else
> nfserr = nfsd4_encode_readv(resp, read, file, maxcount);
>
> - if (ra)
> - nfsd_put_raparams(file, ra);
> -
> if (nfserr)
> xdr_truncate_encode(xdr, starting_len);
> -
> out:
> - if (file)
> - fput(file);
> + if (read->rd_nf)
> + nfsd_file_put(read->rd_nf);
> return nfserr;
> }
>
> diff --git a/fs/nfsd/state.h b/fs/nfsd/state.h
> index 8a317de773b9..cf7e27199507 100644
> --- a/fs/nfsd/state.h
> +++ b/fs/nfsd/state.h
> @@ -585,7 +585,7 @@ struct nfsd_net;
>
> extern __be32 nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
> struct nfsd4_compound_state *cstate, stateid_t *stateid,
> - int flags, struct file **filp, bool *tmp_file);
> + int flags, struct nfsd_file **filp);
> __be32 nfsd4_lookup_stateid(struct nfsd4_compound_state *cstate,
> stateid_t *stateid, unsigned char typemask,
> struct nfs4_stid **s, struct nfsd_net *nn);
> diff --git a/fs/nfsd/xdr4.h b/fs/nfsd/xdr4.h
> index 9f991007a578..ea016fb24675 100644
> --- a/fs/nfsd/xdr4.h
> +++ b/fs/nfsd/xdr4.h
> @@ -268,15 +268,14 @@ struct nfsd4_open_downgrade {
>
>
> struct nfsd4_read {
> - stateid_t rd_stateid; /* request */
> - u64 rd_offset; /* request */
> - u32 rd_length; /* request */
> - int rd_vlen;
> - struct file *rd_filp;
> - bool rd_tmp_file;
> + stateid_t rd_stateid; /* request */
> + u64 rd_offset; /* request */
> + u32 rd_length; /* request */
> + int rd_vlen;
> + struct nfsd_file *rd_nf;
>
> - struct svc_rqst *rd_rqstp; /* response */
> - struct svc_fh * rd_fhp; /* response */
> + struct svc_rqst *rd_rqstp; /* response */
> + struct svc_fh *rd_fhp; /* response */
> };
>
> struct nfsd4_readdir {
> --
> 2.4.3
>
> --
> To unsubscribe from this list: send the line "unsubscribe linux-nfs" in
> the body of a message to [email protected]
> More majordomo info at http://vger.kernel.org/majordomo-info.html

2015-08-21 02:18:46

by Peng Tao

[permalink] [raw]
Subject: Re: [PATCH v3 15/20] nfsd: call flush_delayed_fput from nfsd_file_close_fh

On Thu, Aug 20, 2015 at 6:01 PM, Peng Tao <[email protected]> wrote:
> On Thu, Aug 20, 2015 at 4:17 AM, Jeff Layton <[email protected]> wrote:
>> ...when there are open files to be closed.
>>
>> When knfsd does an fput(), it gets queued to a list and a workqueue job
>> is then scheduled to do the actual __fput work. In the case of knfsd
>> closing down the file prior to a REMOVE or RENAME, we really want to
>> ensure that those files are closed prior to returning. When there are
>> files to be closed, call flush_delayed_fput to ensure this.
>>
>> There are deadlock possibilities if you call flush_delayed_fput while
>> holding locks, however. In the case of nfsd_rename, we don't even do the
>> lookups of the dentries to be renamed until we've locked for rename.
>>
>> Once we've figured out what the target dentry is for a rename, check to
>> see whether there are cached open files associated with it. If there
>> are, then unwind all of the locking, close them all, and then reattempt
>> the rename.
>>
>> Signed-off-by: Jeff Layton <[email protected]>
>> ---
>> fs/file_table.c | 1 +
>> fs/nfsd/filecache.c | 33 ++++++++++++++++++++++++++++++++-
>> fs/nfsd/filecache.h | 1 +
>> fs/nfsd/trace.h | 10 +++++++++-
>> fs/nfsd/vfs.c | 47 +++++++++++++++++++++++++++++++++++++++--------
>> 5 files changed, 82 insertions(+), 10 deletions(-)
>>
>> diff --git a/fs/file_table.c b/fs/file_table.c
>> index 7f9d407c7595..33898e72618c 100644
>> --- a/fs/file_table.c
>> +++ b/fs/file_table.c
>> @@ -257,6 +257,7 @@ void flush_delayed_fput(void)
>> {
>> delayed_fput(NULL);
>> }
>> +EXPORT_SYMBOL_GPL(flush_delayed_fput);
>>
>> static DECLARE_DELAYED_WORK(delayed_fput_work, delayed_fput);
>>
>> diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
>> index 4bd683f03b6e..b62942ba6e7b 100644
>> --- a/fs/nfsd/filecache.c
>> +++ b/fs/nfsd/filecache.c
>> @@ -284,6 +284,34 @@ nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
>> }
>>
>> /**
>> + * nfsd_file_is_cached - are there any cached open files for this fh?
>> + * @inode: inode of the file to check
>> + *
>> + * Scan the hashtable for open files that match this fh. Returns true if there
>> + * are any, and false if not.
>> + */
>> +bool
>> +nfsd_file_is_cached(struct inode *inode)
>> +{
>> + bool ret = false;
>> + struct nfsd_file *nf;
>> + unsigned int hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
>> +
>> + rcu_read_lock();
>> + hlist_for_each_entry_rcu(nf, &nfsd_file_hashtbl[hashval].nfb_head,
>> + nf_node) {
>> + if (inode == nf->nf_inode) {
>> + ret = true;
>> + break;
>> + }
>> + }
>> + rcu_read_unlock();
>> + trace_nfsd_file_is_cached(hashval, inode, (int)ret);
>> + return ret;
>> +}
>> +
>> +
>> +/**
>> * nfsd_file_close_inode - attempt to forcibly close a nfsd_file
>> * @inode: inode of the file to attempt to remove
>> *
>> @@ -305,7 +333,10 @@ nfsd_file_close_inode(struct inode *inode)
>> }
>> spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
>> trace_nfsd_file_close_inode(hashval, inode, !list_empty(&dispose));
>> - nfsd_file_dispose_list(&dispose);
>> + if (!list_empty(&dispose)) {
>> + nfsd_file_dispose_list(&dispose);
>> + flush_delayed_fput();
> It looks like flush_delayed_fput() is not exported symbol?
>
> And if flush_delayed_fput() is acceptable, it looks like __fput_sync()
> is a better fit, because knfsd would not try to do all the delayed
> fput() work, just the dispose list...
oh, just saw that flush_delayed_fput() is exported in this patch!
sorry for the noise. But I still think __fput_sync() might be a better
fit, despite the assertion there... I'm fine with settling with
flush_delayed_fput() though since calling __fput_sync() from a kernel
thread might get more objections.

Cheers,
Tao

>
> Cheers,
> Tao
>
>> + }
>> }
>>
>> __be32
>> diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
>> index 191cdb25aa66..4a873efb7953 100644
>> --- a/fs/nfsd/filecache.h
>> +++ b/fs/nfsd/filecache.h
>> @@ -27,6 +27,7 @@ void nfsd_file_cache_shutdown(void);
>> void nfsd_file_put(struct nfsd_file *nf);
>> struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
>> void nfsd_file_close_inode(struct inode *inode);
>> +bool nfsd_file_is_cached(struct inode *inode);
>> __be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
>> unsigned int may_flags, struct nfsd_file **nfp);
>> #endif /* _FS_NFSD_FILECACHE_H */
>> diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
>> index 95af3b9c7b66..fc6d8ee51a00 100644
>> --- a/fs/nfsd/trace.h
>> +++ b/fs/nfsd/trace.h
>> @@ -140,7 +140,7 @@ TRACE_EVENT(nfsd_file_acquire,
>> be32_to_cpu(__entry->status))
>> );
>>
>> -TRACE_EVENT(nfsd_file_close_inode,
>> +DECLARE_EVENT_CLASS(nfsd_file_search_class,
>> TP_PROTO(unsigned int hash, struct inode *inode, int found),
>> TP_ARGS(hash, inode, found),
>> TP_STRUCT__entry(
>> @@ -156,6 +156,14 @@ TRACE_EVENT(nfsd_file_close_inode,
>> TP_printk("hash=0x%x inode=0x%p found=%d", __entry->hash,
>> __entry->inode, __entry->found)
>> );
>> +
>> +#define DEFINE_NFSD_FILE_SEARCH_EVENT(name) \
>> +DEFINE_EVENT(nfsd_file_search_class, name, \
>> + TP_PROTO(unsigned int hash, struct inode *inode, int found), \
>> + TP_ARGS(hash, inode, found))
>> +
>> +DEFINE_NFSD_FILE_SEARCH_EVENT(nfsd_file_close_inode);
>> +DEFINE_NFSD_FILE_SEARCH_EVENT(nfsd_file_is_cached);
>> #endif /* _NFSD_TRACE_H */
>>
>> #undef TRACE_INCLUDE_PATH
>> diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
>> index 98d3b9d96480..4cc78a4ec694 100644
>> --- a/fs/nfsd/vfs.c
>> +++ b/fs/nfsd/vfs.c
>> @@ -1592,6 +1592,17 @@ nfsd_close_cached_files(struct dentry *dentry)
>> nfsd_file_close_inode(inode);
>> }
>>
>> +static bool
>> +nfsd_has_cached_files(struct dentry *dentry)
>> +{
>> + bool ret = false;
>> + struct inode *inode = d_inode(dentry);
>> +
>> + if (inode && S_ISREG(inode->i_mode))
>> + ret = nfsd_file_is_cached(inode);
>> + return ret;
>> +}
>> +
>> /*
>> * Rename a file
>> * N.B. After this call _both_ ffhp and tfhp need an fh_put
>> @@ -1604,6 +1615,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
>> struct inode *fdir, *tdir;
>> __be32 err;
>> int host_err;
>> + bool has_cached = false;
>>
>> err = fh_verify(rqstp, ffhp, S_IFDIR, NFSD_MAY_REMOVE);
>> if (err)
>> @@ -1622,6 +1634,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
>> if (!flen || isdotent(fname, flen) || !tlen || isdotent(tname, tlen))
>> goto out;
>>
>> +retry:
>> host_err = fh_want_write(ffhp);
>> if (host_err) {
>> err = nfserrno(host_err);
>> @@ -1661,12 +1674,16 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
>> if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
>> goto out_dput_new;
>>
>> - nfsd_close_cached_files(ndentry);
>> - host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
>> - if (!host_err) {
>> - host_err = commit_metadata(tfhp);
>> - if (!host_err)
>> - host_err = commit_metadata(ffhp);
>> + if (nfsd_has_cached_files(ndentry)) {
>> + has_cached = true;
>> + goto out_dput_old;
>> + } else {
>> + host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
>> + if (!host_err) {
>> + host_err = commit_metadata(tfhp);
>> + if (!host_err)
>> + host_err = commit_metadata(ffhp);
>> + }
>> }
>> out_dput_new:
>> dput(ndentry);
>> @@ -1679,12 +1696,26 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
>> * as that would do the wrong thing if the two directories
>> * were the same, so again we do it by hand.
>> */
>> - fill_post_wcc(ffhp);
>> - fill_post_wcc(tfhp);
>> + if (!has_cached) {
>> + fill_post_wcc(ffhp);
>> + fill_post_wcc(tfhp);
>> + }
>> unlock_rename(tdentry, fdentry);
>> ffhp->fh_locked = tfhp->fh_locked = 0;
>> fh_drop_write(ffhp);
>>
>> + /*
>> + * If the target dentry has cached open files, then we need to try to
>> + * close them prior to doing the rename. Flushing delayed fput
>> + * shouldn't be done with locks held however, so we delay it until this
>> + * point and then reattempt the whole shebang.
>> + */
>> + if (has_cached) {
>> + has_cached = false;
>> + nfsd_close_cached_files(ndentry);
>> + dput(ndentry);
>> + goto retry;
>> + }
>> out:
>> return err;
>> }
>> --
>> 2.4.3
>>
>> --
>> To unsubscribe from this list: send the line "unsubscribe linux-nfs" in
>> the body of a message to [email protected]
>> More majordomo info at http://vger.kernel.org/majordomo-info.html

2015-08-21 09:36:48

by Vladimir Davydov

[permalink] [raw]
Subject: Re: [PATCH v3 03/20] list_lru: add list_lru_rotate

On Thu, Aug 20, 2015 at 07:17:03AM -0400, Jeff Layton wrote:
> Add a function that can move an entry to the MRU end of the list.
>
> Cc: Andrew Morton <[email protected]>
> Cc: Vladimir Davydov <[email protected]>
> Cc: [email protected]
> Signed-off-by: Jeff Layton <[email protected]>

Reviewed-by: Vladimir Davydov <[email protected]>

2015-08-21 11:21:23

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH v3 15/20] nfsd: call flush_delayed_fput from nfsd_file_close_fh

On Thu, 20 Aug 2015 19:18:25 -0700
Peng Tao <[email protected]> wrote:

> On Thu, Aug 20, 2015 at 6:01 PM, Peng Tao <[email protected]> wrote:
> > On Thu, Aug 20, 2015 at 4:17 AM, Jeff Layton <[email protected]> wrote:
> >> ...when there are open files to be closed.
> >>
> >> When knfsd does an fput(), it gets queued to a list and a workqueue job
> >> is then scheduled to do the actual __fput work. In the case of knfsd
> >> closing down the file prior to a REMOVE or RENAME, we really want to
> >> ensure that those files are closed prior to returning. When there are
> >> files to be closed, call flush_delayed_fput to ensure this.
> >>
> >> There are deadlock possibilities if you call flush_delayed_fput while
> >> holding locks, however. In the case of nfsd_rename, we don't even do the
> >> lookups of the dentries to be renamed until we've locked for rename.
> >>
> >> Once we've figured out what the target dentry is for a rename, check to
> >> see whether there are cached open files associated with it. If there
> >> are, then unwind all of the locking, close them all, and then reattempt
> >> the rename.
> >>
> >> Signed-off-by: Jeff Layton <[email protected]>
> >> ---
> >> fs/file_table.c | 1 +
> >> fs/nfsd/filecache.c | 33 ++++++++++++++++++++++++++++++++-
> >> fs/nfsd/filecache.h | 1 +
> >> fs/nfsd/trace.h | 10 +++++++++-
> >> fs/nfsd/vfs.c | 47 +++++++++++++++++++++++++++++++++++++++--------
> >> 5 files changed, 82 insertions(+), 10 deletions(-)
> >>
> >> diff --git a/fs/file_table.c b/fs/file_table.c
> >> index 7f9d407c7595..33898e72618c 100644
> >> --- a/fs/file_table.c
> >> +++ b/fs/file_table.c
> >> @@ -257,6 +257,7 @@ void flush_delayed_fput(void)
> >> {
> >> delayed_fput(NULL);
> >> }
> >> +EXPORT_SYMBOL_GPL(flush_delayed_fput);
> >>
> >> static DECLARE_DELAYED_WORK(delayed_fput_work, delayed_fput);
> >>
> >> diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> >> index 4bd683f03b6e..b62942ba6e7b 100644
> >> --- a/fs/nfsd/filecache.c
> >> +++ b/fs/nfsd/filecache.c
> >> @@ -284,6 +284,34 @@ nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
> >> }
> >>
> >> /**
> >> + * nfsd_file_is_cached - are there any cached open files for this fh?
> >> + * @inode: inode of the file to check
> >> + *
> >> + * Scan the hashtable for open files that match this fh. Returns true if there
> >> + * are any, and false if not.
> >> + */
> >> +bool
> >> +nfsd_file_is_cached(struct inode *inode)
> >> +{
> >> + bool ret = false;
> >> + struct nfsd_file *nf;
> >> + unsigned int hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
> >> +
> >> + rcu_read_lock();
> >> + hlist_for_each_entry_rcu(nf, &nfsd_file_hashtbl[hashval].nfb_head,
> >> + nf_node) {
> >> + if (inode == nf->nf_inode) {
> >> + ret = true;
> >> + break;
> >> + }
> >> + }
> >> + rcu_read_unlock();
> >> + trace_nfsd_file_is_cached(hashval, inode, (int)ret);
> >> + return ret;
> >> +}
> >> +
> >> +
> >> +/**
> >> * nfsd_file_close_inode - attempt to forcibly close a nfsd_file
> >> * @inode: inode of the file to attempt to remove
> >> *
> >> @@ -305,7 +333,10 @@ nfsd_file_close_inode(struct inode *inode)
> >> }
> >> spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> >> trace_nfsd_file_close_inode(hashval, inode, !list_empty(&dispose));
> >> - nfsd_file_dispose_list(&dispose);
> >> + if (!list_empty(&dispose)) {
> >> + nfsd_file_dispose_list(&dispose);
> >> + flush_delayed_fput();
> > It looks like flush_delayed_fput() is not exported symbol?
> >
> > And if flush_delayed_fput() is acceptable, it looks like __fput_sync()
> > is a better fit, because knfsd would not try to do all the delayed
> > fput() work, just the dispose list...
> oh, just saw that flush_delayed_fput() is exported in this patch!
> sorry for the noise. But I still think __fput_sync() might be a better
> fit, despite the assertion there... I'm fine with settling with
> flush_delayed_fput() though since calling __fput_sync() from a kernel
> thread might get more objections.
>
> Cheers,
> Tao
>

I looked at __fput_sync when I first rolled this patch, but it's a
little less convenient to use. __fput_sync is a synchronous analogue to
fput -- so you have to ensure that you use it instead of fput.

To make that work here, we'd need a separate set of destruction
routines that uses __fput_sync instead of fput. Certainly we can do
that if necessary, but I don't think it's really worth it.

The downside of course is that we might end up with this thread doing a
little extra __fput work if there happened to be other things queued
onto the delayed_fput_list, but I'm not too concerned about that.

> >
> > Cheers,
> > Tao
> >
> >> + }
> >> }
> >>
> >> __be32
> >> diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
> >> index 191cdb25aa66..4a873efb7953 100644
> >> --- a/fs/nfsd/filecache.h
> >> +++ b/fs/nfsd/filecache.h
> >> @@ -27,6 +27,7 @@ void nfsd_file_cache_shutdown(void);
> >> void nfsd_file_put(struct nfsd_file *nf);
> >> struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
> >> void nfsd_file_close_inode(struct inode *inode);
> >> +bool nfsd_file_is_cached(struct inode *inode);
> >> __be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> >> unsigned int may_flags, struct nfsd_file **nfp);
> >> #endif /* _FS_NFSD_FILECACHE_H */
> >> diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
> >> index 95af3b9c7b66..fc6d8ee51a00 100644
> >> --- a/fs/nfsd/trace.h
> >> +++ b/fs/nfsd/trace.h
> >> @@ -140,7 +140,7 @@ TRACE_EVENT(nfsd_file_acquire,
> >> be32_to_cpu(__entry->status))
> >> );
> >>
> >> -TRACE_EVENT(nfsd_file_close_inode,
> >> +DECLARE_EVENT_CLASS(nfsd_file_search_class,
> >> TP_PROTO(unsigned int hash, struct inode *inode, int found),
> >> TP_ARGS(hash, inode, found),
> >> TP_STRUCT__entry(
> >> @@ -156,6 +156,14 @@ TRACE_EVENT(nfsd_file_close_inode,
> >> TP_printk("hash=0x%x inode=0x%p found=%d", __entry->hash,
> >> __entry->inode, __entry->found)
> >> );
> >> +
> >> +#define DEFINE_NFSD_FILE_SEARCH_EVENT(name) \
> >> +DEFINE_EVENT(nfsd_file_search_class, name, \
> >> + TP_PROTO(unsigned int hash, struct inode *inode, int found), \
> >> + TP_ARGS(hash, inode, found))
> >> +
> >> +DEFINE_NFSD_FILE_SEARCH_EVENT(nfsd_file_close_inode);
> >> +DEFINE_NFSD_FILE_SEARCH_EVENT(nfsd_file_is_cached);
> >> #endif /* _NFSD_TRACE_H */
> >>
> >> #undef TRACE_INCLUDE_PATH
> >> diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
> >> index 98d3b9d96480..4cc78a4ec694 100644
> >> --- a/fs/nfsd/vfs.c
> >> +++ b/fs/nfsd/vfs.c
> >> @@ -1592,6 +1592,17 @@ nfsd_close_cached_files(struct dentry *dentry)
> >> nfsd_file_close_inode(inode);
> >> }
> >>
> >> +static bool
> >> +nfsd_has_cached_files(struct dentry *dentry)
> >> +{
> >> + bool ret = false;
> >> + struct inode *inode = d_inode(dentry);
> >> +
> >> + if (inode && S_ISREG(inode->i_mode))
> >> + ret = nfsd_file_is_cached(inode);
> >> + return ret;
> >> +}
> >> +
> >> /*
> >> * Rename a file
> >> * N.B. After this call _both_ ffhp and tfhp need an fh_put
> >> @@ -1604,6 +1615,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> >> struct inode *fdir, *tdir;
> >> __be32 err;
> >> int host_err;
> >> + bool has_cached = false;
> >>
> >> err = fh_verify(rqstp, ffhp, S_IFDIR, NFSD_MAY_REMOVE);
> >> if (err)
> >> @@ -1622,6 +1634,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> >> if (!flen || isdotent(fname, flen) || !tlen || isdotent(tname, tlen))
> >> goto out;
> >>
> >> +retry:
> >> host_err = fh_want_write(ffhp);
> >> if (host_err) {
> >> err = nfserrno(host_err);
> >> @@ -1661,12 +1674,16 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> >> if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
> >> goto out_dput_new;
> >>
> >> - nfsd_close_cached_files(ndentry);
> >> - host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
> >> - if (!host_err) {
> >> - host_err = commit_metadata(tfhp);
> >> - if (!host_err)
> >> - host_err = commit_metadata(ffhp);
> >> + if (nfsd_has_cached_files(ndentry)) {
> >> + has_cached = true;
> >> + goto out_dput_old;
> >> + } else {
> >> + host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
> >> + if (!host_err) {
> >> + host_err = commit_metadata(tfhp);
> >> + if (!host_err)
> >> + host_err = commit_metadata(ffhp);
> >> + }
> >> }
> >> out_dput_new:
> >> dput(ndentry);
> >> @@ -1679,12 +1696,26 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> >> * as that would do the wrong thing if the two directories
> >> * were the same, so again we do it by hand.
> >> */
> >> - fill_post_wcc(ffhp);
> >> - fill_post_wcc(tfhp);
> >> + if (!has_cached) {
> >> + fill_post_wcc(ffhp);
> >> + fill_post_wcc(tfhp);
> >> + }
> >> unlock_rename(tdentry, fdentry);
> >> ffhp->fh_locked = tfhp->fh_locked = 0;
> >> fh_drop_write(ffhp);
> >>
> >> + /*
> >> + * If the target dentry has cached open files, then we need to try to
> >> + * close them prior to doing the rename. Flushing delayed fput
> >> + * shouldn't be done with locks held however, so we delay it until this
> >> + * point and then reattempt the whole shebang.
> >> + */
> >> + if (has_cached) {
> >> + has_cached = false;
> >> + nfsd_close_cached_files(ndentry);
> >> + dput(ndentry);
> >> + goto retry;
> >> + }
> >> out:
> >> return err;
> >> }
> >> --
> >> 2.4.3
> >>
> >> --
> >> To unsubscribe from this list: send the line "unsubscribe linux-nfs" in
> >> the body of a message to [email protected]
> >> More majordomo info at http://vger.kernel.org/majordomo-info.html


--
Jeff Layton <[email protected]>

2015-08-21 11:23:48

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH v3 19/20] nfsd: hook up nfs4_preprocess_stateid_op to the nfsd_file cache

On Thu, 20 Aug 2015 18:28:05 -0700
Peng Tao <[email protected]> wrote:

> On Thu, Aug 20, 2015 at 4:17 AM, Jeff Layton <[email protected]> wrote:
> > Have nfs4_preprocess_stateid_op pass back a nfsd_file instead of a filp.
> > Since we now presume that the struct file will be persistent in most
> > cases, we can stop fiddling with the raparms in the read code. This
> > also means that we don't really care about the rd_tmp_file field
> > anymore.
> >
> > Signed-off-by: Jeff Layton <[email protected]>
> > ---
> > fs/nfsd/nfs4proc.c | 32 ++++++++++++++++----------------
> > fs/nfsd/nfs4state.c | 20 +++++++-------------
> > fs/nfsd/nfs4xdr.c | 16 +++++-----------
> > fs/nfsd/state.h | 2 +-
> > fs/nfsd/xdr4.h | 15 +++++++--------
> > 5 files changed, 36 insertions(+), 49 deletions(-)
> >
> > diff --git a/fs/nfsd/nfs4proc.c b/fs/nfsd/nfs4proc.c
> > index b9681ee0ed19..42a3f8b50814 100644
> > --- a/fs/nfsd/nfs4proc.c
> > +++ b/fs/nfsd/nfs4proc.c
> > @@ -758,7 +758,7 @@ nfsd4_read(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> > {
> > __be32 status;
> >
> > - read->rd_filp = NULL;
> > + read->rd_nf = NULL;
> > if (read->rd_offset >= OFFSET_MAX)
> > return nfserr_inval;
> >
> > @@ -775,7 +775,7 @@ nfsd4_read(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> >
> > /* check stateid */
> > status = nfs4_preprocess_stateid_op(rqstp, cstate, &read->rd_stateid,
> > - RD_STATE, &read->rd_filp, &read->rd_tmp_file);
> > + RD_STATE, &read->rd_nf);
> > if (status) {
> > dprintk("NFSD: nfsd4_read: couldn't process stateid!\n");
> > goto out;
> > @@ -921,7 +921,7 @@ nfsd4_setattr(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> >
> > if (setattr->sa_iattr.ia_valid & ATTR_SIZE) {
> > status = nfs4_preprocess_stateid_op(rqstp, cstate,
> > - &setattr->sa_stateid, WR_STATE, NULL, NULL);
> > + &setattr->sa_stateid, WR_STATE, NULL);
> > if (status) {
> > dprintk("NFSD: nfsd4_setattr: couldn't process stateid!\n");
> > return status;
> > @@ -977,7 +977,7 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> > struct nfsd4_write *write)
> > {
> > stateid_t *stateid = &write->wr_stateid;
> > - struct file *filp = NULL;
> > + struct nfsd_file *nf = NULL;
> > __be32 status = nfs_ok;
> > unsigned long cnt;
> > int nvecs;
> > @@ -986,7 +986,7 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> > return nfserr_inval;
> >
> > status = nfs4_preprocess_stateid_op(rqstp, cstate, stateid, WR_STATE,
> > - &filp, NULL);
> > + &nf);
> > if (status) {
> > dprintk("NFSD: nfsd4_write: couldn't process stateid!\n");
> > return status;
> > @@ -999,10 +999,10 @@ nfsd4_write(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> > nvecs = fill_in_write_vector(rqstp->rq_vec, write);
> > WARN_ON_ONCE(nvecs > ARRAY_SIZE(rqstp->rq_vec));
> >
> > - status = nfsd_vfs_write(rqstp, &cstate->current_fh, filp,
> > + status = nfsd_vfs_write(rqstp, &cstate->current_fh, nf->nf_file,
> > write->wr_offset, rqstp->rq_vec, nvecs, &cnt,
> > &write->wr_how_written);
> > - fput(filp);
> > + nfsd_file_put(nf);
> >
> > write->wr_bytes_written = cnt;
> >
> > @@ -1014,21 +1014,21 @@ nfsd4_fallocate(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> > struct nfsd4_fallocate *fallocate, int flags)
> > {
> > __be32 status = nfserr_notsupp;
> > - struct file *file;
> > + struct nfsd_file *nf;
> >
> > status = nfs4_preprocess_stateid_op(rqstp, cstate,
> > &fallocate->falloc_stateid,
> > - WR_STATE, &file, NULL);
> > + WR_STATE, &nf);
> > if (status != nfs_ok) {
> > dprintk("NFSD: nfsd4_fallocate: couldn't process stateid!\n");
> > return status;
> > }
> >
> > - status = nfsd4_vfs_fallocate(rqstp, &cstate->current_fh, file,
> > + status = nfsd4_vfs_fallocate(rqstp, &cstate->current_fh, nf->nf_file,
> > fallocate->falloc_offset,
> > fallocate->falloc_length,
> > flags);
> > - fput(file);
> > + nfsd_file_put(nf);
> > return status;
> > }
> >
> > @@ -1053,11 +1053,11 @@ nfsd4_seek(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> > {
> > int whence;
> > __be32 status;
> > - struct file *file;
> > + struct nfsd_file *nf;
> >
> > status = nfs4_preprocess_stateid_op(rqstp, cstate,
> > &seek->seek_stateid,
> > - RD_STATE, &file, NULL);
> > + RD_STATE, &nf);
> > if (status) {
> > dprintk("NFSD: nfsd4_seek: couldn't process stateid!\n");
> > return status;
> > @@ -1079,14 +1079,14 @@ nfsd4_seek(struct svc_rqst *rqstp, struct nfsd4_compound_state *cstate,
> > * Note: This call does change file->f_pos, but nothing in NFSD
> > * should ever file->f_pos.
> > */
> > - seek->seek_pos = vfs_llseek(file, seek->seek_offset, whence);
> > + seek->seek_pos = vfs_llseek(nf->nf_file, seek->seek_offset, whence);
> > if (seek->seek_pos < 0)
> > status = nfserrno(seek->seek_pos);
> > - else if (seek->seek_pos >= i_size_read(file_inode(file)))
> > + else if (seek->seek_pos >= i_size_read(file_inode(nf->nf_file)))
> > seek->seek_eof = true;
> >
> > out:
> > - fput(file);
> > + nfsd_file_put(nf);
> > return status;
> > }
> >
> > diff --git a/fs/nfsd/nfs4state.c b/fs/nfsd/nfs4state.c
> > index f8394a4cd126..c626358c2bad 100644
> > --- a/fs/nfsd/nfs4state.c
> > +++ b/fs/nfsd/nfs4state.c
> > @@ -4615,7 +4615,7 @@ nfs4_check_olstateid(struct svc_fh *fhp, struct nfs4_ol_stateid *ols, int flags)
> >
> > static __be32
> > nfs4_check_file(struct svc_rqst *rqstp, struct svc_fh *fhp, struct nfs4_stid *s,
> > - struct file **filpp, bool *tmp_file, int flags)
> > + struct nfsd_file **nfp, int flags)
> > {
> > int acc = (flags & RD_STATE) ? NFSD_MAY_READ : NFSD_MAY_WRITE;
> > struct nfsd_file *nf;
> > @@ -4631,14 +4631,10 @@ nfs4_check_file(struct svc_rqst *rqstp, struct svc_fh *fhp, struct nfs4_stid *s,
> > status = nfsd_file_acquire(rqstp, fhp, acc, &nf);
> > if (status)
> > return status;
> > -
> > - if (tmp_file)
> > - *tmp_file = true;
> > }
> >
> > - *filpp = get_file(nf->nf_file);
> > + *nfp = nf;
> > out:
> > - nfsd_file_put(nf);
> If nfsd_permission() fails, nf is leaked. Previous patch has:
>
> @@ -4614,21 +4618,17 @@ nfs4_check_file(struct svc_rqst *rqstp, struct
> svc_fh *fhp, struct nfs4_stid *s,
> struct file **filpp, bool *tmp_file, int flags)
> {
> int acc = (flags & RD_STATE) ? NFSD_MAY_READ : NFSD_MAY_WRITE;
> - struct file *file;
> + struct nfsd_file *nf;
> __be32 status;
>
> - file = nfs4_find_file(s, flags);
> - if (file) {
> + nf = nfs4_find_file(s, flags);
> + if (nf) {
> status = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
> acc | NFSD_MAY_OWNER_OVERRIDE);
> - if (status) {
> - fput(file);
> - return status;
> - }
> -
> - *filpp = file;
> + if (status)
> + goto out;
> } else {
> - status = nfsd_open(rqstp, fhp, S_IFREG, acc, filpp);
> + status = nfsd_file_acquire(rqstp, fhp, acc, &nf);
> if (status)
> return status;
>
> Cheers,
> Tao
>

Good catch. I'll fix that.

Thanks,
Jeff

> > return status;
> > }
> >
> > @@ -4648,7 +4644,7 @@ out:
> > __be32
> > nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
> > struct nfsd4_compound_state *cstate, stateid_t *stateid,
> > - int flags, struct file **filpp, bool *tmp_file)
> > + int flags, struct nfsd_file **nfp)
> > {
> > struct svc_fh *fhp = &cstate->current_fh;
> > struct inode *ino = d_inode(fhp->fh_dentry);
> > @@ -4657,10 +4653,8 @@ nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
> > struct nfs4_stid *s = NULL;
> > __be32 status;
> >
> > - if (filpp)
> > - *filpp = NULL;
> > - if (tmp_file)
> > - *tmp_file = false;
> > + if (nfp)
> > + *nfp = NULL;
> >
> > if (grace_disallows_io(net, ino))
> > return nfserr_grace;
> > @@ -4697,8 +4691,8 @@ nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
> > status = nfs4_check_fh(fhp, s);
> >
> > done:
> > - if (!status && filpp)
> > - status = nfs4_check_file(rqstp, fhp, s, filpp, tmp_file, flags);
> > + if (status == nfs_ok && nfp)
> > + status = nfs4_check_file(rqstp, fhp, s, nfp, flags);
> > out:
> > if (s)
> > nfs4_put_stid(s);
> > diff --git a/fs/nfsd/nfs4xdr.c b/fs/nfsd/nfs4xdr.c
> > index 75e0563c09d1..7e25a31f8e60 100644
> > --- a/fs/nfsd/nfs4xdr.c
> > +++ b/fs/nfsd/nfs4xdr.c
> > @@ -49,6 +49,7 @@
> > #include "cache.h"
> > #include "netns.h"
> > #include "pnfs.h"
> > +#include "filecache.h"
> >
> > #ifdef CONFIG_NFSD_V4_SECURITY_LABEL
> > #include <linux/security.h>
> > @@ -3418,14 +3419,14 @@ nfsd4_encode_read(struct nfsd4_compoundres *resp, __be32 nfserr,
> > {
> > unsigned long maxcount;
> > struct xdr_stream *xdr = &resp->xdr;
> > - struct file *file = read->rd_filp;
> > + struct file *file;
> > int starting_len = xdr->buf->len;
> > - struct raparms *ra = NULL;
> > __be32 *p;
> >
> > if (nfserr)
> > goto out;
> >
> > + file = read->rd_nf->nf_file;
> > p = xdr_reserve_space(xdr, 8); /* eof flag and byte count */
> > if (!p) {
> > WARN_ON_ONCE(test_bit(RQ_SPLICE_OK, &resp->rqstp->rq_flags));
> > @@ -3445,24 +3446,17 @@ nfsd4_encode_read(struct nfsd4_compoundres *resp, __be32 nfserr,
> > (xdr->buf->buflen - xdr->buf->len));
> > maxcount = min_t(unsigned long, maxcount, read->rd_length);
> >
> > - if (read->rd_tmp_file)
> > - ra = nfsd_init_raparms(file);
> > -
> > if (file->f_op->splice_read &&
> > test_bit(RQ_SPLICE_OK, &resp->rqstp->rq_flags))
> > nfserr = nfsd4_encode_splice_read(resp, read, file, maxcount);
> > else
> > nfserr = nfsd4_encode_readv(resp, read, file, maxcount);
> >
> > - if (ra)
> > - nfsd_put_raparams(file, ra);
> > -
> > if (nfserr)
> > xdr_truncate_encode(xdr, starting_len);
> > -
> > out:
> > - if (file)
> > - fput(file);
> > + if (read->rd_nf)
> > + nfsd_file_put(read->rd_nf);
> > return nfserr;
> > }
> >
> > diff --git a/fs/nfsd/state.h b/fs/nfsd/state.h
> > index 8a317de773b9..cf7e27199507 100644
> > --- a/fs/nfsd/state.h
> > +++ b/fs/nfsd/state.h
> > @@ -585,7 +585,7 @@ struct nfsd_net;
> >
> > extern __be32 nfs4_preprocess_stateid_op(struct svc_rqst *rqstp,
> > struct nfsd4_compound_state *cstate, stateid_t *stateid,
> > - int flags, struct file **filp, bool *tmp_file);
> > + int flags, struct nfsd_file **filp);
> > __be32 nfsd4_lookup_stateid(struct nfsd4_compound_state *cstate,
> > stateid_t *stateid, unsigned char typemask,
> > struct nfs4_stid **s, struct nfsd_net *nn);
> > diff --git a/fs/nfsd/xdr4.h b/fs/nfsd/xdr4.h
> > index 9f991007a578..ea016fb24675 100644
> > --- a/fs/nfsd/xdr4.h
> > +++ b/fs/nfsd/xdr4.h
> > @@ -268,15 +268,14 @@ struct nfsd4_open_downgrade {
> >
> >
> > struct nfsd4_read {
> > - stateid_t rd_stateid; /* request */
> > - u64 rd_offset; /* request */
> > - u32 rd_length; /* request */
> > - int rd_vlen;
> > - struct file *rd_filp;
> > - bool rd_tmp_file;
> > + stateid_t rd_stateid; /* request */
> > + u64 rd_offset; /* request */
> > + u32 rd_length; /* request */
> > + int rd_vlen;
> > + struct nfsd_file *rd_nf;
> >
> > - struct svc_rqst *rd_rqstp; /* response */
> > - struct svc_fh * rd_fhp; /* response */
> > + struct svc_rqst *rd_rqstp; /* response */
> > + struct svc_fh *rd_fhp; /* response */
> > };
> >
> > struct nfsd4_readdir {
> > --
> > 2.4.3
> >
> > --
> > To unsubscribe from this list: send the line "unsubscribe linux-nfs" in
> > the body of a message to [email protected]
> > More majordomo info at http://vger.kernel.org/majordomo-info.html


--
Jeff Layton <[email protected]>

2015-08-26 19:49:24

by J. Bruce Fields

[permalink] [raw]
Subject: Re: [PATCH v3 06/20] locks/nfsd: create a new notifier chain for lease attempts

On Thu, Aug 20, 2015 at 07:17:06AM -0400, Jeff Layton wrote:
> With the new file caching infrastructure in nfsd, we can end up holding
> files open for an indefinite period of time, even when they are still
> idle. This may prevent the kernel from handing out leases on the file,
> which we don't really want to block.
>
> Fix this by running a blocking notifier call chain whenever on any
> lease attempt. nfsd can then purge the cache for that inode before
> returning.

Could this be expensive? There's potentially a lease attempt on every
NFSv4 open.

Could we use the cache to decide whether a delegation's a good idea?
(So, skip requesting the delegation if the cache gives evidence of
recent activity on the file that would have conflicted with the
delegation we're about to grant.) That might avoid unnecessary cache
purges too.

--b.

>
> Signed-off-by: Jeff Layton <[email protected]>
> ---
> fs/locks.c | 15 +++++++++++++++
> fs/nfsd/filecache.c | 27 +++++++++++++++++++++++++++
> include/linux/fs.h | 1 +
> 3 files changed, 43 insertions(+)
>
> diff --git a/fs/locks.c b/fs/locks.c
> index d3d558ba4da7..c81b96159e5c 100644
> --- a/fs/locks.c
> +++ b/fs/locks.c
> @@ -167,6 +167,13 @@ DEFINE_STATIC_LGLOCK(file_lock_lglock);
> static DEFINE_PER_CPU(struct hlist_head, file_lock_list);
>
> /*
> + * Some subsystems would like to be notified if someone attempts to set a
> + * lease on a file. This notifier chain will be called whenever this occurs.
> + */
> +BLOCKING_NOTIFIER_HEAD(lease_notifier_chain);
> +EXPORT_SYMBOL_GPL(lease_notifier_chain);
> +
> +/*
> * The blocked_hash is used to find POSIX lock loops for deadlock detection.
> * It is protected by blocked_lock_lock.
> *
> @@ -1795,10 +1802,18 @@ EXPORT_SYMBOL(generic_setlease);
> *
> * The "priv" pointer is passed directly to the lm_setup function as-is. It
> * may be NULL if the lm_setup operation doesn't require it.
> + *
> + * Kernel subsystems can also register to be notified on any attempt to set
> + * a new lease with the lease_notifier_chain. This is used by (e.g.) nfsd
> + * to close files that it may have cached when there is an attempt to set a
> + * conflicting lease.
> */
> int
> vfs_setlease(struct file *filp, long arg, struct file_lock **lease, void **priv)
> {
> + if (arg != F_UNLCK)
> + blocking_notifier_call_chain(&lease_notifier_chain, arg, *lease);
> +
> if (filp->f_op->setlease)
> return filp->f_op->setlease(filp, arg, lease, priv);
> else
> diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> index 669e62f6f4f6..77041967d8ff 100644
> --- a/fs/nfsd/filecache.c
> +++ b/fs/nfsd/filecache.c
> @@ -158,6 +158,22 @@ static struct shrinker nfsd_file_shrinker = {
> .seeks = 1,
> };
>
> +static int
> +nfsd_file_lease_notifier_call(struct notifier_block *nb, unsigned long arg,
> + void *data)
> +{
> + struct file_lock *fl = data;
> +
> + /* Don't close files if we're the one trying to set the lease */
> + if (fl->fl_type == FL_LEASE)
> + nfsd_file_close_inode(file_inode(fl->fl_file));
> + return 0;
> +}
> +
> +static struct notifier_block nfsd_file_lease_notifier = {
> + .notifier_call = nfsd_file_lease_notifier_call,
> +};
> +
> int
> nfsd_file_cache_init(void)
> {
> @@ -186,12 +202,21 @@ nfsd_file_cache_init(void)
> goto out_lru;
> }
>
> + ret = blocking_notifier_chain_register(&lease_notifier_chain,
> + &nfsd_file_lease_notifier);
> + if (ret) {
> + pr_err("nfsd: unable to register lease notifier: %d\n", ret);
> + goto out_shrinker;
> + }
> +
> for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
> INIT_HLIST_HEAD(&nfsd_file_hashtbl[i].nfb_head);
> spin_lock_init(&nfsd_file_hashtbl[i].nfb_lock);
> }
> out:
> return ret;
> +out_shrinker:
> + unregister_shrinker(&nfsd_file_shrinker);
> out_lru:
> list_lru_destroy(&nfsd_file_lru);
> out_err:
> @@ -207,6 +232,8 @@ nfsd_file_cache_shutdown(void)
> struct nfsd_file *nf;
> LIST_HEAD(dispose);
>
> + blocking_notifier_chain_unregister(&lease_notifier_chain,
> + &nfsd_file_lease_notifier);
> unregister_shrinker(&nfsd_file_shrinker);
> for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
> spin_lock(&nfsd_file_hashtbl[i].nfb_lock);
> diff --git a/include/linux/fs.h b/include/linux/fs.h
> index 9a9d314f7b27..01bb82eae684 100644
> --- a/include/linux/fs.h
> +++ b/include/linux/fs.h
> @@ -1041,6 +1041,7 @@ extern int fcntl_setlease(unsigned int fd, struct file *filp, long arg);
> extern int fcntl_getlease(struct file *filp);
>
> /* fs/locks.c */
> +extern struct blocking_notifier_head lease_notifier_chain;
> void locks_free_lock_context(struct file_lock_context *ctx);
> void locks_free_lock(struct file_lock *fl);
> extern void locks_init_lock(struct file_lock *);
> --
> 2.4.3

2015-08-26 19:53:37

by J. Bruce Fields

[permalink] [raw]
Subject: Re: [PATCH v3 07/20] nfsd: hook up nfsd_write to the new nfsd_file cache

On Thu, Aug 20, 2015 at 07:17:07AM -0400, Jeff Layton wrote:
> Note that all callers currently pass in NULL for "file" anyway, so
> there was already some dead code in here. Just eliminate that parm
> and have it use the file cache instead of dealing directly with a
> filp.

So this is the first time the cache is actually used, right? (Most of
the code introduced by previous patches looks dead until this patch.)

--b.

>
> Signed-off-by: Jeff Layton <[email protected]>
> ---
> fs/nfsd/nfs3proc.c | 2 +-
> fs/nfsd/nfsproc.c | 2 +-
> fs/nfsd/vfs.c | 33 +++++++++++----------------------
> fs/nfsd/vfs.h | 2 +-
> 4 files changed, 14 insertions(+), 25 deletions(-)
>
> diff --git a/fs/nfsd/nfs3proc.c b/fs/nfsd/nfs3proc.c
> index 7b755b7f785c..4e46ac511479 100644
> --- a/fs/nfsd/nfs3proc.c
> +++ b/fs/nfsd/nfs3proc.c
> @@ -192,7 +192,7 @@ nfsd3_proc_write(struct svc_rqst *rqstp, struct nfsd3_writeargs *argp,
>
> fh_copy(&resp->fh, &argp->fh);
> resp->committed = argp->stable;
> - nfserr = nfsd_write(rqstp, &resp->fh, NULL,
> + nfserr = nfsd_write(rqstp, &resp->fh,
> argp->offset,
> rqstp->rq_vec, argp->vlen,
> &cnt,
> diff --git a/fs/nfsd/nfsproc.c b/fs/nfsd/nfsproc.c
> index 4cd78ef4c95c..9893095cbee1 100644
> --- a/fs/nfsd/nfsproc.c
> +++ b/fs/nfsd/nfsproc.c
> @@ -213,7 +213,7 @@ nfsd_proc_write(struct svc_rqst *rqstp, struct nfsd_writeargs *argp,
> SVCFH_fmt(&argp->fh),
> argp->len, argp->offset);
>
> - nfserr = nfsd_write(rqstp, fh_copy(&resp->fh, &argp->fh), NULL,
> + nfserr = nfsd_write(rqstp, fh_copy(&resp->fh, &argp->fh),
> argp->offset,
> rqstp->rq_vec, argp->vlen,
> &cnt,
> diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
> index b5e077a6e7d4..7c1e7866fec9 100644
> --- a/fs/nfsd/vfs.c
> +++ b/fs/nfsd/vfs.c
> @@ -42,6 +42,7 @@
>
> #include "nfsd.h"
> #include "vfs.h"
> +#include "filecache.h"
>
> #define NFSDDBG_FACILITY NFSDDBG_FILEOP
>
> @@ -1002,30 +1003,18 @@ __be32 nfsd_read(struct svc_rqst *rqstp, struct svc_fh *fhp,
> * N.B. After this call fhp needs an fh_put
> */
> __be32
> -nfsd_write(struct svc_rqst *rqstp, struct svc_fh *fhp, struct file *file,
> - loff_t offset, struct kvec *vec, int vlen, unsigned long *cnt,
> - int *stablep)
> +nfsd_write(struct svc_rqst *rqstp, struct svc_fh *fhp, loff_t offset,
> + struct kvec *vec, int vlen, unsigned long *cnt, int *stablep)
> {
> - __be32 err = 0;
> -
> - if (file) {
> - err = nfsd_permission(rqstp, fhp->fh_export, fhp->fh_dentry,
> - NFSD_MAY_WRITE|NFSD_MAY_OWNER_OVERRIDE);
> - if (err)
> - goto out;
> - err = nfsd_vfs_write(rqstp, fhp, file, offset, vec, vlen, cnt,
> - stablep);
> - } else {
> - err = nfsd_open(rqstp, fhp, S_IFREG, NFSD_MAY_WRITE, &file);
> - if (err)
> - goto out;
> -
> - if (cnt)
> - err = nfsd_vfs_write(rqstp, fhp, file, offset, vec, vlen,
> - cnt, stablep);
> - fput(file);
> + __be32 err;
> + struct nfsd_file *nf;
> +
> + err = nfsd_file_acquire(rqstp, fhp, NFSD_MAY_WRITE, &nf);
> + if (err == nfs_ok) {
> + err = nfsd_vfs_write(rqstp, fhp, nf->nf_file, offset, vec,
> + vlen, cnt, stablep);
> + nfsd_file_put(nf);
> }
> -out:
> return err;
> }
>
> diff --git a/fs/nfsd/vfs.h b/fs/nfsd/vfs.h
> index 5be875e3e638..78b5527cba93 100644
> --- a/fs/nfsd/vfs.h
> +++ b/fs/nfsd/vfs.h
> @@ -78,7 +78,7 @@ __be32 nfsd_readv(struct file *, loff_t, struct kvec *, int,
> unsigned long *);
> __be32 nfsd_read(struct svc_rqst *, struct svc_fh *,
> loff_t, struct kvec *, int, unsigned long *);
> -__be32 nfsd_write(struct svc_rqst *, struct svc_fh *,struct file *,
> +__be32 nfsd_write(struct svc_rqst *, struct svc_fh *,
> loff_t, struct kvec *,int, unsigned long *, int *);
> __be32 nfsd_vfs_write(struct svc_rqst *rqstp, struct svc_fh *fhp,
> struct file *file, loff_t offset,
> --
> 2.4.3

2015-08-26 20:00:33

by J. Bruce Fields

[permalink] [raw]
Subject: Re: [PATCH v3 14/20] nfsd: close cached files prior to a REMOVE or RENAME that would replace target

On Thu, Aug 20, 2015 at 07:17:14AM -0400, Jeff Layton wrote:
> It's not uncommon for some workloads to do a bunch of I/O to a file and
> delete it just afterward. If knfsd has a cached open file however, then
> the file may still be open when the dentry is unlinked. If the
> underlying filesystem is nfs, then that could trigger it to do a
> sillyrename.

Possibly worth noting that situation doesn't currently occur upstream.

(And, another justification worth noting: space used by a file should be
deallocated on last unlink or close. People do sometimes notice if it's
not, especially if the file is large.)

> On a REMOVE or RENAME scan the nfsd_file cache for open files that
> correspond to the inode, and proactively unhash and put their
> references. This should prevent any delete-on-last-close activity from
> occurring, solely due to knfsd's open file cache.

Is there anything here to prevent a new cache entry being added after
nfsd_file_close_inode and before the file is actually removed?

--b.

>
> Signed-off-by: Jeff Layton <[email protected]>
> ---
> fs/nfsd/filecache.c | 25 +++++++++++++++++++++++++
> fs/nfsd/filecache.h | 1 +
> fs/nfsd/trace.h | 17 +++++++++++++++++
> fs/nfsd/vfs.c | 17 +++++++++++++++--
> 4 files changed, 58 insertions(+), 2 deletions(-)
>
> diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> index e48b536762aa..4bd683f03b6e 100644
> --- a/fs/nfsd/filecache.c
> +++ b/fs/nfsd/filecache.c
> @@ -283,6 +283,31 @@ nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
> return NULL;
> }
>
> +/**
> + * nfsd_file_close_inode - attempt to forcibly close a nfsd_file
> + * @inode: inode of the file to attempt to remove
> + *
> + * Walk the whole hash bucket, looking for any files that correspond to "inode".
> + * If any do, then unhash them and put the hashtable reference to them.
> + */
> +void
> +nfsd_file_close_inode(struct inode *inode)
> +{
> + struct nfsd_file *nf;
> + struct hlist_node *tmp;
> + unsigned int hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
> + LIST_HEAD(dispose);
> +
> + spin_lock(&nfsd_file_hashtbl[hashval].nfb_lock);
> + hlist_for_each_entry_safe(nf, tmp, &nfsd_file_hashtbl[hashval].nfb_head, nf_node) {
> + if (inode == nf->nf_inode)
> + nfsd_file_unhash_and_release_locked(nf, &dispose);
> + }
> + spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> + trace_nfsd_file_close_inode(hashval, inode, !list_empty(&dispose));
> + nfsd_file_dispose_list(&dispose);
> +}
> +
> __be32
> nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> unsigned int may_flags, struct nfsd_file **pnf)
> diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
> index debd558ef786..191cdb25aa66 100644
> --- a/fs/nfsd/filecache.h
> +++ b/fs/nfsd/filecache.h
> @@ -26,6 +26,7 @@ int nfsd_file_cache_init(void);
> void nfsd_file_cache_shutdown(void);
> void nfsd_file_put(struct nfsd_file *nf);
> struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
> +void nfsd_file_close_inode(struct inode *inode);
> __be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> unsigned int may_flags, struct nfsd_file **nfp);
> #endif /* _FS_NFSD_FILECACHE_H */
> diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
> index 2dac872d31e8..95af3b9c7b66 100644
> --- a/fs/nfsd/trace.h
> +++ b/fs/nfsd/trace.h
> @@ -139,6 +139,23 @@ TRACE_EVENT(nfsd_file_acquire,
> show_nf_may(__entry->nf_may), __entry->nf_file,
> be32_to_cpu(__entry->status))
> );
> +
> +TRACE_EVENT(nfsd_file_close_inode,
> + TP_PROTO(unsigned int hash, struct inode *inode, int found),
> + TP_ARGS(hash, inode, found),
> + TP_STRUCT__entry(
> + __field(unsigned int, hash)
> + __field(struct inode *, inode)
> + __field(int, found)
> + ),
> + TP_fast_assign(
> + __entry->hash = hash;
> + __entry->inode = inode;
> + __entry->found = found;
> + ),
> + TP_printk("hash=0x%x inode=0x%p found=%d", __entry->hash,
> + __entry->inode, __entry->found)
> +);
> #endif /* _NFSD_TRACE_H */
>
> #undef TRACE_INCLUDE_PATH
> diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
> index 6cfd96adcc71..98d3b9d96480 100644
> --- a/fs/nfsd/vfs.c
> +++ b/fs/nfsd/vfs.c
> @@ -1583,6 +1583,15 @@ out_nfserr:
> goto out_unlock;
> }
>
> +static void
> +nfsd_close_cached_files(struct dentry *dentry)
> +{
> + struct inode *inode = d_inode(dentry);
> +
> + if (inode && S_ISREG(inode->i_mode))
> + nfsd_file_close_inode(inode);
> +}
> +
> /*
> * Rename a file
> * N.B. After this call _both_ ffhp and tfhp need an fh_put
> @@ -1652,6 +1661,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
> goto out_dput_new;
>
> + nfsd_close_cached_files(ndentry);
> host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
> if (!host_err) {
> host_err = commit_metadata(tfhp);
> @@ -1721,10 +1731,13 @@ nfsd_unlink(struct svc_rqst *rqstp, struct svc_fh *fhp, int type,
> if (!type)
> type = d_inode(rdentry)->i_mode & S_IFMT;
>
> - if (type != S_IFDIR)
> + if (type != S_IFDIR) {
> + nfsd_close_cached_files(rdentry);
> host_err = vfs_unlink(dirp, rdentry, NULL);
> - else
> + } else {
> host_err = vfs_rmdir(dirp, rdentry);
> + }
> +
> if (!host_err)
> host_err = commit_metadata(fhp);
> dput(rdentry);
> --
> 2.4.3

2015-08-26 22:39:54

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH v3 06/20] locks/nfsd: create a new notifier chain for lease attempts

On Wed, 26 Aug 2015 15:49:23 -0400
"J. Bruce Fields" <[email protected]> wrote:

> On Thu, Aug 20, 2015 at 07:17:06AM -0400, Jeff Layton wrote:
> > With the new file caching infrastructure in nfsd, we can end up holding
> > files open for an indefinite period of time, even when they are still
> > idle. This may prevent the kernel from handing out leases on the file,
> > which we don't really want to block.
> >
> > Fix this by running a blocking notifier call chain whenever on any
> > lease attempt. nfsd can then purge the cache for that inode before
> > returning.
>
> Could this be expensive? There's potentially a lease attempt on every
> NFSv4 open.
>

That's the reason for the FL_LEASE check. nfsd never sets an FL_LEASE
-- only FL_DELEG and FL_LAYOUT. So nfsd will call into the notifier
callback but it'll never do anything.

We'll take a rwsem (which is icky, granted), call into the notifier and
then return immediately once we see it's not FL_LEASE. We could (in
principle) only call the notifier for FL_LEASE calls, but that seems a
little like a layering violation. Maybe it's worthwhile to consider
though.

> Could we use the cache to decide whether a delegation's a good idea?
> (So, skip requesting the delegation if the cache gives evidence of
> recent activity on the file that would have conflicted with the
> delegation we're about to grant.) That might avoid unnecessary cache
> purges too.
>

That's not a bad idea either. I'd have to think about that...

> --b.
>
> >
> > Signed-off-by: Jeff Layton <[email protected]>
> > ---
> > fs/locks.c | 15 +++++++++++++++
> > fs/nfsd/filecache.c | 27 +++++++++++++++++++++++++++
> > include/linux/fs.h | 1 +
> > 3 files changed, 43 insertions(+)
> >
> > diff --git a/fs/locks.c b/fs/locks.c
> > index d3d558ba4da7..c81b96159e5c 100644
> > --- a/fs/locks.c
> > +++ b/fs/locks.c
> > @@ -167,6 +167,13 @@ DEFINE_STATIC_LGLOCK(file_lock_lglock);
> > static DEFINE_PER_CPU(struct hlist_head, file_lock_list);
> >
> > /*
> > + * Some subsystems would like to be notified if someone attempts to set a
> > + * lease on a file. This notifier chain will be called whenever this occurs.
> > + */
> > +BLOCKING_NOTIFIER_HEAD(lease_notifier_chain);
> > +EXPORT_SYMBOL_GPL(lease_notifier_chain);
> > +
> > +/*
> > * The blocked_hash is used to find POSIX lock loops for deadlock detection.
> > * It is protected by blocked_lock_lock.
> > *
> > @@ -1795,10 +1802,18 @@ EXPORT_SYMBOL(generic_setlease);
> > *
> > * The "priv" pointer is passed directly to the lm_setup function as-is. It
> > * may be NULL if the lm_setup operation doesn't require it.
> > + *
> > + * Kernel subsystems can also register to be notified on any attempt to set
> > + * a new lease with the lease_notifier_chain. This is used by (e.g.) nfsd
> > + * to close files that it may have cached when there is an attempt to set a
> > + * conflicting lease.
> > */
> > int
> > vfs_setlease(struct file *filp, long arg, struct file_lock **lease, void **priv)
> > {
> > + if (arg != F_UNLCK)
> > + blocking_notifier_call_chain(&lease_notifier_chain, arg, *lease);
> > +
> > if (filp->f_op->setlease)
> > return filp->f_op->setlease(filp, arg, lease, priv);
> > else
> > diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> > index 669e62f6f4f6..77041967d8ff 100644
> > --- a/fs/nfsd/filecache.c
> > +++ b/fs/nfsd/filecache.c
> > @@ -158,6 +158,22 @@ static struct shrinker nfsd_file_shrinker = {
> > .seeks = 1,
> > };
> >
> > +static int
> > +nfsd_file_lease_notifier_call(struct notifier_block *nb, unsigned long arg,
> > + void *data)
> > +{
> > + struct file_lock *fl = data;
> > +
> > + /* Don't close files if we're the one trying to set the lease */
> > + if (fl->fl_type == FL_LEASE)
> > + nfsd_file_close_inode(file_inode(fl->fl_file));
> > + return 0;
> > +}
> > +
> > +static struct notifier_block nfsd_file_lease_notifier = {
> > + .notifier_call = nfsd_file_lease_notifier_call,
> > +};
> > +
> > int
> > nfsd_file_cache_init(void)
> > {
> > @@ -186,12 +202,21 @@ nfsd_file_cache_init(void)
> > goto out_lru;
> > }
> >
> > + ret = blocking_notifier_chain_register(&lease_notifier_chain,
> > + &nfsd_file_lease_notifier);
> > + if (ret) {
> > + pr_err("nfsd: unable to register lease notifier: %d\n", ret);
> > + goto out_shrinker;
> > + }
> > +
> > for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
> > INIT_HLIST_HEAD(&nfsd_file_hashtbl[i].nfb_head);
> > spin_lock_init(&nfsd_file_hashtbl[i].nfb_lock);
> > }
> > out:
> > return ret;
> > +out_shrinker:
> > + unregister_shrinker(&nfsd_file_shrinker);
> > out_lru:
> > list_lru_destroy(&nfsd_file_lru);
> > out_err:
> > @@ -207,6 +232,8 @@ nfsd_file_cache_shutdown(void)
> > struct nfsd_file *nf;
> > LIST_HEAD(dispose);
> >
> > + blocking_notifier_chain_unregister(&lease_notifier_chain,
> > + &nfsd_file_lease_notifier);
> > unregister_shrinker(&nfsd_file_shrinker);
> > for (i = 0; i < NFSD_FILE_HASH_SIZE; i++) {
> > spin_lock(&nfsd_file_hashtbl[i].nfb_lock);
> > diff --git a/include/linux/fs.h b/include/linux/fs.h
> > index 9a9d314f7b27..01bb82eae684 100644
> > --- a/include/linux/fs.h
> > +++ b/include/linux/fs.h
> > @@ -1041,6 +1041,7 @@ extern int fcntl_setlease(unsigned int fd, struct file *filp, long arg);
> > extern int fcntl_getlease(struct file *filp);
> >
> > /* fs/locks.c */
> > +extern struct blocking_notifier_head lease_notifier_chain;
> > void locks_free_lock_context(struct file_lock_context *ctx);
> > void locks_free_lock(struct file_lock *fl);
> > extern void locks_init_lock(struct file_lock *);
> > --
> > 2.4.3


--
Jeff Layton <[email protected]>

2015-08-26 22:40:35

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH v3 07/20] nfsd: hook up nfsd_write to the new nfsd_file cache

On Wed, 26 Aug 2015 15:53:37 -0400
"J. Bruce Fields" <[email protected]> wrote:

> On Thu, Aug 20, 2015 at 07:17:07AM -0400, Jeff Layton wrote:
> > Note that all callers currently pass in NULL for "file" anyway, so
> > there was already some dead code in here. Just eliminate that parm
> > and have it use the file cache instead of dealing directly with a
> > filp.
>
> So this is the first time the cache is actually used, right? (Most of
> the code introduced by previous patches looks dead until this patch.)
>

Yes! I should have made that clear in the patch descriptions. I'll do
that for the next respin.

Thanks,
Jeff

> --b.
>
> >
> > Signed-off-by: Jeff Layton <[email protected]>
> > ---
> > fs/nfsd/nfs3proc.c | 2 +-
> > fs/nfsd/nfsproc.c | 2 +-
> > fs/nfsd/vfs.c | 33 +++++++++++----------------------
> > fs/nfsd/vfs.h | 2 +-
> > 4 files changed, 14 insertions(+), 25 deletions(-)
> >
> > diff --git a/fs/nfsd/nfs3proc.c b/fs/nfsd/nfs3proc.c
> > index 7b755b7f785c..4e46ac511479 100644
> > --- a/fs/nfsd/nfs3proc.c
> > +++ b/fs/nfsd/nfs3proc.c
> > @@ -192,7 +192,7 @@ nfsd3_proc_write(struct svc_rqst *rqstp, struct
> > nfsd3_writeargs *argp,
> > fh_copy(&resp->fh, &argp->fh);
> > resp->committed = argp->stable;
> > - nfserr = nfsd_write(rqstp, &resp->fh, NULL,
> > + nfserr = nfsd_write(rqstp, &resp->fh,
> > argp->offset,
> > rqstp->rq_vec, argp->vlen,
> > &cnt,
> > diff --git a/fs/nfsd/nfsproc.c b/fs/nfsd/nfsproc.c
> > index 4cd78ef4c95c..9893095cbee1 100644
> > --- a/fs/nfsd/nfsproc.c
> > +++ b/fs/nfsd/nfsproc.c
> > @@ -213,7 +213,7 @@ nfsd_proc_write(struct svc_rqst *rqstp, struct
> > nfsd_writeargs *argp, SVCFH_fmt(&argp->fh),
> > argp->len, argp->offset);
> >
> > - nfserr = nfsd_write(rqstp, fh_copy(&resp->fh, &argp->fh),
> > NULL,
> > + nfserr = nfsd_write(rqstp, fh_copy(&resp->fh, &argp->fh),
> > argp->offset,
> > rqstp->rq_vec, argp->vlen,
> > &cnt,
> > diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
> > index b5e077a6e7d4..7c1e7866fec9 100644
> > --- a/fs/nfsd/vfs.c
> > +++ b/fs/nfsd/vfs.c
> > @@ -42,6 +42,7 @@
> >
> > #include "nfsd.h"
> > #include "vfs.h"
> > +#include "filecache.h"
> >
> > #define NFSDDBG_FACILITY NFSDDBG_FILEOP
> >
> > @@ -1002,30 +1003,18 @@ __be32 nfsd_read(struct svc_rqst *rqstp,
> > struct svc_fh *fhp,
> > * N.B. After this call fhp needs an fh_put
> > */
> > __be32
> > -nfsd_write(struct svc_rqst *rqstp, struct svc_fh *fhp, struct file
> > *file,
> > - loff_t offset, struct kvec *vec, int vlen,
> > unsigned long *cnt,
> > - int *stablep)
> > +nfsd_write(struct svc_rqst *rqstp, struct svc_fh *fhp, loff_t
> > offset,
> > + struct kvec *vec, int vlen, unsigned long *cnt, int
> > *stablep) {
> > - __be32 err = 0;
> > -
> > - if (file) {
> > - err = nfsd_permission(rqstp, fhp->fh_export,
> > fhp->fh_dentry,
> > -
> > NFSD_MAY_WRITE|NFSD_MAY_OWNER_OVERRIDE);
> > - if (err)
> > - goto out;
> > - err = nfsd_vfs_write(rqstp, fhp, file, offset,
> > vec, vlen, cnt,
> > - stablep);
> > - } else {
> > - err = nfsd_open(rqstp, fhp, S_IFREG,
> > NFSD_MAY_WRITE, &file);
> > - if (err)
> > - goto out;
> > -
> > - if (cnt)
> > - err = nfsd_vfs_write(rqstp, fhp, file,
> > offset, vec, vlen,
> > - cnt, stablep);
> > - fput(file);
> > + __be32 err;
> > + struct nfsd_file *nf;
> > +
> > + err = nfsd_file_acquire(rqstp, fhp, NFSD_MAY_WRITE, &nf);
> > + if (err == nfs_ok) {
> > + err = nfsd_vfs_write(rqstp, fhp, nf->nf_file,
> > offset, vec,
> > + vlen, cnt, stablep);
> > + nfsd_file_put(nf);
> > }
> > -out:
> > return err;
> > }
> >
> > diff --git a/fs/nfsd/vfs.h b/fs/nfsd/vfs.h
> > index 5be875e3e638..78b5527cba93 100644
> > --- a/fs/nfsd/vfs.h
> > +++ b/fs/nfsd/vfs.h
> > @@ -78,7 +78,7 @@ __be32 nfsd_readv(struct file *,
> > loff_t, struct kvec *, int, unsigned long *);
> > __be32 nfsd_read(struct svc_rqst *, struct svc_fh
> > *, loff_t, struct kvec *, int, unsigned long *);
> > -__be32 nfsd_write(struct svc_rqst *, struct svc_fh
> > *,struct file *, +__be32 nfsd_write(struct svc_rqst
> > *, struct svc_fh *, loff_t, struct kvec *,int, unsigned long *, int
> > *); __be32 nfsd_vfs_write(struct svc_rqst *rqstp,
> > struct svc_fh *fhp, struct file *file, loff_t offset,
> > --
> > 2.4.3


--
Jeff Layton <[email protected]>

2015-08-26 22:53:34

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH v3 14/20] nfsd: close cached files prior to a REMOVE or RENAME that would replace target

On Wed, 26 Aug 2015 16:00:32 -0400
"J. Bruce Fields" <[email protected]> wrote:

> On Thu, Aug 20, 2015 at 07:17:14AM -0400, Jeff Layton wrote:
> > It's not uncommon for some workloads to do a bunch of I/O to a file and
> > delete it just afterward. If knfsd has a cached open file however, then
> > the file may still be open when the dentry is unlinked. If the
> > underlying filesystem is nfs, then that could trigger it to do a
> > sillyrename.
>
> Possibly worth noting that situation doesn't currently occur upstream.
>
> (And, another justification worth noting: space used by a file should be
> deallocated on last unlink or close. People do sometimes notice if it's
> not, especially if the file is large.)
>

Good points.

> > On a REMOVE or RENAME scan the nfsd_file cache for open files that
> > correspond to the inode, and proactively unhash and put their
> > references. This should prevent any delete-on-last-close activity from
> > occurring, solely due to knfsd's open file cache.
>
> Is there anything here to prevent a new cache entry being added after
> nfsd_file_close_inode and before the file is actually removed?
>

No, nothing -- it's strictly best effort.

What might make sense is to consider looking at the dentry associated
with the struct file when putting the last reference to the nfsd_file.
If it's unhashed, then we could unhash the nfsd_file and put the hash
reference for it.

That won't prevent silly renames in the case of NFS being reexported,
of course, but it should ensure that we don't leave the thing open
indefinitely in the case of such a race.

I'll have to think about that one as well...

> --b.
>
> >
> > Signed-off-by: Jeff Layton <[email protected]>
> > ---
> > fs/nfsd/filecache.c | 25 +++++++++++++++++++++++++
> > fs/nfsd/filecache.h | 1 +
> > fs/nfsd/trace.h | 17 +++++++++++++++++
> > fs/nfsd/vfs.c | 17 +++++++++++++++--
> > 4 files changed, 58 insertions(+), 2 deletions(-)
> >
> > diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> > index e48b536762aa..4bd683f03b6e 100644
> > --- a/fs/nfsd/filecache.c
> > +++ b/fs/nfsd/filecache.c
> > @@ -283,6 +283,31 @@ nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
> > return NULL;
> > }
> >
> > +/**
> > + * nfsd_file_close_inode - attempt to forcibly close a nfsd_file
> > + * @inode: inode of the file to attempt to remove
> > + *
> > + * Walk the whole hash bucket, looking for any files that correspond to "inode".
> > + * If any do, then unhash them and put the hashtable reference to them.
> > + */
> > +void
> > +nfsd_file_close_inode(struct inode *inode)
> > +{
> > + struct nfsd_file *nf;
> > + struct hlist_node *tmp;
> > + unsigned int hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
> > + LIST_HEAD(dispose);
> > +
> > + spin_lock(&nfsd_file_hashtbl[hashval].nfb_lock);
> > + hlist_for_each_entry_safe(nf, tmp, &nfsd_file_hashtbl[hashval].nfb_head, nf_node) {
> > + if (inode == nf->nf_inode)
> > + nfsd_file_unhash_and_release_locked(nf, &dispose);
> > + }
> > + spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> > + trace_nfsd_file_close_inode(hashval, inode, !list_empty(&dispose));
> > + nfsd_file_dispose_list(&dispose);
> > +}
> > +
> > __be32
> > nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> > unsigned int may_flags, struct nfsd_file **pnf)
> > diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
> > index debd558ef786..191cdb25aa66 100644
> > --- a/fs/nfsd/filecache.h
> > +++ b/fs/nfsd/filecache.h
> > @@ -26,6 +26,7 @@ int nfsd_file_cache_init(void);
> > void nfsd_file_cache_shutdown(void);
> > void nfsd_file_put(struct nfsd_file *nf);
> > struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
> > +void nfsd_file_close_inode(struct inode *inode);
> > __be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> > unsigned int may_flags, struct nfsd_file **nfp);
> > #endif /* _FS_NFSD_FILECACHE_H */
> > diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
> > index 2dac872d31e8..95af3b9c7b66 100644
> > --- a/fs/nfsd/trace.h
> > +++ b/fs/nfsd/trace.h
> > @@ -139,6 +139,23 @@ TRACE_EVENT(nfsd_file_acquire,
> > show_nf_may(__entry->nf_may), __entry->nf_file,
> > be32_to_cpu(__entry->status))
> > );
> > +
> > +TRACE_EVENT(nfsd_file_close_inode,
> > + TP_PROTO(unsigned int hash, struct inode *inode, int found),
> > + TP_ARGS(hash, inode, found),
> > + TP_STRUCT__entry(
> > + __field(unsigned int, hash)
> > + __field(struct inode *, inode)
> > + __field(int, found)
> > + ),
> > + TP_fast_assign(
> > + __entry->hash = hash;
> > + __entry->inode = inode;
> > + __entry->found = found;
> > + ),
> > + TP_printk("hash=0x%x inode=0x%p found=%d", __entry->hash,
> > + __entry->inode, __entry->found)
> > +);
> > #endif /* _NFSD_TRACE_H */
> >
> > #undef TRACE_INCLUDE_PATH
> > diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
> > index 6cfd96adcc71..98d3b9d96480 100644
> > --- a/fs/nfsd/vfs.c
> > +++ b/fs/nfsd/vfs.c
> > @@ -1583,6 +1583,15 @@ out_nfserr:
> > goto out_unlock;
> > }
> >
> > +static void
> > +nfsd_close_cached_files(struct dentry *dentry)
> > +{
> > + struct inode *inode = d_inode(dentry);
> > +
> > + if (inode && S_ISREG(inode->i_mode))
> > + nfsd_file_close_inode(inode);
> > +}
> > +
> > /*
> > * Rename a file
> > * N.B. After this call _both_ ffhp and tfhp need an fh_put
> > @@ -1652,6 +1661,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> > if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
> > goto out_dput_new;
> >
> > + nfsd_close_cached_files(ndentry);
> > host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
> > if (!host_err) {
> > host_err = commit_metadata(tfhp);
> > @@ -1721,10 +1731,13 @@ nfsd_unlink(struct svc_rqst *rqstp, struct svc_fh *fhp, int type,
> > if (!type)
> > type = d_inode(rdentry)->i_mode & S_IFMT;
> >
> > - if (type != S_IFDIR)
> > + if (type != S_IFDIR) {
> > + nfsd_close_cached_files(rdentry);
> > host_err = vfs_unlink(dirp, rdentry, NULL);
> > - else
> > + } else {
> > host_err = vfs_rmdir(dirp, rdentry);
> > + }
> > +
> > if (!host_err)
> > host_err = commit_metadata(fhp);
> > dput(rdentry);
> > --
> > 2.4.3


--
Jeff Layton <[email protected]>

2015-08-27 13:38:07

by J. Bruce Fields

[permalink] [raw]
Subject: Re: [PATCH v3 14/20] nfsd: close cached files prior to a REMOVE or RENAME that would replace target

On Wed, Aug 26, 2015 at 06:53:31PM -0400, Jeff Layton wrote:
> On Wed, 26 Aug 2015 16:00:32 -0400
> "J. Bruce Fields" <[email protected]> wrote:
>
> > On Thu, Aug 20, 2015 at 07:17:14AM -0400, Jeff Layton wrote:
> > > It's not uncommon for some workloads to do a bunch of I/O to a file and
> > > delete it just afterward. If knfsd has a cached open file however, then
> > > the file may still be open when the dentry is unlinked. If the
> > > underlying filesystem is nfs, then that could trigger it to do a
> > > sillyrename.
> >
> > Possibly worth noting that situation doesn't currently occur upstream.
> >
> > (And, another justification worth noting: space used by a file should be
> > deallocated on last unlink or close. People do sometimes notice if it's
> > not, especially if the file is large.)
> >
>
> Good points.
>
> > > On a REMOVE or RENAME scan the nfsd_file cache for open files that
> > > correspond to the inode, and proactively unhash and put their
> > > references. This should prevent any delete-on-last-close activity from
> > > occurring, solely due to knfsd's open file cache.
> >
> > Is there anything here to prevent a new cache entry being added after
> > nfsd_file_close_inode and before the file is actually removed?
> >
>
> No, nothing -- it's strictly best effort.

Unfortunately I think this is something we really want to guarantee.

--b.

> What might make sense is to consider looking at the dentry associated
> with the struct file when putting the last reference to the nfsd_file.
> If it's unhashed, then we could unhash the nfsd_file and put the hash
> reference for it.
>
> That won't prevent silly renames in the case of NFS being reexported,
> of course, but it should ensure that we don't leave the thing open
> indefinitely in the case of such a race.
>
> I'll have to think about that one as well...
>
> > --b.
> >
> > >
> > > Signed-off-by: Jeff Layton <[email protected]>
> > > ---
> > > fs/nfsd/filecache.c | 25 +++++++++++++++++++++++++
> > > fs/nfsd/filecache.h | 1 +
> > > fs/nfsd/trace.h | 17 +++++++++++++++++
> > > fs/nfsd/vfs.c | 17 +++++++++++++++--
> > > 4 files changed, 58 insertions(+), 2 deletions(-)
> > >
> > > diff --git a/fs/nfsd/filecache.c b/fs/nfsd/filecache.c
> > > index e48b536762aa..4bd683f03b6e 100644
> > > --- a/fs/nfsd/filecache.c
> > > +++ b/fs/nfsd/filecache.c
> > > @@ -283,6 +283,31 @@ nfsd_file_find_locked(struct inode *inode, unsigned int may_flags,
> > > return NULL;
> > > }
> > >
> > > +/**
> > > + * nfsd_file_close_inode - attempt to forcibly close a nfsd_file
> > > + * @inode: inode of the file to attempt to remove
> > > + *
> > > + * Walk the whole hash bucket, looking for any files that correspond to "inode".
> > > + * If any do, then unhash them and put the hashtable reference to them.
> > > + */
> > > +void
> > > +nfsd_file_close_inode(struct inode *inode)
> > > +{
> > > + struct nfsd_file *nf;
> > > + struct hlist_node *tmp;
> > > + unsigned int hashval = (unsigned int)hash_ptr(inode, NFSD_FILE_HASH_BITS);
> > > + LIST_HEAD(dispose);
> > > +
> > > + spin_lock(&nfsd_file_hashtbl[hashval].nfb_lock);
> > > + hlist_for_each_entry_safe(nf, tmp, &nfsd_file_hashtbl[hashval].nfb_head, nf_node) {
> > > + if (inode == nf->nf_inode)
> > > + nfsd_file_unhash_and_release_locked(nf, &dispose);
> > > + }
> > > + spin_unlock(&nfsd_file_hashtbl[hashval].nfb_lock);
> > > + trace_nfsd_file_close_inode(hashval, inode, !list_empty(&dispose));
> > > + nfsd_file_dispose_list(&dispose);
> > > +}
> > > +
> > > __be32
> > > nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> > > unsigned int may_flags, struct nfsd_file **pnf)
> > > diff --git a/fs/nfsd/filecache.h b/fs/nfsd/filecache.h
> > > index debd558ef786..191cdb25aa66 100644
> > > --- a/fs/nfsd/filecache.h
> > > +++ b/fs/nfsd/filecache.h
> > > @@ -26,6 +26,7 @@ int nfsd_file_cache_init(void);
> > > void nfsd_file_cache_shutdown(void);
> > > void nfsd_file_put(struct nfsd_file *nf);
> > > struct nfsd_file *nfsd_file_get(struct nfsd_file *nf);
> > > +void nfsd_file_close_inode(struct inode *inode);
> > > __be32 nfsd_file_acquire(struct svc_rqst *rqstp, struct svc_fh *fhp,
> > > unsigned int may_flags, struct nfsd_file **nfp);
> > > #endif /* _FS_NFSD_FILECACHE_H */
> > > diff --git a/fs/nfsd/trace.h b/fs/nfsd/trace.h
> > > index 2dac872d31e8..95af3b9c7b66 100644
> > > --- a/fs/nfsd/trace.h
> > > +++ b/fs/nfsd/trace.h
> > > @@ -139,6 +139,23 @@ TRACE_EVENT(nfsd_file_acquire,
> > > show_nf_may(__entry->nf_may), __entry->nf_file,
> > > be32_to_cpu(__entry->status))
> > > );
> > > +
> > > +TRACE_EVENT(nfsd_file_close_inode,
> > > + TP_PROTO(unsigned int hash, struct inode *inode, int found),
> > > + TP_ARGS(hash, inode, found),
> > > + TP_STRUCT__entry(
> > > + __field(unsigned int, hash)
> > > + __field(struct inode *, inode)
> > > + __field(int, found)
> > > + ),
> > > + TP_fast_assign(
> > > + __entry->hash = hash;
> > > + __entry->inode = inode;
> > > + __entry->found = found;
> > > + ),
> > > + TP_printk("hash=0x%x inode=0x%p found=%d", __entry->hash,
> > > + __entry->inode, __entry->found)
> > > +);
> > > #endif /* _NFSD_TRACE_H */
> > >
> > > #undef TRACE_INCLUDE_PATH
> > > diff --git a/fs/nfsd/vfs.c b/fs/nfsd/vfs.c
> > > index 6cfd96adcc71..98d3b9d96480 100644
> > > --- a/fs/nfsd/vfs.c
> > > +++ b/fs/nfsd/vfs.c
> > > @@ -1583,6 +1583,15 @@ out_nfserr:
> > > goto out_unlock;
> > > }
> > >
> > > +static void
> > > +nfsd_close_cached_files(struct dentry *dentry)
> > > +{
> > > + struct inode *inode = d_inode(dentry);
> > > +
> > > + if (inode && S_ISREG(inode->i_mode))
> > > + nfsd_file_close_inode(inode);
> > > +}
> > > +
> > > /*
> > > * Rename a file
> > > * N.B. After this call _both_ ffhp and tfhp need an fh_put
> > > @@ -1652,6 +1661,7 @@ nfsd_rename(struct svc_rqst *rqstp, struct svc_fh *ffhp, char *fname, int flen,
> > > if (ffhp->fh_export->ex_path.dentry != tfhp->fh_export->ex_path.dentry)
> > > goto out_dput_new;
> > >
> > > + nfsd_close_cached_files(ndentry);
> > > host_err = vfs_rename(fdir, odentry, tdir, ndentry, NULL, 0);
> > > if (!host_err) {
> > > host_err = commit_metadata(tfhp);
> > > @@ -1721,10 +1731,13 @@ nfsd_unlink(struct svc_rqst *rqstp, struct svc_fh *fhp, int type,
> > > if (!type)
> > > type = d_inode(rdentry)->i_mode & S_IFMT;
> > >
> > > - if (type != S_IFDIR)
> > > + if (type != S_IFDIR) {
> > > + nfsd_close_cached_files(rdentry);
> > > host_err = vfs_unlink(dirp, rdentry, NULL);
> > > - else
> > > + } else {
> > > host_err = vfs_rmdir(dirp, rdentry);
> > > + }
> > > +
> > > if (!host_err)
> > > host_err = commit_metadata(fhp);
> > > dput(rdentry);
> > > --
> > > 2.4.3
>
>
> --
> Jeff Layton <[email protected]>

2015-08-28 12:19:58

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH v3 14/20] nfsd: close cached files prior to a REMOVE or RENAME that would replace target

On Thu, 27 Aug 2015 09:38:06 -0400
"J. Bruce Fields" <[email protected]> wrote:

> On Wed, Aug 26, 2015 at 06:53:31PM -0400, Jeff Layton wrote:
> > On Wed, 26 Aug 2015 16:00:32 -0400
> > "J. Bruce Fields" <[email protected]> wrote:
> >
> > > On Thu, Aug 20, 2015 at 07:17:14AM -0400, Jeff Layton wrote:
> > > > It's not uncommon for some workloads to do a bunch of I/O to a file and
> > > > delete it just afterward. If knfsd has a cached open file however, then
> > > > the file may still be open when the dentry is unlinked. If the
> > > > underlying filesystem is nfs, then that could trigger it to do a
> > > > sillyrename.
> > >
> > > Possibly worth noting that situation doesn't currently occur upstream.
> > >
> > > (And, another justification worth noting: space used by a file should be
> > > deallocated on last unlink or close. People do sometimes notice if it's
> > > not, especially if the file is large.)
> > >
> >
> > Good points.
> >
> > > > On a REMOVE or RENAME scan the nfsd_file cache for open files that
> > > > correspond to the inode, and proactively unhash and put their
> > > > references. This should prevent any delete-on-last-close activity from
> > > > occurring, solely due to knfsd's open file cache.
> > >
> > > Is there anything here to prevent a new cache entry being added after
> > > nfsd_file_close_inode and before the file is actually removed?
> > >
> >
> > No, nothing -- it's strictly best effort.
>
> Unfortunately I think this is something we really want to guarantee.
>

That should be doable.

One question though -- if we hit this race, what's the right way to
handle it?

We don't want to return nfserr_stale or anything since we won't know if
the inode will really be stale after the remove completes. We could
just be removing one link from a multiply-linked inode.

We also don't want to make the caller wait out nfsd_file_acquire, as the
file could be open via NFSv4 and those references might not get put for
quite some time.

What semantics should we be aiming for?
--
Jeff Layton <[email protected]>

2015-08-28 17:58:35

by J. Bruce Fields

[permalink] [raw]
Subject: Re: [PATCH v3 14/20] nfsd: close cached files prior to a REMOVE or RENAME that would replace target

On Fri, Aug 28, 2015 at 08:19:46AM -0400, Jeff Layton wrote:
> On Thu, 27 Aug 2015 09:38:06 -0400
> "J. Bruce Fields" <[email protected]> wrote:
>
> > On Wed, Aug 26, 2015 at 06:53:31PM -0400, Jeff Layton wrote:
> > > On Wed, 26 Aug 2015 16:00:32 -0400
> > > "J. Bruce Fields" <[email protected]> wrote:
> > >
> > > > On Thu, Aug 20, 2015 at 07:17:14AM -0400, Jeff Layton wrote:
> > > > > It's not uncommon for some workloads to do a bunch of I/O to a file and
> > > > > delete it just afterward. If knfsd has a cached open file however, then
> > > > > the file may still be open when the dentry is unlinked. If the
> > > > > underlying filesystem is nfs, then that could trigger it to do a
> > > > > sillyrename.
> > > >
> > > > Possibly worth noting that situation doesn't currently occur upstream.
> > > >
> > > > (And, another justification worth noting: space used by a file should be
> > > > deallocated on last unlink or close. People do sometimes notice if it's
> > > > not, especially if the file is large.)
> > > >
> > >
> > > Good points.
> > >
> > > > > On a REMOVE or RENAME scan the nfsd_file cache for open files that
> > > > > correspond to the inode, and proactively unhash and put their
> > > > > references. This should prevent any delete-on-last-close activity from
> > > > > occurring, solely due to knfsd's open file cache.
> > > >
> > > > Is there anything here to prevent a new cache entry being added after
> > > > nfsd_file_close_inode and before the file is actually removed?
> > > >
> > >
> > > No, nothing -- it's strictly best effort.
> >
> > Unfortunately I think this is something we really want to guarantee.
> >
>
> That should be doable.
>
> One question though -- if we hit this race, what's the right way to
> handle it?
>
> We don't want to return nfserr_stale or anything since we won't know if
> the inode will really be stale after the remove completes. We could
> just be removing one link from a multiply-linked inode.
>
> We also don't want to make the caller wait out nfsd_file_acquire, as the
> file could be open via NFSv4 and those references might not get put for
> quite some time.
>
> What semantics should we be aiming for?

Hm.

The RENAME or REMOVE has to succeed regardless.

If the other operation is an open or something lookup-like, then it has
to succeed or see ENOENT. If it's a filehandle-based operation like
READ or WRITE then it's got to succeed or get STALE, and if the latter
then it better be really and truly gone. Continuing to process a READ
or WRITE after the REMOVE continues is fine, I think, it just means some
application on some other v3 client has an open that we're honoring a
moment longer than we're required to.

Now I'd have to look back at your code.... If the caching were just an
optimization, we could just it off temporarily, but I don't think we
can.

Marking the file dead somehow and failing further attempts to use it
might work. I guess that's equivalent to your suggestion to unhash it
in this case.

--b.

2015-08-31 16:50:55

by Jeff Layton

[permalink] [raw]
Subject: Re: [PATCH v3 14/20] nfsd: close cached files prior to a REMOVE or RENAME that would replace target

On Fri, 28 Aug 2015 13:58:34 -0400
"J. Bruce Fields" <[email protected]> wrote:

> On Fri, Aug 28, 2015 at 08:19:46AM -0400, Jeff Layton wrote:
> > On Thu, 27 Aug 2015 09:38:06 -0400
> > "J. Bruce Fields" <[email protected]> wrote:
> >
> > > On Wed, Aug 26, 2015 at 06:53:31PM -0400, Jeff Layton wrote:
> > > > On Wed, 26 Aug 2015 16:00:32 -0400
> > > > "J. Bruce Fields" <[email protected]> wrote:
> > > >
> > > > > On Thu, Aug 20, 2015 at 07:17:14AM -0400, Jeff Layton wrote:
> > > > > > It's not uncommon for some workloads to do a bunch of I/O to a file and
> > > > > > delete it just afterward. If knfsd has a cached open file however, then
> > > > > > the file may still be open when the dentry is unlinked. If the
> > > > > > underlying filesystem is nfs, then that could trigger it to do a
> > > > > > sillyrename.
> > > > >
> > > > > Possibly worth noting that situation doesn't currently occur upstream.
> > > > >
> > > > > (And, another justification worth noting: space used by a file should be
> > > > > deallocated on last unlink or close. People do sometimes notice if it's
> > > > > not, especially if the file is large.)
> > > > >
> > > >
> > > > Good points.
> > > >
> > > > > > On a REMOVE or RENAME scan the nfsd_file cache for open files that
> > > > > > correspond to the inode, and proactively unhash and put their
> > > > > > references. This should prevent any delete-on-last-close activity from
> > > > > > occurring, solely due to knfsd's open file cache.
> > > > >
> > > > > Is there anything here to prevent a new cache entry being added after
> > > > > nfsd_file_close_inode and before the file is actually removed?
> > > > >
> > > >
> > > > No, nothing -- it's strictly best effort.
> > >
> > > Unfortunately I think this is something we really want to guarantee.
> > >
> >
> > That should be doable.
> >
> > One question though -- if we hit this race, what's the right way to
> > handle it?
> >
> > We don't want to return nfserr_stale or anything since we won't know if
> > the inode will really be stale after the remove completes. We could
> > just be removing one link from a multiply-linked inode.
> >
> > We also don't want to make the caller wait out nfsd_file_acquire, as the
> > file could be open via NFSv4 and those references might not get put for
> > quite some time.
> >
> > What semantics should we be aiming for?
>
> Hm.
>
> The RENAME or REMOVE has to succeed regardless.
>
> If the other operation is an open or something lookup-like, then it has
> to succeed or see ENOENT. If it's a filehandle-based operation like
> READ or WRITE then it's got to succeed or get STALE, and if the latter
> then it better be really and truly gone. Continuing to process a READ
> or WRITE after the REMOVE continues is fine, I think, it just means some
> application on some other v3 client has an open that we're honoring a
> moment longer than we're required to.
>
> Now I'd have to look back at your code.... If the caching were just an
> optimization, we could just it off temporarily, but I don't think we
> can.
>
> Marking the file dead somehow and failing further attempts to use it
> might work. I guess that's equivalent to your suggestion to unhash it
> in this case.
>

Yeah, though that is a rather substantive change. Right now, we just
tear these things down when we put the last reference. If we have to
prevent new references from being acquired during the duration of the
unlink, then that changes the entire semantics of the cache.

We'll need to allow some sort of object to persist in the cache (to act
as a placeholder), but still allow the struct file to be closed. I'll
have to think about how we can make that work. It'll probably make this
quite a bit more complex...
--
Jeff Layton <[email protected]>