summaryrefslogtreecommitdiff
path: root/fs/nfs/write.c
diff options
context:
space:
mode:
Diffstat (limited to 'fs/nfs/write.c')
-rw-r--r--fs/nfs/write.c2164
1 files changed, 1222 insertions, 942 deletions
diff --git a/fs/nfs/write.c b/fs/nfs/write.c
index f1bdb7254776..336c510f3750 100644
--- a/fs/nfs/write.c
+++ b/fs/nfs/write.c
@@ -1,3 +1,4 @@
+// SPDX-License-Identifier: GPL-2.0-only
/*
* linux/fs/nfs/write.c
*
@@ -21,8 +22,13 @@
#include <linux/nfs_page.h>
#include <linux/backing-dev.h>
#include <linux/export.h>
+#include <linux/freezer.h>
+#include <linux/wait.h>
+#include <linux/iversion.h>
+#include <linux/filelock.h>
-#include <asm/uaccess.h>
+#include <linux/uaccess.h>
+#include <linux/sched/mm.h>
#include "delegation.h"
#include "internal.h"
@@ -31,19 +37,32 @@
#include "fscache.h"
#include "pnfs.h"
+#include "nfstrace.h"
+
#define NFSDBG_FACILITY NFSDBG_PAGECACHE
#define MIN_POOL_WRITE (32)
#define MIN_POOL_COMMIT (4)
+struct nfs_io_completion {
+ void (*complete)(void *data);
+ void *data;
+ struct kref refcount;
+};
+
/*
* Local function declarations
*/
static void nfs_redirty_request(struct nfs_page *req);
-static const struct rpc_call_ops nfs_write_common_ops;
static const struct rpc_call_ops nfs_commit_ops;
static const struct nfs_pgio_completion_ops nfs_async_write_completion_ops;
static const struct nfs_commit_completion_ops nfs_commit_completion_ops;
+static const struct nfs_rw_ops nfs_rw_write_ops;
+static void nfs_inode_remove_request(struct nfs_page *req);
+static void nfs_clear_request_commit(struct nfs_commit_info *cinfo,
+ struct nfs_page *req);
+static void nfs_init_cinfo_from_inode(struct nfs_commit_info *cinfo,
+ struct inode *inode);
static struct kmem_cache *nfs_wdata_cachep;
static mempool_t *nfs_wdata_mempool;
@@ -52,12 +71,16 @@ static mempool_t *nfs_commit_mempool;
struct nfs_commit_data *nfs_commitdata_alloc(void)
{
- struct nfs_commit_data *p = mempool_alloc(nfs_commit_mempool, GFP_NOIO);
+ struct nfs_commit_data *p;
- if (p) {
+ p = kmem_cache_zalloc(nfs_cdata_cachep, nfs_io_gfp_mask());
+ if (!p) {
+ p = mempool_alloc(nfs_commit_mempool, GFP_NOWAIT);
+ if (!p)
+ return NULL;
memset(p, 0, sizeof(*p));
- INIT_LIST_HEAD(&p->pages);
}
+ INIT_LIST_HEAD(&p->pages);
return p;
}
EXPORT_SYMBOL_GPL(nfs_commitdata_alloc);
@@ -68,168 +91,186 @@ void nfs_commit_free(struct nfs_commit_data *p)
}
EXPORT_SYMBOL_GPL(nfs_commit_free);
-struct nfs_write_header *nfs_writehdr_alloc(void)
+static struct nfs_pgio_header *nfs_writehdr_alloc(void)
{
- struct nfs_write_header *p = mempool_alloc(nfs_wdata_mempool, GFP_NOIO);
-
- if (p) {
- struct nfs_pgio_header *hdr = &p->header;
+ struct nfs_pgio_header *p;
+ p = kmem_cache_zalloc(nfs_wdata_cachep, nfs_io_gfp_mask());
+ if (!p) {
+ p = mempool_alloc(nfs_wdata_mempool, GFP_NOWAIT);
+ if (!p)
+ return NULL;
memset(p, 0, sizeof(*p));
- INIT_LIST_HEAD(&hdr->pages);
- INIT_LIST_HEAD(&hdr->rpc_list);
- spin_lock_init(&hdr->lock);
- atomic_set(&hdr->refcnt, 0);
- hdr->verf = &p->verf;
}
+ p->rw_mode = FMODE_WRITE;
return p;
}
-EXPORT_SYMBOL_GPL(nfs_writehdr_alloc);
-static struct nfs_write_data *nfs_writedata_alloc(struct nfs_pgio_header *hdr,
- unsigned int pagecount)
+static void nfs_writehdr_free(struct nfs_pgio_header *hdr)
{
- struct nfs_write_data *data, *prealloc;
-
- prealloc = &container_of(hdr, struct nfs_write_header, header)->rpc_data;
- if (prealloc->header == NULL)
- data = prealloc;
- else
- data = kzalloc(sizeof(*data), GFP_KERNEL);
- if (!data)
- goto out;
-
- if (nfs_pgarray_set(&data->pages, pagecount)) {
- data->header = hdr;
- atomic_inc(&hdr->refcnt);
- } else {
- if (data != prealloc)
- kfree(data);
- data = NULL;
- }
-out:
- return data;
+ mempool_free(hdr, nfs_wdata_mempool);
}
-void nfs_writehdr_free(struct nfs_pgio_header *hdr)
+static struct nfs_io_completion *nfs_io_completion_alloc(gfp_t gfp_flags)
{
- struct nfs_write_header *whdr = container_of(hdr, struct nfs_write_header, header);
- mempool_free(whdr, nfs_wdata_mempool);
+ return kmalloc(sizeof(struct nfs_io_completion), gfp_flags);
}
-EXPORT_SYMBOL_GPL(nfs_writehdr_free);
-void nfs_writedata_release(struct nfs_write_data *wdata)
+static void nfs_io_completion_init(struct nfs_io_completion *ioc,
+ void (*complete)(void *), void *data)
{
- struct nfs_pgio_header *hdr = wdata->header;
- struct nfs_write_header *write_header = container_of(hdr, struct nfs_write_header, header);
-
- put_nfs_open_context(wdata->args.context);
- if (wdata->pages.pagevec != wdata->pages.page_array)
- kfree(wdata->pages.pagevec);
- if (wdata == &write_header->rpc_data) {
- wdata->header = NULL;
- wdata = NULL;
- }
- if (atomic_dec_and_test(&hdr->refcnt))
- hdr->completion_ops->completion(hdr);
- /* Note: we only free the rpc_task after callbacks are done.
- * See the comment in rpc_free_task() for why
- */
- kfree(wdata);
+ ioc->complete = complete;
+ ioc->data = data;
+ kref_init(&ioc->refcount);
}
-EXPORT_SYMBOL_GPL(nfs_writedata_release);
-static void nfs_context_set_write_error(struct nfs_open_context *ctx, int error)
+static void nfs_io_completion_release(struct kref *kref)
{
- ctx->error = error;
- smp_wmb();
- set_bit(NFS_CONTEXT_ERROR_WRITE, &ctx->flags);
+ struct nfs_io_completion *ioc = container_of(kref,
+ struct nfs_io_completion, refcount);
+ ioc->complete(ioc->data);
+ kfree(ioc);
}
-static struct nfs_page *
-nfs_page_find_request_locked(struct nfs_inode *nfsi, struct page *page)
+static void nfs_io_completion_get(struct nfs_io_completion *ioc)
{
- struct nfs_page *req = NULL;
-
- if (PagePrivate(page))
- req = (struct nfs_page *)page_private(page);
- else if (unlikely(PageSwapCache(page))) {
- struct nfs_page *freq, *t;
+ if (ioc != NULL)
+ kref_get(&ioc->refcount);
+}
- /* Linearly search the commit list for the correct req */
- list_for_each_entry_safe(freq, t, &nfsi->commit_info.list, wb_list) {
- if (freq->wb_page == page) {
- req = freq;
- break;
- }
- }
- }
+static void nfs_io_completion_put(struct nfs_io_completion *ioc)
+{
+ if (ioc != NULL)
+ kref_put(&ioc->refcount, nfs_io_completion_release);
+}
- if (req)
+static void
+nfs_page_set_inode_ref(struct nfs_page *req, struct inode *inode)
+{
+ if (!test_and_set_bit(PG_INODE_REF, &req->wb_flags)) {
kref_get(&req->wb_kref);
+ atomic_long_inc(&NFS_I(inode)->nrequests);
+ }
+}
- return req;
+static void nfs_cancel_remove_inode(struct nfs_page *req, struct inode *inode)
+{
+ if (test_and_clear_bit(PG_REMOVE, &req->wb_flags))
+ nfs_page_set_inode_ref(req, inode);
}
-static struct nfs_page *nfs_page_find_request(struct page *page)
+/**
+ * nfs_folio_find_head_request - find head request associated with a folio
+ * @folio: pointer to folio
+ *
+ * must be called while holding the inode lock.
+ *
+ * returns matching head request with reference held, or NULL if not found.
+ */
+static struct nfs_page *nfs_folio_find_head_request(struct folio *folio)
{
- struct inode *inode = page_file_mapping(page)->host;
- struct nfs_page *req = NULL;
+ struct address_space *mapping = folio->mapping;
+ struct nfs_page *req;
- spin_lock(&inode->i_lock);
- req = nfs_page_find_request_locked(NFS_I(inode), page);
- spin_unlock(&inode->i_lock);
+ if (!folio_test_private(folio))
+ return NULL;
+ spin_lock(&mapping->i_private_lock);
+ req = folio->private;
+ if (req) {
+ WARN_ON_ONCE(req->wb_head != req);
+ kref_get(&req->wb_kref);
+ }
+ spin_unlock(&mapping->i_private_lock);
return req;
}
/* Adjust the file length if we're writing beyond the end */
-static void nfs_grow_file(struct page *page, unsigned int offset, unsigned int count)
+static void nfs_grow_file(struct folio *folio, unsigned int offset,
+ unsigned int count)
{
- struct inode *inode = page_file_mapping(page)->host;
+ struct inode *inode = folio->mapping->host;
loff_t end, i_size;
pgoff_t end_index;
spin_lock(&inode->i_lock);
i_size = i_size_read(inode);
- end_index = (i_size - 1) >> PAGE_CACHE_SHIFT;
- if (i_size > 0 && page_file_index(page) < end_index)
+ end_index = ((i_size - 1) >> folio_shift(folio)) << folio_order(folio);
+ if (i_size > 0 && folio->index < end_index)
goto out;
- end = page_file_offset(page) + ((loff_t)offset+count);
+ end = folio_pos(folio) + (loff_t)offset + (loff_t)count;
if (i_size >= end)
goto out;
+ trace_nfs_size_grow(inode, end);
i_size_write(inode, end);
+ NFS_I(inode)->cache_validity &= ~NFS_INO_INVALID_SIZE;
nfs_inc_stats(inode, NFSIOS_EXTENDWRITE);
out:
+ /* Atomically update timestamps if they are delegated to us. */
+ nfs_update_delegated_mtime_locked(inode);
spin_unlock(&inode->i_lock);
+ nfs_fscache_invalidate(inode, 0);
}
/* A writeback failed: mark the page as bad, and invalidate the page cache */
-static void nfs_set_pageerror(struct page *page)
+static void nfs_set_pageerror(struct address_space *mapping)
+{
+ struct inode *inode = mapping->host;
+
+ nfs_zap_mapping(mapping->host, mapping);
+ /* Force file size revalidation */
+ spin_lock(&inode->i_lock);
+ nfs_set_cache_invalid(inode, NFS_INO_REVAL_FORCED |
+ NFS_INO_INVALID_CHANGE |
+ NFS_INO_INVALID_SIZE);
+ spin_unlock(&inode->i_lock);
+}
+
+static void nfs_mapping_set_error(struct folio *folio, int error)
{
- nfs_zap_mapping(page_file_mapping(page)->host, page_file_mapping(page));
+ struct address_space *mapping = folio->mapping;
+
+ filemap_set_wb_err(mapping, error);
+ if (mapping->host)
+ errseq_set(&mapping->host->i_sb->s_wb_err,
+ error == -ENOSPC ? -ENOSPC : -EIO);
+ nfs_set_pageerror(mapping);
+}
+
+/*
+ * nfs_page_covers_folio
+ * @req: struct nfs_page
+ *
+ * Return true if the request covers the whole folio.
+ * Note that the caller should ensure all subrequests have been joined
+ */
+static bool nfs_page_group_covers_page(struct nfs_page *req)
+{
+ unsigned int len = nfs_folio_length(nfs_page_to_folio(req));
+
+ return req->wb_pgbase == 0 && req->wb_bytes == len;
}
/* We can set the PG_uptodate flag if we see that a write request
* covers the full page.
*/
-static void nfs_mark_uptodate(struct page *page, unsigned int base, unsigned int count)
+static void nfs_mark_uptodate(struct nfs_page *req)
{
- if (PageUptodate(page))
- return;
- if (base != 0)
+ struct folio *folio = nfs_page_to_folio(req);
+
+ if (folio_test_uptodate(folio))
return;
- if (count != nfs_page_length(page))
+ if (!nfs_page_group_covers_page(req))
return;
- SetPageUptodate(page);
+ folio_mark_uptodate(folio);
}
static int wb_priority(struct writeback_control *wbc)
{
- if (wbc->for_reclaim)
- return FLUSH_HIGHPRI | FLUSH_STABLE;
- if (wbc->for_kupdate || wbc->for_background)
- return FLUSH_LOWPRI | FLUSH_COND_STABLE;
- return FLUSH_COND_STABLE;
+ int ret = 0;
+
+ if (wbc->sync_mode == WB_SYNC_ALL)
+ ret = FLUSH_COND_STABLE;
+ return ret;
}
/*
@@ -242,205 +283,450 @@ int nfs_congestion_kb;
#define NFS_CONGESTION_OFF_THRESH \
(NFS_CONGESTION_ON_THRESH - (NFS_CONGESTION_ON_THRESH >> 2))
-static void nfs_set_page_writeback(struct page *page)
+static void nfs_folio_set_writeback(struct folio *folio)
{
- struct nfs_server *nfss = NFS_SERVER(page_file_mapping(page)->host);
- int ret = test_set_page_writeback(page);
+ struct nfs_server *nfss = NFS_SERVER(folio->mapping->host);
- WARN_ON_ONCE(ret != 0);
+ folio_start_writeback(folio);
+ if (atomic_long_inc_return(&nfss->writeback) > NFS_CONGESTION_ON_THRESH)
+ nfss->write_congested = 1;
+}
+
+static void nfs_folio_end_writeback(struct folio *folio)
+{
+ struct nfs_server *nfss = NFS_SERVER(folio->mapping->host);
- if (atomic_long_inc_return(&nfss->writeback) >
- NFS_CONGESTION_ON_THRESH) {
- set_bdi_congested(&nfss->backing_dev_info,
- BLK_RW_ASYNC);
+ folio_end_writeback_no_dropbehind(folio);
+ if (atomic_long_dec_return(&nfss->writeback) <
+ NFS_CONGESTION_OFF_THRESH) {
+ nfss->write_congested = 0;
+ wake_up_all(&nfss->write_congestion_wait);
}
}
-static void nfs_end_page_writeback(struct page *page)
+static void nfs_page_end_writeback(struct nfs_page *req)
{
- struct inode *inode = page_file_mapping(page)->host;
- struct nfs_server *nfss = NFS_SERVER(inode);
+ if (nfs_page_group_sync_on_bit(req, PG_WB_END)) {
+ nfs_unlock_request(req);
+ nfs_folio_end_writeback(nfs_page_to_folio(req));
+ } else
+ nfs_unlock_request(req);
+}
+
+/*
+ * nfs_destroy_unlinked_subrequests - destroy recently unlinked subrequests
+ *
+ * @destroy_list - request list (using wb_this_page) terminated by @old_head
+ * @old_head - the old head of the list
+ *
+ * All subrequests must be locked and removed from all lists, so at this point
+ * they are only "active" in this function, and possibly in nfs_wait_on_request
+ * with a reference held by some other context.
+ */
+static void
+nfs_destroy_unlinked_subrequests(struct nfs_page *destroy_list,
+ struct nfs_page *old_head,
+ struct inode *inode)
+{
+ while (destroy_list) {
+ struct nfs_page *subreq = destroy_list;
+
+ destroy_list = (subreq->wb_this_page == old_head) ?
+ NULL : subreq->wb_this_page;
+
+ /* Note: lock subreq in order to change subreq->wb_head */
+ nfs_page_set_headlock(subreq);
+ WARN_ON_ONCE(old_head != subreq->wb_head);
+
+ /* make sure old group is not used */
+ subreq->wb_this_page = subreq;
+ subreq->wb_head = subreq;
+
+ clear_bit(PG_REMOVE, &subreq->wb_flags);
+
+ /* Note: races with nfs_page_group_destroy() */
+ if (!kref_read(&subreq->wb_kref)) {
+ /* Check if we raced with nfs_page_group_destroy() */
+ if (test_and_clear_bit(PG_TEARDOWN, &subreq->wb_flags)) {
+ nfs_page_clear_headlock(subreq);
+ nfs_free_request(subreq);
+ } else
+ nfs_page_clear_headlock(subreq);
+ continue;
+ }
+ nfs_page_clear_headlock(subreq);
+
+ nfs_release_request(old_head);
+
+ if (test_and_clear_bit(PG_INODE_REF, &subreq->wb_flags)) {
+ nfs_release_request(subreq);
+ atomic_long_dec(&NFS_I(inode)->nrequests);
+ }
+
+ /* subreq is now totally disconnected from page group or any
+ * write / commit lists. last chance to wake any waiters */
+ nfs_unlock_and_release_request(subreq);
+ }
+}
+
+/*
+ * nfs_join_page_group - destroy subrequests of the head req
+ * @head: the page used to lookup the "page group" of nfs_page structures
+ * @inode: Inode to which the request belongs.
+ *
+ * This function joins all sub requests to the head request by first
+ * locking all requests in the group, cancelling any pending operations
+ * and finally updating the head request to cover the whole range covered by
+ * the (former) group. All subrequests are removed from any write or commit
+ * lists, unlinked from the group and destroyed.
+ */
+void nfs_join_page_group(struct nfs_page *head, struct nfs_commit_info *cinfo,
+ struct inode *inode)
+{
+ struct nfs_page *subreq;
+ struct nfs_page *destroy_list = NULL;
+ unsigned int pgbase, off, bytes;
+
+ pgbase = head->wb_pgbase;
+ bytes = head->wb_bytes;
+ off = head->wb_offset;
+ for (subreq = head->wb_this_page; subreq != head;
+ subreq = subreq->wb_this_page) {
+ /* Subrequests should always form a contiguous range */
+ if (pgbase > subreq->wb_pgbase) {
+ off -= pgbase - subreq->wb_pgbase;
+ bytes += pgbase - subreq->wb_pgbase;
+ pgbase = subreq->wb_pgbase;
+ }
+ bytes = max(subreq->wb_pgbase + subreq->wb_bytes
+ - pgbase, bytes);
+ }
- end_page_writeback(page);
- if (atomic_long_dec_return(&nfss->writeback) < NFS_CONGESTION_OFF_THRESH)
- clear_bdi_congested(&nfss->backing_dev_info, BLK_RW_ASYNC);
+ /* Set the head request's range to cover the former page group */
+ head->wb_pgbase = pgbase;
+ head->wb_bytes = bytes;
+ head->wb_offset = off;
+
+ /* Now that all requests are locked, make sure they aren't on any list.
+ * Commit list removal accounting is done after locks are dropped */
+ subreq = head;
+ do {
+ nfs_clear_request_commit(cinfo, subreq);
+ subreq = subreq->wb_this_page;
+ } while (subreq != head);
+
+ /* unlink subrequests from head, destroy them later */
+ if (head->wb_this_page != head) {
+ /* destroy list will be terminated by head */
+ destroy_list = head->wb_this_page;
+ head->wb_this_page = head;
+ }
+
+ nfs_destroy_unlinked_subrequests(destroy_list, head, inode);
}
-static struct nfs_page *nfs_find_and_lock_request(struct page *page, bool nonblock)
+/**
+ * nfs_wait_on_request - Wait for a request to complete.
+ * @req: request to wait upon.
+ *
+ * Interruptible by fatal signals only.
+ * The user is responsible for holding a count on the request.
+ */
+static int nfs_wait_on_request(struct nfs_page *req)
+{
+ if (!test_bit(PG_BUSY, &req->wb_flags))
+ return 0;
+ set_bit(PG_CONTENDED2, &req->wb_flags);
+ smp_mb__after_atomic();
+ return wait_on_bit_io(&req->wb_flags, PG_BUSY,
+ TASK_UNINTERRUPTIBLE);
+}
+
+/*
+ * nfs_unroll_locks - unlock all newly locked reqs and wait on @req
+ * @head: head request of page group, must be holding head lock
+ * @req: request that couldn't lock and needs to wait on the req bit lock
+ *
+ * This is a helper function for nfs_lock_and_join_requests
+ * returns 0 on success, < 0 on error.
+ */
+static void
+nfs_unroll_locks(struct nfs_page *head, struct nfs_page *req)
+{
+ struct nfs_page *tmp;
+
+ /* relinquish all the locks successfully grabbed this run */
+ for (tmp = head->wb_this_page ; tmp != req; tmp = tmp->wb_this_page) {
+ if (!kref_read(&tmp->wb_kref))
+ continue;
+ nfs_unlock_and_release_request(tmp);
+ }
+}
+
+/*
+ * nfs_page_group_lock_subreq - try to lock a subrequest
+ * @head: head request of page group
+ * @subreq: request to lock
+ *
+ * This is a helper function for nfs_lock_and_join_requests which
+ * must be called with the head request and page group both locked.
+ * On error, it returns with the page group unlocked.
+ */
+static int
+nfs_page_group_lock_subreq(struct nfs_page *head, struct nfs_page *subreq)
{
- struct inode *inode = page_file_mapping(page)->host;
- struct nfs_page *req;
int ret;
- spin_lock(&inode->i_lock);
- for (;;) {
- req = nfs_page_find_request_locked(NFS_I(inode), page);
- if (req == NULL)
- break;
- if (nfs_lock_request(req))
- break;
- /* Note: If we hold the page lock, as is the case in nfs_writepage,
- * then the call to nfs_lock_request() will always
- * succeed provided that someone hasn't already marked the
- * request as dirty (in which case we don't care).
- */
- spin_unlock(&inode->i_lock);
- if (!nonblock)
- ret = nfs_wait_on_request(req);
- else
- ret = -EAGAIN;
- nfs_release_request(req);
- if (ret != 0)
+ if (!kref_get_unless_zero(&subreq->wb_kref))
+ return 0;
+ while (!nfs_lock_request(subreq)) {
+ nfs_page_group_unlock(head);
+ ret = nfs_wait_on_request(subreq);
+ if (!ret)
+ ret = nfs_page_group_lock(head);
+ if (ret < 0) {
+ nfs_unroll_locks(head, subreq);
+ nfs_release_request(subreq);
+ return ret;
+ }
+ }
+ return 0;
+}
+
+/*
+ * nfs_lock_and_join_requests - join all subreqs to the head req
+ * @folio: the folio used to lookup the "page group" of nfs_page structures
+ *
+ * This function joins all sub requests to the head request by first
+ * locking all requests in the group, cancelling any pending operations
+ * and finally updating the head request to cover the whole range covered by
+ * the (former) group. All subrequests are removed from any write or commit
+ * lists, unlinked from the group and destroyed.
+ *
+ * Returns a locked, referenced pointer to the head request - which after
+ * this call is guaranteed to be the only request associated with the page.
+ * Returns NULL if no requests are found for @folio, or a ERR_PTR if an
+ * error was encountered.
+ */
+static struct nfs_page *nfs_lock_and_join_requests(struct folio *folio)
+{
+ struct inode *inode = folio->mapping->host;
+ struct nfs_page *head, *subreq;
+ struct nfs_commit_info cinfo;
+ int ret;
+
+ /*
+ * A reference is taken only on the head request which acts as a
+ * reference to the whole page group - the group will not be destroyed
+ * until the head reference is released.
+ */
+retry:
+ head = nfs_folio_find_head_request(folio);
+ if (!head)
+ return NULL;
+
+ while (!nfs_lock_request(head)) {
+ ret = nfs_wait_on_request(head);
+ if (ret < 0) {
+ nfs_release_request(head);
return ERR_PTR(ret);
- spin_lock(&inode->i_lock);
+ }
}
- spin_unlock(&inode->i_lock);
- return req;
+
+ ret = nfs_page_group_lock(head);
+ if (ret < 0)
+ goto out_unlock;
+
+ /* Ensure that nobody removed the request before we locked it */
+ if (head != folio->private) {
+ nfs_page_group_unlock(head);
+ nfs_unlock_and_release_request(head);
+ goto retry;
+ }
+
+ nfs_cancel_remove_inode(head, inode);
+
+ /* lock each request in the page group */
+ for (subreq = head->wb_this_page;
+ subreq != head;
+ subreq = subreq->wb_this_page) {
+ ret = nfs_page_group_lock_subreq(head, subreq);
+ if (ret < 0)
+ goto out_unlock;
+ }
+
+ nfs_page_group_unlock(head);
+
+ nfs_init_cinfo_from_inode(&cinfo, inode);
+ nfs_join_page_group(head, &cinfo, inode);
+ return head;
+
+out_unlock:
+ nfs_unlock_and_release_request(head);
+ return ERR_PTR(ret);
+}
+
+static void nfs_write_error(struct nfs_page *req, int error)
+{
+ trace_nfs_write_error(nfs_page_to_inode(req), req, error);
+ nfs_mapping_set_error(nfs_page_to_folio(req), error);
+ nfs_inode_remove_request(req);
+ nfs_page_end_writeback(req);
+ nfs_release_request(req);
}
/*
* Find an associated nfs write request, and prepare to flush it out
* May return an error if the user signalled nfs_wait_on_request().
*/
-static int nfs_page_async_flush(struct nfs_pageio_descriptor *pgio,
- struct page *page, bool nonblock)
+static int nfs_do_writepage(struct folio *folio, struct writeback_control *wbc,
+ struct nfs_pageio_descriptor *pgio)
{
struct nfs_page *req;
- int ret = 0;
+ int ret;
+
+ nfs_pageio_cond_complete(pgio, folio->index);
- req = nfs_find_and_lock_request(page, nonblock);
+ req = nfs_lock_and_join_requests(folio);
if (!req)
- goto out;
- ret = PTR_ERR(req);
+ return 0;
if (IS_ERR(req))
- goto out;
+ return PTR_ERR(req);
- nfs_set_page_writeback(page);
+ trace_nfs_do_writepage(req);
+ nfs_folio_set_writeback(folio);
WARN_ON_ONCE(test_bit(PG_CLEAN, &req->wb_flags));
- ret = 0;
+ /* If there is a fatal error that covers this write, just exit */
+ ret = pgio->pg_error;
+ if (nfs_error_is_fatal_on_server(ret))
+ goto out_launder;
+
if (!nfs_pageio_add_request(pgio, req)) {
- nfs_redirty_request(req);
ret = pgio->pg_error;
+ /*
+ * Remove the problematic req upon fatal errors on the server
+ */
+ if (nfs_error_is_fatal_on_server(ret))
+ goto out_launder;
+ folio_redirty_for_writepage(wbc, folio);
+ nfs_redirty_request(req);
+ pgio->pg_error = 0;
+ return ret;
}
-out:
- return ret;
-}
-static int nfs_do_writepage(struct page *page, struct writeback_control *wbc, struct nfs_pageio_descriptor *pgio)
-{
- struct inode *inode = page_file_mapping(page)->host;
- int ret;
-
- nfs_inc_stats(inode, NFSIOS_VFSWRITEPAGE);
- nfs_add_stats(inode, NFSIOS_WRITEPAGES, 1);
+ nfs_add_stats(folio->mapping->host, NFSIOS_WRITEPAGES, 1);
+ return 0;
- nfs_pageio_cond_complete(pgio, page_file_index(page));
- ret = nfs_page_async_flush(pgio, page, wbc->sync_mode == WB_SYNC_NONE);
- if (ret == -EAGAIN) {
- redirty_page_for_writepage(wbc, page);
- ret = 0;
- }
- return ret;
+out_launder:
+ nfs_write_error(req, ret);
+ return 0;
}
/*
* Write an mmapped page to the server.
*/
-static int nfs_writepage_locked(struct page *page, struct writeback_control *wbc)
+static int nfs_writepage_locked(struct folio *folio,
+ struct writeback_control *wbc)
{
struct nfs_pageio_descriptor pgio;
+ struct inode *inode = folio->mapping->host;
int err;
- NFS_PROTO(page_file_mapping(page)->host)->write_pageio_init(&pgio,
- page->mapping->host,
- wb_priority(wbc),
- &nfs_async_write_completion_ops);
- err = nfs_do_writepage(page, wbc, &pgio);
+ nfs_inc_stats(inode, NFSIOS_VFSWRITEPAGE);
+ nfs_pageio_init_write(&pgio, inode, 0, false,
+ &nfs_async_write_completion_ops);
+ err = nfs_do_writepage(folio, wbc, &pgio);
+ pgio.pg_error = 0;
nfs_pageio_complete(&pgio);
- if (err < 0)
- return err;
- if (pgio.pg_error < 0)
- return pgio.pg_error;
- return 0;
-}
-
-int nfs_writepage(struct page *page, struct writeback_control *wbc)
-{
- int ret;
-
- ret = nfs_writepage_locked(page, wbc);
- unlock_page(page);
- return ret;
+ return err;
}
-static int nfs_writepages_callback(struct page *page, struct writeback_control *wbc, void *data)
+static void nfs_io_completion_commit(void *inode)
{
- int ret;
-
- ret = nfs_do_writepage(page, wbc, data);
- unlock_page(page);
- return ret;
+ nfs_commit_inode(inode, 0);
}
int nfs_writepages(struct address_space *mapping, struct writeback_control *wbc)
{
struct inode *inode = mapping->host;
- unsigned long *bitlock = &NFS_I(inode)->flags;
struct nfs_pageio_descriptor pgio;
+ struct nfs_io_completion *ioc = NULL;
+ unsigned int mntflags = NFS_SERVER(inode)->flags;
+ struct nfs_server *nfss = NFS_SERVER(inode);
+ int priority = 0;
int err;
- /* Stop dirtying of new pages while we sync */
- err = wait_on_bit_lock(bitlock, NFS_INO_FLUSHING,
- nfs_wait_bit_killable, TASK_KILLABLE);
- if (err)
- goto out_err;
+ trace_nfs_writepages(inode, wbc->range_start, wbc->range_end - wbc->range_start);
+
+ /* Wait with writeback until write congestion eases */
+ if (wbc->sync_mode == WB_SYNC_NONE && nfss->write_congested) {
+ err = wait_event_killable(nfss->write_congestion_wait,
+ nfss->write_congested == 0);
+ if (err)
+ goto out_err;
+ }
nfs_inc_stats(inode, NFSIOS_VFSWRITEPAGES);
- NFS_PROTO(inode)->write_pageio_init(&pgio, inode, wb_priority(wbc), &nfs_async_write_completion_ops);
- err = write_cache_pages(mapping, wbc, nfs_writepages_callback, &pgio);
- nfs_pageio_complete(&pgio);
+ if (!(mntflags & NFS_MOUNT_WRITE_EAGER) || wbc->for_kupdate ||
+ wbc->for_background || wbc->for_sync) {
+ ioc = nfs_io_completion_alloc(GFP_KERNEL);
+ if (ioc)
+ nfs_io_completion_init(ioc, nfs_io_completion_commit,
+ inode);
+ priority = wb_priority(wbc);
+ }
- clear_bit_unlock(NFS_INO_FLUSHING, bitlock);
- smp_mb__after_clear_bit();
- wake_up_bit(bitlock, NFS_INO_FLUSHING);
+ do {
+ struct folio *folio = NULL;
+
+ nfs_pageio_init_write(&pgio, inode, priority, false,
+ &nfs_async_write_completion_ops);
+ pgio.pg_io_completion = ioc;
+ while ((folio = writeback_iter(mapping, wbc, folio, &err))) {
+ err = nfs_do_writepage(folio, wbc, &pgio);
+ folio_unlock(folio);
+ }
+ pgio.pg_error = 0;
+ nfs_pageio_complete(&pgio);
+ if (err == -EAGAIN && mntflags & NFS_MOUNT_SOFTERR)
+ break;
+ } while (err < 0 && !nfs_error_is_fatal(err));
+ nfs_io_completion_put(ioc);
- if (err < 0)
- goto out_err;
- err = pgio.pg_error;
- if (err < 0)
- goto out_err;
- return 0;
+ if (err > 0)
+ err = 0;
out_err:
+ trace_nfs_writepages_done(inode, wbc->range_start, wbc->range_end - wbc->range_start, err);
return err;
}
/*
* Insert a write request into an inode
*/
-static void nfs_inode_add_request(struct inode *inode, struct nfs_page *req)
+static void nfs_inode_add_request(struct nfs_page *req)
{
- struct nfs_inode *nfsi = NFS_I(inode);
+ struct folio *folio = nfs_page_to_folio(req);
+ struct address_space *mapping = folio->mapping;
+ struct nfs_inode *nfsi = NFS_I(mapping->host);
+
+ WARN_ON_ONCE(req->wb_this_page != req);
/* Lock the request! */
nfs_lock_request(req);
-
- spin_lock(&inode->i_lock);
- if (!nfsi->npages && NFS_PROTO(inode)->have_delegation(inode, FMODE_WRITE))
- inode->i_version++;
- /*
- * Swap-space should not get truncated. Hence no need to plug the race
- * with invalidate/truncate.
- */
- if (likely(!PageSwapCache(req->wb_page))) {
- set_bit(PG_MAPPED, &req->wb_flags);
- SetPagePrivate(req->wb_page);
- set_page_private(req->wb_page, (unsigned long)req);
- }
- nfsi->npages++;
+ spin_lock(&mapping->i_private_lock);
+ set_bit(PG_MAPPED, &req->wb_flags);
+ folio_set_private(folio);
+ folio->private = req;
+ spin_unlock(&mapping->i_private_lock);
+ atomic_long_inc(&nfsi->nrequests);
+ /* this a head request for a page group - mark it as having an
+ * extra reference so sub groups can follow suit.
+ * This flag also informs pgio layer when to bump nrequests when
+ * adding subrequests. */
+ WARN_ON(test_and_set_bit(PG_INODE_REF, &req->wb_flags));
kref_get(&req->wb_kref);
- spin_unlock(&inode->i_lock);
}
/*
@@ -448,29 +734,40 @@ static void nfs_inode_add_request(struct inode *inode, struct nfs_page *req)
*/
static void nfs_inode_remove_request(struct nfs_page *req)
{
- struct inode *inode = req->wb_context->dentry->d_inode;
- struct nfs_inode *nfsi = NFS_I(inode);
+ struct nfs_inode *nfsi = NFS_I(nfs_page_to_inode(req));
- spin_lock(&inode->i_lock);
- if (likely(!PageSwapCache(req->wb_page))) {
- set_page_private(req->wb_page, 0);
- ClearPagePrivate(req->wb_page);
- clear_bit(PG_MAPPED, &req->wb_flags);
+ nfs_page_group_lock(req);
+ if (nfs_page_group_sync_on_bit_locked(req, PG_REMOVE)) {
+ struct folio *folio = nfs_page_to_folio(req->wb_head);
+ struct address_space *mapping = folio->mapping;
+
+ spin_lock(&mapping->i_private_lock);
+ if (likely(folio)) {
+ folio->private = NULL;
+ folio_clear_private(folio);
+ clear_bit(PG_MAPPED, &req->wb_head->wb_flags);
+ }
+ spin_unlock(&mapping->i_private_lock);
+
+ folio_end_dropbehind(folio);
+ }
+ nfs_page_group_unlock(req);
+
+ if (test_and_clear_bit(PG_INODE_REF, &req->wb_flags)) {
+ atomic_long_dec(&nfsi->nrequests);
+ nfs_release_request(req);
}
- nfsi->npages--;
- spin_unlock(&inode->i_lock);
- nfs_release_request(req);
}
-static void
-nfs_mark_request_dirty(struct nfs_page *req)
+static void nfs_mark_request_dirty(struct nfs_page *req)
{
- __set_page_dirty_nobuffers(req->wb_page);
+ struct folio *folio = nfs_page_to_folio(req);
+ if (folio)
+ filemap_dirty_folio(folio_mapping(folio), folio);
}
-#if IS_ENABLED(CONFIG_NFS_V3) || IS_ENABLED(CONFIG_NFS_V4)
/**
- * nfs_request_add_commit_list - add request to a commit list
+ * nfs_request_add_commit_list_locked - add request to a commit list
* @req: pointer to a struct nfs_page
* @dst: commit list head
* @cinfo: holds list lock and accounting info
@@ -479,25 +776,38 @@ nfs_mark_request_dirty(struct nfs_page *req)
* number of outstanding requests requiring a commit as well as
* the MM page stats.
*
- * The caller must _not_ hold the cinfo->lock, but must be
- * holding the nfs_page lock.
+ * The caller must hold NFS_I(cinfo->inode)->commit_mutex, and the
+ * nfs_page lock.
*/
void
-nfs_request_add_commit_list(struct nfs_page *req, struct list_head *dst,
+nfs_request_add_commit_list_locked(struct nfs_page *req, struct list_head *dst,
struct nfs_commit_info *cinfo)
{
- set_bit(PG_CLEAN, &(req)->wb_flags);
- spin_lock(cinfo->lock);
+ set_bit(PG_CLEAN, &req->wb_flags);
nfs_list_add_request(req, dst);
- cinfo->mds->ncommit++;
- spin_unlock(cinfo->lock);
- if (!cinfo->dreq) {
- inc_zone_page_state(req->wb_page, NR_UNSTABLE_NFS);
- inc_bdi_stat(page_file_mapping(req->wb_page)->backing_dev_info,
- BDI_RECLAIMABLE);
- __mark_inode_dirty(req->wb_context->dentry->d_inode,
- I_DIRTY_DATASYNC);
- }
+ atomic_long_inc(&cinfo->mds->ncommit);
+}
+EXPORT_SYMBOL_GPL(nfs_request_add_commit_list_locked);
+
+/**
+ * nfs_request_add_commit_list - add request to a commit list
+ * @req: pointer to a struct nfs_page
+ * @cinfo: holds list lock and accounting info
+ *
+ * This sets the PG_CLEAN bit, updates the cinfo count of
+ * number of outstanding requests requiring a commit as well as
+ * the MM page stats.
+ *
+ * The caller must _not_ hold the cinfo->lock, but must be
+ * holding the nfs_page lock.
+ */
+void
+nfs_request_add_commit_list(struct nfs_page *req, struct nfs_commit_info *cinfo)
+{
+ mutex_lock(&NFS_I(cinfo->inode)->commit_mutex);
+ nfs_request_add_commit_list_locked(req, &cinfo->mds->list, cinfo);
+ mutex_unlock(&NFS_I(cinfo->inode)->commit_mutex);
+ nfs_folio_mark_unstable(nfs_page_to_folio(req), cinfo);
}
EXPORT_SYMBOL_GPL(nfs_request_add_commit_list);
@@ -519,14 +829,14 @@ nfs_request_remove_commit_list(struct nfs_page *req,
if (!test_and_clear_bit(PG_CLEAN, &(req)->wb_flags))
return;
nfs_list_remove_request(req);
- cinfo->mds->ncommit--;
+ atomic_long_dec(&cinfo->mds->ncommit);
}
EXPORT_SYMBOL_GPL(nfs_request_remove_commit_list);
static void nfs_init_cinfo_from_inode(struct nfs_commit_info *cinfo,
struct inode *inode)
{
- cinfo->lock = &inode->i_lock;
+ cinfo->inode = inode;
cinfo->mds = &NFS_I(inode)->commit_info;
cinfo->ds = pnfs_get_ds_info(inode);
cinfo->dreq = NULL;
@@ -549,76 +859,53 @@ EXPORT_SYMBOL_GPL(nfs_init_cinfo);
*/
void
nfs_mark_request_commit(struct nfs_page *req, struct pnfs_layout_segment *lseg,
- struct nfs_commit_info *cinfo)
+ struct nfs_commit_info *cinfo, u32 ds_commit_idx)
{
- if (pnfs_mark_request_commit(req, lseg, cinfo))
+ if (pnfs_mark_request_commit(req, lseg, cinfo, ds_commit_idx))
return;
- nfs_request_add_commit_list(req, &cinfo->mds->list, cinfo);
+ nfs_request_add_commit_list(req, cinfo);
}
-static void
-nfs_clear_page_commit(struct page *page)
+static void nfs_folio_clear_commit(struct folio *folio)
{
- dec_zone_page_state(page, NR_UNSTABLE_NFS);
- dec_bdi_stat(page_file_mapping(page)->backing_dev_info, BDI_RECLAIMABLE);
-}
+ if (folio) {
+ long nr = folio_nr_pages(folio);
-static void
-nfs_clear_request_commit(struct nfs_page *req)
-{
- if (test_bit(PG_CLEAN, &req->wb_flags)) {
- struct inode *inode = req->wb_context->dentry->d_inode;
- struct nfs_commit_info cinfo;
-
- nfs_init_cinfo_from_inode(&cinfo, inode);
- if (!pnfs_clear_request_commit(req, &cinfo)) {
- spin_lock(cinfo.lock);
- nfs_request_remove_commit_list(req, &cinfo);
- spin_unlock(cinfo.lock);
- }
- nfs_clear_page_commit(req->wb_page);
+ node_stat_mod_folio(folio, NR_WRITEBACK, -nr);
+ wb_stat_mod(&inode_to_bdi(folio->mapping->host)->wb,
+ WB_WRITEBACK, -nr);
}
}
-static inline
-int nfs_write_need_commit(struct nfs_write_data *data)
-{
- if (data->verf.committed == NFS_DATA_SYNC)
- return data->header->lseg == NULL;
- return data->verf.committed != NFS_FILE_SYNC;
-}
-
-#else
-static void nfs_init_cinfo_from_inode(struct nfs_commit_info *cinfo,
- struct inode *inode)
+/* Called holding the request lock on @req */
+static void nfs_clear_request_commit(struct nfs_commit_info *cinfo,
+ struct nfs_page *req)
{
-}
-
-void nfs_init_cinfo(struct nfs_commit_info *cinfo,
- struct inode *inode,
- struct nfs_direct_req *dreq)
-{
-}
+ if (test_bit(PG_CLEAN, &req->wb_flags)) {
+ struct nfs_open_context *ctx = nfs_req_openctx(req);
+ struct inode *inode = d_inode(ctx->dentry);
-void
-nfs_mark_request_commit(struct nfs_page *req, struct pnfs_layout_segment *lseg,
- struct nfs_commit_info *cinfo)
-{
+ mutex_lock(&NFS_I(inode)->commit_mutex);
+ if (!pnfs_clear_request_commit(req, cinfo)) {
+ nfs_request_remove_commit_list(req, cinfo);
+ }
+ mutex_unlock(&NFS_I(inode)->commit_mutex);
+ nfs_folio_clear_commit(nfs_page_to_folio(req));
+ }
}
-static void
-nfs_clear_request_commit(struct nfs_page *req)
+int nfs_write_need_commit(struct nfs_pgio_header *hdr)
{
+ if (hdr->verf.committed == NFS_DATA_SYNC)
+ return hdr->lseg == NULL;
+ return hdr->verf.committed != NFS_FILE_SYNC;
}
-static inline
-int nfs_write_need_commit(struct nfs_write_data *data)
+static void nfs_async_write_init(struct nfs_pgio_header *hdr)
{
- return 0;
+ nfs_io_completion_get(hdr->io_completion);
}
-#endif
-
static void nfs_write_completion(struct nfs_pgio_header *hdr)
{
struct nfs_commit_info cinfo;
@@ -634,38 +921,37 @@ static void nfs_write_completion(struct nfs_pgio_header *hdr)
nfs_list_remove_request(req);
if (test_bit(NFS_IOHDR_ERROR, &hdr->flags) &&
(hdr->good_bytes < bytes)) {
- nfs_set_pageerror(req->wb_page);
- nfs_context_set_write_error(req->wb_context, hdr->error);
+ trace_nfs_comp_error(hdr->inode, req, hdr->error);
+ nfs_mapping_set_error(nfs_page_to_folio(req),
+ hdr->error);
goto remove_req;
}
- if (test_bit(NFS_IOHDR_NEED_RESCHED, &hdr->flags)) {
- nfs_mark_request_dirty(req);
- goto next;
- }
- if (test_bit(NFS_IOHDR_NEED_COMMIT, &hdr->flags)) {
- memcpy(&req->wb_verf, &hdr->verf->verifier, sizeof(req->wb_verf));
- nfs_mark_request_commit(req, hdr->lseg, &cinfo);
+ if (nfs_write_need_commit(hdr)) {
+ /* Reset wb_nio, since the write was successful. */
+ req->wb_nio = 0;
+ memcpy(&req->wb_verf, &hdr->verf.verifier, sizeof(req->wb_verf));
+ nfs_mark_request_commit(req, hdr->lseg, &cinfo,
+ hdr->ds_commit_idx);
goto next;
}
remove_req:
nfs_inode_remove_request(req);
next:
- nfs_unlock_request(req);
- nfs_end_page_writeback(req->wb_page);
+ nfs_page_end_writeback(req);
nfs_release_request(req);
}
out:
+ nfs_io_completion_put(hdr->io_completion);
hdr->release(hdr);
}
-#if IS_ENABLED(CONFIG_NFS_V3) || IS_ENABLED(CONFIG_NFS_V4)
-static unsigned long
+unsigned long
nfs_reqs_to_commit(struct nfs_commit_info *cinfo)
{
- return cinfo->mds->ncommit;
+ return atomic_long_read(&cinfo->mds->ncommit);
}
-/* cinfo->lock held by caller */
+/* NFS_I(cinfo->inode)->commit_mutex held by caller */
int
nfs_scan_commit_list(struct list_head *src, struct list_head *dst,
struct nfs_commit_info *cinfo, int max)
@@ -674,19 +960,22 @@ nfs_scan_commit_list(struct list_head *src, struct list_head *dst,
int ret = 0;
list_for_each_entry_safe(req, tmp, src, wb_list) {
- if (!nfs_lock_request(req))
- continue;
kref_get(&req->wb_kref);
- if (cond_resched_lock(cinfo->lock))
- list_safe_reset_next(req, tmp, wb_list);
+ if (!nfs_lock_request(req)) {
+ nfs_release_request(req);
+ continue;
+ }
nfs_request_remove_commit_list(req, cinfo);
+ clear_bit(PG_COMMIT_TO_DS, &req->wb_flags);
nfs_list_add_request(req, dst);
ret++;
if ((ret == max) && !cinfo->dreq)
break;
+ cond_resched();
}
return ret;
}
+EXPORT_SYMBOL_GPL(nfs_scan_commit_list);
/*
* nfs_scan_commit - Scan an inode for commit requests
@@ -703,31 +992,20 @@ nfs_scan_commit(struct inode *inode, struct list_head *dst,
{
int ret = 0;
- spin_lock(cinfo->lock);
- if (cinfo->mds->ncommit > 0) {
+ if (!atomic_long_read(&cinfo->mds->ncommit))
+ return 0;
+ mutex_lock(&NFS_I(cinfo->inode)->commit_mutex);
+ if (atomic_long_read(&cinfo->mds->ncommit) > 0) {
const int max = INT_MAX;
ret = nfs_scan_commit_list(&cinfo->mds->list, dst,
cinfo, max);
ret += pnfs_scan_commit_lists(inode, cinfo, max - ret);
}
- spin_unlock(cinfo->lock);
+ mutex_unlock(&NFS_I(cinfo->inode)->commit_mutex);
return ret;
}
-#else
-static unsigned long nfs_reqs_to_commit(struct nfs_commit_info *cinfo)
-{
- return 0;
-}
-
-int nfs_scan_commit(struct inode *inode, struct list_head *dst,
- struct nfs_commit_info *cinfo)
-{
- return 0;
-}
-#endif
-
/*
* Search for an existing write request, and attempt to update
* it to reflect a new dirty region on a given page.
@@ -735,49 +1013,31 @@ int nfs_scan_commit(struct inode *inode, struct list_head *dst,
* If the attempt fails, then the existing request is flushed out
* to disk.
*/
-static struct nfs_page *nfs_try_to_update_request(struct inode *inode,
- struct page *page,
- unsigned int offset,
- unsigned int bytes)
+static struct nfs_page *nfs_try_to_update_request(struct folio *folio,
+ unsigned int offset,
+ unsigned int bytes)
{
struct nfs_page *req;
unsigned int rqend;
unsigned int end;
int error;
- if (!PagePrivate(page))
- return NULL;
-
+ trace_nfs_try_to_update_request(folio_inode(folio), offset, bytes);
end = offset + bytes;
- spin_lock(&inode->i_lock);
-
- for (;;) {
- req = nfs_page_find_request_locked(NFS_I(inode), page);
- if (req == NULL)
- goto out_unlock;
- rqend = req->wb_offset + req->wb_bytes;
- /*
- * Tell the caller to flush out the request if
- * the offsets are non-contiguous.
- * Note: nfs_flush_incompatible() will already
- * have flushed out requests having wrong owners.
- */
- if (offset > rqend
- || end < req->wb_offset)
- goto out_flushme;
-
- if (nfs_lock_request(req))
- break;
+ req = nfs_lock_and_join_requests(folio);
+ if (IS_ERR_OR_NULL(req))
+ goto out;
- /* The request is locked, so wait and then retry */
- spin_unlock(&inode->i_lock);
- error = nfs_wait_on_request(req);
- nfs_release_request(req);
- if (error != 0)
- goto out_err;
- spin_lock(&inode->i_lock);
- }
+ rqend = req->wb_offset + req->wb_bytes;
+ /*
+ * Tell the caller to flush out the request if
+ * the offsets are non-contiguous.
+ * Note: nfs_flush_incompatible() will already
+ * have flushed out requests having wrong owners.
+ */
+ if (offset > rqend || end < req->wb_offset)
+ goto out_flushme;
/* Okay, the request matches. Update the region */
if (offset < req->wb_offset) {
@@ -788,17 +1048,22 @@ static struct nfs_page *nfs_try_to_update_request(struct inode *inode,
req->wb_bytes = end - req->wb_offset;
else
req->wb_bytes = rqend - req->wb_offset;
-out_unlock:
- spin_unlock(&inode->i_lock);
- if (req)
- nfs_clear_request_commit(req);
+ req->wb_nio = 0;
+out:
+ trace_nfs_try_to_update_request_done(folio_inode(folio), offset, bytes,
+ PTR_ERR_OR_ZERO(req));
return req;
out_flushme:
- spin_unlock(&inode->i_lock);
- nfs_release_request(req);
- error = nfs_wb_page(inode, page);
-out_err:
- return ERR_PTR(error);
+ /*
+ * Note: we mark the request dirty here because
+ * nfs_lock_and_join_requests() cannot preserve
+ * commit flags, so we have to replay the write.
+ */
+ nfs_mark_request_dirty(req);
+ nfs_unlock_and_release_request(req);
+ error = nfs_wb_folio(folio->mapping->host, folio);
+ trace_nfs_try_to_update_request_done(folio_inode(folio), offset, bytes, error);
+ return (error < 0) ? ERR_PTR(error) : NULL;
}
/*
@@ -808,43 +1073,47 @@ out_err:
* if we have to add a new request. Also assumes that the caller has
* already called nfs_flush_incompatible() if necessary.
*/
-static struct nfs_page * nfs_setup_write_request(struct nfs_open_context* ctx,
- struct page *page, unsigned int offset, unsigned int bytes)
+static struct nfs_page *nfs_setup_write_request(struct nfs_open_context *ctx,
+ struct folio *folio,
+ unsigned int offset,
+ unsigned int bytes)
{
- struct inode *inode = page_file_mapping(page)->host;
- struct nfs_page *req;
+ struct nfs_page *req;
- req = nfs_try_to_update_request(inode, page, offset, bytes);
+ req = nfs_try_to_update_request(folio, offset, bytes);
if (req != NULL)
goto out;
- req = nfs_create_request(ctx, inode, page, offset, bytes);
+ req = nfs_page_create_from_folio(ctx, folio, offset, bytes);
if (IS_ERR(req))
goto out;
- nfs_inode_add_request(inode, req);
+ nfs_inode_add_request(req);
out:
return req;
}
-static int nfs_writepage_setup(struct nfs_open_context *ctx, struct page *page,
- unsigned int offset, unsigned int count)
+static int nfs_writepage_setup(struct nfs_open_context *ctx,
+ struct folio *folio, unsigned int offset,
+ unsigned int count)
{
- struct nfs_page *req;
+ struct nfs_page *req;
- req = nfs_setup_write_request(ctx, page, offset, count);
+ req = nfs_setup_write_request(ctx, folio, offset, count);
if (IS_ERR(req))
return PTR_ERR(req);
+ trace_nfs_writepage_setup(req);
/* Update file length */
- nfs_grow_file(page, offset, count);
- nfs_mark_uptodate(page, req->wb_pgbase, req->wb_bytes);
+ nfs_grow_file(folio, offset, count);
+ nfs_mark_uptodate(req);
nfs_mark_request_dirty(req);
nfs_unlock_and_release_request(req);
return 0;
}
-int nfs_flush_incompatible(struct file *file, struct page *page)
+int nfs_flush_incompatible(struct file *file, struct folio *folio)
{
struct nfs_open_context *ctx = nfs_file_open_context(file);
struct nfs_lock_context *l_ctx;
+ struct file_lock_context *flctx = locks_inode_context(file_inode(file));
struct nfs_page *req;
int do_flush, status;
/*
@@ -856,36 +1125,116 @@ int nfs_flush_incompatible(struct file *file, struct page *page)
* dropped page.
*/
do {
- req = nfs_page_find_request(page);
+ req = nfs_folio_find_head_request(folio);
if (req == NULL)
return 0;
l_ctx = req->wb_lock_context;
- do_flush = req->wb_page != page || req->wb_context != ctx;
- if (l_ctx) {
- do_flush |= l_ctx->lockowner.l_owner != current->files
- || l_ctx->lockowner.l_pid != current->tgid;
+ do_flush = nfs_page_to_folio(req) != folio ||
+ !nfs_match_open_context(nfs_req_openctx(req), ctx);
+ if (l_ctx && flctx &&
+ !(list_empty_careful(&flctx->flc_posix) &&
+ list_empty_careful(&flctx->flc_flock))) {
+ do_flush |= l_ctx->lockowner != current->files;
}
nfs_release_request(req);
if (!do_flush)
return 0;
- status = nfs_wb_page(page_file_mapping(page)->host, page);
+ status = nfs_wb_folio(folio->mapping->host, folio);
} while (status == 0);
return status;
}
/*
+ * Avoid buffered writes when a open context credential's key would
+ * expire soon.
+ *
+ * Returns -EACCES if the key will expire within RPC_KEY_EXPIRE_FAIL.
+ *
+ * Return 0 and set a credential flag which triggers the inode to flush
+ * and performs NFS_FILE_SYNC writes if the key will expired within
+ * RPC_KEY_EXPIRE_TIMEO.
+ */
+int
+nfs_key_timeout_notify(struct file *filp, struct inode *inode)
+{
+ struct nfs_open_context *ctx = nfs_file_open_context(filp);
+
+ if (nfs_ctx_key_to_expire(ctx, inode) &&
+ !rcu_access_pointer(ctx->ll_cred))
+ /* Already expired! */
+ return -EACCES;
+ return 0;
+}
+
+/*
+ * Test if the open context credential key is marked to expire soon.
+ */
+bool nfs_ctx_key_to_expire(struct nfs_open_context *ctx, struct inode *inode)
+{
+ struct rpc_auth *auth = NFS_SERVER(inode)->client->cl_auth;
+ struct rpc_cred *cred, *new, *old = NULL;
+ struct auth_cred acred = {
+ .cred = ctx->cred,
+ };
+ bool ret = false;
+
+ rcu_read_lock();
+ cred = rcu_dereference(ctx->ll_cred);
+ if (cred && !(cred->cr_ops->crkey_timeout &&
+ cred->cr_ops->crkey_timeout(cred)))
+ goto out;
+ rcu_read_unlock();
+
+ new = auth->au_ops->lookup_cred(auth, &acred, 0);
+ if (new == cred) {
+ put_rpccred(new);
+ return true;
+ }
+ if (IS_ERR_OR_NULL(new)) {
+ new = NULL;
+ ret = true;
+ } else if (new->cr_ops->crkey_timeout &&
+ new->cr_ops->crkey_timeout(new))
+ ret = true;
+
+ rcu_read_lock();
+ old = rcu_dereference_protected(xchg(&ctx->ll_cred,
+ RCU_INITIALIZER(new)), 1);
+out:
+ rcu_read_unlock();
+ put_rpccred(old);
+ return ret;
+}
+
+/*
* If the page cache is marked as unsafe or invalid, then we can't rely on
* the PageUptodate() flag. In this case, we will need to turn off
* write optimisations that depend on the page contents being correct.
*/
-static bool nfs_write_pageuptodate(struct page *page, struct inode *inode)
+static bool nfs_folio_write_uptodate(struct folio *folio, unsigned int pagelen)
{
+ struct inode *inode = folio->mapping->host;
+ struct nfs_inode *nfsi = NFS_I(inode);
+
if (nfs_have_delegated_attributes(inode))
goto out;
- if (NFS_I(inode)->cache_validity & (NFS_INO_INVALID_DATA|NFS_INO_REVAL_PAGECACHE))
+ if (nfsi->cache_validity &
+ (NFS_INO_INVALID_CHANGE | NFS_INO_INVALID_SIZE))
+ return false;
+ smp_rmb();
+ if (test_bit(NFS_INO_INVALIDATING, &nfsi->flags) && pagelen != 0)
return false;
out:
- return PageUptodate(page) != 0;
+ if (nfsi->cache_validity & NFS_INO_INVALID_DATA && pagelen != 0)
+ return false;
+ return folio_test_uptodate(folio) != 0;
+}
+
+static bool
+is_whole_file_wrlock(struct file_lock *fl)
+{
+ return fl->fl_start == 0 && fl->fl_end == OFFSET_MAX &&
+ lock_is_write(fl);
}
/* If we know the page is up to date, and we're not using byte range locks (or
@@ -893,21 +1242,46 @@ out:
* extend the write to cover the entire page in order to avoid fragmentation
* inefficiencies.
*
- * If the file is opened for synchronous writes or if we have a write delegation
- * from the server then we can just skip the rest of the checks.
+ * If the file is opened for synchronous writes then we can just skip the rest
+ * of the checks.
*/
-static int nfs_can_extend_write(struct file *file, struct page *page, struct inode *inode)
+static int nfs_can_extend_write(struct file *file, struct folio *folio,
+ unsigned int pagelen)
{
+ struct inode *inode = file_inode(file);
+ struct file_lock_context *flctx = locks_inode_context(inode);
+ struct file_lock *fl;
+ int ret;
+ unsigned int mntflags = NFS_SERVER(inode)->flags;
+
+ if (mntflags & NFS_MOUNT_NO_ALIGNWRITE)
+ return 0;
if (file->f_flags & O_DSYNC)
return 0;
- if (NFS_PROTO(inode)->have_delegation(inode, FMODE_WRITE))
+ if (!nfs_folio_write_uptodate(folio, pagelen))
+ return 0;
+ if (nfs_have_write_delegation(inode))
return 1;
- if (nfs_write_pageuptodate(page, inode) && (inode->i_flock == NULL ||
- (inode->i_flock->fl_start == 0 &&
- inode->i_flock->fl_end == OFFSET_MAX &&
- inode->i_flock->fl_type != F_RDLCK)))
+ if (!flctx || (list_empty_careful(&flctx->flc_flock) &&
+ list_empty_careful(&flctx->flc_posix)))
return 1;
- return 0;
+
+ /* Check to see if there are whole file write locks */
+ ret = 0;
+ spin_lock(&flctx->flc_lock);
+ if (!list_empty(&flctx->flc_posix)) {
+ fl = list_first_entry(&flctx->flc_posix, struct file_lock,
+ c.flc_list);
+ if (is_whole_file_wrlock(fl))
+ ret = 1;
+ } else if (!list_empty(&flctx->flc_flock)) {
+ fl = list_first_entry(&flctx->flc_flock, struct file_lock,
+ c.flc_list);
+ if (lock_is_write(fl))
+ ret = 1;
+ }
+ spin_unlock(&flctx->flc_lock);
+ return ret;
}
/*
@@ -916,32 +1290,40 @@ static int nfs_can_extend_write(struct file *file, struct page *page, struct ino
* XXX: Keep an eye on generic_file_read to make sure it doesn't do bad
* things with a page scheduled for an RPC call (e.g. invalidate it).
*/
-int nfs_updatepage(struct file *file, struct page *page,
- unsigned int offset, unsigned int count)
+int nfs_update_folio(struct file *file, struct folio *folio,
+ unsigned int offset, unsigned int count)
{
struct nfs_open_context *ctx = nfs_file_open_context(file);
- struct inode *inode = page_file_mapping(page)->host;
+ struct address_space *mapping = folio->mapping;
+ struct inode *inode = mapping->host;
+ unsigned int pagelen = nfs_folio_length(folio);
int status = 0;
nfs_inc_stats(inode, NFSIOS_VFSUPDATEPAGE);
- dprintk("NFS: nfs_updatepage(%s/%s %d@%lld)\n",
- file->f_path.dentry->d_parent->d_name.name,
- file->f_path.dentry->d_name.name, count,
- (long long)(page_file_offset(page) + offset));
+ trace_nfs_update_folio(inode, offset, count);
+
+ dprintk("NFS: nfs_update_folio(%pD2 %d@%lld)\n", file, count,
+ (long long)(folio_pos(folio) + offset));
- if (nfs_can_extend_write(file, page, inode)) {
- count = max(count + offset, nfs_page_length(page));
- offset = 0;
+ if (!count)
+ goto out;
+
+ if (nfs_can_extend_write(file, folio, pagelen)) {
+ unsigned int end = count + offset;
+
+ offset = round_down(offset, PAGE_SIZE);
+ if (end < pagelen)
+ end = min(round_up(end, PAGE_SIZE), pagelen);
+ count = end - offset;
}
- status = nfs_writepage_setup(ctx, page, offset, count);
+ status = nfs_writepage_setup(ctx, folio, offset, count);
if (status < 0)
- nfs_set_pageerror(page);
- else
- __set_page_dirty_nobuffers(page);
-
- dprintk("NFS: nfs_updatepage returns %d (isize %lld)\n",
+ nfs_set_pageerror(mapping);
+out:
+ trace_nfs_update_folio_done(inode, offset, count, status);
+ dprintk("NFS: nfs_update_folio returns %d (isize %lld)\n",
status, (long long)i_size_read(inode));
return status;
}
@@ -957,123 +1339,18 @@ static int flush_task_priority(int how)
return RPC_PRIORITY_NORMAL;
}
-int nfs_initiate_write(struct rpc_clnt *clnt,
- struct nfs_write_data *data,
- const struct rpc_call_ops *call_ops,
- int how, int flags)
+static void nfs_initiate_write(struct nfs_pgio_header *hdr,
+ struct rpc_message *msg,
+ const struct nfs_rpc_ops *rpc_ops,
+ struct rpc_task_setup *task_setup_data, int how)
{
- struct inode *inode = data->header->inode;
int priority = flush_task_priority(how);
- struct rpc_task *task;
- struct rpc_message msg = {
- .rpc_argp = &data->args,
- .rpc_resp = &data->res,
- .rpc_cred = data->header->cred,
- };
- struct rpc_task_setup task_setup_data = {
- .rpc_client = clnt,
- .task = &data->task,
- .rpc_message = &msg,
- .callback_ops = call_ops,
- .callback_data = data,
- .workqueue = nfsiod_workqueue,
- .flags = RPC_TASK_ASYNC | flags,
- .priority = priority,
- };
- int ret = 0;
-
- /* Set up the initial task struct. */
- NFS_PROTO(inode)->write_setup(data, &msg);
- dprintk("NFS: %5u initiated write call "
- "(req %s/%lld, %u bytes @ offset %llu)\n",
- data->task.tk_pid,
- inode->i_sb->s_id,
- (long long)NFS_FILEID(inode),
- data->args.count,
- (unsigned long long)data->args.offset);
-
- task = rpc_run_task(&task_setup_data);
- if (IS_ERR(task)) {
- ret = PTR_ERR(task);
- goto out;
- }
- if (how & FLUSH_SYNC) {
- ret = rpc_wait_for_completion_task(task);
- if (ret == 0)
- ret = task->tk_status;
- }
- rpc_put_task(task);
-out:
- return ret;
-}
-EXPORT_SYMBOL_GPL(nfs_initiate_write);
-
-/*
- * Set up the argument/result storage required for the RPC call.
- */
-static void nfs_write_rpcsetup(struct nfs_write_data *data,
- unsigned int count, unsigned int offset,
- int how, struct nfs_commit_info *cinfo)
-{
- struct nfs_page *req = data->header->req;
-
- /* Set up the RPC argument and reply structs
- * NB: take care not to mess about with data->commit et al. */
-
- data->args.fh = NFS_FH(data->header->inode);
- data->args.offset = req_offset(req) + offset;
- /* pnfs_set_layoutcommit needs this */
- data->mds_offset = data->args.offset;
- data->args.pgbase = req->wb_pgbase + offset;
- data->args.pages = data->pages.pagevec;
- data->args.count = count;
- data->args.context = get_nfs_open_context(req->wb_context);
- data->args.lock_context = req->wb_lock_context;
- data->args.stable = NFS_UNSTABLE;
- switch (how & (FLUSH_STABLE | FLUSH_COND_STABLE)) {
- case 0:
- break;
- case FLUSH_COND_STABLE:
- if (nfs_reqs_to_commit(cinfo))
- break;
- default:
- data->args.stable = NFS_FILE_SYNC;
- }
-
- data->res.fattr = &data->fattr;
- data->res.count = count;
- data->res.verf = &data->verf;
- nfs_fattr_init(&data->fattr);
-}
-
-static int nfs_do_write(struct nfs_write_data *data,
- const struct rpc_call_ops *call_ops,
- int how)
-{
- struct inode *inode = data->header->inode;
-
- return nfs_initiate_write(NFS_CLIENT(inode), data, call_ops, how, 0);
-}
-
-static int nfs_do_multiple_writes(struct list_head *head,
- const struct rpc_call_ops *call_ops,
- int how)
-{
- struct nfs_write_data *data;
- int ret = 0;
-
- while (!list_empty(head)) {
- int ret2;
-
- data = list_first_entry(head, struct nfs_write_data, list);
- list_del_init(&data->list);
-
- ret2 = nfs_do_write(data, call_ops, how);
- if (ret == 0)
- ret = ret2;
- }
- return ret;
+ if (IS_SWAPFILE(hdr->inode))
+ task_setup_data->flags |= RPC_TASK_SWAPPER;
+ task_setup_data->priority = priority;
+ rpc_ops->write_setup(hdr, msg, &task_setup_data->rpc_client);
+ trace_nfs_initiate_write(hdr);
}
/* If a nfs_flush_* function fails, it should remove reqs from @head and
@@ -1082,194 +1359,75 @@ static int nfs_do_multiple_writes(struct list_head *head,
*/
static void nfs_redirty_request(struct nfs_page *req)
{
+ struct nfs_inode *nfsi = NFS_I(nfs_page_to_inode(req));
+
+ /* Bump the transmission count */
+ req->wb_nio++;
nfs_mark_request_dirty(req);
- nfs_unlock_request(req);
- nfs_end_page_writeback(req->wb_page);
+ atomic_long_inc(&nfsi->redirtied_pages);
+ nfs_page_end_writeback(req);
nfs_release_request(req);
}
-static void nfs_async_write_error(struct list_head *head)
+static void nfs_async_write_error(struct list_head *head, int error)
{
struct nfs_page *req;
while (!list_empty(head)) {
req = nfs_list_entry(head->next);
nfs_list_remove_request(req);
- nfs_redirty_request(req);
+ if (nfs_error_is_fatal_on_server(error))
+ nfs_write_error(req, error);
+ else
+ nfs_redirty_request(req);
}
}
+static void nfs_async_write_reschedule_io(struct nfs_pgio_header *hdr)
+{
+ nfs_async_write_error(&hdr->pages, 0);
+}
+
static const struct nfs_pgio_completion_ops nfs_async_write_completion_ops = {
+ .init_hdr = nfs_async_write_init,
.error_cleanup = nfs_async_write_error,
.completion = nfs_write_completion,
+ .reschedule_io = nfs_async_write_reschedule_io,
};
-static void nfs_flush_error(struct nfs_pageio_descriptor *desc,
- struct nfs_pgio_header *hdr)
-{
- set_bit(NFS_IOHDR_REDO, &hdr->flags);
- while (!list_empty(&hdr->rpc_list)) {
- struct nfs_write_data *data = list_first_entry(&hdr->rpc_list,
- struct nfs_write_data, list);
- list_del(&data->list);
- nfs_writedata_release(data);
- }
- desc->pg_completion_ops->error_cleanup(&desc->pg_list);
-}
-
-/*
- * Generate multiple small requests to write out a single
- * contiguous dirty area on one page.
- */
-static int nfs_flush_multi(struct nfs_pageio_descriptor *desc,
- struct nfs_pgio_header *hdr)
-{
- struct nfs_page *req = hdr->req;
- struct page *page = req->wb_page;
- struct nfs_write_data *data;
- size_t wsize = desc->pg_bsize, nbytes;
- unsigned int offset;
- int requests = 0;
- struct nfs_commit_info cinfo;
-
- nfs_init_cinfo(&cinfo, desc->pg_inode, desc->pg_dreq);
-
- if ((desc->pg_ioflags & FLUSH_COND_STABLE) &&
- (desc->pg_moreio || nfs_reqs_to_commit(&cinfo) ||
- desc->pg_count > wsize))
- desc->pg_ioflags &= ~FLUSH_COND_STABLE;
-
-
- offset = 0;
- nbytes = desc->pg_count;
- do {
- size_t len = min(nbytes, wsize);
-
- data = nfs_writedata_alloc(hdr, 1);
- if (!data) {
- nfs_flush_error(desc, hdr);
- return -ENOMEM;
- }
- data->pages.pagevec[0] = page;
- nfs_write_rpcsetup(data, len, offset, desc->pg_ioflags, &cinfo);
- list_add(&data->list, &hdr->rpc_list);
- requests++;
- nbytes -= len;
- offset += len;
- } while (nbytes != 0);
- nfs_list_remove_request(req);
- nfs_list_add_request(req, &hdr->pages);
- desc->pg_rpc_callops = &nfs_write_common_ops;
- return 0;
-}
-
-/*
- * 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 int nfs_flush_one(struct nfs_pageio_descriptor *desc,
- struct nfs_pgio_header *hdr)
+void nfs_pageio_init_write(struct nfs_pageio_descriptor *pgio,
+ struct inode *inode, int ioflags, bool force_mds,
+ const struct nfs_pgio_completion_ops *compl_ops)
{
- struct nfs_page *req;
- struct page **pages;
- struct nfs_write_data *data;
- struct list_head *head = &desc->pg_list;
- struct nfs_commit_info cinfo;
-
- data = nfs_writedata_alloc(hdr, nfs_page_array_len(desc->pg_base,
- desc->pg_count));
- if (!data) {
- nfs_flush_error(desc, hdr);
- return -ENOMEM;
- }
-
- nfs_init_cinfo(&cinfo, desc->pg_inode, desc->pg_dreq);
- pages = data->pages.pagevec;
- while (!list_empty(head)) {
- req = nfs_list_entry(head->next);
- nfs_list_remove_request(req);
- nfs_list_add_request(req, &hdr->pages);
- *pages++ = req->wb_page;
- }
-
- if ((desc->pg_ioflags & FLUSH_COND_STABLE) &&
- (desc->pg_moreio || nfs_reqs_to_commit(&cinfo)))
- desc->pg_ioflags &= ~FLUSH_COND_STABLE;
-
- /* Set up the argument struct */
- nfs_write_rpcsetup(data, desc->pg_count, 0, desc->pg_ioflags, &cinfo);
- list_add(&data->list, &hdr->rpc_list);
- desc->pg_rpc_callops = &nfs_write_common_ops;
- return 0;
-}
+ struct nfs_server *server = NFS_SERVER(inode);
+ const struct nfs_pageio_ops *pg_ops = &nfs_pgio_rw_ops;
-int nfs_generic_flush(struct nfs_pageio_descriptor *desc,
- struct nfs_pgio_header *hdr)
-{
- if (desc->pg_bsize < PAGE_CACHE_SIZE)
- return nfs_flush_multi(desc, hdr);
- return nfs_flush_one(desc, hdr);
+#ifdef CONFIG_NFS_V4_1
+ if (server->pnfs_curr_ld && !force_mds)
+ pg_ops = server->pnfs_curr_ld->pg_write_ops;
+#endif
+ nfs_pageio_init(pgio, inode, pg_ops, compl_ops, &nfs_rw_write_ops,
+ server->wsize, ioflags);
}
-EXPORT_SYMBOL_GPL(nfs_generic_flush);
+EXPORT_SYMBOL_GPL(nfs_pageio_init_write);
-static int nfs_generic_pg_writepages(struct nfs_pageio_descriptor *desc)
+void nfs_pageio_reset_write_mds(struct nfs_pageio_descriptor *pgio)
{
- struct nfs_write_header *whdr;
- struct nfs_pgio_header *hdr;
- int ret;
+ struct nfs_pgio_mirror *mirror;
- whdr = nfs_writehdr_alloc();
- if (!whdr) {
- desc->pg_completion_ops->error_cleanup(&desc->pg_list);
- return -ENOMEM;
- }
- hdr = &whdr->header;
- nfs_pgheader_init(desc, hdr, nfs_writehdr_free);
- atomic_inc(&hdr->refcnt);
- ret = nfs_generic_flush(desc, hdr);
- if (ret == 0)
- ret = nfs_do_multiple_writes(&hdr->rpc_list,
- desc->pg_rpc_callops,
- desc->pg_ioflags);
- if (atomic_dec_and_test(&hdr->refcnt))
- hdr->completion_ops->completion(hdr);
- return ret;
-}
+ if (pgio->pg_ops && pgio->pg_ops->pg_cleanup)
+ pgio->pg_ops->pg_cleanup(pgio);
-static const struct nfs_pageio_ops nfs_pageio_write_ops = {
- .pg_test = nfs_generic_pg_test,
- .pg_doio = nfs_generic_pg_writepages,
-};
+ pgio->pg_ops = &nfs_pgio_rw_ops;
-void nfs_pageio_init_write(struct nfs_pageio_descriptor *pgio,
- struct inode *inode, int ioflags,
- const struct nfs_pgio_completion_ops *compl_ops)
-{
- nfs_pageio_init(pgio, inode, &nfs_pageio_write_ops, compl_ops,
- NFS_SERVER(inode)->wsize, ioflags);
-}
-EXPORT_SYMBOL_GPL(nfs_pageio_init_write);
+ nfs_pageio_stop_mirroring(pgio);
-void nfs_pageio_reset_write_mds(struct nfs_pageio_descriptor *pgio)
-{
- pgio->pg_ops = &nfs_pageio_write_ops;
- pgio->pg_bsize = NFS_SERVER(pgio->pg_inode)->wsize;
+ mirror = &pgio->pg_mirrors[0];
+ mirror->pg_bsize = NFS_SERVER(pgio->pg_inode)->wsize;
}
EXPORT_SYMBOL_GPL(nfs_pageio_reset_write_mds);
-void nfs_write_prepare(struct rpc_task *task, void *calldata)
-{
- struct nfs_write_data *data = calldata;
- NFS_PROTO(data->header->inode)->write_rpc_prepare(task, data);
- if (unlikely(test_bit(NFS_CONTEXT_BAD, &data->args.context->flags)))
- rpc_exit(task, -EIO);
-}
-
void nfs_commit_prepare(struct rpc_task *task, void *calldata)
{
struct nfs_commit_data *data = calldata;
@@ -1277,59 +1435,55 @@ void nfs_commit_prepare(struct rpc_task *task, void *calldata)
NFS_PROTO(data->inode)->commit_rpc_prepare(task, data);
}
-/*
- * Handle a write reply that flushes a whole page.
- *
- * FIXME: There is an inherent race with invalidate_inode_pages and
- * writebacks since the page->count is kept > 1 for as long
- * as the page has a write request pending.
- */
-static void nfs_writeback_done_common(struct rpc_task *task, void *calldata)
+static void nfs_writeback_check_extend(struct nfs_pgio_header *hdr,
+ struct nfs_fattr *fattr)
{
- struct nfs_write_data *data = calldata;
+ struct nfs_pgio_args *argp = &hdr->args;
+ struct nfs_pgio_res *resp = &hdr->res;
+ u64 size = argp->offset + resp->count;
- nfs_writeback_done(task, data);
+ if (!(fattr->valid & NFS_ATTR_FATTR_SIZE))
+ fattr->size = size;
+ if (nfs_size_to_loff_t(fattr->size) < i_size_read(hdr->inode)) {
+ fattr->valid &= ~NFS_ATTR_FATTR_SIZE;
+ return;
+ }
+ if (size != fattr->size)
+ return;
+ /* Set attribute barrier */
+ nfs_fattr_set_barrier(fattr);
+ /* ...and update size */
+ fattr->valid |= NFS_ATTR_FATTR_SIZE;
}
-static void nfs_writeback_release_common(void *calldata)
+void nfs_writeback_update_inode(struct nfs_pgio_header *hdr)
{
- struct nfs_write_data *data = calldata;
- struct nfs_pgio_header *hdr = data->header;
- int status = data->task.tk_status;
+ struct nfs_fattr *fattr = &hdr->fattr;
+ struct inode *inode = hdr->inode;
- if ((status >= 0) && nfs_write_need_commit(data)) {
- spin_lock(&hdr->lock);
- if (test_bit(NFS_IOHDR_NEED_RESCHED, &hdr->flags))
- ; /* Do nothing */
- else if (!test_and_set_bit(NFS_IOHDR_NEED_COMMIT, &hdr->flags))
- memcpy(hdr->verf, &data->verf, sizeof(*hdr->verf));
- else if (memcmp(hdr->verf, &data->verf, sizeof(*hdr->verf)))
- set_bit(NFS_IOHDR_NEED_RESCHED, &hdr->flags);
- spin_unlock(&hdr->lock);
+ if (nfs_have_delegated_mtime(inode)) {
+ spin_lock(&inode->i_lock);
+ nfs_set_cache_invalid(inode, NFS_INO_INVALID_BLOCKS);
+ spin_unlock(&inode->i_lock);
+ return;
}
- nfs_writedata_release(data);
-}
-
-static const struct rpc_call_ops nfs_write_common_ops = {
- .rpc_call_prepare = nfs_write_prepare,
- .rpc_call_done = nfs_writeback_done_common,
- .rpc_release = nfs_writeback_release_common,
-};
+ spin_lock(&inode->i_lock);
+ nfs_writeback_check_extend(hdr, fattr);
+ nfs_post_op_update_inode_force_wcc_locked(inode, fattr);
+ spin_unlock(&inode->i_lock);
+}
+EXPORT_SYMBOL_GPL(nfs_writeback_update_inode);
/*
* This function is called when the WRITE call is complete.
*/
-void nfs_writeback_done(struct rpc_task *task, struct nfs_write_data *data)
+static int nfs_writeback_done(struct rpc_task *task,
+ struct nfs_pgio_header *hdr,
+ struct inode *inode)
{
- struct nfs_writeargs *argp = &data->args;
- struct nfs_writeres *resp = &data->res;
- struct inode *inode = data->header->inode;
int status;
- dprintk("NFS: %5u nfs_writeback_done (status %d)\n",
- task->tk_pid, task->tk_status);
-
/*
* ->write_done will attempt to use post-op attributes to detect
* conflicting writes by other clients. A strict interpretation
@@ -1337,40 +1491,71 @@ void nfs_writeback_done(struct rpc_task *task, struct nfs_write_data *data)
* another writer had changed the file, but some applications
* depend on tighter cache coherency when writing.
*/
- status = NFS_PROTO(inode)->write_done(task, data);
+ status = NFS_PROTO(inode)->write_done(task, hdr);
if (status != 0)
- return;
- nfs_add_stats(inode, NFSIOS_SERVERWRITTENBYTES, resp->count);
-
-#if IS_ENABLED(CONFIG_NFS_V3) || IS_ENABLED(CONFIG_NFS_V4)
- 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.
- * Note: There is a known bug in Tru64 < 5.0 in which
- * the server reports NFS_DATA_SYNC, but performs
- * NFS_FILE_SYNC. We therefore implement this checking
- * as a dprintk() in order to avoid filling syslog.
- */
- static unsigned long complain;
+ return status;
+
+ nfs_add_stats(inode, NFSIOS_SERVERWRITTENBYTES, hdr->res.count);
+ trace_nfs_writeback_done(task, hdr);
- /* Note this will print the MDS for a DS write */
- if (time_before(complain, jiffies)) {
- dprintk("NFS: faulty NFS server %s:"
- " (committed = %d) != (stable = %d)\n",
- NFS_SERVER(inode)->nfs_client->cl_hostname,
- resp->verf->committed, argp->stable);
- complain = jiffies + 300 * HZ;
+ if (task->tk_status >= 0) {
+ enum nfs3_stable_how committed = hdr->res.verf->committed;
+
+ if (committed == NFS_UNSTABLE) {
+ /*
+ * We have some uncommitted data on the server at
+ * this point, so ensure that we keep track of that
+ * fact irrespective of what later writes do.
+ */
+ set_bit(NFS_IOHDR_UNSTABLE_WRITES, &hdr->flags);
+ }
+
+ if (committed < hdr->args.stable) {
+ /* We tried a write call, but the server did not
+ * commit data to stable storage even though we
+ * requested it.
+ * Note: There is a known bug in Tru64 < 5.0 in which
+ * the server reports NFS_DATA_SYNC, but performs
+ * NFS_FILE_SYNC. We therefore implement this checking
+ * as a dprintk() in order to avoid filling syslog.
+ */
+ static unsigned long complain;
+
+ /* Note this will print the MDS for a DS write */
+ if (time_before(complain, jiffies)) {
+ dprintk("NFS: faulty NFS server %s:"
+ " (committed = %d) != (stable = %d)\n",
+ NFS_SERVER(inode)->nfs_client->cl_hostname,
+ committed, hdr->args.stable);
+ complain = jiffies + 300 * HZ;
+ }
}
}
-#endif
- if (task->tk_status < 0)
- nfs_set_pgio_error(data->header, task->tk_status, argp->offset);
- else if (resp->count < argp->count) {
+
+ /* Deal with the suid/sgid bit corner case */
+ if (nfs_should_remove_suid(inode)) {
+ spin_lock(&inode->i_lock);
+ nfs_set_cache_invalid(inode, NFS_INO_INVALID_MODE
+ | NFS_INO_REVAL_FORCED);
+ spin_unlock(&inode->i_lock);
+ }
+ return 0;
+}
+
+/*
+ * This function is called when the WRITE call is complete.
+ */
+static void nfs_writeback_result(struct rpc_task *task,
+ struct nfs_pgio_header *hdr)
+{
+ struct nfs_pgio_args *argp = &hdr->args;
+ struct nfs_pgio_res *resp = &hdr->res;
+
+ if (resp->count < argp->count) {
static unsigned long complain;
/* This a short write! */
- nfs_inc_stats(inode, NFSIOS_SHORTWRITE);
+ nfs_inc_stats(hdr->inode, NFSIOS_SHORTWRITE);
/* Has the server at least made some progress? */
if (resp->count == 0) {
@@ -1380,14 +1565,21 @@ void nfs_writeback_done(struct rpc_task *task, struct nfs_write_data *data)
argp->count);
complain = jiffies + 300 * HZ;
}
- nfs_set_pgio_error(data->header, -EIO, argp->offset);
+ nfs_set_pgio_error(hdr, -EIO, argp->offset);
task->tk_status = -EIO;
return;
}
+
+ /* For non rpc-based layout drivers, retry-through-MDS */
+ if (!task->tk_ops) {
+ hdr->pnfs_error = -EAGAIN;
+ return;
+ }
+
/* Was this an NFSv2 write or an NFSv3 stable write? */
if (resp->verf->committed != NFS_UNSTABLE) {
/* Resend from where the server left off */
- data->mds_offset += resp->count;
+ hdr->mds_offset += resp->count;
argp->offset += resp->count;
argp->pgbase += resp->count;
argp->count -= resp->count;
@@ -1397,32 +1589,30 @@ void nfs_writeback_done(struct rpc_task *task, struct nfs_write_data *data)
*/
argp->stable = NFS_FILE_SYNC;
}
+ resp->count = 0;
+ resp->verf->committed = 0;
rpc_restart_call_prepare(task);
}
}
-
-#if IS_ENABLED(CONFIG_NFS_V3) || IS_ENABLED(CONFIG_NFS_V4)
-static int nfs_commit_set_lock(struct nfs_inode *nfsi, int may_wait)
+static int wait_on_commit(struct nfs_mds_commit_info *cinfo)
{
- int ret;
+ return wait_var_event_killable(&cinfo->rpcs_out,
+ !atomic_read(&cinfo->rpcs_out));
+}
- if (!test_and_set_bit(NFS_INO_COMMIT, &nfsi->flags))
- return 1;
- if (!may_wait)
- return 0;
- ret = out_of_line_wait_on_bit_lock(&nfsi->flags,
- NFS_INO_COMMIT,
- nfs_wait_bit_killable,
- TASK_KILLABLE);
- return (ret < 0) ? ret : 1;
+void nfs_commit_begin(struct nfs_mds_commit_info *cinfo)
+{
+ atomic_inc(&cinfo->rpcs_out);
}
-static void nfs_commit_clear_lock(struct nfs_inode *nfsi)
+bool nfs_commit_end(struct nfs_mds_commit_info *cinfo)
{
- clear_bit(NFS_INO_COMMIT, &nfsi->flags);
- smp_mb__after_clear_bit();
- wake_up_bit(&nfsi->flags, NFS_INO_COMMIT);
+ if (atomic_dec_and_test(&cinfo->rpcs_out)) {
+ wake_up_var(&cinfo->rpcs_out);
+ return true;
+ }
+ return false;
}
void nfs_commitdata_release(struct nfs_commit_data *data)
@@ -1433,8 +1623,10 @@ void nfs_commitdata_release(struct nfs_commit_data *data)
EXPORT_SYMBOL_GPL(nfs_commitdata_release);
int nfs_initiate_commit(struct rpc_clnt *clnt, struct nfs_commit_data *data,
+ const struct nfs_rpc_ops *nfs_ops,
const struct rpc_call_ops *call_ops,
- int how, int flags)
+ int how, int flags,
+ struct nfsd_file *localio)
{
struct rpc_task *task;
int priority = flush_task_priority(how);
@@ -1453,10 +1645,18 @@ int nfs_initiate_commit(struct rpc_clnt *clnt, struct nfs_commit_data *data,
.flags = RPC_TASK_ASYNC | flags,
.priority = priority,
};
+
+ if (nfs_server_capable(data->inode, NFS_CAP_MOVEABLE))
+ task_setup_data.flags |= RPC_TASK_MOVEABLE;
+
/* Set up the initial task struct. */
- NFS_PROTO(data->inode)->commit_setup(data, &msg);
+ nfs_ops->commit_setup(data, &msg, &task_setup_data.rpc_client);
+ trace_nfs_initiate_commit(data);
- dprintk("NFS: %5u initiated commit call\n", data->task.tk_pid);
+ dprintk("NFS: initiated commit call\n");
+
+ if (localio)
+ return nfs_local_commit(localio, data, call_ops, how);
task = rpc_run_task(&task_setup_data);
if (IS_ERR(task))
@@ -1468,6 +1668,18 @@ int nfs_initiate_commit(struct rpc_clnt *clnt, struct nfs_commit_data *data,
}
EXPORT_SYMBOL_GPL(nfs_initiate_commit);
+static loff_t nfs_get_lwb(struct list_head *head)
+{
+ loff_t lwb = 0;
+ struct nfs_page *req;
+
+ list_for_each_entry(req, head, wb_list)
+ if (lwb < (req_offset(req) + req->wb_bytes))
+ lwb = req_offset(req) + req->wb_bytes;
+
+ return lwb;
+}
+
/*
* Set up the argument/result storage required for the RPC call.
*/
@@ -1476,17 +1688,26 @@ void nfs_init_commit(struct nfs_commit_data *data,
struct pnfs_layout_segment *lseg,
struct nfs_commit_info *cinfo)
{
- struct nfs_page *first = nfs_list_entry(head->next);
- struct inode *inode = first->wb_context->dentry->d_inode;
+ struct nfs_page *first;
+ struct nfs_open_context *ctx;
+ struct inode *inode;
/* Set up the RPC argument and reply structs
* NB: take care not to mess about with data->commit et al. */
- list_splice_init(head, &data->pages);
+ if (head)
+ list_splice_init(head, &data->pages);
+
+ first = nfs_list_entry(data->pages.next);
+ ctx = nfs_req_openctx(first);
+ inode = d_inode(ctx->dentry);
data->inode = inode;
- data->cred = first->wb_context->cred;
+ data->cred = ctx->cred;
data->lseg = lseg; /* reference transferred */
+ /* only set lwb for pnfs commit */
+ if (lseg)
+ data->lwb = nfs_get_lwb(&data->pages);
data->mds_ops = &nfs_commit_ops;
data->completion_ops = cinfo->completion_ops;
data->dreq = cinfo->dreq;
@@ -1495,33 +1716,39 @@ void nfs_init_commit(struct nfs_commit_data *data,
/* Note: we always request a commit of the entire inode */
data->args.offset = 0;
data->args.count = 0;
- data->context = get_nfs_open_context(first->wb_context);
+ data->context = get_nfs_open_context(ctx);
data->res.fattr = &data->fattr;
data->res.verf = &data->verf;
nfs_fattr_init(&data->fattr);
+ nfs_commit_begin(cinfo->mds);
}
EXPORT_SYMBOL_GPL(nfs_init_commit);
void nfs_retry_commit(struct list_head *page_list,
struct pnfs_layout_segment *lseg,
- struct nfs_commit_info *cinfo)
+ struct nfs_commit_info *cinfo,
+ u32 ds_commit_idx)
{
struct nfs_page *req;
while (!list_empty(page_list)) {
req = nfs_list_entry(page_list->next);
nfs_list_remove_request(req);
- nfs_mark_request_commit(req, lseg, cinfo);
- if (!cinfo->dreq) {
- dec_zone_page_state(req->wb_page, NR_UNSTABLE_NFS);
- dec_bdi_stat(page_file_mapping(req->wb_page)->backing_dev_info,
- BDI_RECLAIMABLE);
- }
+ nfs_mark_request_commit(req, lseg, cinfo, ds_commit_idx);
+ nfs_folio_clear_commit(nfs_page_to_folio(req));
nfs_unlock_and_release_request(req);
}
}
EXPORT_SYMBOL_GPL(nfs_retry_commit);
+static void nfs_commit_resched_write(struct nfs_commit_info *cinfo,
+ struct nfs_page *req)
+{
+ struct folio *folio = nfs_page_to_folio(req);
+
+ filemap_dirty_folio(folio_mapping(folio), folio);
+}
+
/*
* Commit dirty pages
*/
@@ -1530,21 +1757,30 @@ nfs_commit_list(struct inode *inode, struct list_head *head, int how,
struct nfs_commit_info *cinfo)
{
struct nfs_commit_data *data;
+ struct nfsd_file *localio;
+ unsigned short task_flags = 0;
- data = nfs_commitdata_alloc();
+ /* another commit raced with us */
+ if (list_empty(head))
+ return 0;
- if (!data)
- goto out_bad;
+ data = nfs_commitdata_alloc();
+ if (!data) {
+ nfs_retry_commit(head, NULL, cinfo, -1);
+ return -ENOMEM;
+ }
/* Set up the argument struct */
nfs_init_commit(data, head, NULL, cinfo);
- atomic_inc(&cinfo->mds->rpcs_out);
- return nfs_initiate_commit(NFS_CLIENT(inode), data, data->mds_ops,
- how, 0);
- out_bad:
- nfs_retry_commit(head, NULL, cinfo);
- cinfo->completion_ops->error_cleanup(NFS_I(inode));
- return -ENOMEM;
+ if (NFS_SERVER(inode)->nfs_client->cl_minorversion)
+ task_flags = RPC_TASK_MOVEABLE;
+
+ localio = nfs_local_open_fh(NFS_SERVER(inode)->nfs_client, data->cred,
+ data->args.fh, &data->context->nfl,
+ data->context->mode);
+ return nfs_initiate_commit(NFS_CLIENT(inode), data, NFS_PROTO(inode),
+ data->mds_ops, how,
+ RPC_TASK_CRED_NOREF | task_flags, localio);
}
/*
@@ -1554,54 +1790,62 @@ static void nfs_commit_done(struct rpc_task *task, void *calldata)
{
struct nfs_commit_data *data = calldata;
- dprintk("NFS: %5u nfs_commit_done (status %d)\n",
- task->tk_pid, task->tk_status);
-
/* Call the NFS version-specific code */
NFS_PROTO(data->inode)->commit_done(task, data);
+ trace_nfs_commit_done(task, data);
}
static void nfs_commit_release_pages(struct nfs_commit_data *data)
{
+ const struct nfs_writeverf *verf = data->res.verf;
struct nfs_page *req;
int status = data->task.tk_status;
struct nfs_commit_info cinfo;
+ struct folio *folio;
while (!list_empty(&data->pages)) {
req = nfs_list_entry(data->pages.next);
nfs_list_remove_request(req);
- nfs_clear_page_commit(req->wb_page);
+ folio = nfs_page_to_folio(req);
+ nfs_folio_clear_commit(folio);
- dprintk("NFS: commit (%s/%lld %d@%lld)",
- req->wb_context->dentry->d_sb->s_id,
- (long long)NFS_FILEID(req->wb_context->dentry->d_inode),
+ dprintk("NFS: commit (%s/%llu %d@%lld)",
+ nfs_req_openctx(req)->dentry->d_sb->s_id,
+ (unsigned long long)NFS_FILEID(d_inode(nfs_req_openctx(req)->dentry)),
req->wb_bytes,
(long long)req_offset(req));
if (status < 0) {
- nfs_context_set_write_error(req->wb_context, status);
- nfs_inode_remove_request(req);
+ if (folio) {
+ trace_nfs_commit_error(data->inode, req,
+ status);
+ nfs_mapping_set_error(folio, status);
+ nfs_inode_remove_request(req);
+ }
dprintk(", error = %d\n", status);
goto next;
}
/* Okay, COMMIT succeeded, apparently. Check the verifier
* returned by the server against all stored verfs. */
- if (!memcmp(&req->wb_verf, &data->verf.verifier, sizeof(req->wb_verf))) {
+ if (nfs_write_match_verf(verf, req)) {
/* We have a match */
- nfs_inode_remove_request(req);
+ if (folio)
+ 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);
- set_bit(NFS_CONTEXT_RESEND_WRITES, &req->wb_context->flags);
+ atomic_long_inc(&NFS_I(data->inode)->redirtied_pages);
next:
nfs_unlock_and_release_request(req);
+ /* Latency breaker */
+ cond_resched();
}
+
nfs_init_cinfo(&cinfo, data->inode, data->dreq);
- if (atomic_dec_and_test(&cinfo.mds->rpcs_out))
- nfs_commit_clear_lock(NFS_I(data->inode));
+ nfs_commit_end(cinfo.mds);
}
static void nfs_commit_release(void *calldata)
@@ -1620,7 +1864,7 @@ static const struct rpc_call_ops nfs_commit_ops = {
static const struct nfs_commit_completion_ops nfs_commit_completion_ops = {
.completion = nfs_commit_release_pages,
- .error_cleanup = nfs_commit_clear_lock,
+ .resched_write = nfs_commit_resched_write,
};
int nfs_generic_commit_list(struct inode *inode, struct list_head *head,
@@ -1634,198 +1878,227 @@ int nfs_generic_commit_list(struct inode *inode, struct list_head *head,
return status;
}
-int nfs_commit_inode(struct inode *inode, int how)
+static int __nfs_commit_inode(struct inode *inode, int how,
+ struct writeback_control *wbc)
{
LIST_HEAD(head);
struct nfs_commit_info cinfo;
int may_wait = how & FLUSH_SYNC;
- int res;
+ int ret, nscan;
- res = nfs_commit_set_lock(NFS_I(inode), may_wait);
- if (res <= 0)
- goto out_mark_dirty;
+ how &= ~FLUSH_SYNC;
nfs_init_cinfo_from_inode(&cinfo, inode);
- res = nfs_scan_commit(inode, &head, &cinfo);
- if (res) {
- int error;
-
- error = nfs_generic_commit_list(inode, &head, how, &cinfo);
- if (error < 0)
- return error;
- if (!may_wait)
- goto out_mark_dirty;
- error = wait_on_bit(&NFS_I(inode)->flags,
- NFS_INO_COMMIT,
- nfs_wait_bit_killable,
- TASK_KILLABLE);
- if (error < 0)
- return error;
- } else
- nfs_commit_clear_lock(NFS_I(inode));
- return res;
- /* Note: If we exit without ensuring that the commit is complete,
- * we must mark the inode as dirty. Otherwise, future calls to
- * sync_inode() with the WB_SYNC_ALL flag set will fail to ensure
- * that the data is on the disk.
- */
-out_mark_dirty:
- __mark_inode_dirty(inode, I_DIRTY_DATASYNC);
- return res;
+ nfs_commit_begin(cinfo.mds);
+ for (;;) {
+ ret = nscan = nfs_scan_commit(inode, &head, &cinfo);
+ if (ret <= 0)
+ break;
+ ret = nfs_generic_commit_list(inode, &head, how, &cinfo);
+ if (ret < 0)
+ break;
+ ret = 0;
+ if (wbc && wbc->sync_mode == WB_SYNC_NONE) {
+ if (nscan < wbc->nr_to_write)
+ wbc->nr_to_write -= nscan;
+ else
+ wbc->nr_to_write = 0;
+ }
+ if (nscan < INT_MAX)
+ break;
+ cond_resched();
+ }
+ nfs_commit_end(cinfo.mds);
+ if (ret || !may_wait)
+ return ret;
+ return wait_on_commit(cinfo.mds);
+}
+
+int nfs_commit_inode(struct inode *inode, int how)
+{
+ return __nfs_commit_inode(inode, how, NULL);
}
+EXPORT_SYMBOL_GPL(nfs_commit_inode);
-static int nfs_commit_unstable_pages(struct inode *inode, struct writeback_control *wbc)
+int nfs_write_inode(struct inode *inode, struct writeback_control *wbc)
{
struct nfs_inode *nfsi = NFS_I(inode);
int flags = FLUSH_SYNC;
int ret = 0;
- /* no commits means nothing needs to be done */
- if (!nfsi->commit_info.ncommit)
- return ret;
-
if (wbc->sync_mode == WB_SYNC_NONE) {
+ /* no commits means nothing needs to be done */
+ if (!atomic_long_read(&nfsi->commit_info.ncommit))
+ goto check_requests_outstanding;
+
/* Don't commit yet if this is a non-blocking flush and there
* are a lot of outstanding writes for this mapping.
*/
- if (nfsi->commit_info.ncommit <= (nfsi->npages >> 1))
+ if (mapping_tagged(inode->i_mapping, PAGECACHE_TAG_WRITEBACK))
goto out_mark_dirty;
/* don't wait for the COMMIT response */
flags = 0;
}
- ret = nfs_commit_inode(inode, flags);
- if (ret >= 0) {
- if (wbc->sync_mode == WB_SYNC_NONE) {
- if (ret < wbc->nr_to_write)
- wbc->nr_to_write -= ret;
- else
- wbc->nr_to_write = 0;
- }
- return 0;
- }
+ ret = __nfs_commit_inode(inode, flags, wbc);
+ if (!ret) {
+ if (flags & FLUSH_SYNC)
+ return 0;
+ } else if (atomic_long_read(&nfsi->commit_info.ncommit))
+ goto out_mark_dirty;
+
+check_requests_outstanding:
+ if (!atomic_read(&nfsi->commit_info.rpcs_out))
+ return ret;
out_mark_dirty:
__mark_inode_dirty(inode, I_DIRTY_DATASYNC);
return ret;
}
-#else
-static int nfs_commit_unstable_pages(struct inode *inode, struct writeback_control *wbc)
-{
- return 0;
-}
-#endif
+EXPORT_SYMBOL_GPL(nfs_write_inode);
-int nfs_write_inode(struct inode *inode, struct writeback_control *wbc)
+/*
+ * Wrapper for filemap_write_and_wait_range()
+ *
+ * Needed for pNFS in order to ensure data becomes visible to the
+ * client.
+ */
+int nfs_filemap_write_and_wait_range(struct address_space *mapping,
+ loff_t lstart, loff_t lend)
{
- return nfs_commit_unstable_pages(inode, wbc);
+ int ret;
+
+ ret = filemap_write_and_wait_range(mapping, lstart, lend);
+ if (ret == 0)
+ ret = pnfs_sync_inode(mapping->host, true);
+ return ret;
}
-EXPORT_SYMBOL_GPL(nfs_write_inode);
+EXPORT_SYMBOL_GPL(nfs_filemap_write_and_wait_range);
/*
* flush the inode to disk.
*/
int nfs_wb_all(struct inode *inode)
{
- struct writeback_control wbc = {
- .sync_mode = WB_SYNC_ALL,
- .nr_to_write = LONG_MAX,
- .range_start = 0,
- .range_end = LLONG_MAX,
- };
+ int ret;
+
+ trace_nfs_writeback_inode_enter(inode);
- return sync_inode(inode, &wbc);
+ ret = filemap_write_and_wait(inode->i_mapping);
+ if (ret)
+ goto out;
+ ret = nfs_commit_inode(inode, FLUSH_SYNC);
+ if (ret < 0)
+ goto out;
+ pnfs_sync_inode(inode, true);
+ ret = 0;
+
+out:
+ trace_nfs_writeback_inode_exit(inode, ret);
+ return ret;
}
EXPORT_SYMBOL_GPL(nfs_wb_all);
-int nfs_wb_page_cancel(struct inode *inode, struct page *page)
+int nfs_wb_folio_cancel(struct inode *inode, struct folio *folio)
{
struct nfs_page *req;
int ret = 0;
- for (;;) {
- wait_on_page_writeback(page);
- req = nfs_page_find_request(page);
- if (req == NULL)
- break;
- if (nfs_lock_request(req)) {
- nfs_clear_request_commit(req);
- nfs_inode_remove_request(req);
- /*
- * In case nfs_inode_remove_request has marked the
- * page as being dirty
- */
- cancel_dirty_page(page, PAGE_CACHE_SIZE);
- nfs_unlock_and_release_request(req);
- break;
- }
- ret = nfs_wait_on_request(req);
- nfs_release_request(req);
- if (ret < 0)
- break;
+ folio_wait_writeback(folio);
+
+ /* blocking call to cancel all requests and join to a single (head)
+ * request */
+ req = nfs_lock_and_join_requests(folio);
+
+ if (IS_ERR(req)) {
+ ret = PTR_ERR(req);
+ } else if (req) {
+ /* all requests from this folio have been cancelled by
+ * nfs_lock_and_join_requests, so just remove the head
+ * request from the inode / page_private pointer and
+ * release it */
+ nfs_inode_remove_request(req);
+ nfs_unlock_and_release_request(req);
+ folio_cancel_dirty(folio);
}
+
return ret;
}
-/*
- * Write back all requests on one page - we do this before reading it.
+/**
+ * nfs_wb_folio - Write back all requests on one page
+ * @inode: pointer to page
+ * @folio: pointer to folio
+ *
+ * Assumes that the folio has been locked by the caller, and will
+ * not unlock it.
*/
-int nfs_wb_page(struct inode *inode, struct page *page)
+int nfs_wb_folio(struct inode *inode, struct folio *folio)
{
- loff_t range_start = page_file_offset(page);
- loff_t range_end = range_start + (loff_t)(PAGE_CACHE_SIZE - 1);
+ loff_t range_start = folio_pos(folio);
+ size_t len = folio_size(folio);
struct writeback_control wbc = {
.sync_mode = WB_SYNC_ALL,
.nr_to_write = 0,
.range_start = range_start,
- .range_end = range_end,
+ .range_end = range_start + len - 1,
};
int ret;
+ trace_nfs_writeback_folio(inode, range_start, len);
+
for (;;) {
- wait_on_page_writeback(page);
- if (clear_page_dirty_for_io(page)) {
- ret = nfs_writepage_locked(page, &wbc);
+ folio_wait_writeback(folio);
+ if (folio_clear_dirty_for_io(folio)) {
+ ret = nfs_writepage_locked(folio, &wbc);
if (ret < 0)
goto out_error;
continue;
}
- if (!PagePrivate(page))
+ ret = 0;
+ if (!folio_test_private(folio))
break;
ret = nfs_commit_inode(inode, FLUSH_SYNC);
if (ret < 0)
goto out_error;
}
- return 0;
out_error:
+ trace_nfs_writeback_folio_done(inode, range_start, len, ret);
return ret;
}
#ifdef CONFIG_MIGRATION
-int nfs_migrate_page(struct address_space *mapping, struct page *newpage,
- struct page *page, enum migrate_mode mode)
+int nfs_migrate_folio(struct address_space *mapping, struct folio *dst,
+ struct folio *src, enum migrate_mode mode)
{
/*
- * If PagePrivate is set, then the page is currently associated with
+ * If the private flag is set, the folio is currently associated with
* an in-progress read or write request. Don't try to migrate it.
*
* FIXME: we could do this in principle, but we'll need a way to ensure
* that we can safely release the inode reference while holding
- * the page lock.
+ * the folio lock.
*/
- if (PagePrivate(page))
- return -EBUSY;
+ if (folio_test_private(src)) {
+ if (mode == MIGRATE_SYNC)
+ nfs_wb_folio(src->mapping->host, src);
+ if (folio_test_private(src))
+ return -EBUSY;
+ }
- if (!nfs_fscache_release_page(page, GFP_KERNEL))
- return -EBUSY;
+ if (folio_test_private_2(src)) { /* [DEPRECATED] */
+ if (mode == MIGRATE_ASYNC)
+ return -EBUSY;
+ folio_wait_private_2(src);
+ }
- return migrate_page(mapping, newpage, page, mode);
+ return migrate_folio(mapping, dst, src, mode);
}
#endif
int __init nfs_init_writepagecache(void)
{
nfs_wdata_cachep = kmem_cache_create("nfs_write_data",
- sizeof(struct nfs_write_header),
+ sizeof(struct nfs_pgio_header),
0, SLAB_HWCACHE_ALIGN,
NULL);
if (nfs_wdata_cachep == NULL)
@@ -1864,7 +2137,7 @@ int __init nfs_init_writepagecache(void)
* This allows larger machines to have larger/more transfers.
* Limit the default to 256M
*/
- nfs_congestion_kb = (16*int_sqrt(totalram_pages)) << (PAGE_SHIFT-10);
+ nfs_congestion_kb = (16*int_sqrt(totalram_pages())) << (PAGE_SHIFT-10);
if (nfs_congestion_kb > 256*1024)
nfs_congestion_kb = 256*1024;
@@ -1887,3 +2160,10 @@ void nfs_destroy_writepagecache(void)
kmem_cache_destroy(nfs_wdata_cachep);
}
+static const struct nfs_rw_ops nfs_rw_write_ops = {
+ .rw_alloc_header = nfs_writehdr_alloc,
+ .rw_free_header = nfs_writehdr_free,
+ .rw_done = nfs_writeback_done,
+ .rw_result = nfs_writeback_result,
+ .rw_initiate = nfs_initiate_write,
+};