summaryrefslogtreecommitdiffstats
path: root/fs/cifs/file.c
diff options
context:
space:
mode:
authorDavid Howells <dhowells@redhat.com>2022-01-24 21:13:24 +0000
committerSteve French <stfrench@microsoft.com>2023-02-20 18:36:02 -0600
commitd08089f649a0cfb2099c8551ac47eef0cc23fdf2 (patch)
tree1165d72f35311d1017bf84c417cee6c37b3c01d9 /fs/cifs/file.c
parent16541195c6d9bcad568b7c6afbf855ddc3a856aa (diff)
downloadlinux-stable-d08089f649a0cfb2099c8551ac47eef0cc23fdf2.tar.gz
linux-stable-d08089f649a0cfb2099c8551ac47eef0cc23fdf2.tar.bz2
linux-stable-d08089f649a0cfb2099c8551ac47eef0cc23fdf2.zip
cifs: Change the I/O paths to use an iterator rather than a page list
Currently, the cifs I/O paths hand lists of pages from the VM interface routines at the top all the way through the intervening layers to the socket interface at the bottom. This is a problem, however, for interfacing with netfslib which passes an iterator through to the ->issue_read() method (and will pass an iterator through to the ->issue_write() method in future). Netfslib takes over bounce buffering for direct I/O, async I/O and encrypted content, so cifs doesn't need to do that. Netfslib also converts IOVEC-type iterators into BVEC-type iterators if necessary. Further, cifs needs foliating - and folios may come in a variety of sizes, so a page list pointing to an array of heterogeneous pages may cause problems in places such as where crypto is done. Change the cifs I/O paths to hand iov_iter iterators all the way through instead. Notes: (1) Some old routines are #if'd out to be removed in a follow up patch so as to avoid confusing diff, thereby making the diff output easier to follow. I've removed functions that don't overlap with anything added. (2) struct smb_rqst loses rq_pages, rq_offset, rq_npages, rq_pagesz and rq_tailsz which describe the pages forming the buffer; instead there's an rq_iter describing the source buffer and an rq_buffer which is used to hold the buffer for encryption. (3) struct cifs_readdata and cifs_writedata are similarly modified to smb_rqst. The ->read_into_pages() and ->copy_into_pages() are then replaced with passing the iterator directly to the socket. The iterators are stored in these structs so that they are persistent and don't get deallocated when the function returns (unlike if they were stack variables). (4) Buffered writeback is overhauled, borrowing the code from the afs filesystem to gather up contiguous runs of folios. The XARRAY-type iterator is then used to refer directly to the pagecache and can be passed to the socket to transmit data directly from there. This includes: cifs_extend_writeback() cifs_write_back_from_locked_folio() cifs_writepages_region() cifs_writepages() (5) Pages are converted to folios. (6) Direct I/O uses netfs_extract_user_iter() to create a BVEC-type iterator from an IOBUF/UBUF-type source iterator. (7) smb2_get_aead_req() uses netfs_extract_iter_to_sg() to extract page fragments from the iterator into the scatterlists that the crypto layer prefers. (8) smb2_init_transform_rq() attached pages to smb_rqst::rq_buffer, an xarray, to use as a bounce buffer for encryption. An XARRAY-type iterator can then be used to pass the bounce buffer to lower layers. Signed-off-by: David Howells <dhowells@redhat.com> cc: Steve French <sfrench@samba.org> cc: Shyam Prasad N <nspmangalore@gmail.com> cc: Rohith Surabattula <rohiths.msft@gmail.com> cc: Paulo Alcantara <pc@cjr.nz> cc: Jeff Layton <jlayton@kernel.org> cc: linux-cifs@vger.kernel.org Link: https://lore.kernel.org/r/164311907995.2806745.400147335497304099.stgit@warthog.procyon.org.uk/ # rfc Link: https://lore.kernel.org/r/164928620163.457102.11602306234438271112.stgit@warthog.procyon.org.uk/ # v1 Link: https://lore.kernel.org/r/165211420279.3154751.15923591172438186144.stgit@warthog.procyon.org.uk/ # v1 Link: https://lore.kernel.org/r/165348880385.2106726.3220789453472800240.stgit@warthog.procyon.org.uk/ # v1 Link: https://lore.kernel.org/r/165364827111.3334034.934805882842932881.stgit@warthog.procyon.org.uk/ # v3 Link: https://lore.kernel.org/r/166126396180.708021.271013668175370826.stgit@warthog.procyon.org.uk/ # v1 Link: https://lore.kernel.org/r/166697259595.61150.5982032408321852414.stgit@warthog.procyon.org.uk/ # rfc Link: https://lore.kernel.org/r/166732031756.3186319.12528413619888902872.stgit@warthog.procyon.org.uk/ # rfc Signed-off-by: Steve French <stfrench@microsoft.com>
Diffstat (limited to 'fs/cifs/file.c')
-rw-r--r--fs/cifs/file.c1195
1 files changed, 748 insertions, 447 deletions
diff --git a/fs/cifs/file.c b/fs/cifs/file.c
index 09240b8b018a..599578f7e961 100644
--- a/fs/cifs/file.c
+++ b/fs/cifs/file.c
@@ -37,6 +37,32 @@
#include "cached_dir.h"
/*
+ * Remove the dirty flags from a span of pages.
+ */
+static void cifs_undirty_folios(struct inode *inode, loff_t start, unsigned int len)
+{
+ struct address_space *mapping = inode->i_mapping;
+ struct folio *folio;
+ pgoff_t end;
+
+ XA_STATE(xas, &mapping->i_pages, start / PAGE_SIZE);
+
+ rcu_read_lock();
+
+ end = (start + len - 1) / PAGE_SIZE;
+ xas_for_each_marked(&xas, folio, end, PAGECACHE_TAG_DIRTY) {
+ xas_pause(&xas);
+ rcu_read_unlock();
+ folio_lock(folio);
+ folio_clear_dirty_for_io(folio);
+ folio_unlock(folio);
+ rcu_read_lock();
+ }
+
+ rcu_read_unlock();
+}
+
+/*
* Completion of write to server.
*/
void cifs_pages_written_back(struct inode *inode, loff_t start, unsigned int len)
@@ -2391,7 +2417,6 @@ cifs_writedata_release(struct kref *refcount)
if (wdata->cfile)
cifsFileInfo_put(wdata->cfile);
- kvfree(wdata->pages);
kfree(wdata);
}
@@ -2402,51 +2427,49 @@ cifs_writedata_release(struct kref *refcount)
static void
cifs_writev_requeue(struct cifs_writedata *wdata)
{
- int i, rc = 0;
+ int rc = 0;
struct inode *inode = d_inode(wdata->cfile->dentry);
struct TCP_Server_Info *server;
- unsigned int rest_len;
+ unsigned int rest_len = wdata->bytes;
+ loff_t fpos = wdata->offset;
server = tlink_tcon(wdata->cfile->tlink)->ses->server;
- i = 0;
- rest_len = wdata->bytes;
do {
struct cifs_writedata *wdata2;
- unsigned int j, nr_pages, wsize, tailsz, cur_len;
+ unsigned int wsize, cur_len;
wsize = server->ops->wp_retry_size(inode);
if (wsize < rest_len) {
- nr_pages = wsize / PAGE_SIZE;
- if (!nr_pages) {
+ if (wsize < PAGE_SIZE) {
rc = -EOPNOTSUPP;
break;
}
- cur_len = nr_pages * PAGE_SIZE;
- tailsz = PAGE_SIZE;
+ cur_len = min(round_down(wsize, PAGE_SIZE), rest_len);
} else {
- nr_pages = DIV_ROUND_UP(rest_len, PAGE_SIZE);
cur_len = rest_len;
- tailsz = rest_len - (nr_pages - 1) * PAGE_SIZE;
}
- wdata2 = cifs_writedata_alloc(nr_pages, cifs_writev_complete);
+ wdata2 = cifs_writedata_alloc(cifs_writev_complete);
if (!wdata2) {
rc = -ENOMEM;
break;
}
- for (j = 0; j < nr_pages; j++) {
- wdata2->pages[j] = wdata->pages[i + j];
- lock_page(wdata2->pages[j]);
- clear_page_dirty_for_io(wdata2->pages[j]);
- }
-
wdata2->sync_mode = wdata->sync_mode;
- wdata2->nr_pages = nr_pages;
- wdata2->offset = page_offset(wdata2->pages[0]);
- wdata2->pagesz = PAGE_SIZE;
- wdata2->tailsz = tailsz;
- wdata2->bytes = cur_len;
+ wdata2->offset = fpos;
+ wdata2->bytes = cur_len;
+ wdata2->iter = wdata->iter;
+
+ iov_iter_advance(&wdata2->iter, fpos - wdata->offset);
+ iov_iter_truncate(&wdata2->iter, wdata2->bytes);
+
+ if (iov_iter_is_xarray(&wdata2->iter))
+ /* Check for pages having been redirtied and clean
+ * them. We can do this by walking the xarray. If
+ * it's not an xarray, then it's a DIO and we shouldn't
+ * be mucking around with the page bits.
+ */
+ cifs_undirty_folios(inode, fpos, cur_len);
rc = cifs_get_writable_file(CIFS_I(inode), FIND_WR_ANY,
&wdata2->cfile);
@@ -2461,33 +2484,22 @@ cifs_writev_requeue(struct cifs_writedata *wdata)
cifs_writedata_release);
}
- for (j = 0; j < nr_pages; j++) {
- unlock_page(wdata2->pages[j]);
- if (rc != 0 && !is_retryable_error(rc)) {
- SetPageError(wdata2->pages[j]);
- end_page_writeback(wdata2->pages[j]);
- put_page(wdata2->pages[j]);
- }
- }
-
kref_put(&wdata2->refcount, cifs_writedata_release);
if (rc) {
if (is_retryable_error(rc))
continue;
- i += nr_pages;
+ fpos += cur_len;
+ rest_len -= cur_len;
break;
}
+ fpos += cur_len;
rest_len -= cur_len;
- i += nr_pages;
- } while (i < wdata->nr_pages);
+ } while (rest_len > 0);
- /* cleanup remaining pages from the original wdata */
- for (; i < wdata->nr_pages; i++) {
- SetPageError(wdata->pages[i]);
- end_page_writeback(wdata->pages[i]);
- put_page(wdata->pages[i]);
- }
+ /* Clean up remaining pages from the original wdata */
+ if (iov_iter_is_xarray(&wdata->iter))
+ cifs_pages_write_failed(inode, fpos, rest_len);
if (rc != 0 && !is_retryable_error(rc))
mapping_set_error(inode->i_mapping, rc);
@@ -2500,7 +2512,6 @@ cifs_writev_complete(struct work_struct *work)
struct cifs_writedata *wdata = container_of(work,
struct cifs_writedata, work);
struct inode *inode = d_inode(wdata->cfile->dentry);
- int i = 0;
if (wdata->result == 0) {
spin_lock(&inode->i_lock);
@@ -2511,45 +2522,24 @@ cifs_writev_complete(struct work_struct *work)
} else if (wdata->sync_mode == WB_SYNC_ALL && wdata->result == -EAGAIN)
return cifs_writev_requeue(wdata);
- for (i = 0; i < wdata->nr_pages; i++) {
- struct page *page = wdata->pages[i];
+ if (wdata->result == -EAGAIN)
+ cifs_pages_write_redirty(inode, wdata->offset, wdata->bytes);
+ else if (wdata->result < 0)
+ cifs_pages_write_failed(inode, wdata->offset, wdata->bytes);
+ else
+ cifs_pages_written_back(inode, wdata->offset, wdata->bytes);
- if (wdata->result == -EAGAIN)
- __set_page_dirty_nobuffers(page);
- else if (wdata->result < 0)
- SetPageError(page);
- end_page_writeback(page);
- cifs_readpage_to_fscache(inode, page);
- put_page(page);
- }
if (wdata->result != -EAGAIN)
mapping_set_error(inode->i_mapping, wdata->result);
kref_put(&wdata->refcount, cifs_writedata_release);
}
-struct cifs_writedata *
-cifs_writedata_alloc(unsigned int nr_pages, work_func_t complete)
-{
- struct cifs_writedata *writedata = NULL;
- struct page **pages =
- kcalloc(nr_pages, sizeof(struct page *), GFP_NOFS);
- if (pages) {
- writedata = cifs_writedata_direct_alloc(pages, complete);
- if (!writedata)
- kvfree(pages);
- }
-
- return writedata;
-}
-
-struct cifs_writedata *
-cifs_writedata_direct_alloc(struct page **pages, work_func_t complete)
+struct cifs_writedata *cifs_writedata_alloc(work_func_t complete)
{
struct cifs_writedata *wdata;
wdata = kzalloc(sizeof(*wdata), GFP_NOFS);
if (wdata != NULL) {
- wdata->pages = pages;
kref_init(&wdata->refcount);
INIT_LIST_HEAD(&wdata->list);
init_completion(&wdata->done);
@@ -2558,7 +2548,6 @@ cifs_writedata_direct_alloc(struct page **pages, work_func_t complete)
return wdata;
}
-
static int cifs_partialpagewrite(struct page *page, unsigned from, unsigned to)
{
struct address_space *mapping = page->mapping;
@@ -2617,6 +2606,7 @@ static int cifs_partialpagewrite(struct page *page, unsigned from, unsigned to)
return rc;
}
+#if 0 // TODO: Remove for iov_iter support
static struct cifs_writedata *
wdata_alloc_and_fillpages(pgoff_t tofind, struct address_space *mapping,
pgoff_t end, pgoff_t *index,
@@ -2922,6 +2912,375 @@ retry:
set_bit(CIFS_INO_MODIFIED_ATTR, &CIFS_I(inode)->flags);
return rc;
}
+#endif
+
+/*
+ * Extend the region to be written back to include subsequent contiguously
+ * dirty pages if possible, but don't sleep while doing so.
+ */
+static void cifs_extend_writeback(struct address_space *mapping,
+ long *_count,
+ loff_t start,
+ int max_pages,
+ size_t max_len,
+ unsigned int *_len)
+{
+ struct folio_batch batch;
+ struct folio *folio;
+ unsigned int psize, nr_pages;
+ size_t len = *_len;
+ pgoff_t index = (start + len) / PAGE_SIZE;
+ bool stop = true;
+ unsigned int i;
+ XA_STATE(xas, &mapping->i_pages, index);
+
+ folio_batch_init(&batch);
+
+ do {
+ /* Firstly, we gather up a batch of contiguous dirty pages
+ * under the RCU read lock - but we can't clear the dirty flags
+ * there if any of those pages are mapped.
+ */
+ rcu_read_lock();
+
+ xas_for_each(&xas, folio, ULONG_MAX) {
+ stop = true;
+ if (xas_retry(&xas, folio))
+ continue;
+ if (xa_is_value(folio))
+ break;
+ if (folio_index(folio) != index)
+ break;
+ if (!folio_try_get_rcu(folio)) {
+ xas_reset(&xas);
+ continue;
+ }
+ nr_pages = folio_nr_pages(folio);
+ if (nr_pages > max_pages)
+ break;
+
+ /* Has the page moved or been split? */
+ if (unlikely(folio != xas_reload(&xas))) {
+ folio_put(folio);
+ break;
+ }
+
+ if (!folio_trylock(folio)) {
+ folio_put(folio);
+ break;
+ }
+ if (!folio_test_dirty(folio) || folio_test_writeback(folio)) {
+ folio_unlock(folio);
+ folio_put(folio);
+ break;
+ }
+
+ max_pages -= nr_pages;
+ psize = folio_size(folio);
+ len += psize;
+ stop = false;
+ if (max_pages <= 0 || len >= max_len || *_count <= 0)
+ stop = true;
+
+ index += nr_pages;
+ if (!folio_batch_add(&batch, folio))
+ break;
+ if (stop)
+ break;
+ }
+
+ if (!stop)
+ xas_pause(&xas);
+ rcu_read_unlock();
+
+ /* Now, if we obtained any pages, we can shift them to being
+ * writable and mark them for caching.
+ */
+ if (!folio_batch_count(&batch))
+ break;
+
+ for (i = 0; i < folio_batch_count(&batch); i++) {
+ folio = batch.folios[i];
+ /* The folio should be locked, dirty and not undergoing
+ * writeback from the loop above.
+ */
+ if (!folio_clear_dirty_for_io(folio))
+ WARN_ON(1);
+ if (folio_start_writeback(folio))
+ WARN_ON(1);
+
+ *_count -= folio_nr_pages(folio);
+ folio_unlock(folio);
+ }
+
+ folio_batch_release(&batch);
+ cond_resched();
+ } while (!stop);
+
+ *_len = len;
+}
+
+/*
+ * Write back the locked page and any subsequent non-locked dirty pages.
+ */
+static ssize_t cifs_write_back_from_locked_folio(struct address_space *mapping,
+ struct writeback_control *wbc,
+ struct folio *folio,
+ loff_t start, loff_t end)
+{
+ struct inode *inode = mapping->host;
+ struct TCP_Server_Info *server;
+ struct cifs_writedata *wdata;
+ struct cifs_sb_info *cifs_sb = CIFS_SB(inode->i_sb);
+ struct cifs_credits credits_on_stack;
+ struct cifs_credits *credits = &credits_on_stack;
+ struct cifsFileInfo *cfile = NULL;
+ unsigned int xid, wsize, len;
+ loff_t i_size = i_size_read(inode);
+ size_t max_len;
+ long count = wbc->nr_to_write;
+ int rc;
+
+ /* The folio should be locked, dirty and not undergoing writeback. */
+ if (folio_start_writeback(folio))
+ WARN_ON(1);
+
+ count -= folio_nr_pages(folio);
+ len = folio_size(folio);
+
+ xid = get_xid();
+ server = cifs_pick_channel(cifs_sb_master_tcon(cifs_sb)->ses);
+
+ rc = cifs_get_writable_file(CIFS_I(inode), FIND_WR_ANY, &cfile);
+ if (rc) {
+ cifs_dbg(VFS, "No writable handle in writepages rc=%d\n", rc);
+ goto err_xid;
+ }
+
+ rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->wsize,
+ &wsize, credits);
+ if (rc != 0)
+ goto err_close;
+
+ wdata = cifs_writedata_alloc(cifs_writev_complete);
+ if (!wdata) {
+ rc = -ENOMEM;
+ goto err_uncredit;
+ }
+
+ wdata->sync_mode = wbc->sync_mode;
+ wdata->offset = folio_pos(folio);
+ wdata->pid = cfile->pid;
+ wdata->credits = credits_on_stack;
+ wdata->cfile = cfile;
+ wdata->server = server;
+ cfile = NULL;
+
+ /* Find all consecutive lockable dirty pages, stopping when we find a
+ * page that is not immediately lockable, is not dirty or is missing,
+ * or we reach the end of the range.
+ */
+ if (start < i_size) {
+ /* Trim the write to the EOF; the extra data is ignored. Also
+ * put an upper limit on the size of a single storedata op.
+ */
+ max_len = wsize;
+ max_len = min_t(unsigned long long, max_len, end - start + 1);
+ max_len = min_t(unsigned long long, max_len, i_size - start);
+
+ if (len < max_len) {
+ int max_pages = INT_MAX;
+
+#ifdef CONFIG_CIFS_SMB_DIRECT
+ if (server->smbd_conn)
+ max_pages = server->smbd_conn->max_frmr_depth;
+#endif
+ max_pages -= folio_nr_pages(folio);
+
+ if (max_pages > 0)
+ cifs_extend_writeback(mapping, &count, start,
+ max_pages, max_len, &len);
+ }
+ len = min_t(loff_t, len, max_len);
+ }
+
+ wdata->bytes = len;
+
+ /* We now have a contiguous set of dirty pages, each with writeback
+ * set; the first page is still locked at this point, but all the rest
+ * have been unlocked.
+ */
+ folio_unlock(folio);
+
+ if (start < i_size) {
+ iov_iter_xarray(&wdata->iter, ITER_SOURCE, &mapping->i_pages,
+ start, len);
+
+ rc = adjust_credits(wdata->server, &wdata->credits, wdata->bytes);
+ if (rc)
+ goto err_wdata;
+
+ if (wdata->cfile->invalidHandle)
+ rc = -EAGAIN;
+ else
+ rc = wdata->server->ops->async_writev(wdata,
+ cifs_writedata_release);
+ if (rc >= 0) {
+ kref_put(&wdata->refcount, cifs_writedata_release);
+ goto err_close;
+ }
+ } else {
+ /* The dirty region was entirely beyond the EOF. */
+ cifs_pages_written_back(inode, start, len);
+ rc = 0;
+ }
+
+err_wdata:
+ kref_put(&wdata->refcount, cifs_writedata_release);
+err_uncredit:
+ add_credits_and_wake_if(server, credits, 0);
+err_close:
+ if (cfile)
+ cifsFileInfo_put(cfile);
+err_xid:
+ free_xid(xid);
+ if (rc == 0) {
+ wbc->nr_to_write = count;
+ } else if (is_retryable_error(rc)) {
+ cifs_pages_write_redirty(inode, start, len);
+ } else {
+ cifs_pages_write_failed(inode, start, len);
+ mapping_set_error(mapping, rc);
+ }
+ /* Indication to update ctime and mtime as close is deferred */
+ set_bit(CIFS_INO_MODIFIED_ATTR, &CIFS_I(inode)->flags);
+ return rc;
+}
+
+/*
+ * write a region of pages back to the server
+ */
+static int cifs_writepages_region(struct address_space *mapping,
+ struct writeback_control *wbc,
+ loff_t start, loff_t end, loff_t *_next)
+{
+ struct folio *folio;
+ struct page *head_page;
+ ssize_t ret;
+ int n, skips = 0;
+
+ do {
+ pgoff_t index = start / PAGE_SIZE;
+
+ n = find_get_pages_range_tag(mapping, &index, end / PAGE_SIZE,
+ PAGECACHE_TAG_DIRTY, 1, &head_page);
+ if (!n)
+ break;
+
+ folio = page_folio(head_page);
+ start = folio_pos(folio); /* May regress with THPs */
+
+ /* At this point we hold neither the i_pages lock nor the
+ * page lock: the page may be truncated or invalidated
+ * (changing page->mapping to NULL), or even swizzled
+ * back from swapper_space to tmpfs file mapping
+ */
+ if (wbc->sync_mode != WB_SYNC_NONE) {
+ ret = folio_lock_killable(folio);
+ if (ret < 0) {
+ folio_put(folio);
+ return ret;
+ }
+ } else {
+ if (!folio_trylock(folio)) {
+ folio_put(folio);
+ return 0;
+ }
+ }
+
+ if (folio_mapping(folio) != mapping ||
+ !folio_test_dirty(folio)) {
+ start += folio_size(folio);
+ folio_unlock(folio);
+ folio_put(folio);
+ continue;
+ }
+
+ if (folio_test_writeback(folio) ||
+ folio_test_fscache(folio)) {
+ folio_unlock(folio);
+ if (wbc->sync_mode != WB_SYNC_NONE) {
+ folio_wait_writeback(folio);
+#ifdef CONFIG_CIFS_FSCACHE
+ folio_wait_fscache(folio);
+#endif
+ } else {
+ start += folio_size(folio);
+ }
+ folio_put(folio);
+ if (wbc->sync_mode == WB_SYNC_NONE) {
+ if (skips >= 5 || need_resched())
+ break;
+ skips++;
+ }
+ continue;
+ }
+
+ if (!folio_clear_dirty_for_io(folio))
+ /* We hold the page lock - it should've been dirty. */
+ WARN_ON(1);
+
+ ret = cifs_write_back_from_locked_folio(mapping, wbc, folio, start, end);
+ folio_put(folio);
+ if (ret < 0)
+ return ret;
+
+ start += ret;
+ cond_resched();
+ } while (wbc->nr_to_write > 0);
+
+ *_next = start;
+ return 0;
+}
+
+/*
+ * Write some of the pending data back to the server
+ */
+static int cifs_writepages(struct address_space *mapping,
+ struct writeback_control *wbc)
+{
+ loff_t start, next;
+ int ret;
+
+ /* We have to be careful as we can end up racing with setattr()
+ * truncating the pagecache since the caller doesn't take a lock here
+ * to prevent it.
+ */
+
+ if (wbc->range_cyclic) {
+ start = mapping->writeback_index * PAGE_SIZE;
+ ret = cifs_writepages_region(mapping, wbc, start, LLONG_MAX, &next);
+ if (ret == 0) {
+ mapping->writeback_index = next / PAGE_SIZE;
+ if (start > 0 && wbc->nr_to_write > 0) {
+ ret = cifs_writepages_region(mapping, wbc, 0,
+ start, &next);
+ if (ret == 0)
+ mapping->writeback_index =
+ next / PAGE_SIZE;
+ }
+ }
+ } else if (wbc->range_start == 0 && wbc->range_end == LLONG_MAX) {
+ ret = cifs_writepages_region(mapping, wbc, 0, LLONG_MAX, &next);
+ if (wbc->nr_to_write > 0 && ret == 0)
+ mapping->writeback_index = next / PAGE_SIZE;
+ } else {
+ ret = cifs_writepages_region(mapping, wbc,
+ wbc->range_start, wbc->range_end, &next);
+ }
+
+ return ret;
+}
static int
cifs_writepage_locked(struct page *page, struct writeback_control *wbc)
@@ -2972,6 +3331,7 @@ static int cifs_write_end(struct file *file, struct address_space *mapping,
struct inode *inode = mapping->host;
struct cifsFileInfo *cfile = file->private_data;
struct cifs_sb_info *cifs_sb = CIFS_SB(cfile->dentry->d_sb);
+ struct folio *folio = page_folio(page);
__u32 pid;
if (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_RWPIDFORWARD)
@@ -2982,14 +3342,14 @@ static int cifs_write_end(struct file *file, struct address_space *mapping,
cifs_dbg(FYI, "write_end for page %p from pos %lld with %d bytes\n",
page, pos, copied);
- if (PageChecked(page)) {
+ if (folio_test_checked(folio)) {
if (copied == len)
- SetPageUptodate(page);
- ClearPageChecked(page);
- } else if (!PageUptodate(page) && copied == PAGE_SIZE)
- SetPageUptodate(page);
+ folio_mark_uptodate(folio);
+ folio_clear_checked(folio);
+ } else if (!folio_test_uptodate(folio) && copied == PAGE_SIZE)
+ folio_mark_uptodate(folio);
- if (!PageUptodate(page)) {
+ if (!folio_test_uptodate(folio)) {
char *page_data;
unsigned offset = pos & (PAGE_SIZE - 1);
unsigned int xid;
@@ -3149,6 +3509,7 @@ int cifs_flush(struct file *file, fl_owner_t id)
return rc;
}
+#if 0 // TODO: Remove for iov_iter support
static int
cifs_write_allocate_pages(struct page **pages, unsigned long num_pages)
{
@@ -3189,17 +3550,15 @@ size_t get_numpages(const size_t wsize, const size_t len, size_t *cur_len)
return num_pages;
}
+#endif
static void
cifs_uncached_writedata_release(struct kref *refcount)
{
- int i;
struct cifs_writedata *wdata = container_of(refcount,
struct cifs_writedata, refcount);
kref_put(&wdata->ctx->refcount, cifs_aio_ctx_release);
- for (i = 0; i < wdata->nr_pages; i++)
- put_page(wdata->pages[i]);
cifs_writedata_release(refcount);
}
@@ -3225,6 +3584,7 @@ cifs_uncached_writev_complete(struct work_struct *work)
kref_put(&wdata->refcount, cifs_uncached_writedata_release);
}
+#if 0 // TODO: Remove for iov_iter support
static int
wdata_fill_from_iovec(struct cifs_writedata *wdata, struct iov_iter *from,
size_t *len, unsigned long *num_pages)
@@ -3266,6 +3626,7 @@ wdata_fill_from_iovec(struct cifs_writedata *wdata, struct iov_iter *from,
*num_pages = i + 1;
return 0;
}
+#endif
static int
cifs_resend_wdata(struct cifs_writedata *wdata, struct list_head *wdata_list,
@@ -3337,23 +3698,57 @@ fail:
return rc;
}
+/*
+ * Select span of a bvec iterator we're going to use. Limit it by both maximum
+ * size and maximum number of segments.
+ */
+static size_t cifs_limit_bvec_subset(const struct iov_iter *iter, size_t max_size,
+ size_t max_segs, unsigned int *_nsegs)
+{
+ const struct bio_vec *bvecs = iter->bvec;
+ unsigned int nbv = iter->nr_segs, ix = 0, nsegs = 0;
+ size_t len, span = 0, n = iter->count;
+ size_t skip = iter->iov_offset;
+
+ if (WARN_ON(!iov_iter_is_bvec(iter)) || n == 0)
+ return 0;
+
+ while (n && ix < nbv && skip) {
+ len = bvecs[ix].bv_len;
+ if (skip < len)
+ break;
+ skip -= len;
+ n -= len;
+ ix++;
+ }
+
+ while (n && ix < nbv) {
+ len = min3(n, bvecs[ix].bv_len - skip, max_size);
+ span += len;
+ nsegs++;
+ ix++;
+ if (span >= max_size || nsegs >= max_segs)
+ break;
+ skip = 0;
+ n -= len;
+ }
+
+ *_nsegs = nsegs;
+ return span;
+}
+
static int
-cifs_write_from_iter(loff_t offset, size_t len, struct iov_iter *from,
+cifs_write_from_iter(loff_t fpos, size_t len, struct iov_iter *from,
struct cifsFileInfo *open_file,
struct cifs_sb_info *cifs_sb, struct list_head *wdata_list,
struct cifs_aio_ctx *ctx)
{
int rc = 0;
- size_t cur_len;
- unsigned long nr_pages, num_pages, i;
+ size_t cur_len, max_len;
struct cifs_writedata *wdata;
- struct iov_iter saved_from = *from;
- loff_t saved_offset = offset;
pid_t pid;
struct TCP_Server_Info *server;
- struct page **pagevec;
- size_t start;
- unsigned int xid;
+ unsigned int xid, max_segs = INT_MAX;
if (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_RWPIDFORWARD)
pid = open_file->pid;
@@ -3363,10 +3758,20 @@ cifs_write_from_iter(loff_t offset, size_t len, struct iov_iter *from,
server = cifs_pick_channel(tlink_tcon(open_file->tlink)->ses);
xid = get_xid();
+#ifdef CONFIG_CIFS_SMB_DIRECT
+ if (server->smbd_conn)
+ max_segs = server->smbd_conn->max_frmr_depth;
+#endif
+
do {
- unsigned int wsize;
struct cifs_credits credits_on_stack;
struct cifs_credits *credits = &credits_on_stack;
+ unsigned int wsize, nsegs = 0;
+
+ if (signal_pending(current)) {
+ rc = -EINTR;
+ break;
+ }
if (open_file->invalidHandle) {
rc = cifs_reopen_file(open_file, false);
@@ -3381,99 +3786,42 @@ cifs_write_from_iter(loff_t offset, size_t len, struct iov_iter *from,
if (rc)
break;
- cur_len = min_t(const size_t, len, wsize);
-
- if (ctx->direct_io) {
- ssize_t result;
-
- result = iov_iter_get_pages_alloc2(
- from, &pagevec, cur_len, &start);
- if (result < 0) {
- cifs_dbg(VFS,
- "direct_writev couldn't get user pages (rc=%zd) iter type %d iov_offset %zd count %zd\n",
- result, iov_iter_type(from),
- from->iov_offset, from->count);
- dump_stack();
-
- rc = result;
- add_credits_and_wake_if(server, credits, 0);
- break;
- }
- cur_len = (size_t)result;
-
- nr_pages =
- (cur_len + start + PAGE_SIZE - 1) / PAGE_SIZE;
-
- wdata = cifs_writedata_direct_alloc(pagevec,
- cifs_uncached_writev_complete);
- if (!wdata) {
- rc = -ENOMEM;
- for (i = 0; i < nr_pages; i++)
- put_page(pagevec[i]);
- kvfree(pagevec);
- add_credits_and_wake_if(server, credits, 0);
- break;
- }
-
-
- wdata->page_offset = start;
- wdata->tailsz =
- nr_pages > 1 ?
- cur_len - (PAGE_SIZE - start) -
- (nr_pages - 2) * PAGE_SIZE :
- cur_len;
- } else {
- nr_pages = get_numpages(wsize, len, &cur_len);
- wdata = cifs_writedata_alloc(nr_pages,
- cifs_uncached_writev_complete);
- if (!wdata) {
- rc = -ENOMEM;
- add_credits_and_wake_if(server, credits, 0);
- break;
- }
-
- rc = cifs_write_allocate_pages(wdata->pages, nr_pages);
- if (rc) {
- kvfree(wdata->pages);
- kfree(wdata);
- add_credits_and_wake_if(server, credits, 0);
- break;
- }
-
- num_pages = nr_pages;
- rc = wdata_fill_from_iovec(
- wdata, from, &cur_len, &num_pages);
- if (rc) {
- for (i = 0; i < nr_pages; i++)
- put_page(wdata->pages[i]);
- kvfree(wdata->pages);
- kfree(wdata);
- add_credits_and_wake_if(server, credits, 0);
- break;
- }
+ max_len = min_t(const size_t, len, wsize);
+ if (!max_len) {
+ rc = -EAGAIN;
+ add_credits_and_wake_if(server, credits, 0);
+ break;
+ }
- /*
- * Bring nr_pages down to the number of pages we
- * actually used, and free any pages that we didn't use.
- */
- for ( ; nr_pages > num_pages; nr_pages--)
- put_page(wdata->pages[nr_pages - 1]);
+ cur_len = cifs_limit_bvec_subset(from, max_len, max_segs, &nsegs);
+ cifs_dbg(FYI, "write_from_iter len=%zx/%zx nsegs=%u/%lu/%u\n",
+ cur_len, max_len, nsegs, from->nr_segs, max_segs);
+ if (cur_len == 0) {
+ rc = -EIO;
+ add_credits_and_wake_if(server, credits, 0);
+ break;
+ }
- wdata->tailsz = cur_len - ((nr_pages - 1) * PAGE_SIZE);
+ wdata = cifs_writedata_alloc(cifs_uncached_writev_complete);
+ if (!wdata) {
+ rc = -ENOMEM;
+ add_credits_and_wake_if(server, credits, 0);
+ break;
}
wdata->sync_mode = WB_SYNC_ALL;
- wdata->nr_pages = nr_pages;
- wdata->offset = (__u64)offset;
- wdata->cfile = cifsFileInfo_get(open_file);
- wdata->server = server;
- wdata->pid = pid;
- wdata->bytes = cur_len;
- wdata->pagesz = PAGE_SIZE;
- wdata->credits = credits_on_stack;
- wdata->ctx = ctx;
+ wdata->offset = (__u64)fpos;
+ wdata->cfile = cifsFileInfo_get(open_file);
+ wdata->server = server;
+ wdata->pid = pid;
+ wdata->bytes = cur_len;
+ wdata->credits = credits_on_stack;
+ wdata->iter = *from;
+ wdata->ctx = ctx;
kref_get(&ctx->refcount);
+ iov_iter_truncate(&wdata->iter, cur_len);
+
rc = adjust_credits(server, &wdata->credits, wdata->bytes);
if (!rc) {
@@ -3488,16 +3836,14 @@ cifs_write_from_iter(loff_t offset, size_t len, struct iov_iter *from,
add_credits_and_wake_if(server, &wdata->credits, 0);
kref_put(&wdata->refcount,
cifs_uncached_writedata_release);
- if (rc == -EAGAIN) {
- *from = saved_from;
- iov_iter_advance(from, offset - saved_offset);
+ if (rc == -EAGAIN)
continue;
- }
break;
}
list_add_tail(&wdata->list, wdata_list);
- offset += cur_len;
+ iov_iter_advance(from, cur_len);
+ fpos += cur_len;
len -= cur_len;
} while (len > 0);
@@ -3596,8 +3942,6 @@ static ssize_t __cifs_writev(
struct cifs_tcon *tcon;
struct cifs_sb_info *cifs_sb;
struct cifs_aio_ctx *ctx;
- struct iov_iter saved_from = *from;
- size_t len = iov_iter_count(from);
int rc;
/*
@@ -3631,23 +3975,54 @@ static ssize_t __cifs_writev(
ctx->iocb = iocb;
ctx->pos = iocb->ki_pos;
+ ctx->direct_io = direct;
+ ctx->nr_pinned_pages = 0;
- if (direct) {
- ctx->direct_io = true;
- ctx->iter = *from;
- ctx->len = len;
- } else {
- rc = setup_aio_ctx_iter(ctx, from, ITER_SOURCE);
- if (rc) {
+ if (user_backed_iter(from)) {
+ /*
+ * Extract IOVEC/UBUF-type iterators to a BVEC-type iterator as
+ * they contain references to the calling process's virtual
+ * memory layout which won't be available in an async worker
+ * thread. This also takes a pin on every folio involved.
+ */
+ rc = netfs_extract_user_iter(from, iov_iter_count(from),
+ &ctx->iter, 0);
+ if (rc < 0) {
kref_put(&ctx->refcount, cifs_aio_ctx_release);
return rc;
}
+
+ ctx->nr_pinned_pages = rc;
+ ctx->bv = (void *)ctx->iter.bvec;
+ ctx->bv_need_unpin = iov_iter_extract_will_pin(&ctx->iter);
+ } else if ((iov_iter_is_bvec(from) || iov_iter_is_kvec(from)) &&
+ !is_sync_kiocb(iocb)) {
+ /*
+ * If the op is asynchronous, we need to copy the list attached
+ * to a BVEC/KVEC-type iterator, but we assume that the storage
+ * will be pinned by the caller; in any case, we may or may not
+ * be able to pin the pages, so we don't try.
+ */
+ ctx->bv = (void *)dup_iter(&ctx->iter, from, GFP_KERNEL);
+ if (!ctx->bv) {
+ kref_put(&ctx->refcount, cifs_aio_ctx_release);
+ return -ENOMEM;
+ }
+ } else {
+ /*
+ * Otherwise, we just pass the iterator down as-is and rely on
+ * the caller to make sure the pages referred to by the
+ * iterator don't evaporate.
+ */
+ ctx->iter = *from;
}
+ ctx->len = iov_iter_count(&ctx->iter);
+
/* grab a lock here due to read response handlers can access ctx */
mutex_lock(&ctx->aio_mutex);
- rc = cifs_write_from_iter(iocb->ki_pos, ctx->len, &saved_from,
+ rc = cifs_write_from_iter(iocb->ki_pos, ctx->len, &ctx->iter,
cfile, cifs_sb, &ctx->list, ctx);
/*
@@ -3790,14 +4165,12 @@ out:
return written;
}
-static struct cifs_readdata *
-cifs_readdata_direct_alloc(struct page **pages, work_func_t complete)
+static struct cifs_readdata *cifs_readdata_alloc(work_func_t complete)
{
struct cifs_readdata *rdata;
rdata = kzalloc(sizeof(*rdata), GFP_KERNEL);
- if (rdata != NULL) {
- rdata->pages = pages;
+ if (rdata) {
kref_init(&rdata->refcount);
INIT_LIST_HEAD(&rdata->list);
init_completion(&rdata->done);
@@ -3807,27 +4180,14 @@ cifs_readdata_direct_alloc(struct page **pages, work_func_t complete)
return rdata;
}
-static struct cifs_readdata *
-cifs_readdata_alloc(unsigned int nr_pages, work_func_t complete)
-{
- struct page **pages =
- kcalloc(nr_pages, sizeof(struct page *), GFP_KERNEL);
- struct cifs_readdata *ret = NULL;
-
- if (pages) {
- ret = cifs_readdata_direct_alloc(pages, complete);
- if (!ret)
- kfree(pages);
- }
-
- return ret;
-}
-
void
cifs_readdata_release(struct kref *refcount)
{
struct cifs_readdata *rdata = container_of(refcount,
struct cifs_readdata, refcount);
+
+ if (rdata->ctx)
+ kref_put(&rdata->ctx->refcount, cifs_aio_ctx_release);
#ifdef CONFIG_CIFS_SMB_DIRECT
if (rdata->mr) {
smbd_deregister_mr(rdata->mr);
@@ -3837,85 +4197,9 @@ cifs_readdata_release(struct kref *refcount)
if (rdata->cfile)
cifsFileInfo_put(rdata->cfile);
- kvfree(rdata->pages);
kfree(rdata);
}
-static int
-cifs_read_allocate_pages(struct cifs_readdata *rdata, unsigned int nr_pages)
-{
- int rc = 0;
- struct page *page;
- unsigned int i;
-
- for (i = 0; i < nr_pages; i++) {
- page = alloc_page(GFP_KERNEL|__GFP_HIGHMEM);
- if (!page) {
- rc = -ENOMEM;
- break;
- }
- rdata->pages[i] = page;
- }
-
- if (rc) {
- unsigned int nr_page_failed = i;
-
- for (i = 0; i < nr_page_failed; i++) {
- put_page(rdata->pages[i]);
- rdata->pages[i] = NULL;
- }
- }
- return rc;
-}
-
-static void
-cifs_uncached_readdata_release(struct kref *refcount)
-{
- struct cifs_readdata *rdata = container_of(refcount,
- struct cifs_readdata, refcount);
- unsigned int i;
-
- kref_put(&rdata->ctx->refcount, cifs_aio_ctx_release);
- for (i = 0; i < rdata->nr_pages; i++) {
- put_page(rdata->pages[i]);
- }
- cifs_readdata_release(refcount);
-}
-
-/**
- * cifs_readdata_to_iov - copy data from pages in response to an iovec
- * @rdata: the readdata response with list of pages holding data
- * @iter: destination for our data
- *
- * This function copies data from a list of pages in a readdata response into
- * an array of iovecs. It will first calculate where the data should go
- * based on the info in the readdata and then copy the data into that spot.
- */
-static int
-cifs_readdata_to_iov(struct cifs_readdata *rdata, struct iov_iter *iter)
-{
- size_t remaining = rdata->got_bytes;
- unsigned int i;
-
- for (i = 0; i < rdata->nr_pages; i++) {
- struct page *page = rdata->pages[i];
- size_t copy = min_t(size_t, remaining, PAGE_SIZE);
- size_t written;
-
- if (unlikely(iov_iter_is_pipe(iter))) {
- void *addr = kmap_atomic(page);
-
- written = copy_to_iter(addr, copy, iter);
- kunmap_atomic(addr);
- } else
- written = copy_page_to_iter(page, 0, copy, iter);
- remaining -= written;
- if (written < copy && iov_iter_count(iter) > 0)
- break;
- }
- return remaining ? -EFAULT : 0;
-}
-
static void collect_uncached_read_data(struct cifs_aio_ctx *ctx);
static void
@@ -3927,9 +4211,11 @@ cifs_uncached_readv_complete(struct work_struct *work)
complete(&rdata->done);
collect_uncached_read_data(rdata->ctx);
/* the below call can possibly free the last ref to aio ctx */
- kref_put(&rdata->refcount, cifs_uncached_readdata_release);
+ kref_put(&rdata->refcount, cifs_readdata_release);
}
+#if 0 // TODO: Remove for iov_iter support
+
static int
uncached_fill_pages(struct TCP_Server_Info *server,
struct cifs_readdata *rdata, struct iov_iter *iter,
@@ -4003,6 +4289,7 @@ cifs_uncached_copy_into_pages(struct TCP_Server_Info *server,
{
return uncached_fill_pages(server, rdata, iter, iter->count);
}
+#endif
static int cifs_resend_rdata(struct cifs_readdata *rdata,
struct list_head *rdata_list,
@@ -4072,37 +4359,36 @@ static int cifs_resend_rdata(struct cifs_readdata *rdata,
} while (rc == -EAGAIN);
fail:
- kref_put(&rdata->refcount, cifs_uncached_readdata_release);
+ kref_put(&rdata->refcount, cifs_readdata_release);
return rc;
}
static int
-cifs_send_async_read(loff_t offset, size_t len, struct cifsFileInfo *open_file,
+cifs_send_async_read(loff_t fpos, size_t len, struct cifsFileInfo *open_file,
struct cifs_sb_info *cifs_sb, struct list_head *rdata_list,
struct cifs_aio_ctx *ctx)
{
struct cifs_readdata *rdata;
- unsigned int npages, rsize;
+ unsigned int rsize, nsegs, max_segs = INT_MAX;
struct cifs_credits credits_on_stack;
struct cifs_credits *credits = &credits_on_stack;
- size_t cur_len;
+ size_t cur_len, max_len;
int rc;
pid_t pid;
struct TCP_Server_Info *server;
- struct page **pagevec;
- size_t start;
- struct iov_iter direct_iov = ctx->iter;
server = cifs_pick_channel(tlink_tcon(open_file->tlink)->ses);
+#ifdef CONFIG_CIFS_SMB_DIRECT
+ if (server->smbd_conn)
+ max_segs = server->smbd_conn->max_frmr_depth;
+#endif
+
if (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_RWPIDFORWARD)
pid = open_file->pid;
else
pid = current->tgid;
- if (ctx->direct_io)
- iov_iter_advance(&direct_iov, offset - ctx->pos);
-
do {
if (open_file->invalidHandle) {
rc = cifs_reopen_file(open_file, true);
@@ -4122,78 +4408,37 @@ cifs_send_async_read(loff_t offset, size_t len, struct cifsFileInfo *open_file,
if (rc)
break;
- cur_len = min_t(const size_t, len, rsize);
-
- if (ctx->direct_io) {
- ssize_t result;
-
- result = iov_iter_get_pages_alloc2(
- &direct_iov, &pagevec,
- cur_len, &start);
- if (result < 0) {
- cifs_dbg(VFS,
- "Couldn't get user pages (rc=%zd) iter type %d iov_offset %zd count %zd\n",
- result, iov_iter_type(&direct_iov),
- direct_iov.iov_offset,
- direct_iov.count);
- dump_stack();
-
- rc = result;
- add_credits_and_wake_if(server, credits, 0);
- break;
- }
- cur_len = (size_t)result;
-
- rdata = cifs_readdata_direct_alloc(
- pagevec, cifs_uncached_readv_complete);
- if (!rdata) {
- add_credits_and_wake_if(server, credits, 0);
- rc = -ENOMEM;
- break;
- }
-
- npages = (cur_len + start + PAGE_SIZE-1) / PAGE_SIZE;
- rdata->page_offset = start;
- rdata->tailsz = npages > 1 ?
- cur_len-(PAGE_SIZE-start)-(npages-2)*PAGE_SIZE :
- cur_len;
-
- } else {
-
- npages = DIV_ROUND_UP(cur_len, PAGE_SIZE);
- /* allocate a readdata struct */
- rdata = cifs_readdata_alloc(npages,
- cifs_uncached_readv_complete);
- if (!rdata) {
- add_credits_and_wake_if(server, credits, 0);
- rc = -ENOMEM;
- break;
- }
+ max_len = min_t(size_t, len, rsize);
- rc = cifs_read_allocate_pages(rdata, npages);
- if (rc) {
- kvfree(rdata->pages);
- kfree(rdata);
- add_credits_and_wake_if(server, credits, 0);
- break;
- }
+ cur_len = cifs_limit_bvec_subset(&ctx->iter, max_len,
+ max_segs, &nsegs);
+ cifs_dbg(FYI, "read-to-iter len=%zx/%zx nsegs=%u/%lu/%u\n",
+ cur_len, max_len, nsegs, ctx->iter.nr_segs, max_segs);
+ if (cur_len == 0) {
+ rc = -EIO;
+ add_credits_and_wake_if(server, credits, 0);
+ break;
+ }
- rdata->tailsz = PAGE_SIZE;
+ rdata = cifs_readdata_alloc(cifs_uncached_readv_complete);
+ if (!rdata) {
+ add_credits_and_wake_if(server, credits, 0);
+ rc = -ENOMEM;
+ break;
}
- rdata->server = server;
- rdata->cfile = cifsFileInfo_get(open_file);
- rdata->nr_pages = npages;
- rdata->offset = offset;
- rdata->bytes = cur_len;
- rdata->pid = pid;
- rdata->pagesz = PAGE_SIZE;
- rdata->read_into_pages = cifs_uncached_read_into_pages;
- rdata->copy_into_pages = cifs_uncached_copy_into_pages;
- rdata->credits = credits_on_stack;
- rdata->ctx = ctx;
+ rdata->server = server;
+ rdata->cfile = cifsFileInfo_get(open_file);
+ rdata->offset = fpos;
+ rdata->bytes = cur_len;
+ rdata->pid = pid;
+ rdata->credits = credits_on_stack;
+ rdata->ctx = ctx;
kref_get(&ctx->refcount);
+ rdata->iter = ctx->iter;
+ iov_iter_truncate(&rdata->iter, cur_len);
+
rc = adjust_credits(server, &rdata->credits, rdata->bytes);
if (!rc) {
@@ -4205,17 +4450,15 @@ cifs_send_async_read(loff_t offset, size_t len, struct cifsFileInfo *open_file,
if (rc) {
add_credits_and_wake_if(server, &rdata->credits, 0);
- kref_put(&rdata->refcount,
- cifs_uncached_readdata_release);
- if (rc == -EAGAIN) {
- iov_iter_revert(&direct_iov, cur_len);
+ kref_put(&rdata->refcount, cifs_readdata_release);
+ if (rc == -EAGAIN)
continue;
- }
break;
}
list_add_tail(&rdata->list, rdata_list);
- offset += cur_len;
+ iov_iter_advance(&ctx->iter, cur_len);
+ fpos += cur_len;
len -= cur_len;
} while (len > 0);
@@ -4257,22 +4500,6 @@ again:
list_del_init(&rdata->list);
INIT_LIST_HEAD(&tmp_list);
- /*
- * Got a part of data and then reconnect has
- * happened -- fill the buffer and continue
- * reading.
- */
- if (got_bytes && got_bytes < rdata->bytes) {
- rc = 0;
- if (!ctx->direct_io)
- rc = cifs_readdata_to_iov(rdata, to);
- if (rc) {
- kref_put(&rdata->refcount,
- cifs_uncached_readdata_release);
- continue;
- }
- }
-
if (ctx->direct_io) {
/*
* Re-use rdata as this is a
@@ -4289,7 +4516,7 @@ again:
&tmp_list, ctx);
kref_put(&rdata->refcount,
- cifs_uncached_readdata_release);
+ cifs_readdata_release);
}
list_splice(&tmp_list, &ctx->list);
@@ -4297,8 +4524,6 @@ again:
goto again;
} else if (rdata->result)
rc = rdata->result;
- else if (!ctx->direct_io)
- rc = cifs_readdata_to_iov(rdata, to);
/* if there was a short read -- discard anything left */
if (rdata->got_bytes && rdata->got_bytes < rdata->bytes)
@@ -4307,7 +4532,7 @@ again:
ctx->total_len += rdata->got_bytes;
}
list_del_init(&rdata->list);
- kref_put(&rdata->refcount, cifs_uncached_readdata_release);
+ kref_put(&rdata->refcount, cifs_readdata_release);
}
if (!ctx->direct_io)
@@ -4367,26 +4592,53 @@ static ssize_t __cifs_readv(
if (!ctx)
return -ENOMEM;
- ctx->cfile = cifsFileInfo_get(cfile);
+ ctx->pos = offset;
+ ctx->direct_io = direct;
+ ctx->len = len;
+ ctx->cfile = cifsFileInfo_get(cfile);
+ ctx->nr_pinned_pages = 0;
if (!is_sync_kiocb(iocb))
ctx->iocb = iocb;
- if (user_backed_iter(to))
- ctx->should_dirty = true;
-
- if (direct) {
- ctx->pos = offset;
- ctx->direct_io = true;
- ctx->iter = *to;
- ctx->len = len;
- } else {
- rc = setup_aio_ctx_iter(ctx, to, ITER_DEST);
- if (rc) {
+ if (user_backed_iter(to)) {
+ /*
+ * Extract IOVEC/UBUF-type iterators to a BVEC-type iterator as
+ * they contain references to the calling process's virtual
+ * memory layout which won't be available in an async worker
+ * thread. This also takes a pin on every folio involved.
+ */
+ rc = netfs_extract_user_iter(to, iov_iter_count(to),
+ &ctx->iter, 0);
+ if (rc < 0) {
kref_put(&ctx->refcount, cifs_aio_ctx_release);
return rc;
}
- len = ctx->len;
+
+ ctx->nr_pinned_pages = rc;
+ ctx->bv = (void *)ctx->iter.bvec;
+ ctx->bv_need_unpin = iov_iter_extract_will_pin(&ctx->iter);
+ ctx->should_dirty = true;
+ } else if ((iov_iter_is_bvec(to) || iov_iter_is_kvec(to)) &&
+ !is_sync_kiocb(iocb)) {
+ /*
+ * If the op is asynchronous, we need to copy the list attached
+ * to a BVEC/KVEC-type iterator, but we assume that the storage
+ * will be retained by the caller; in any case, we may or may
+ * not be able to pin the pages, so we don't try.
+ */
+ ctx->bv = (void *)dup_iter(&ctx->iter, to, GFP_KERNEL);
+ if (!ctx->bv) {
+ kref_put(&ctx->refcount, cifs_aio_ctx_release);
+ return -ENOMEM;
+ }
+ } else {
+ /*
+ * Otherwise, we just pass the iterator down as-is and rely on
+ * the caller to make sure the pages referred to by the
+ * iterator don't evaporate.
+ */
+ ctx->iter = *to;
}
if (direct) {
@@ -4648,6 +4900,8 @@ int cifs_file_mmap(struct file *file, struct vm_area_struct *vma)
return rc;
}
+#if 0 // TODO: Remove for iov_iter support
+
static void
cifs_readv_complete(struct work_struct *work)
{
@@ -4778,19 +5032,74 @@ cifs_readpages_copy_into_pages(struct TCP_Server_Info *server,
{
return readpages_fill_pages(server, rdata, iter, iter->count);
}
+#endif
+
+/*
+ * Unlock a bunch of folios in the pagecache.
+ */
+static void cifs_unlock_folios(struct address_space *mapping, pgoff_t first, pgoff_t last)
+{
+ struct folio *folio;
+ XA_STATE(xas, &mapping->i_pages, first);
+
+ rcu_read_lock();
+ xas_for_each(&xas, folio, last) {
+ folio_unlock(folio);
+ }
+ rcu_read_unlock();
+}
+
+static void cifs_readahead_complete(struct work_struct *work)
+{
+ struct cifs_readdata *rdata = container_of(work,
+ struct cifs_readdata, work);
+ struct folio *folio;
+ pgoff_t last;
+ bool good = rdata->result == 0 || (rdata->result == -EAGAIN && rdata->got_bytes);
+
+ XA_STATE(xas, &rdata->mapping->i_pages, rdata->offset / PAGE_SIZE);
+
+ if (good)
+ cifs_readahead_to_fscache(rdata->mapping->host,
+ rdata->offset, rdata->bytes);
+
+ if (iov_iter_count(&rdata->iter) > 0)
+ iov_iter_zero(iov_iter_count(&rdata->iter), &rdata->iter);
+
+ last = (rdata->offset + rdata->bytes - 1) / PAGE_SIZE;
+
+ rcu_read_lock();
+ xas_for_each(&xas, folio, last) {
+ if (good) {
+ flush_dcache_folio(folio);
+ folio_mark_uptodate(folio);
+ }
+ folio_unlock(folio);
+ }
+ rcu_read_unlock();
+
+ kref_put(&rdata->refcount, cifs_readdata_release);
+}
static void cifs_readahead(struct readahead_control *ractl)
{
- int rc;
struct cifsFileInfo *open_file = ractl->file->private_data;
struct cifs_sb_info *cifs_sb = CIFS_FILE_SB(ractl->file);
struct TCP_Server_Info *server;
- pid_t pid;
- unsigned int xid, nr_pages, last_batch_size = 0, cache_nr_pages = 0;
- pgoff_t next_cached = ULONG_MAX;
+ unsigned int xid, nr_pages, cache_nr_pages = 0;
+ unsigned int ra_pages;
+ pgoff_t next_cached = ULONG_MAX, ra_index;
bool caching = fscache_cookie_enabled(cifs_inode_cookie(ractl->mapping->host)) &&
cifs_inode_cookie(ractl->mapping->host)->cache_priv;
bool check_cache = caching;
+ pid_t pid;
+ int rc = 0;
+
+ /* Note that readahead_count() lags behind our dequeuing of pages from
+ * the ractl, wo we have to keep track for ourselves.
+ */
+ ra_pages = readahead_count(ractl);
+ ra_index = readahead_index(ractl);
xid = get_xid();
@@ -4799,22 +5108,21 @@ static void cifs_readahead(struct readahead_control *ractl)
else
pid = current->tgid;
- rc = 0;
server = cifs_pick_channel(tlink_tcon(open_file->tlink)->ses);
cifs_dbg(FYI, "%s: file=%p mapping=%p num_pages=%u\n",
- __func__, ractl->file, ractl->mapping, readahead_count(ractl));
+ __func__, ractl->file, ractl->mapping, ra_pages);
/*
* Chop the readahead request up into rsize-sized read requests.
*/
- while ((nr_pages = readahead_count(ractl) - last_batch_size)) {
- unsigned int i, got, rsize;
- struct page *page;
+ while ((nr_pages = ra_pages)) {
+ unsigned int i, rsize;
struct cifs_readdata *rdata;
struct cifs_credits credits_on_stack;
struct cifs_credits *credits = &credits_on_stack;
- pgoff_t index = readahead_index(ractl) + last_batch_size;
+ struct folio *folio;
+ pgoff_t fsize;
/*
* Find out if we have anything cached in the range of
@@ -4823,21 +5131,22 @@ static void cifs_readahead(struct readahead_control *ractl)
if (caching) {
if (check_cache) {
rc = cifs_fscache_query_occupancy(
- ractl->mapping->host, index, nr_pages,
+ ractl->mapping->host, ra_index, nr_pages,
&next_cached, &cache_nr_pages);
if (rc < 0)
caching = false;
check_cache = false;
}
- if (index == next_cached) {
+ if (ra_index == next_cached) {
/*
* TODO: Send a whole batch of pages to be read
* by the cache.
*/
- struct folio *folio = readahead_folio(ractl);
-
- last_batch_size = folio_nr_pages(folio);
+ folio = readahead_folio(ractl);
+ fsize = folio_nr_pages(folio);
+ ra_pages -= fsize;
+ ra_index += fsize;
if (cifs_readpage_from_fscache(ractl->mapping->host,
&folio->page) < 0) {
/*
@@ -4848,8 +5157,8 @@ static void cifs_readahead(struct readahead_control *ractl)
caching = false;
}
folio_unlock(folio);
- next_cached++;
- cache_nr_pages--;
+ next_cached += fsize;
+ cache_nr_pages -= fsize;
if (cache_nr_pages == 0)
check_cache = true;
continue;
@@ -4874,8 +5183,9 @@ static void cifs_readahead(struct readahead_control *ractl)
&rsize, credits);
if (rc)
break;
- nr_pages = min_t(size_t, rsize / PAGE_SIZE, readahead_count(ractl));
- nr_pages = min_t(size_t, nr_pages, next_cached - index);
+ nr_pages = min_t(size_t, rsize / PAGE_SIZE, ra_pages);
+ if (next_cached != ULONG_MAX)
+ nr_pages = min_t(size_t, nr_pages, next_cached - ra_index);
/*
* Give up immediately if rsize is too small to read an entire
@@ -4888,33 +5198,31 @@ static void cifs_readahead(struct readahead_control *ractl)
break;
}
- rdata = cifs_readdata_alloc(nr_pages, cifs_readv_complete);
+ rdata = cifs_readdata_alloc(cifs_readahead_complete);
if (!rdata) {
/* best to give up if we're out of mem */
add_credits_and_wake_if(server, credits, 0);
break;
}
- got = __readahead_batch(ractl, rdata->pages, nr_pages);
- if (got != nr_pages) {
- pr_warn("__readahead_batch() returned %u/%u\n",
- got, nr_pages);
- nr_pages = got;
- }
-
- rdata->nr_pages = nr_pages;
- rdata->bytes = readahead_batch_length(ractl);
+ rdata->offset = ra_index * PAGE_SIZE;
+ rdata->bytes = nr_pages * PAGE_SIZE;
rdata->cfile = cifsFileInfo_get(open_file);
rdata->server = server;
rdata->mapping = ractl->mapping;
- rdata->offset = readahead_pos(ractl);
rdata->pid = pid;
- rdata->pagesz = PAGE_SIZE;
- rdata->tailsz = PAGE_SIZE;
- rdata->read_into_pages = cifs_readpages_read_into_pages;
- rdata->copy_into_pages = cifs_readpages_copy_into_pages;
rdata->credits = credits_on_stack;
+ for (i = 0; i < nr_pages; i++) {
+ if (!readahead_folio(ractl))
+ WARN_ON(1);
+ }
+ ra_pages -= nr_pages;
+ ra_index += nr_pages;
+
+ iov_iter_xarray(&rdata->iter, ITER_DEST, &rdata->mapping->i_pages,
+ rdata->offset, rdata->bytes);
+
rc = adjust_credits(server, &rdata->credits, rdata->bytes);
if (!rc) {
if (rdata->cfile->invalidHandle)
@@ -4925,18 +5233,15 @@ static void cifs_readahead(struct readahead_control *ractl)
if (rc) {
add_credits_and_wake_if(server, &rdata->credits, 0);
- for (i = 0; i < rdata->nr_pages; i++) {
- page = rdata->pages[i];
- unlock_page(page);
- put_page(page);
- }
+ cifs_unlock_folios(rdata->mapping,
+ rdata->offset / PAGE_SIZE,
+ (rdata->offset + rdata->bytes - 1) / PAGE_SIZE);
/* Fallback to the readpage in error/reconnect cases */
kref_put(&rdata->refcount, cifs_readdata_release);
break;
}
kref_put(&rdata->refcount, cifs_readdata_release);
- last_batch_size = nr_pages;
}
free_xid(xid);
@@ -4978,10 +5283,6 @@ static int cifs_readpage_worker(struct file *file, struct page *page,
flush_dcache_page(page);
SetPageUptodate(page);
-
- /* send this page to the cache */
- cifs_readpage_to_fscache(file_inode(file), page);
-
rc = 0;
io_error: