patch-2.2.18 linux/fs/nfs/write.c
Next file: linux/fs/nfsd/Makefile
Previous file: linux/fs/nfs/unlink.c
Back to the patch index
Back to the overall index
- Lines: 1820
- Date:
Fri Sep 15 22:10:44 2000
- Orig file:
v2.2.17/fs/nfs/write.c
- Orig date:
Fri Apr 21 12:46:44 2000
diff -u --new-file --recursive --exclude-from /usr/src/exclude v2.2.17/fs/nfs/write.c linux/fs/nfs/write.c
@@ -10,26 +10,19 @@
* RPC call to write the page is scheduled immediately; otherwise, the call
* is delayed for a few seconds.
*
- * Just like readahead, no async I/O is performed if wsize < PAGE_SIZE.
+ * Just like readahead, no async I/O is performed if wsize < PAGE_CACHE_SIZE.
*
- * Write requests are kept on the inode's writeback list. Each entry in
+ * Write requests are kept on the inode's dirty list. Each entry in
* that list references the page (portion) to be written. When the
- * cache timeout has expired, the RPC task is woken up, and tries to
- * lock the page. As soon as it manages to do so, the request is moved
- * from the writeback list to the writelock list.
+ * cache timeout has expired, the flushd task is woken up, and tries to
+ * lock the request. As soon as it manages to do so, the request is removed
+ * from the dirty list. When the RPC write call completes, the request
+ * is either released or put on the inode's commit list.
*
* Note: we must make sure never to confuse the inode passed in the
* write_page request with the one in page->inode. As far as I understand
* it, these are different when doing a swap-out.
*
- * To understand everything that goes on here and in the NFS read code,
- * one should be aware that a page is locked in exactly one of the following
- * cases:
- *
- * - A write request is in progress.
- * - A user process is in generic_file_write/nfs_update_page
- * - A user process is in generic_file_read
- *
* Also note that because of the way pages are invalidated in
* nfs_revalidate_inode, the following assertions hold:
*
@@ -38,75 +31,147 @@
* - If the page is not uptodate, there will be no pending write
* requests, and no process will be in nfs_update_page.
*
- * FIXME: Interaction with the vmscan routines is not optimal yet.
- * Either vmscan must be made nfs-savvy, or we need a different page
- * reclaim concept that supports something like FS-independent
- * buffer_heads with a b_ops-> field.
- *
* Copyright (C) 1996, 1997, Olaf Kirch <okir@monad.swb.de>
+ *
+ * Rewritten 6/3/2000 by Trond Myklebust.
+ * Copyright (C) 1999, 2000, Trond Myklebust <trond.myklebust@fys.uio.no>
*/
+#include <linux/config.h>
#include <linux/types.h>
#include <linux/malloc.h>
#include <linux/swap.h>
+#include <asm/pgtable.h>
#include <linux/pagemap.h>
-#include <linux/file.h>
+#include <linux/sunrpc/auth.h>
#include <linux/sunrpc/clnt.h>
+#include <linux/nfs.h>
#include <linux/nfs_fs.h>
+#include <linux/nfs_mount.h>
+#include <linux/nfs_flushd.h>
+#include <linux/nfs_page.h>
#include <asm/uaccess.h>
+#include <linux/file.h>
+
#define NFS_PARANOIA 1
#define NFSDBG_FACILITY NFSDBG_PAGECACHE
-static void nfs_wback_begin(struct rpc_task *task);
-static void nfs_wback_result(struct rpc_task *task);
-static void nfs_cancel_request(struct nfs_wreq *req);
+static unsigned int nfs_nr_requests = 0;
/*
- * Cache parameters
+ * Local structures
+ * This is the struct where the WRITE/COMMIT arguments go.
*/
-#define NFS_WRITEBACK_DELAY (10 * HZ)
-#define NFS_WRITEBACK_MAX 64
+struct nfs_write_data {
+ struct rpc_task task;
+ struct dentry *dentry;
+ struct rpc_cred *cred;
+ struct nfs_writeargs args; /* argument struct */
+ struct nfs_writeres res; /* result struct */
+ struct nfs_fattr fattr;
+ struct nfs_writeverf verf;
+ struct list_head pages; /* Coalesced requests we wish to flush */
+};
/*
- * Limit number of delayed writes
+ * Local function declarations
*/
-static int nr_write_requests = 0;
-static struct rpc_wait_queue write_queue = RPC_INIT_WAITQ("write_chain");
+static void nfs_writeback_done(struct rpc_task *);
+#ifdef CONFIG_NFS_V3
+static void nfs_commit_done(struct rpc_task *);
+#endif
/* Hack for future NFS swap support */
#ifndef IS_SWAPFILE
# define IS_SWAPFILE(inode) (0)
#endif
+static __inline__ struct nfs_page *nfs_page_alloc(void)
+{
+ struct nfs_page *p;
+ p = (struct nfs_page *) kmalloc(sizeof(*p), GFP_KERNEL);
+ if (p) {
+ memset(p, 0, sizeof(*p));
+ INIT_LIST_HEAD(&p->wb_hash);
+ INIT_LIST_HEAD(&p->wb_list);
+ }
+ return p;
+}
+
+static __inline__ void nfs_page_free(struct nfs_page *p)
+{
+ kfree(p);
+}
+
+static __inline__ struct nfs_write_data *nfs_writedata_alloc(void)
+{
+ struct nfs_write_data *p;
+ p = (struct nfs_write_data *) kmalloc(sizeof(*p), GFP_NFS);
+ if (p) {
+ memset(p, 0, sizeof(*p));
+ INIT_LIST_HEAD(&p->pages);
+ }
+ return p;
+}
+
+static __inline__ void nfs_writedata_free(struct nfs_write_data *p)
+{
+ kfree(p);
+}
+
+static void nfs_writedata_release(struct rpc_task *task)
+{
+ nfs_writedata_free(task->tk_calldata);
+}
+
+static __inline__ int
+nfs_write_attributes(struct inode *inode, struct nfs_fattr *fattr)
+{
+ if ((fattr->valid & NFS_ATTR_FATTR) && !(fattr->valid & NFS_ATTR_WCC)) {
+ fattr->pre_size = NFS_CACHE_ISIZE(inode);
+ fattr->pre_mtime = NFS_CACHE_MTIME(inode);
+ fattr->pre_ctime = NFS_CACHE_CTIME(inode);
+ fattr->valid |= NFS_ATTR_WCC;
+ }
+ return nfs_refresh_inode(inode, fattr);
+}
+
/*
* Write a page synchronously.
* Offset is the data offset within the page.
*/
static int
-nfs_writepage_sync(struct dentry *dentry, struct inode *inode,
- struct page *page, unsigned long offset, unsigned int count)
+nfs_writepage_sync(struct file *file, struct page *page,
+ unsigned long offset, unsigned int count)
{
+ struct dentry *dentry = file->f_dentry;
+ struct inode *inode = dentry->d_inode;
+ struct rpc_cred *cred = nfs_file_cred(file);
unsigned int wsize = NFS_SERVER(inode)->wsize;
- int result, refresh = 0, written = 0;
+ int result, refresh = 0, written = 0, flags;
u8 *buffer;
struct nfs_fattr fattr;
+ struct nfs_writeverf verifier;
dprintk("NFS: nfs_writepage_sync(%s/%s %d@%ld)\n",
dentry->d_parent->d_name.name, dentry->d_name.name,
- count, page->offset + offset);
+ count, nfs_page_offset(page) + offset);
buffer = (u8 *) page_address(page) + offset;
- offset += page->offset;
+ offset += nfs_page_offset(page);
+
+ flags = ((IS_SWAPFILE(inode)) ? NFS_RW_SWAP : 0) | NFS_RW_SYNC;
do {
if (count < wsize && !IS_SWAPFILE(inode))
wsize = count;
- result = nfs_proc_write(NFS_DSERVER(dentry), NFS_FH(dentry),
- IS_SWAPFILE(inode), offset, wsize,
- buffer, &fattr);
+ result = NFS_PROTO(inode)->write(dentry, &fattr, cred, flags,
+ offset, wsize, buffer,
+ &verifier);
+ nfs_write_attributes(inode, &fattr);
if (result < 0) {
/* Must mark the page invalid after I/O error */
@@ -114,7 +179,7 @@
goto io_error;
}
if (result != wsize)
- printk("NFS: short write, wsize=%u, result=%d\n",
+ printk(KERN_ERR "NFS: short write, size=%u, result=%d\n",
wsize, result);
refresh = 1;
buffer += wsize;
@@ -130,274 +195,677 @@
} while (count);
io_error:
- /* Note: we don't refresh if the call failed (fattr invalid) */
- if (refresh && result >= 0) {
- /* See comments in nfs_wback_result */
- /* N.B. I don't think this is right -- sync writes in order */
- if (fattr.size < inode->i_size)
- fattr.size = inode->i_size;
- if (fattr.mtime.seconds < inode->i_mtime)
- printk("nfs_writepage_sync: prior time??\n");
- /* Solaris 2.5 server seems to send garbled
- * fattrs occasionally */
- if (inode->i_ino == fattr.fileid) {
- /*
- * We expect the mtime value to change, and
- * don't want to invalidate the caches.
- */
- inode->i_mtime = fattr.mtime.seconds;
- nfs_refresh_inode(inode, &fattr);
- }
- else
- printk("nfs_writepage_sync: inode %ld, got %u?\n",
- inode->i_ino, fattr.fileid);
- }
-
return written? written : result;
}
/*
- * Append a writeback request to a list
+ * Write a page to the server. This will be used for NFS swapping only
+ * (for now), and we currently do this synchronously only.
*/
-static inline void
-append_write_request(struct nfs_wreq **q, struct nfs_wreq *wreq)
+int
+nfs_writepage(struct file * file, struct page *page)
+{
+ struct inode *inode = file->f_dentry->d_inode;
+ unsigned offset = PAGE_CACHE_SIZE;
+
+ if (page->offset >= inode->i_size)
+ return -EIO;
+ if (page->offset + offset > inode->i_size)
+ offset = inode->i_size & (PAGE_CACHE_SIZE-1);
+ return nfs_writepage_sync(file, page, 0, offset);
+}
+
+/*
+ * Check whether the file range we want to write to is locked by
+ * us.
+ */
+static int
+region_locked(struct inode *inode, struct nfs_page *req)
{
- dprintk("NFS: append_write_request(%p, %p)\n", q, wreq);
- rpc_append_list(q, wreq);
+ struct file_lock *fl;
+ unsigned long rqstart, rqend;
+
+ /* Don't optimize writes if we don't use NLM */
+ if (NFS_SERVER(inode)->flags & NFS_MOUNT_NONLM)
+ return 0;
+
+ rqstart = nfs_page_offset(req->wb_page) + req->wb_offset;
+ rqend = rqstart + req->wb_bytes;
+ for (fl = inode->i_flock; fl; fl = fl->fl_next) {
+ if (fl->fl_owner == current->files && (fl->fl_flags & FL_POSIX)
+ && fl->fl_type == F_WRLCK
+ && fl->fl_start <= rqstart && rqend <= fl->fl_end) {
+ return 1;
+ }
+ }
+
+ return 0;
}
/*
- * Remove a writeback request from a list
+ * Insert a write request into an inode
*/
static inline void
-remove_write_request(struct nfs_wreq **q, struct nfs_wreq *wreq)
+nfs_inode_add_request(struct inode *inode, struct nfs_page *req)
{
- dprintk("NFS: remove_write_request(%p, %p)\n", q, wreq);
- rpc_remove_list(q, wreq);
+ if (!list_empty(&req->wb_hash))
+ return;
+ if (!NFS_WBACK_BUSY(req))
+ printk(KERN_ERR "NFS: unlocked request attempted hashed!\n");
+ inode->u.nfs_i.npages++;
+ list_add(&req->wb_hash, &inode->u.nfs_i.writeback);
+ req->wb_count++;
}
/*
- * Find a non-busy write request for a given page to
- * try to combine with.
+ * Insert a write request into an inode
*/
-static inline struct nfs_wreq *
-find_write_request(struct inode *inode, struct page *page)
+static inline void
+nfs_inode_remove_request(struct nfs_page *req)
{
- pid_t pid = current->pid;
- struct nfs_wreq *head, *req;
+ struct inode *inode;
+ if (list_empty(&req->wb_hash))
+ return;
+
+ if (!NFS_WBACK_BUSY(req))
+ printk(KERN_ERR "NFS: unlocked request attempted unhashed!\n");
+ inode = req->wb_dentry->d_inode;
+ list_del(&req->wb_hash);
+ INIT_LIST_HEAD(&req->wb_hash);
+ inode->u.nfs_i.npages--;
+ if ((inode->u.nfs_i.npages == 0) != list_empty(&inode->u.nfs_i.writeback))
+ printk(KERN_ERR "NFS: desynchronized value of nfs_i.npages.\n");
+ if (!nfs_have_writebacks(inode) && !nfs_have_read(inode))
+ inode_remove_flushd(inode);
+ nfs_release_request(req);
+}
- dprintk("NFS: find_write_request(%x/%ld, %p)\n",
- inode->i_dev, inode->i_ino, page);
- if (!(req = head = NFS_WRITEBACK(inode)))
- return NULL;
- do {
- /*
- * We can't combine with canceled requests or
- * requests that have already been started..
- */
- if (req->wb_flags & (NFS_WRITE_CANCELLED | NFS_WRITE_INPROGRESS))
+/*
+ * Find a request
+ */
+static inline struct nfs_page *
+_nfs_find_request(struct inode *inode, struct page *page)
+{
+ struct list_head *head, *next;
+
+ head = &inode->u.nfs_i.writeback;
+ next = head->next;
+ while (next != head) {
+ struct nfs_page *req = nfs_inode_wb_entry(next);
+ next = next->next;
+ if (page_index(req->wb_page) != page_index(page))
continue;
+ req->wb_count++;
+ return req;
+ }
+ return NULL;
+}
- if (req->wb_page == page && req->wb_pid == pid)
- return req;
+static struct nfs_page *
+nfs_find_request(struct inode *inode, struct page *page)
+{
+ struct nfs_page *req;
- /*
- * Ehh, don't keep too many tasks queued..
- */
- rpc_wake_up_task(&req->wb_task);
+ req = _nfs_find_request(inode, page);
+ return req;
+}
- } while ((req = WB_NEXT(req)) != head);
- return NULL;
+/*
+ * Insert a write request into a sorted list
+ */
+void nfs_list_add_request(struct nfs_page *req, struct list_head *head)
+{
+ struct list_head *prev;
+
+ if (!list_empty(&req->wb_list)) {
+ printk(KERN_ERR "NFS: Add to list failed!\n");
+ return;
+ }
+ if (!NFS_WBACK_BUSY(req))
+ printk(KERN_ERR "NFS: unlocked request attempted added to list!\n");
+ prev = head->prev;
+ while (prev != head) {
+ struct nfs_page *p = nfs_list_entry(prev);
+ if (page_index(p->wb_page) < page_index(req->wb_page))
+ break;
+ prev = prev->prev;
+ }
+ list_add(&req->wb_list, prev);
+ req->wb_list_head = head;
}
/*
- * Find and release all failed requests for this inode.
+ * Insert a write request into an inode
*/
-int
-nfs_check_failed_request(struct inode * inode)
+void nfs_list_remove_request(struct nfs_page *req)
{
- /* FIXME! */
- return 0;
+ if (list_empty(&req->wb_list))
+ return;
+ if (!NFS_WBACK_BUSY(req))
+ printk(KERN_ERR "NFS: unlocked request attempted removed from list!\n");
+ list_del(&req->wb_list);
+ INIT_LIST_HEAD(&req->wb_list);
+ req->wb_list_head = NULL;
}
/*
- * Try to merge adjacent write requests. This works only for requests
- * issued by the same user.
+ * Add a request to the inode's dirty list.
+ */
+static inline void
+nfs_mark_request_dirty(struct nfs_page *req)
+{
+ struct inode *inode = req->wb_dentry->d_inode;
+
+ if (list_empty(&req->wb_list)) {
+ nfs_list_add_request(req, &inode->u.nfs_i.dirty);
+ inode->u.nfs_i.ndirty++;
+ }
+ inode_schedule_scan(inode, req->wb_timeout);
+}
+
+/*
+ * Check if a request is dirty
*/
static inline int
-update_write_request(struct nfs_wreq *req, unsigned int first,
- unsigned int bytes)
+nfs_dirty_request(struct nfs_page *req)
{
- unsigned int rqfirst = req->wb_offset,
- rqlast = rqfirst + req->wb_bytes,
- last = first + bytes;
+ struct inode *inode = req->wb_dentry->d_inode;
+ return !list_empty(&req->wb_list) && req->wb_list_head == &inode->u.nfs_i.dirty;
+}
- dprintk("nfs: trying to update write request %p\n", req);
+#ifdef CONFIG_NFS_V3
+/*
+ * Add a request to the inode's commit list.
+ */
+static inline void
+nfs_mark_request_commit(struct nfs_page *req)
+{
+ struct inode *inode = req->wb_dentry->d_inode;
- /* not contiguous? */
- if (rqlast < first || last < rqfirst)
- return 0;
+ if (list_empty(&req->wb_list)) {
+ nfs_list_add_request(req, &inode->u.nfs_i.commit);
+ inode->u.nfs_i.ncommit++;
+ }
+ inode_schedule_scan(inode, req->wb_timeout);
+}
+#endif
- if (first < rqfirst)
- rqfirst = first;
- if (rqlast < last)
- rqlast = last;
+/*
+ * Create a write request.
+ * Page must be locked by the caller. This makes sure we never create
+ * two different requests for the same page, and avoids possible deadlock
+ * when we reach the hard limit on the number of dirty pages.
+ */
+struct nfs_page *nfs_create_request(struct file *file, struct page *page,
+ unsigned int offset, unsigned int count)
+{
+ struct dentry *dentry = file->f_dentry;
+ struct inode *inode = dentry->d_inode;
+ struct nfs_reqlist *cache = NFS_REQUESTLIST(inode);
+ struct nfs_page *req = NULL;
+ long timeout;
- req->wb_offset = rqfirst;
- req->wb_bytes = rqlast - rqfirst;
- req->wb_count++;
+ /* Deal with hard/soft limits.
+ */
+ do {
+ /* If we're still over the global soft limit, wake up all requests */
+ if (nfs_nr_requests >= MAX_REQUEST_SOFT) {
+ dprintk("NFS: hit soft limit (%d requests)\n",
+ nfs_nr_requests);
+ if (!cache->task)
+ nfs_reqlist_init(NFS_SERVER(inode));
+ nfs_wake_flushd();
+ }
+
+ /* If we haven't reached the hard limit yet,
+ * try to allocate the request struct */
+ if (cache->nr_requests < MAX_REQUEST_HARD) {
+ req = nfs_page_alloc();
+ if (req != NULL)
+ break;
+ }
+
+ /* We're over the hard limit. Wait for better times */
+ dprintk("NFS: create_request sleeping (total %d pid %d)\n",
+ nfs_nr_requests, current->pid);
+
+ timeout = 1 * HZ;
+ if (NFS_SERVER(inode)->flags & NFS_MOUNT_INTR) {
+ interruptible_sleep_on_timeout(&cache->request_wait,
+ timeout);
+ if (signalled())
+ break;
+ } else
+ sleep_on_timeout(&cache->request_wait, timeout);
+
+ dprintk("NFS: create_request waking up (tot %d pid %d)\n",
+ nfs_nr_requests, current->pid);
+ } while (!req);
+ if (!req)
+ return NULL;
- return 1;
+ /* Initialize the request struct. Initially, we assume a
+ * long write-back delay. This will be adjusted in
+ * update_nfs_request below if the region is not locked. */
+ req->wb_page = page;
+ atomic_inc(&page->count);
+ req->wb_offset = offset;
+ req->wb_bytes = count;
+ req->wb_file = file;
+ file->f_count++;
+ req->wb_dentry = dentry;
+ req->wb_cred = nfs_file_cred(file);
+ req->wb_count = 1;
+
+ /* register request's existence */
+ cache->nr_requests++;
+ nfs_nr_requests++;
+ return req;
}
-static inline void
-free_write_request(struct nfs_wreq * req)
+
+/*
+ * Release all resources associated with a write request after it
+ * has been committed to stable storage
+ */
+void
+nfs_release_request(struct nfs_page *req)
{
- if (!--req->wb_count)
- kfree(req);
+ struct inode *inode = req->wb_dentry->d_inode;
+ struct nfs_reqlist *cache = NFS_REQUESTLIST(inode);
+ struct page *page = req->wb_page;
+
+ if (--req->wb_count)
+ return;
+
+ if (!list_empty(&req->wb_list)) {
+ printk(KERN_ERR "NFS: Request released while still on a list!\n");
+ nfs_list_remove_request(req);
+ }
+ if (!list_empty(&req->wb_hash)) {
+ printk(KERN_ERR "NFS: Request released while still hashed!\n");
+ nfs_inode_remove_request(req);
+ }
+ if (NFS_WBACK_BUSY(req))
+ printk(KERN_ERR "NFS: Request released while still locked!\n");
+
+ fput(req->wb_file);
+ page_cache_release(page);
+ nfs_page_free(req);
+ /* wake up anyone waiting to allocate a request */
+ cache->nr_requests--;
+ nfs_nr_requests--;
+ wake_up(&cache->request_wait);
}
/*
- * Create and initialize a writeback request
+ * Wait for a request to complete.
+ *
+ * Interruptible by signals only if mounted with intr flag.
*/
-static inline struct nfs_wreq *
-create_write_request(struct file * file, struct page *page, unsigned int offset, unsigned int bytes)
+static int
+nfs_wait_on_request(struct nfs_page *req)
{
- struct dentry *dentry = file->f_dentry;
- struct inode *inode = dentry->d_inode;
- struct rpc_clnt *clnt = NFS_CLIENT(inode);
- struct nfs_wreq *wreq;
- struct rpc_task *task;
+ struct inode *inode = req->wb_dentry->d_inode;
+ struct rpc_clnt *clnt = NFS_CLIENT(inode);
+ int retval;
- dprintk("NFS: create_write_request(%s/%s, %ld+%d)\n",
- dentry->d_parent->d_name.name, dentry->d_name.name,
- page->offset + offset, bytes);
+ if (!NFS_WBACK_BUSY(req))
+ return 0;
+ req->wb_count++;
+ retval = nfs_wait_event(clnt, req->wb_wait, !NFS_WBACK_BUSY(req));
+ nfs_release_request(req);
+ return retval;
+}
- /* FIXME: Enforce hard limit on number of concurrent writes? */
- wreq = (struct nfs_wreq *) kmalloc(sizeof(*wreq), GFP_KERNEL);
- if (!wreq)
- goto out_fail;
- memset(wreq, 0, sizeof(*wreq));
-
- task = &wreq->wb_task;
- rpc_init_task(task, clnt, nfs_wback_result, RPC_TASK_NFSWRITE);
- task->tk_calldata = wreq;
- task->tk_action = nfs_wback_begin;
-
- rpcauth_lookupcred(task); /* Obtain user creds */
- if (task->tk_status < 0)
- goto out_req;
-
- /* Put the task on inode's writeback request list. */
- wreq->wb_file = file;
- wreq->wb_pid = current->pid;
- wreq->wb_page = page;
- wreq->wb_offset = offset;
- wreq->wb_bytes = bytes;
- wreq->wb_count = 2; /* One for the IO, one for us */
-
- append_write_request(&NFS_WRITEBACK(inode), wreq);
-
- if (nr_write_requests++ > NFS_WRITEBACK_MAX*3/4)
- rpc_wake_up_next(&write_queue);
-
- return wreq;
-
-out_req:
- rpc_release_task(task);
- kfree(wreq);
-out_fail:
- return NULL;
+/*
+ * Wait for a request to complete.
+ *
+ * Interruptible by signals only if mounted with intr flag.
+ */
+static int
+nfs_wait_on_requests(struct inode *inode, struct file *file, unsigned long idx_start, unsigned int npages)
+{
+ struct list_head *p, *head;
+ unsigned long idx_end;
+ unsigned int res = 0;
+ int error;
+
+ if (npages == 0)
+ idx_end = ~0;
+ else
+ idx_end = idx_start + npages - 1;
+
+ head = &inode->u.nfs_i.writeback;
+ p = head->next;
+ while (p != head) {
+ unsigned long pg_idx;
+ struct nfs_page *req = nfs_inode_wb_entry(p);
+
+ p = p->next;
+
+ if (file && req->wb_file != file)
+ continue;
+
+ pg_idx = page_index(req->wb_page);
+ if (pg_idx < idx_start || pg_idx > idx_end)
+ continue;
+
+ if (!NFS_WBACK_BUSY(req))
+ continue;
+ req->wb_count++;
+ error = nfs_wait_on_request(req);
+ nfs_release_request(req);
+ if (error < 0)
+ return error;
+ p = head->next;
+ res++;
+ }
+ return res;
}
/*
- * Schedule a writeback RPC call.
- * If the server is congested, don't add to our backlog of queued
- * requests but call it synchronously.
- * The function returns whether we should wait for the thing or not.
- *
- * FIXME: Here we could walk the inode's lock list to see whether the
- * page we're currently writing to has been write-locked by the caller.
- * If it is, we could schedule an async write request with a long
- * delay in order to avoid writing back the page until the lock is
- * released.
+ * Scan cluster for dirty pages and send as many of them to the
+ * server as possible.
*/
-static inline int
-schedule_write_request(struct nfs_wreq *req, int sync)
+int nfs_scan_list_timeout(struct list_head *head, struct list_head *dst,
+ struct inode *inode)
{
- struct rpc_task *task = &req->wb_task;
- struct file *file = req->wb_file;
- struct dentry *dentry = file->f_dentry;
- struct inode *inode = dentry->d_inode;
+ struct list_head *p;
+ struct nfs_page *req;
+ int pages = 0;
- if (NFS_CONGESTED(inode) || nr_write_requests >= NFS_WRITEBACK_MAX)
- sync = 1;
+ p = head->next;
+ while (p != head) {
+ req = nfs_list_entry(p);
+ p = p->next;
+ if (time_after(req->wb_timeout, jiffies)) {
+ if (time_after(NFS_NEXTSCAN(inode), req->wb_timeout))
+ NFS_NEXTSCAN(inode) = req->wb_timeout;
+ continue;
+ }
+ if (!nfs_lock_request(req))
+ continue;
+ nfs_list_remove_request(req);
+ nfs_list_add_request(req, dst);
+ pages++;
+ }
+ return pages;
+}
- if (sync) {
- sigset_t oldmask;
- struct rpc_clnt *clnt = NFS_CLIENT(inode);
- dprintk("NFS: %4d schedule_write_request (sync)\n",
- task->tk_pid);
- /* Page is already locked */
- rpc_clnt_sigmask(clnt, &oldmask);
- rpc_execute(task);
- rpc_clnt_sigunmask(clnt, &oldmask);
- } else {
- dprintk("NFS: %4d schedule_write_request (async)\n",
- task->tk_pid);
- task->tk_flags |= RPC_TASK_ASYNC;
- task->tk_timeout = NFS_WRITEBACK_DELAY;
- rpc_sleep_on(&write_queue, task, NULL, NULL);
+static int
+nfs_scan_dirty_timeout(struct inode *inode, struct list_head *dst)
+{
+ int pages;
+ pages = nfs_scan_list_timeout(&inode->u.nfs_i.dirty, dst, inode);
+ inode->u.nfs_i.ndirty -= pages;
+ if ((inode->u.nfs_i.ndirty == 0) != list_empty(&inode->u.nfs_i.dirty))
+ printk(KERN_ERR "NFS: desynchronized value of nfs_i.ndirty.\n");
+ return pages;
+}
+
+#ifdef CONFIG_NFS_V3
+static int
+nfs_scan_commit_timeout(struct inode *inode, struct list_head *dst)
+{
+ int pages;
+ pages = nfs_scan_list_timeout(&inode->u.nfs_i.commit, dst, inode);
+ inode->u.nfs_i.ncommit -= pages;
+ if ((inode->u.nfs_i.ncommit == 0) != list_empty(&inode->u.nfs_i.commit))
+ printk(KERN_ERR "NFS: desynchronized value of nfs_i.ncommit.\n");
+ return pages;
+}
+#endif
+
+int nfs_scan_list(struct list_head *src, struct list_head *dst,
+ struct file *file, unsigned long idx_start,
+ unsigned int npages)
+{
+ struct list_head *p;
+ struct nfs_page *req;
+ unsigned long idx_end;
+ int res;
+
+ res = 0;
+ if (npages == 0)
+ idx_end = ~0;
+ else
+ idx_end = idx_start + npages - 1;
+ p = src->next;
+ while (p != src) {
+ unsigned long pg_idx;
+
+ req = nfs_list_entry(p);
+ p = p->next;
+
+ if (file && req->wb_file != file)
+ continue;
+
+ pg_idx = page_index(req->wb_page);
+ if (pg_idx < idx_start || pg_idx > idx_end)
+ continue;
+
+ if (!nfs_lock_request(req))
+ continue;
+ nfs_list_remove_request(req);
+ nfs_list_add_request(req, dst);
+ res++;
}
+ return res;
+}
- return sync;
+static int
+nfs_scan_dirty(struct inode *inode, struct list_head *dst, struct file *file, unsigned long idx_start, unsigned int npages)
+{
+ int res;
+ res = nfs_scan_list(&inode->u.nfs_i.dirty, dst, file, idx_start, npages);
+ inode->u.nfs_i.ndirty -= res;
+ if ((inode->u.nfs_i.ndirty == 0) != list_empty(&inode->u.nfs_i.dirty))
+ printk(KERN_ERR "NFS: desynchronized value of nfs_i.ndirty.\n");
+ return res;
+}
+
+#ifdef CONFIG_NFS_V3
+static int
+nfs_scan_commit(struct inode *inode, struct list_head *dst, struct file *file, unsigned long idx_start, unsigned int npages)
+{
+ int res;
+ res = nfs_scan_list(&inode->u.nfs_i.commit, dst, file, idx_start, npages);
+ inode->u.nfs_i.ncommit -= res;
+ if ((inode->u.nfs_i.ncommit == 0) != list_empty(&inode->u.nfs_i.commit))
+ printk(KERN_ERR "NFS: desynchronized value of nfs_i.ncommit.\n");
+ return res;
+}
+#endif
+
+int nfs_coalesce_requests(struct list_head *src, struct list_head *dst, unsigned int maxpages)
+{
+ struct nfs_page *req = NULL;
+ unsigned int pages = 0;
+
+ while (!list_empty(src)) {
+ struct nfs_page *prev = req;
+
+ req = nfs_list_entry(src->next);
+ if (prev) {
+ if (req->wb_file != prev->wb_file)
+ break;
+
+ if (page_index(req->wb_page) != page_index(prev->wb_page)+1)
+ break;
+
+ if (req->wb_offset != 0)
+ break;
+ }
+ nfs_list_remove_request(req);
+ nfs_list_add_request(req, dst);
+ pages++;
+ if (req->wb_offset + req->wb_bytes != PAGE_CACHE_SIZE)
+ break;
+ if (pages >= maxpages)
+ break;
+ }
+ return pages;
}
/*
- * Wait for request to complete.
+ * Try to update any existing write request, or create one if there is none.
+ * In order to match, the request's credentials must match those of
+ * the calling process.
+ *
+ * Note: Should always be called with the Page Lock held!
*/
-static int
-wait_on_write_request(struct nfs_wreq *req)
+static struct nfs_page *
+nfs_update_request(struct file* file, struct page *page,
+ unsigned int offset, unsigned int bytes)
{
- struct file *file = req->wb_file;
- struct dentry *dentry = file->f_dentry;
- struct inode *inode = dentry->d_inode;
- struct rpc_clnt *clnt = NFS_CLIENT(inode);
- struct wait_queue wait = { current, NULL };
- sigset_t oldmask;
- int retval;
-
- /* Make sure it's started.. */
- if (!WB_INPROGRESS(req))
- rpc_wake_up_task(&req->wb_task);
+ struct inode *inode = file->f_dentry->d_inode;
+ struct nfs_page *req, *new = NULL;
+ unsigned long rqend, end;
+
+ end = offset + bytes;
- rpc_clnt_sigmask(clnt, &oldmask);
- add_wait_queue(&req->wb_wait, &wait);
for (;;) {
- current->state = TASK_INTERRUPTIBLE;
- retval = 0;
- if (req->wb_flags & NFS_WRITE_COMPLETE)
+ /* Loop over all inode entries and see if we find
+ * A request for the page we wish to update
+ */
+ req = _nfs_find_request(inode, page);
+ if (req) {
+ if (!nfs_lock_request(req)) {
+ nfs_wait_on_request(req);
+ nfs_release_request(req);
+ continue;
+ }
+ if (new)
+ nfs_release_request(new);
break;
- retval = -ERESTARTSYS;
- if (signalled())
+ }
+
+ req = new;
+ if (req) {
+ nfs_lock_request(req);
+ nfs_inode_add_request(inode, req);
+ nfs_mark_request_dirty(req);
break;
- schedule();
+ }
+
+ /*
+ * If we're over the soft limit, flush out old requests
+ */
+ if (inode->u.nfs_i.npages >= MAX_REQUEST_SOFT)
+ nfs_wb_file(inode, file);
+ new = nfs_create_request(file, page, offset, bytes);
+ if (!new)
+ return ERR_PTR(-ENOMEM);
+ /* If the region is locked, adjust the timeout */
+ if (region_locked(inode, new))
+ new->wb_timeout = jiffies + NFS_WRITEBACK_LOCKDELAY;
+ else
+ new->wb_timeout = jiffies + NFS_WRITEBACK_DELAY;
+ }
+
+ /* We have a request for our page.
+ * If the creds don't match, or the
+ * page addresses don't match,
+ * tell the caller to wait on the conflicting
+ * request.
+ */
+ rqend = req->wb_offset + req->wb_bytes;
+ if (req->wb_file != file
+ || req->wb_page != page
+ || !nfs_dirty_request(req)
+ || offset > rqend || end < req->wb_offset) {
+ nfs_unlock_request(req);
+ nfs_release_request(req);
+ return ERR_PTR(-EBUSY);
+ }
+
+ /* Okay, the request matches. Update the region */
+ if (offset < req->wb_offset) {
+ req->wb_offset = offset;
+ req->wb_bytes = rqend - req->wb_offset;
}
- remove_wait_queue(&req->wb_wait, &wait);
- current->state = TASK_RUNNING;
- rpc_clnt_sigunmask(clnt, &oldmask);
- return retval;
+
+ if (end > rqend)
+ req->wb_bytes = end - req->wb_offset;
+
+ nfs_unlock_request(req);
+
+ return req;
}
/*
- * Write a page to the server. This will be used for NFS swapping only
- * (for now), and we currently do this synchronously only.
+ * This is the strategy routine for NFS.
+ * It is called by nfs_updatepage whenever the user wrote up to the end
+ * of a page.
+ *
+ * We always try to submit a set of requests in parallel so that the
+ * server's write code can gather writes. This is mainly for the benefit
+ * of NFSv2.
+ *
+ * We never submit more requests than we think the remote can handle.
+ * For UDP sockets, we make sure we don't exceed the congestion window;
+ * for TCP, we limit the number of requests to 8.
+ *
+ * NFS_STRATEGY_PAGES gives the minimum number of requests for NFSv2 that
+ * should be sent out in one go. This is for the benefit of NFSv2 servers
+ * that perform write gathering.
+ *
+ * FIXME: Different servers may have different sweet spots.
+ * Record the average congestion window in server struct?
*/
+#define NFS_STRATEGY_PAGES 8
+static void
+nfs_strategy(struct inode *inode)
+{
+ unsigned int dirty, wpages;
+
+ dirty = inode->u.nfs_i.ndirty;
+ wpages = NFS_SERVER(inode)->wpages;
+#ifdef CONFIG_NFS_V3
+ if (NFS_PROTO(inode)->version == 2) {
+ if (dirty >= NFS_STRATEGY_PAGES * wpages)
+ nfs_flush_file(inode, NULL, 0, 0, 0);
+ } else {
+ if (dirty >= wpages)
+ nfs_flush_file(inode, NULL, 0, 0, 0);
+ if (inode->u.nfs_i.ncommit > NFS_STRATEGY_PAGES * wpages &&
+ nfs_nr_requests > MAX_REQUEST_SOFT)
+ nfs_commit_file(inode, NULL, 0, 0, 0);
+ }
+#else
+ if (dirty >= NFS_STRATEGY_PAGES * wpages)
+ nfs_flush_file(inode, NULL, 0, 0, 0);
+#endif
+ /*
+ * If we're running out of requests, flush out everything
+ * in order to reduce memory useage...
+ */
+ if (inode->u.nfs_i.npages > MAX_REQUEST_SOFT)
+ nfs_wb_all(inode);
+}
+
int
-nfs_writepage(struct file * file, struct page *page)
+nfs_flush_incompatible(struct file *file, struct page *page)
{
- struct dentry *dentry = file->f_dentry;
- return nfs_writepage_sync(dentry, dentry->d_inode, page, 0, PAGE_SIZE);
+ struct inode *inode = file->f_dentry->d_inode;
+ struct nfs_page *req;
+ int status = 0;
+ /*
+ * Look for a request corresponding to this page. If there
+ * is one, and it belongs to another file, we flush it out
+ * before we try to copy anything into the page. Do this
+ * due to the lack of an ACCESS-type call in NFSv2.
+ * Also do the same if we find a request from an existing
+ * dropped page.
+ */
+ req = nfs_find_request(inode,page);
+ if (req) {
+ if (req->wb_file != file || req->wb_page != page)
+ status = nfs_wb_page(inode, page);
+ nfs_release_request(req);
+ }
+ return (status < 0) ? status : 0;
}
/*
@@ -411,270 +879,532 @@
{
struct dentry *dentry = file->f_dentry;
struct inode *inode = dentry->d_inode;
- struct nfs_wreq *req;
- int synchronous = sync;
- int retval;
+ struct nfs_page *req;
+ int status = 0;
dprintk("NFS: nfs_updatepage(%s/%s %d@%ld, sync=%d)\n",
dentry->d_parent->d_name.name, dentry->d_name.name,
- count, page->offset+offset, sync);
-
- /*
- * Try to find a corresponding request on the writeback queue.
- * If there is one, we can be sure that this request is not
- * yet being processed, because we hold a lock on the page.
- *
- * If the request was created by us, update it. Otherwise,
- * transfer the page lock and flush out the dirty page now.
- * After returning, generic_file_write will wait on the
- * page and retry the update.
- */
- req = find_write_request(inode, page);
- if (req && req->wb_file == file && update_write_request(req, offset, count))
- goto updated;
+ count, nfs_page_offset(page)+offset, sync);
/*
* If wsize is smaller than page size, update and write
* page synchronously.
*/
- if (NFS_SERVER(inode)->wsize < PAGE_SIZE)
- return nfs_writepage_sync(dentry, inode, page, offset, count);
+ if (NFS_SERVER(inode)->wsize < PAGE_CACHE_SIZE)
+ return nfs_writepage_sync(file, page, offset, count);
- /* Create the write request. */
- req = create_write_request(file, page, offset, count);
- if (!req)
- return -ENOBUFS;
-
- /*
- * Ok, there's another user of this page with the new request..
- * The IO completion will then free the page and the dentry.
- */
- atomic_inc(&page->count);
- file->f_count++;
-
- /* Schedule request */
- synchronous = schedule_write_request(req, sync);
+ /*
+ * Try to find an NFS request corresponding to this page
+ * and update it.
+ * If the existing request cannot be updated, we must flush
+ * it out now.
+ */
+ do {
+ req = nfs_update_request(file, page, offset, count);
+ if (IS_ERR(req))
+ status = PTR_ERR(req);
+ if (status != -EBUSY)
+ break;
+ /* Request could not be updated. Flush it out and try again */
+ status = nfs_wb_page(inode, page);
+ } while (status >= 0);
+ if (status < 0)
+ goto done;
-updated:
- if (req->wb_bytes == PAGE_SIZE)
+ if (req->wb_bytes == PAGE_CACHE_SIZE)
set_bit(PG_uptodate, &page->flags);
- retval = count;
- if (synchronous) {
- int status = wait_on_write_request(req);
- if (status) {
- nfs_cancel_request(req);
- retval = status;
- } else {
- status = req->wb_status;
- if (status < 0)
- retval = status;
- }
+ status = count; /* unless we detect an error */
- if (retval < 0)
- clear_bit(PG_uptodate, &page->flags);
- }
+ /* If the user requested a sync write, do it now */
+ if (sync) {
+ int error;
- free_write_request(req);
- return retval;
+ error = nfs_sync_file(inode, file, page_index(page), 1, FLUSH_SYNC|FLUSH_STABLE);
+ if (error < 0 || (error = file->f_error) < 0)
+ status = error;
+ file->f_error = 0;
+ } else {
+ /* If we wrote past the end of the page.
+ * Call the strategy routine so it can send out a bunch
+ * of requests.
+ */
+ if (req->wb_offset == 0 && req->wb_bytes == PAGE_CACHE_SIZE)
+ nfs_strategy(inode);
+ }
+ nfs_release_request(req);
+done:
+ dprintk("NFS: nfs_updatepage returns %d (isize %ld)\n",
+ status, inode->i_size);
+ if (status < 0)
+ clear_bit(PG_uptodate, &page->flags);
+ return status;
}
/*
- * Cancel a write request. We always mark it cancelled,
- * but if it's already in progress there's no point in
- * calling rpc_exit, and we don't want to overwrite the
- * tk_status field.
- */
+ * Set up the argument/result storage required for the RPC call.
+ */
static void
-nfs_cancel_request(struct nfs_wreq *req)
+nfs_write_rpcsetup(struct list_head *head, struct nfs_write_data *data)
{
- req->wb_flags |= NFS_WRITE_CANCELLED;
- if (!WB_INPROGRESS(req)) {
- rpc_exit(&req->wb_task, 0);
- rpc_wake_up_task(&req->wb_task);
+ struct nfs_page *req;
+ struct iovec *iov;
+ unsigned int count;
+
+ /* Set up the RPC argument and reply structs
+ * NB: take care not to mess about with data->commit et al. */
+
+ iov = data->args.iov;
+ count = 0;
+ while (!list_empty(head)) {
+ struct nfs_page *req = nfs_list_entry(head->next);
+ nfs_list_remove_request(req);
+ nfs_list_add_request(req, &data->pages);
+ iov->iov_base = (void *)(page_address(req->wb_page) + req->wb_offset);
+ iov->iov_len = req->wb_bytes;
+ count += req->wb_bytes;
+ iov++;
+ data->args.nriov++;
}
+ req = nfs_list_entry(data->pages.next);
+ data->dentry = req->wb_dentry;
+ data->cred = req->wb_cred;
+ data->args.fh = NFS_FH(req->wb_dentry);
+ data->args.offset = nfs_page_offset(req->wb_page) + req->wb_offset;
+ data->args.count = count;
+ data->res.fattr = &data->fattr;
+ data->res.count = count;
+ data->res.verf = &data->verf;
}
+
/*
- * Cancel all writeback requests, both pending and in progress.
+ * Create an RPC task for the given write request and kick it.
+ * The page must have been locked by the caller.
+ *
+ * It may happen that the page we're passed is not marked dirty.
+ * This is the case if nfs_updatepage detects a conflicting request
+ * that has been written but not committed.
*/
-static void
-nfs_cancel_dirty(struct inode *inode, pid_t pid)
+static int
+nfs_flush_one(struct list_head *head, struct dentry *dentry, int how)
{
- struct nfs_wreq *head, *req;
+ struct inode *inode = dentry->d_inode;
+ struct rpc_clnt *clnt = NFS_CLIENT(inode);
+ struct nfs_write_data *data;
+ struct rpc_task *task;
+ struct rpc_message msg;
+ int flags,
+ async = !(how & FLUSH_SYNC),
+ stable = (how & FLUSH_STABLE);
+ sigset_t oldset;
+
+
+ data = nfs_writedata_alloc();
+ if (!data)
+ goto out_bad;
+ task = &data->task;
+
+ /* Set the initial flags for the task. */
+ flags = (async) ? RPC_TASK_ASYNC : 0;
+
+ /* Set up the argument struct */
+ nfs_write_rpcsetup(head, data);
+ if (stable) {
+ if (!inode->u.nfs_i.ncommit)
+ data->args.stable = NFS_FILE_SYNC;
+ else
+ data->args.stable = NFS_DATA_SYNC;
+ } else
+ data->args.stable = NFS_UNSTABLE;
+
+ /* Finalize the task. */
+ rpc_init_task(task, clnt, nfs_writeback_done, flags);
+ task->tk_calldata = data;
+ /* Release requests */
+ task->tk_release = nfs_writedata_release;
+
+#ifdef CONFIG_NFS_V3
+ msg.rpc_proc = (NFS_PROTO(inode)->version == 3) ? NFS3PROC_WRITE : NFSPROC_WRITE;
+#else
+ msg.rpc_proc = NFSPROC_WRITE;
+#endif
+ msg.rpc_argp = &data->args;
+ msg.rpc_resp = &data->res;
+ msg.rpc_cred = data->cred;
+
+ dprintk("NFS: %4d initiated write call (req %s/%s count %d nriov %d)\n",
+ task->tk_pid,
+ dentry->d_parent->d_name.name,
+ dentry->d_name.name,
+ data->args.count, data->args.nriov);
+
+ rpc_clnt_sigmask(clnt, &oldset);
+ rpc_call_setup(task, &msg, 0);
+ rpc_execute(task);
+ rpc_clnt_sigunmask(clnt, &oldset);
+ return 0;
+ out_bad:
+ while (!list_empty(head)) {
+ struct nfs_page *req = nfs_list_entry(head->next);
+ nfs_list_remove_request(req);
+ nfs_mark_request_dirty(req);
+ nfs_unlock_request(req);
+ }
+ return -ENOMEM;
+}
- req = head = NFS_WRITEBACK(inode);
- while (req != NULL) {
- if (pid == 0 || req->wb_pid == pid)
- nfs_cancel_request(req);
- if ((req = WB_NEXT(req)) == head)
+static int
+nfs_flush_list(struct inode *inode, struct list_head *head, int how)
+{
+ LIST_HEAD(one_request);
+ struct nfs_page *req;
+ int error = 0;
+ unsigned int pages = 0,
+ wpages = NFS_SERVER(inode)->wpages;
+
+ while (!list_empty(head)) {
+ pages += nfs_coalesce_requests(head, &one_request, wpages);
+ req = nfs_list_entry(one_request.next);
+ error = nfs_flush_one(&one_request, req->wb_dentry, how);
+ if (error < 0)
break;
}
+ if (error >= 0)
+ return pages;
+
+ while (!list_empty(head)) {
+ req = nfs_list_entry(head->next);
+ nfs_list_remove_request(req);
+ nfs_mark_request_dirty(req);
+ nfs_unlock_request(req);
+ }
+ return error;
}
+
/*
- * If we're waiting on somebody else's request
- * we need to increment the counter during the
- * wait so that the request doesn't disappear
- * from under us during the wait..
+ * This function is called when the WRITE call is complete.
*/
-static int FASTCALL(wait_on_other_req(struct nfs_wreq *));
-static int wait_on_other_req(struct nfs_wreq *req)
+static void
+nfs_writeback_done(struct rpc_task *task)
{
- int retval;
- req->wb_count++;
- retval = wait_on_write_request(req);
- free_write_request(req);
- return retval;
-}
+ struct nfs_write_data *data = (struct nfs_write_data *) task->tk_calldata;
+ struct nfs_writeargs *argp = &data->args;
+ struct nfs_writeres *resp = &data->res;
+ struct dentry *dentry = data->dentry;
+ struct inode *inode = dentry->d_inode;
+ struct nfs_page *req;
+ struct page *page;
-/*
- * This writes back a set of requests according to the condition.
- *
- * If this ever gets much more convoluted, use a fn pointer for
- * the condition..
- */
-#define NFS_WB(inode, cond) { int retval = 0 ; \
- do { \
- struct nfs_wreq *req = NFS_WRITEBACK(inode); \
- struct nfs_wreq *head = req; \
- if (!req) break; \
- for (;;) { \
- if (!(req->wb_flags & NFS_WRITE_COMPLETE)) \
- if (cond) break; \
- req = WB_NEXT(req); \
- if (req == head) goto out; \
- } \
- retval = wait_on_other_req(req); \
- } while (!retval); \
-out: return retval; \
-}
+ dprintk("NFS: %4d nfs_writeback_done (status %d)\n",
+ task->tk_pid, task->tk_status);
-int
-nfs_wb_all(struct inode *inode)
-{
- NFS_WB(inode, 1);
+ /* We can't handle that yet but we check for it nevertheless */
+ if (resp->count < argp->count && task->tk_status >= 0) {
+ static unsigned long complain = 0;
+ if (time_before(complain, jiffies)) {
+ printk(KERN_WARNING
+ "NFS: Server wrote less than requested.\n");
+ complain = jiffies + 300 * HZ;
+ }
+ /* Can't do anything about it right now except throw
+ * an error. */
+ task->tk_status = -EIO;
+ }
+#ifdef CONFIG_NFS_V3
+ if (resp->verf->committed < argp->stable && task->tk_status >= 0) {
+ /* We tried a write call, but the server did not
+ * commit data to stable storage even though we
+ * requested it.
+ */
+ static unsigned long complain = 0;
+
+ if (time_before(complain, jiffies)) {
+ dprintk("NFS: faulty NFSv3 server %s:"
+ " (committed = %d) != (stable = %d)\n",
+ NFS_SERVER(inode)->hostname,
+ resp->verf->committed, argp->stable);
+ complain = jiffies + 300 * HZ;
+ }
+ }
+#endif
+
+ /* Update attributes as result of writeback. */
+ nfs_write_attributes(inode, resp->fattr);
+
+ while (!list_empty(&data->pages)) {
+ req = nfs_list_entry(data->pages.next);
+ nfs_list_remove_request(req);
+ page = req->wb_page;
+
+ dprintk("NFS: write (%s/%s %d@%Ld)",
+ req->wb_dentry->d_parent->d_name.name,
+ req->wb_dentry->d_name.name,
+ req->wb_bytes,
+ (long long)(nfs_page_offset(page) + req->wb_offset));
+
+ if (task->tk_status < 0) {
+ clear_bit(PG_uptodate, &page->flags);
+ set_bit(PG_error, &page->flags);
+ nfs_inode_remove_request(req);
+ if (req->wb_file)
+ req->wb_file->f_error = task->tk_status;
+ dprintk(", error = %d\n", task->tk_status);
+ goto next;
+ }
+
+#ifdef CONFIG_NFS_V3
+ if (resp->verf->committed != NFS_UNSTABLE) {
+ nfs_inode_remove_request(req);
+ dprintk(" OK\n");
+ goto next;
+ }
+
+ memcpy(&req->wb_verf, resp->verf, sizeof(req->wb_verf));
+ req->wb_timeout = jiffies + NFS_COMMIT_DELAY;
+ nfs_mark_request_commit(req);
+ dprintk(" marked for commit\n");
+#else
+ nfs_inode_remove_request(req);
+#endif
+ next:
+ nfs_unlock_request(req);
+ }
}
+#ifdef CONFIG_NFS_V3
/*
- * Write back all requests on one page - we do this before reading it.
+ * Set up the argument/result storage required for the RPC call.
*/
-int
-nfs_wb_page(struct inode *inode, struct page *page)
+static void
+nfs_commit_rpcsetup(struct list_head *head, struct nfs_write_data *data)
{
- NFS_WB(inode, req->wb_page == page);
+ struct nfs_page *req;
+ struct dentry *dentry;
+ struct inode *inode;
+ unsigned long start, end, len;
+
+ /* Set up the RPC argument and reply structs
+ * NB: take care not to mess about with data->commit et al. */
+
+ end = 0;
+ start = ~0;
+ req = nfs_list_entry(head->next);
+ dentry = req->wb_dentry;
+ data->dentry = dentry;
+ data->cred = req->wb_cred;
+ inode = dentry->d_inode;
+ while (!list_empty(head)) {
+ struct nfs_page *req;
+ unsigned long rqstart, rqend;
+ req = nfs_list_entry(head->next);
+ nfs_list_remove_request(req);
+ nfs_list_add_request(req, &data->pages);
+ rqstart = nfs_page_offset(req->wb_page) + req->wb_offset;
+ rqend = rqstart + req->wb_bytes;
+ if (rqstart < start)
+ start = rqstart;
+ if (rqend > end)
+ end = rqend;
+ }
+ data->args.fh = NFS_FH(dentry);
+ data->args.offset = start;
+ len = end - start;
+ if (end >= inode->i_size || len > (~((u32)0) >> 1))
+ len = 0;
+ data->res.count = data->args.count = (u32)len;
+ data->res.fattr = &data->fattr;
+ data->res.verf = &data->verf;
}
/*
- * Write back all pending writes from one file descriptor..
+ * Commit dirty pages
*/
-int
-nfs_wb_file(struct inode *inode, struct file *file)
-{
- NFS_WB(inode, req->wb_file == file);
-}
-
-void
-nfs_inval(struct inode *inode)
+static int
+nfs_commit_list(struct list_head *head, int how)
{
- nfs_cancel_dirty(inode,0);
+ struct rpc_message msg;
+ struct rpc_clnt *clnt;
+ struct nfs_write_data *data;
+ struct rpc_task *task;
+ struct nfs_page *req;
+ int flags,
+ async = !(how & FLUSH_SYNC);
+ sigset_t oldset;
+
+ data = nfs_writedata_alloc();
+ if (!data)
+ goto out_bad;
+ task = &data->task;
+
+ flags = (async) ? RPC_TASK_ASYNC : 0;
+
+ /* Set up the argument struct */
+ nfs_commit_rpcsetup(head, data);
+ req = nfs_list_entry(data->pages.next);
+ clnt = NFS_CLIENT(req->wb_dentry->d_inode);
+
+ rpc_init_task(task, clnt, nfs_commit_done, flags);
+ task->tk_calldata = data;
+ task->tk_release = nfs_writedata_release;
+
+ msg.rpc_proc = NFS3PROC_COMMIT;
+ msg.rpc_argp = &data->args;
+ msg.rpc_resp = &data->res;
+ msg.rpc_cred = data->cred;
+
+ dprintk("NFS: %4d initiated commit call\n", task->tk_pid);
+ rpc_clnt_sigmask(clnt, &oldset);
+ rpc_call_setup(task, &msg, 0);
+ rpc_execute(task);
+ rpc_clnt_sigunmask(clnt, &oldset);
+ return 0;
+ out_bad:
+ while (!list_empty(head)) {
+ req = nfs_list_entry(head->next);
+ nfs_list_remove_request(req);
+ nfs_mark_request_commit(req);
+ nfs_unlock_request(req);
+ }
+ return -ENOMEM;
}
/*
- * The following procedures make up the writeback finite state machinery:
- *
- * 1. Try to lock the page if not yet locked by us,
- * set up the RPC call info, and pass to the call FSM.
+ * COMMIT call returned
*/
static void
-nfs_wback_begin(struct rpc_task *task)
+nfs_commit_done(struct rpc_task *task)
{
- struct nfs_wreq *req = (struct nfs_wreq *) task->tk_calldata;
- struct page *page = req->wb_page;
- struct file *file = req->wb_file;
- struct dentry *dentry = file->f_dentry;
+ struct nfs_write_data *data = (struct nfs_write_data *)task->tk_calldata;
+ struct nfs_writeres *resp = &data->res;
+ struct nfs_page *req;
+ struct dentry *dentry = data->dentry;
+ struct inode *inode = dentry->d_inode;
- dprintk("NFS: %4d nfs_wback_begin (%s/%s, status=%d flags=%x)\n",
- task->tk_pid, dentry->d_parent->d_name.name,
- dentry->d_name.name, task->tk_status, req->wb_flags);
+ dprintk("NFS: %4d nfs_commit_done (status %d)\n",
+ task->tk_pid, task->tk_status);
- task->tk_status = 0;
+ nfs_write_attributes(inode, resp->fattr);
+ while (!list_empty(&data->pages)) {
+ req = nfs_list_entry(data->pages.next);
+ nfs_list_remove_request(req);
+
+ dprintk("NFS: commit (%s/%s %d@%ld)",
+ req->wb_dentry->d_parent->d_name.name,
+ req->wb_dentry->d_name.name,
+ req->wb_bytes,
+ nfs_page_offset(req->wb_page) + req->wb_offset);
+ if (task->tk_status < 0) {
+ if (req->wb_file)
+ req->wb_file->f_error = task->tk_status;
+ nfs_inode_remove_request(req);
+ dprintk(", error = %d\n", task->tk_status);
+ goto next;
+ }
- /* Setup the task struct for a writeback call */
- req->wb_flags |= NFS_WRITE_INPROGRESS;
- req->wb_args.fh = NFS_FH(dentry);
- req->wb_args.offset = page->offset + req->wb_offset;
- req->wb_args.count = req->wb_bytes;
- req->wb_args.buffer = (void *) (page_address(page) + req->wb_offset);
+ /* Okay, COMMIT succeeded, apparently. Check the verifier
+ * returned by the server against all stored verfs. */
+ if (!memcmp(req->wb_verf.verifier, data->verf.verifier, sizeof(data->verf.verifier))) {
+ /* We have a match */
+ nfs_inode_remove_request(req);
+ dprintk(" OK\n");
+ goto next;
+ }
+ /* We have a mismatch. Write the page again */
+ dprintk(" mismatch\n");
+ nfs_mark_request_dirty(req);
+ next:
+ nfs_unlock_request(req);
+ }
+}
+#endif
- rpc_call_setup(task, NFSPROC_WRITE, &req->wb_args, &req->wb_fattr, 0);
+int nfs_flush_file(struct inode *inode, struct file *file, unsigned long idx_start,
+ unsigned int npages, int how)
+{
+ LIST_HEAD(head);
+ int res,
+ error = 0;
- return;
+ res = nfs_scan_dirty(inode, &head, file, idx_start, npages);
+ if (res)
+ error = nfs_flush_list(inode, &head, how);
+ if (error < 0)
+ return error;
+ return res;
}
-/*
- * 2. Collect the result
- */
-static void
-nfs_wback_result(struct rpc_task *task)
+int nfs_flush_timeout(struct inode *inode, int how)
{
- struct nfs_wreq *req = (struct nfs_wreq *) task->tk_calldata;
- struct file *file = req->wb_file;
- struct page *page = req->wb_page;
- int status = task->tk_status;
- struct dentry *dentry = file->f_dentry;
- struct inode *inode = dentry->d_inode;
+ LIST_HEAD(head);
+ int pages,
+ error = 0;
- dprintk("NFS: %4d nfs_wback_result (%s/%s, status=%d, flags=%x)\n",
- task->tk_pid, dentry->d_parent->d_name.name,
- dentry->d_name.name, status, req->wb_flags);
-
- /* Set the WRITE_COMPLETE flag, but leave WRITE_INPROGRESS set */
- req->wb_flags |= NFS_WRITE_COMPLETE;
- req->wb_status = status;
-
- if (status < 0) {
- req->wb_flags |= NFS_WRITE_INVALIDATE;
- file->f_error = status;
- } else if (!WB_CANCELLED(req)) {
- struct nfs_fattr *fattr = &req->wb_fattr;
- /* Update attributes as result of writeback.
- * Beware: when UDP replies arrive out of order, we
- * may end up overwriting a previous, bigger file size.
- *
- * When the file size shrinks we cancel all pending
- * writebacks.
- */
- if (fattr->mtime.seconds >= inode->i_mtime) {
- if (fattr->size < inode->i_size)
- fattr->size = inode->i_size;
-
- /* possible Solaris 2.5 server bug workaround */
- if (inode->i_ino == fattr->fileid) {
- /*
- * We expect these values to change, and
- * don't want to invalidate the caches.
- */
- inode->i_size = fattr->size;
- inode->i_mtime = fattr->mtime.seconds;
- nfs_refresh_inode(inode, fattr);
- }
- else
- printk("nfs_wback_result: inode %ld, got %u?\n",
- inode->i_ino, fattr->fileid);
- }
+ pages = nfs_scan_dirty_timeout(inode, &head);
+ if (pages)
+ error = nfs_flush_list(inode, &head, how);
+ if (error < 0)
+ return error;
+ return pages;
+}
+
+#ifdef CONFIG_NFS_V3
+int nfs_commit_file(struct inode *inode, struct file *file, unsigned long idx_start,
+ unsigned int npages, int how)
+{
+ LIST_HEAD(head);
+ int res,
+ error = 0;
+
+ res = nfs_scan_commit(inode, &head, file, idx_start, npages);
+ if (res)
+ error = nfs_commit_list(&head, how);
+ if (error < 0)
+ return error;
+ return res;
+}
+
+int nfs_commit_timeout(struct inode *inode, int how)
+{
+ LIST_HEAD(head);
+ int pages,
+ error = 0;
+
+ pages = nfs_scan_commit_timeout(inode, &head);
+ if (pages) {
+ pages += nfs_scan_commit(inode, &head, NULL, 0, 0);
+ error = nfs_commit_list(&head, how);
}
+ if (error < 0)
+ return error;
+ return pages;
+}
+#endif
- rpc_release_task(task);
+int nfs_sync_file(struct inode *inode, struct file *file, unsigned long idx_start,
+ unsigned int npages, int how)
+{
+ int error,
+ wait;
- if (WB_INVALIDATE(req))
- clear_bit(PG_uptodate, &page->flags);
+ wait = how & FLUSH_WAIT;
+ how &= ~FLUSH_WAIT;
- __free_page(page);
- remove_write_request(&NFS_WRITEBACK(inode), req);
- nr_write_requests--;
- fput(req->wb_file);
+ if (!inode && file)
+ inode = file->f_dentry->d_inode;
- wake_up(&req->wb_wait);
- free_write_request(req);
+ do {
+ error = 0;
+ if (wait)
+ error = nfs_wait_on_requests(inode, file, idx_start, npages);
+ if (error == 0)
+ error = nfs_flush_file(inode, file, idx_start, npages, how);
+#ifdef CONFIG_NFS_V3
+ if (error == 0)
+ error = nfs_commit_file(inode, file, idx_start, npages, how);
+#endif
+ } while (error > 0);
+ return error;
}
+
FUNET's LINUX-ADM group, linux-adm@nic.funet.fi
TCL-scripts by Sam Shen (who was at: slshen@lbl.gov)