2022-04-07 01:43:21

by David Howells

[permalink] [raw]
Subject: [PATCH 00/14] cifs: Iterators, netfslib and folios


Here's a set of patches to make the following changes to the cifs
filesystem driver:

(1) Convert cifs to use I/O iterators to pass data/buffers from the the VM
interface layer down to the socket rather than passing page lists
about.

(2) Convert cifs to use netfslib for buffered and direct read operations.

(3) A partial conversion to folios.

This branch is built on top of my netfs-lib branch[1].

The patches can be found here also:

https://git.kernel.org/pub/scm/linux/kernel/git/dhowells/linux-fs.git/log/?h=cifs-experimental


David

Link: https://git.kernel.org/pub/scm/linux/kernel/git/dhowells/linux-fs.git/log/?h=netfs-lib [1]
---
David Howells (14):
cifs: Add some helper functions
cifs: Add a function to read into an iter from a socket
cifs: Check the IOCB_DIRECT flag, not O_DIRECT
cifs: Change the I/O paths to use an iterator rather than a page list
cifs: Remove unused code
cifs: Use netfslib to handle reads
cifs: Share server EOF pos with netfslib
netfs: Allow the netfs to make the io (sub)request alloc larger
cifs: Put credits into cifs_io_subrequest, not on the stack
cifs: Hold the open file on netfs_io_request, not netfs_io_subrequest
cifs: Clamp length according to credits and rsize
cifs: Expose netfs subrequest debug ID and index in read tracepoints
cifs: Split the smb3_add_credits tracepoint
mm, netfs, fscache: Stop read optimisation when folio removed from pagecache


fs/afs/file.c | 1 +
fs/afs/inode.c | 1 +
fs/afs/internal.h | 2 +
fs/cifs/Kconfig | 1 +
fs/cifs/cifsencrypt.c | 40 +-
fs/cifs/cifsfs.c | 11 +-
fs/cifs/cifsfs.h | 6 +-
fs/cifs/cifsglob.h | 53 +-
fs/cifs/cifsproto.h | 13 +-
fs/cifs/cifssmb.c | 252 +++--
fs/cifs/connect.c | 18 +-
fs/cifs/file.c | 2096 ++++++++++-----------------------------
fs/cifs/fscache.c | 120 +--
fs/cifs/fscache.h | 70 --
fs/cifs/inode.c | 22 +-
fs/cifs/misc.c | 109 --
fs/cifs/smb2ops.c | 387 ++++----
fs/cifs/smb2pdu.c | 85 +-
fs/cifs/smb2proto.h | 2 +-
fs/cifs/trace.h | 151 ++-
fs/cifs/transport.c | 41 +-
fs/netfs/internal.h | 1 +
fs/netfs/io.c | 7 +-
fs/netfs/misc.c | 13 +-
fs/netfs/stats.c | 9 +-
include/linux/fs.h | 2 +
include/linux/netfs.h | 1 +
include/linux/pagemap.h | 1 +
mm/filemap.c | 15 +
29 files changed, 1203 insertions(+), 2327 deletions(-)



2022-04-07 01:43:40

by David Howells

[permalink] [raw]
Subject: [PATCH 06/14] cifs: Use netfslib to handle reads

Make the cifs filesystem use netfslib to handle both buffered and direct
reads and alo to handle the preloading of pages to be modified.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/Kconfig | 1
fs/cifs/cifsfs.c | 6
fs/cifs/cifsfs.h | 3
fs/cifs/cifsglob.h | 6
fs/cifs/cifssmb.c | 9 -
fs/cifs/file.c | 918 ++++++++--------------------------------------------
fs/cifs/fscache.c | 117 -------
fs/cifs/fscache.h | 70 ----
fs/cifs/inode.c | 16 +
fs/cifs/smb2pdu.c | 15 +
fs/netfs/io.c | 7
11 files changed, 195 insertions(+), 973 deletions(-)

diff --git a/fs/cifs/Kconfig b/fs/cifs/Kconfig
index 3b7e3b9e4fd2..c47e2d3a101f 100644
--- a/fs/cifs/Kconfig
+++ b/fs/cifs/Kconfig
@@ -2,6 +2,7 @@
config CIFS
tristate "SMB3 and CIFS support (advanced network filesystem)"
depends on INET
+ select NETFS_SUPPORT
select NLS
select CRYPTO
select CRYPTO_MD5
diff --git a/fs/cifs/cifsfs.c b/fs/cifs/cifsfs.c
index fb60b5410789..b33f5eb3233c 100644
--- a/fs/cifs/cifsfs.c
+++ b/fs/cifs/cifsfs.c
@@ -945,7 +945,7 @@ cifs_loose_read_iter(struct kiocb *iocb, struct iov_iter *iter)
struct inode *inode = file_inode(iocb->ki_filp);

if (iocb->ki_flags & IOCB_DIRECT)
- return cifs_user_readv(iocb, iter);
+ return netfs_direct_read_iter(iocb, iter);

rc = cifs_revalidate_mapping(inode);
if (rc)
@@ -1323,7 +1323,7 @@ const struct file_operations cifs_file_strict_ops = {
};

const struct file_operations cifs_file_direct_ops = {
- .read_iter = cifs_direct_readv,
+ .read_iter = netfs_direct_read_iter,
.write_iter = cifs_direct_writev,
.open = cifs_open,
.release = cifs_close,
@@ -1379,7 +1379,7 @@ const struct file_operations cifs_file_strict_nobrl_ops = {
};

const struct file_operations cifs_file_direct_nobrl_ops = {
- .read_iter = cifs_direct_readv,
+ .read_iter = netfs_direct_read_iter,
.write_iter = cifs_direct_writev,
.open = cifs_open,
.release = cifs_close,
diff --git a/fs/cifs/cifsfs.h b/fs/cifs/cifsfs.h
index 1c77bbc0815f..c7d5c268fc47 100644
--- a/fs/cifs/cifsfs.h
+++ b/fs/cifs/cifsfs.h
@@ -85,6 +85,7 @@ extern const struct inode_operations cifs_dfs_referral_inode_operations;


/* Functions related to files and directories */
+extern const struct netfs_request_ops cifs_req_ops;
extern const struct file_operations cifs_file_ops;
extern const struct file_operations cifs_file_direct_ops; /* if directio mnt */
extern const struct file_operations cifs_file_strict_ops; /* if strictio mnt */
@@ -94,8 +95,6 @@ extern const struct file_operations cifs_file_strict_nobrl_ops;
extern int cifs_open(struct inode *inode, struct file *file);
extern int cifs_close(struct inode *inode, struct file *file);
extern int cifs_closedir(struct inode *inode, struct file *file);
-extern ssize_t cifs_user_readv(struct kiocb *iocb, struct iov_iter *to);
-extern ssize_t cifs_direct_readv(struct kiocb *iocb, struct iov_iter *to);
extern ssize_t cifs_strict_readv(struct kiocb *iocb, struct iov_iter *to);
extern ssize_t cifs_user_writev(struct kiocb *iocb, struct iov_iter *from);
extern ssize_t cifs_direct_writev(struct kiocb *iocb, struct iov_iter *from);
diff --git a/fs/cifs/cifsglob.h b/fs/cifs/cifsglob.h
index 06e0dd2c408d..e7db00396391 100644
--- a/fs/cifs/cifsglob.h
+++ b/fs/cifs/cifsglob.h
@@ -1315,18 +1315,14 @@ struct cifs_aio_ctx {

/* asynchronous read support */
struct cifs_readdata {
+ struct netfs_io_subrequest *subreq;
struct kref refcount;
- struct list_head list;
- struct completion done;
struct cifsFileInfo *cfile;
- struct address_space *mapping;
- struct cifs_aio_ctx *ctx;
__u64 offset;
ssize_t got_bytes;
unsigned int bytes;
pid_t pid;
int result;
- struct work_struct work;
struct iov_iter iter;
struct kvec iov[2];
struct TCP_Server_Info *server;
diff --git a/fs/cifs/cifssmb.c b/fs/cifs/cifssmb.c
index 1256cafe08b1..5956caaec0e3 100644
--- a/fs/cifs/cifssmb.c
+++ b/fs/cifs/cifssmb.c
@@ -23,6 +23,7 @@
#include <linux/swap.h>
#include <linux/task_io_accounting_ops.h>
#include <linux/uaccess.h>
+#include <linux/netfs.h>
#include "cifspdu.h"
#include "cifsfs.h"
#include "cifsglob.h"
@@ -1609,7 +1610,13 @@ cifs_readv_callback(struct mid_q_entry *mid)
rdata->result = -EIO;
}

- queue_work(cifsiod_wq, &rdata->work);
+ if (rdata->result == 0 || rdata->result == -EAGAIN)
+ iov_iter_advance(&rdata->subreq->iter, rdata->got_bytes);
+ netfs_subreq_terminated(rdata->subreq,
+ (rdata->result == 0 || rdata->result == -EAGAIN) ?
+ rdata->got_bytes : rdata->result,
+ false);
+ kref_put(&rdata->refcount, cifs_readdata_release);
DeleteMidQEntry(mid);
add_credits(server, &credits, 0);
}
diff --git a/fs/cifs/file.c b/fs/cifs/file.c
index aa72be9dcf4e..200280be064d 100644
--- a/fs/cifs/file.c
+++ b/fs/cifs/file.c
@@ -21,6 +21,7 @@
#include <linux/slab.h>
#include <linux/swap.h>
#include <linux/mm.h>
+#include <linux/netfs.h>
#include <asm/div64.h>
#include "cifsfs.h"
#include "cifspdu.h"
@@ -3326,12 +3327,8 @@ static struct cifs_readdata *cifs_readdata_alloc(work_func_t complete)
struct cifs_readdata *rdata;

rdata = kzalloc(sizeof(*rdata), GFP_KERNEL);
- if (rdata) {
+ if (rdata)
kref_init(&rdata->refcount);
- INIT_LIST_HEAD(&rdata->list);
- init_completion(&rdata->done);
- INIT_WORK(&rdata->work, complete);
- }

return rdata;
}
@@ -3342,8 +3339,6 @@ 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);
@@ -3356,375 +3351,6 @@ cifs_readdata_release(struct kref *refcount)
kfree(rdata);
}

-static void collect_uncached_read_data(struct cifs_aio_ctx *ctx);
-
-static void
-cifs_uncached_readv_complete(struct work_struct *work)
-{
- struct cifs_readdata *rdata = container_of(work,
- struct cifs_readdata, 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_readdata_release);
-}
-
-static int cifs_resend_rdata(struct cifs_readdata *rdata,
- struct list_head *rdata_list,
- struct cifs_aio_ctx *ctx)
-{
- unsigned int rsize;
- struct cifs_credits credits;
- int rc;
- struct TCP_Server_Info *server;
-
- /* XXX: should we pick a new channel here? */
- server = rdata->server;
-
- do {
- if (rdata->cfile->invalidHandle) {
- rc = cifs_reopen_file(rdata->cfile, true);
- if (rc == -EAGAIN)
- continue;
- else if (rc)
- break;
- }
-
- /*
- * Wait for credits to resend this rdata.
- * Note: we are attempting to resend the whole rdata not in
- * segments
- */
- do {
- rc = server->ops->wait_mtu_credits(server, rdata->bytes,
- &rsize, &credits);
-
- if (rc)
- goto fail;
-
- if (rsize < rdata->bytes) {
- add_credits_and_wake_if(server, &credits, 0);
- msleep(1000);
- }
- } while (rsize < rdata->bytes);
- rdata->credits = credits;
-
- rc = adjust_credits(server, &rdata->credits, rdata->bytes);
- if (!rc) {
- if (rdata->cfile->invalidHandle)
- rc = -EAGAIN;
- else {
-#ifdef CONFIG_CIFS_SMB_DIRECT
- if (rdata->mr) {
- rdata->mr->need_invalidate = true;
- smbd_deregister_mr(rdata->mr);
- rdata->mr = NULL;
- }
-#endif
- rc = server->ops->async_readv(rdata);
- }
- }
-
- /* If the read was successfully sent, we are done */
- if (!rc) {
- /* Add to aio pending list */
- list_add_tail(&rdata->list, rdata_list);
- return 0;
- }
-
- /* Roll back credits and retry if needed */
- add_credits_and_wake_if(server, &rdata->credits, 0);
- } while (rc == -EAGAIN);
-
-fail:
- 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,
- struct cifs_sb_info *cifs_sb, struct list_head *rdata_list,
- struct cifs_aio_ctx *ctx)
-{
- struct cifs_readdata *rdata;
- unsigned int rsize;
- struct cifs_credits credits_on_stack;
- struct cifs_credits *credits = &credits_on_stack;
- size_t cur_len;
- int rc;
- pid_t pid;
- struct TCP_Server_Info *server;
-
- server = cifs_pick_channel(tlink_tcon(open_file->tlink)->ses);
-
- if (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_RWPIDFORWARD)
- pid = open_file->pid;
- else
- pid = current->tgid;
-
- do {
- if (open_file->invalidHandle) {
- rc = cifs_reopen_file(open_file, true);
- if (rc == -EAGAIN)
- continue;
- else if (rc)
- break;
- }
-
- if (cifs_sb->ctx->rsize == 0)
- cifs_sb->ctx->rsize =
- server->ops->negotiate_rsize(tlink_tcon(open_file->tlink),
- cifs_sb->ctx);
-
- rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->rsize,
- &rsize, credits);
- if (rc)
- break;
-
- cur_len = min_t(const size_t, len, rsize);
-
- 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->offset = offset;
- 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_advance(&rdata->iter, offset - ctx->pos);
- iov_iter_truncate(&rdata->iter, cur_len);
-
- rc = adjust_credits(server, &rdata->credits, rdata->bytes);
-
- if (!rc) {
- if (rdata->cfile->invalidHandle)
- rc = -EAGAIN;
- else
- rc = server->ops->async_readv(rdata);
- }
-
- if (rc) {
- add_credits_and_wake_if(server, &rdata->credits, 0);
- kref_put(&rdata->refcount, cifs_readdata_release);
- if (rc == -EAGAIN)
- continue;
- break;
- }
-
- list_add_tail(&rdata->list, rdata_list);
- offset += cur_len;
- len -= cur_len;
- } while (len > 0);
-
- return rc;
-}
-
-static void
-collect_uncached_read_data(struct cifs_aio_ctx *ctx)
-{
- struct cifs_readdata *rdata, *tmp;
- struct iov_iter *to = &ctx->iter;
- struct cifs_sb_info *cifs_sb;
- int rc;
-
- cifs_sb = CIFS_SB(ctx->cfile->dentry->d_sb);
-
- mutex_lock(&ctx->aio_mutex);
-
- if (list_empty(&ctx->list)) {
- mutex_unlock(&ctx->aio_mutex);
- return;
- }
-
- rc = ctx->rc;
- /* the loop below should proceed in the order of increasing offsets */
-again:
- list_for_each_entry_safe(rdata, tmp, &ctx->list, list) {
- if (!rc) {
- if (!try_wait_for_completion(&rdata->done)) {
- mutex_unlock(&ctx->aio_mutex);
- return;
- }
-
- if (rdata->result == -EAGAIN) {
- /* resend call if it's a retryable error */
- struct list_head tmp_list;
- unsigned int got_bytes = rdata->got_bytes;
-
- list_del_init(&rdata->list);
- INIT_LIST_HEAD(&tmp_list);
-
- if (ctx->direct_io) {
- /*
- * Re-use rdata as this is a
- * direct I/O
- */
- rc = cifs_resend_rdata(
- rdata,
- &tmp_list, ctx);
- } else {
- rc = cifs_send_async_read(
- rdata->offset + got_bytes,
- rdata->bytes - got_bytes,
- rdata->cfile, cifs_sb,
- &tmp_list, ctx);
-
- kref_put(&rdata->refcount,
- cifs_readdata_release);
- }
-
- list_splice(&tmp_list, &ctx->list);
-
- goto again;
- } else if (rdata->result)
- rc = rdata->result;
-
- /* if there was a short read -- discard anything left */
- if (rdata->got_bytes && rdata->got_bytes < rdata->bytes)
- rc = -ENODATA;
-
- ctx->total_len += rdata->got_bytes;
- }
- list_del_init(&rdata->list);
- kref_put(&rdata->refcount, cifs_readdata_release);
- }
-
- if (!ctx->direct_io)
- ctx->total_len = ctx->len - iov_iter_count(to);
-
- /* mask nodata case */
- if (rc == -ENODATA)
- rc = 0;
-
- ctx->rc = (rc == 0) ? (ssize_t)ctx->total_len : rc;
-
- mutex_unlock(&ctx->aio_mutex);
-
- if (ctx->iocb && ctx->iocb->ki_complete)
- ctx->iocb->ki_complete(ctx->iocb, ctx->rc);
- else
- complete(&ctx->done);
-}
-
-static ssize_t __cifs_readv(
- struct kiocb *iocb, struct iov_iter *to, bool direct)
-{
- size_t len;
- struct file *file = iocb->ki_filp;
- struct cifs_sb_info *cifs_sb;
- struct cifsFileInfo *cfile;
- struct cifs_tcon *tcon;
- ssize_t rc, total_read = 0;
- loff_t offset = iocb->ki_pos;
- struct cifs_aio_ctx *ctx;
-
- /*
- * iov_iter_get_pages_alloc() doesn't work with ITER_KVEC,
- * fall back to data copy read path
- * this could be improved by getting pages directly in ITER_KVEC
- */
- if (direct && iov_iter_is_kvec(to)) {
- cifs_dbg(FYI, "use non-direct cifs_user_readv for kvec I/O\n");
- direct = false;
- }
-
- len = iov_iter_count(to);
- if (!len)
- return 0;
-
- cifs_sb = CIFS_FILE_SB(file);
- cfile = file->private_data;
- tcon = tlink_tcon(cfile->tlink);
-
- if (!tcon->ses->server->ops->async_readv)
- return -ENOSYS;
-
- if ((file->f_flags & O_ACCMODE) == O_WRONLY)
- cifs_dbg(FYI, "attempting read on write only file instance\n");
-
- ctx = cifs_aio_ctx_alloc();
- if (!ctx)
- return -ENOMEM;
-
- ctx->pos = offset;
- ctx->direct_io = direct;
- ctx->len = len;
- ctx->cfile = cifsFileInfo_get(cfile);
-
- if (!is_sync_kiocb(iocb))
- ctx->iocb = iocb;
-
- if (iter_is_iovec(to))
- ctx->should_dirty = true;
-
- rc = extract_iter_to_iter(to, len, &ctx->iter, &ctx->bv);
- if (rc < 0) {
- kref_put(&ctx->refcount, cifs_aio_ctx_release);
- return rc;
- }
- ctx->npages = rc;
-
- /* grab a lock here due to read response handlers can access ctx */
- mutex_lock(&ctx->aio_mutex);
-
- rc = cifs_send_async_read(offset, len, cfile, cifs_sb, &ctx->list, ctx);
-
- /* if at least one read request send succeeded, then reset rc */
- if (!list_empty(&ctx->list))
- rc = 0;
-
- mutex_unlock(&ctx->aio_mutex);
-
- if (rc) {
- kref_put(&ctx->refcount, cifs_aio_ctx_release);
- return rc;
- }
-
- if (!is_sync_kiocb(iocb)) {
- kref_put(&ctx->refcount, cifs_aio_ctx_release);
- return -EIOCBQUEUED;
- }
-
- rc = wait_for_completion_killable(&ctx->done);
- if (rc) {
- mutex_lock(&ctx->aio_mutex);
- ctx->rc = rc = -EINTR;
- total_read = ctx->total_len;
- mutex_unlock(&ctx->aio_mutex);
- } else {
- rc = ctx->rc;
- total_read = ctx->total_len;
- }
-
- kref_put(&ctx->refcount, cifs_aio_ctx_release);
-
- if (total_read) {
- iocb->ki_pos += total_read;
- return total_read;
- }
- return rc;
-}
-
-ssize_t cifs_direct_readv(struct kiocb *iocb, struct iov_iter *to)
-{
- return __cifs_readv(iocb, to, true);
-}
-
-ssize_t cifs_user_readv(struct kiocb *iocb, struct iov_iter *to)
-{
- return __cifs_readv(iocb, to, false);
-}
-
ssize_t
cifs_strict_readv(struct kiocb *iocb, struct iov_iter *to)
{
@@ -3745,12 +3371,15 @@ cifs_strict_readv(struct kiocb *iocb, struct iov_iter *to)
* pos+len-1.
*/
if (!CIFS_CACHE_READ(cinode))
- return cifs_user_readv(iocb, to);
+ return netfs_direct_read_iter(iocb, to);

if (cap_unix(tcon->ses) &&
(CIFS_UNIX_FCNTL_CAP & le64_to_cpu(tcon->fsUnixInfo.Capability)) &&
- ((cifs_sb->mnt_cifs_flags & CIFS_MOUNT_NOPOSIXBRL) == 0))
+ ((cifs_sb->mnt_cifs_flags & CIFS_MOUNT_NOPOSIXBRL) == 0)) {
+ if (iocb->ki_flags & IOCB_DIRECT)
+ return netfs_direct_read_iter(iocb, to);
return generic_file_read_iter(iocb, to);
+ }

/*
* We need to hold the sem to be sure nobody modifies lock list
@@ -3759,104 +3388,16 @@ cifs_strict_readv(struct kiocb *iocb, struct iov_iter *to)
down_read(&cinode->lock_sem);
if (!cifs_find_lock_conflict(cfile, iocb->ki_pos, iov_iter_count(to),
tcon->ses->server->vals->shared_lock_type,
- 0, NULL, CIFS_READ_OP))
- rc = generic_file_read_iter(iocb, to);
+ 0, NULL, CIFS_READ_OP)) {
+ if (iocb->ki_flags & IOCB_DIRECT)
+ rc = netfs_direct_read_iter(iocb, to);
+ else
+ rc = generic_file_read_iter(iocb, to);
+ }
up_read(&cinode->lock_sem);
return rc;
}

-static ssize_t
-cifs_read(struct file *file, char *read_data, size_t read_size, loff_t *offset)
-{
- int rc = -EACCES;
- unsigned int bytes_read = 0;
- unsigned int total_read;
- unsigned int current_read_size;
- unsigned int rsize;
- struct cifs_sb_info *cifs_sb;
- struct cifs_tcon *tcon;
- struct TCP_Server_Info *server;
- unsigned int xid;
- char *cur_offset;
- struct cifsFileInfo *open_file;
- struct cifs_io_parms io_parms = {0};
- int buf_type = CIFS_NO_BUFFER;
- __u32 pid;
-
- xid = get_xid();
- cifs_sb = CIFS_FILE_SB(file);
-
- /* FIXME: set up handlers for larger reads and/or convert to async */
- rsize = min_t(unsigned int, cifs_sb->ctx->rsize, CIFSMaxBufSize);
-
- if (file->private_data == NULL) {
- rc = -EBADF;
- free_xid(xid);
- return rc;
- }
- open_file = file->private_data;
- tcon = tlink_tcon(open_file->tlink);
- server = cifs_pick_channel(tcon->ses);
-
- if (!server->ops->sync_read) {
- free_xid(xid);
- return -ENOSYS;
- }
-
- if (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_RWPIDFORWARD)
- pid = open_file->pid;
- else
- pid = current->tgid;
-
- if ((file->f_flags & O_ACCMODE) == O_WRONLY)
- cifs_dbg(FYI, "attempting read on write only file instance\n");
-
- for (total_read = 0, cur_offset = read_data; read_size > total_read;
- total_read += bytes_read, cur_offset += bytes_read) {
- do {
- current_read_size = min_t(uint, read_size - total_read,
- rsize);
- /*
- * For windows me and 9x we do not want to request more
- * than it negotiated since it will refuse the read
- * then.
- */
- if (!(tcon->ses->capabilities &
- tcon->ses->server->vals->cap_large_files)) {
- current_read_size = min_t(uint,
- current_read_size, CIFSMaxBufSize);
- }
- if (open_file->invalidHandle) {
- rc = cifs_reopen_file(open_file, true);
- if (rc != 0)
- break;
- }
- io_parms.pid = pid;
- io_parms.tcon = tcon;
- io_parms.offset = *offset;
- io_parms.length = current_read_size;
- io_parms.server = server;
- rc = server->ops->sync_read(xid, &open_file->fid, &io_parms,
- &bytes_read, &cur_offset,
- &buf_type);
- } while (rc == -EAGAIN);
-
- if (rc || (bytes_read == 0)) {
- if (total_read) {
- break;
- } else {
- free_xid(xid);
- return rc;
- }
- } else {
- cifs_stats_bytes_read(tcon, total_read);
- *offset += bytes_read;
- }
- }
- free_xid(xid);
- return total_read;
-}
-
/*
* If the page is mmap'ed into a process' page tables, then we need to make
* sure that it doesn't change while being written back.
@@ -3926,278 +3467,161 @@ int cifs_file_mmap(struct file *file, struct vm_area_struct *vma)
}

/*
- * Unlock a bunch of folios in the pagecache.
+ * Issue a read operation on behalf of the netfs helper functions. We're asked
+ * to make a read of a certain size at a point in the file. We are permitted
+ * to only read a portion of that, but as long as we read something, the netfs
+ * helper will call us again so that we can issue another read.
*/
-static void cifs_unlock_folios(struct address_space *mapping, pgoff_t first, pgoff_t last)
+static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
{
- 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 0
- if (good)
- cifs_readpage_to_fscache(rdata->mapping->host, page);
-#endif
-
- if (iov_iter_count(&rdata->iter) > 0)
- iov_iter_zero(iov_iter_count(&rdata->iter), &rdata->iter);
-
- last = round_down(rdata->offset + rdata->got_bytes - 1, PAGE_SIZE);
-
- xas_for_each(&xas, folio, last) {
- if (good) {
- flush_dcache_folio(folio);
- folio_mark_uptodate(folio);
- }
- folio_unlock(folio);
- }
-
- kref_put(&rdata->refcount, cifs_readdata_release);
-}
-
-static void cifs_readahead(struct readahead_control *ractl)
-{
- struct cifsFileInfo *open_file = ractl->file->private_data;
- struct cifs_sb_info *cifs_sb = CIFS_FILE_SB(ractl->file);
+ struct netfs_io_request *rreq = subreq->rreq;
struct TCP_Server_Info *server;
- unsigned int xid, nr_pages, last_batch_size = 0, cache_nr_pages = 0;
- pgoff_t next_cached = ULONG_MAX;
- bool caching = fscache_cookie_enabled(cifs_inode_cookie(ractl->mapping->host)) &&
- cifs_inode_cookie(ractl->mapping->host)->cache_priv;
- bool check_cache = caching;
+ struct cifs_readdata *rdata;
+ struct cifsFileInfo *open_file = rreq->netfs_priv;
+ struct cifs_sb_info *cifs_sb = CIFS_SB(rreq->inode->i_sb);
+ struct cifs_credits credits_on_stack, *credits = &credits_on_stack;
+ unsigned int xid;
pid_t pid;
int rc = 0;
+ unsigned int rsize;

xid = get_xid();

if (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_RWPIDFORWARD)
pid = open_file->pid;
else
- pid = current->tgid;
+ pid = current->tgid; // Ummm... This may be a workqueue

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));
-
- /*
- * Chop the readahead request up into rsize-sized read requests.
- */
- while ((nr_pages = readahead_count(ractl) - last_batch_size)) {
- unsigned int i, rsize;
- struct page *page;
- 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;
-
- /*
- * Find out if we have anything cached in the range of
- * interest, and if so, where the next chunk of cached data is.
- */
- if (caching) {
- if (check_cache) {
- rc = cifs_fscache_query_occupancy(
- ractl->mapping->host, index, nr_pages,
- &next_cached, &cache_nr_pages);
- if (rc < 0)
- caching = false;
- check_cache = false;
- }
-
- if (index == next_cached) {
- /*
- * TODO: Send a whole batch of pages to be read
- * by the cache.
- */
- page = readahead_page(ractl);
- last_batch_size = 1 << thp_order(page);
- if (cifs_readpage_from_fscache(ractl->mapping->host,
- page) < 0) {
- /*
- * TODO: Deal with cache read failure
- * here, but for the moment, delegate
- * that to readpage.
- */
- caching = false;
- }
- unlock_page(page);
- next_cached++;
- cache_nr_pages--;
- if (cache_nr_pages == 0)
- check_cache = true;
- continue;
- }
- }
+ cifs_dbg(FYI, "%s: op=%08x[%x] mapping=%p len=%zu/%zu\n",
+ __func__, rreq->debug_id, subreq->debug_index, rreq->mapping,
+ subreq->transferred, subreq->len);

- if (open_file->invalidHandle) {
+ if (open_file->invalidHandle) {
+ do {
rc = cifs_reopen_file(open_file, true);
- if (rc) {
- if (rc == -EAGAIN)
- continue;
- break;
- }
- }
-
- if (cifs_sb->ctx->rsize == 0)
- cifs_sb->ctx->rsize =
- server->ops->negotiate_rsize(tlink_tcon(open_file->tlink),
- cifs_sb->ctx);
-
- rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->rsize,
- &rsize, credits);
+ } while (rc == -EAGAIN);
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);
-
- /*
- * Give up immediately if rsize is too small to read an entire
- * page. The VFS will fall back to readpage. We should never
- * reach this point however since we set ra_pages to 0 when the
- * rsize is smaller than a cache page.
- */
- if (unlikely(!nr_pages)) {
- add_credits_and_wake_if(server, credits, 0);
- break;
- }
+ goto out;
+ }

- 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;
- }
+ if (cifs_sb->ctx->rsize == 0)
+ cifs_sb->ctx->rsize =
+ server->ops->negotiate_rsize(tlink_tcon(open_file->tlink),
+ cifs_sb->ctx);

- rdata->offset = readahead_pos(ractl);
- rdata->bytes = nr_pages * PAGE_SIZE;
- rdata->cfile = cifsFileInfo_get(open_file);
- rdata->server = server;
- rdata->mapping = ractl->mapping;
- rdata->pid = pid;
- rdata->credits = credits_on_stack;
+ rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->rsize, &rsize, credits);
+ if (rc)
+ goto out;

- for (i = 0; i < nr_pages; i++)
- if (!readahead_folio(ractl))
- BUG();
+ rdata = cifs_readdata_alloc(NULL);
+ if (!rdata) {
+ add_credits_and_wake_if(server, credits, 0);
+ rc = -ENOMEM;
+ goto out;
+ }

- iov_iter_xarray(&rdata->iter, READ, &rdata->mapping->i_pages,
- rdata->offset, rdata->bytes);
+ __set_bit(NETFS_SREQ_CLEAR_TAIL, &subreq->flags);
+ rdata->subreq = subreq;
+ rdata->cfile = cifsFileInfo_get(open_file);
+ rdata->server = server;
+ rdata->offset = subreq->start + subreq->transferred;
+ rdata->bytes = subreq->len - subreq->transferred;
+ rdata->pid = pid;
+ rdata->credits = credits_on_stack;
+ rdata->iter = subreq->iter;

- rc = adjust_credits(server, &rdata->credits, rdata->bytes);
- if (!rc) {
- if (rdata->cfile->invalidHandle)
- rc = -EAGAIN;
- else
- rc = server->ops->async_readv(rdata);
- }
-
- if (rc) {
- add_credits_and_wake_if(server, &rdata->credits, 0);
- 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;
- }
+ rc = adjust_credits(server, &rdata->credits, rdata->bytes);
+ if (!rc) {
+ if (rdata->cfile->invalidHandle)
+ rc = -EAGAIN;
+ else
+ rc = server->ops->async_readv(rdata);
+ }

+ if (rc) {
+ add_credits_and_wake_if(server, &rdata->credits, 0);
+ /* Fallback to the readpage in error/reconnect cases */
kref_put(&rdata->refcount, cifs_readdata_release);
- last_batch_size = ractl->_batch_count;
+ goto out;
}

+ kref_put(&rdata->refcount, cifs_readdata_release);
+
+out:
free_xid(xid);
+ if (rc)
+ netfs_subreq_terminated(subreq, rc, false);
+}
+
+static int cifs_init_request(struct netfs_io_request *rreq, struct file *file)
+{
+ rreq->netfs_priv = file->private_data;
+ return 0;
}

/*
- * cifs_readpage_worker must be called with the page pinned
+ * Expand the size of a readahead to the size of the rsize, if at least as
+ * large as a page, allowing for the possibility that rsize is not pow-2
+ * aligned.
*/
-static int cifs_readpage_worker(struct file *file, struct page *page,
- loff_t *poffset)
+static void cifs_expand_readahead(struct netfs_io_request *rreq)
{
- char *read_data;
- int rc;
-
- /* Is the page cached? */
- rc = cifs_readpage_from_fscache(file_inode(file), page);
- if (rc == 0)
- goto read_complete;
+ struct cifs_sb_info *cifs_sb = CIFS_SB(rreq->inode->i_sb);
+ unsigned int rsize = cifs_sb->ctx->rsize;
+ loff_t misalignment, i_size = i_size_read(rreq->inode);

- read_data = kmap(page);
- /* for reads over a certain size could initiate async read ahead */
-
- rc = cifs_read(file, read_data, PAGE_SIZE, poffset);
-
- if (rc < 0)
- goto io_error;
- else
- cifs_dbg(FYI, "Bytes read %d\n", rc);
+ if (rsize < PAGE_SIZE)
+ return;

- /* we do not want atime to be less than mtime, it broke some apps */
- file_inode(file)->i_atime = current_time(file_inode(file));
- if (timespec64_compare(&(file_inode(file)->i_atime), &(file_inode(file)->i_mtime)))
- file_inode(file)->i_atime = file_inode(file)->i_mtime;
+ if (rsize < INT_MAX)
+ rsize = roundup_pow_of_two(rsize);
else
- file_inode(file)->i_atime = current_time(file_inode(file));
-
- if (PAGE_SIZE > rc)
- memset(read_data + rc, 0, PAGE_SIZE - rc);
+ rsize = ((unsigned int)INT_MAX + 1) / 2;

- flush_dcache_page(page);
- SetPageUptodate(page);
-
- /* send this page to the cache */
- cifs_readpage_to_fscache(file_inode(file), page);
-
- rc = 0;
-
-io_error:
- kunmap(page);
- unlock_page(page);
+ misalignment = rreq->start & (rsize - 1);
+ if (misalignment) {
+ rreq->start -= misalignment;
+ rreq->len += misalignment;
+ }

-read_complete:
- return rc;
+ rreq->len = round_up(rreq->len, rsize);
+ if (rreq->start < i_size && rreq->len > i_size - rreq->start)
+ rreq->len = i_size - rreq->start;
}

-static int cifs_readpage(struct file *file, struct page *page)
+static void cifs_rreq_done(struct netfs_io_request *rreq)
{
- loff_t offset = page_file_offset(page);
- int rc = -EACCES;
- unsigned int xid;
-
- xid = get_xid();
-
- if (file->private_data == NULL) {
- rc = -EBADF;
- free_xid(xid);
- return rc;
- }
+ struct inode *inode = rreq->inode;

- cifs_dbg(FYI, "readpage %p at offset %d 0x%x\n",
- page, (int)offset, (int)offset);
+ /* we do not want atime to be less than mtime, it broke some apps */
+ inode->i_atime = current_time(inode);
+ if (timespec64_compare(&inode->i_atime, &inode->i_mtime))
+ inode->i_atime = inode->i_mtime;
+ else
+ inode->i_atime = current_time(inode);
+}

- rc = cifs_readpage_worker(file, page, &offset);
+static int cifs_begin_cache_operation(struct netfs_io_request *rreq)
+{
+#ifdef CONFIG_CIFS_FSCACHE
+ struct fscache_cookie *cookie = cifs_inode_cookie(rreq->inode);

- free_xid(xid);
- return rc;
+ return fscache_begin_read_operation(&rreq->cache_resources, cookie);
+#else
+ return -ENOBUFS;
+#endif
}

+const struct netfs_request_ops cifs_req_ops = {
+ .init_request = cifs_init_request,
+ .begin_cache_operation = cifs_begin_cache_operation,
+ .expand_readahead = cifs_expand_readahead,
+ .issue_read = cifs_req_issue_read,
+ .done = cifs_rreq_done,
+};
+
static int is_inode_writable(struct cifsInodeInfo *cifs_inode)
{
struct cifsFileInfo *open_file;
@@ -4247,34 +3671,20 @@ static int cifs_write_begin(struct file *file, struct address_space *mapping,
loff_t pos, unsigned len, unsigned flags,
struct page **pagep, void **fsdata)
{
- int oncethru = 0;
- pgoff_t index = pos >> PAGE_SHIFT;
- loff_t offset = pos & (PAGE_SIZE - 1);
- loff_t page_start = pos & PAGE_MASK;
- loff_t i_size;
- struct page *page;
- int rc = 0;
+ struct folio *folio;
+ int rc;

cifs_dbg(FYI, "write_begin from %lld len %d\n", (long long)pos, len);

-start:
- page = grab_cache_page_write_begin(mapping, index, flags);
- if (!page) {
- rc = -ENOMEM;
- goto out;
- }
-
- if (PageUptodate(page))
- goto out;
-
- /*
- * If we write a full page it will be up to date, no need to read from
- * the server. If the write is short, we'll end up doing a sync write
- * instead.
+ /* Prefetch area to be written into the cache if we're caching this
+ * file. We need to do this before we get a lock on the page in case
+ * there's more than one writer competing for the same cache block.
*/
- if (len == PAGE_SIZE)
- goto out;
+ rc = netfs_write_begin(file, mapping, pos, len, flags, &folio, fsdata);
+ if (rc < 0)
+ return rc;

+#if 0
/*
* optimize away the read when we have an oplock, and we're not
* expecting to use any of the data we'd be reading in. That
@@ -4289,56 +3699,20 @@ static int cifs_write_begin(struct file *file, struct address_space *mapping,
offset + len,
PAGE_SIZE);
/*
- * PageChecked means that the parts of the page
- * to which we're not writing are considered up
- * to date. Once the data is copied to the
- * page, it can be set uptodate.
+ * Marking a folio checked means that the parts of the
+ * page to which we're not writing are considered up to
+ * date. Once the data is copied to the page, it can be
+ * set uptodate.
*/
- SetPageChecked(page);
+ folio_set_checked(folio);
goto out;
}
}
-
- if ((file->f_flags & O_ACCMODE) != O_WRONLY && !oncethru) {
- /*
- * might as well read a page, it is fast enough. If we get
- * an error, we don't need to return it. cifs_write_end will
- * do a sync write instead since PG_uptodate isn't set.
- */
- cifs_readpage_worker(file, page, &page_start);
- put_page(page);
- oncethru = 1;
- goto start;
- } else {
- /* we could try using another file handle if there is one -
- but how would we lock it to prevent close of that handle
- racing with this read? In any case
- this will be written out by write_end so is fine */
- }
-out:
- *pagep = page;
+#endif
+ *pagep = folio_page(folio, (pos - folio_pos(folio)) / PAGE_SIZE);
return rc;
}

-static int cifs_release_page(struct page *page, gfp_t gfp)
-{
- if (PagePrivate(page))
- return 0;
- if (PageFsCache(page)) {
- if (current_is_kswapd() || !(gfp & __GFP_FS))
- return false;
- wait_on_page_fscache(page);
- }
- fscache_note_page_release(cifs_inode_cookie(page->mapping->host));
- return true;
-}
-
-static void cifs_invalidate_folio(struct folio *folio, size_t offset,
- size_t length)
-{
- folio_wait_fscache(folio);
-}
-
static int cifs_launder_folio(struct folio *folio)
{
int rc = 0;
@@ -4528,16 +3902,16 @@ static bool cifs_dirty_folio(struct address_space *mapping, struct folio *folio)
#endif

const struct address_space_operations cifs_addr_ops = {
- .readpage = cifs_readpage,
- .readahead = cifs_readahead,
+ .readpage = netfs_readpage,
+ .readahead = netfs_readahead,
.writepage = cifs_writepage,
.writepages = cifs_writepages,
.write_begin = cifs_write_begin,
.write_end = cifs_write_end,
.dirty_folio = cifs_dirty_folio,
- .releasepage = cifs_release_page,
+ .releasepage = netfs_releasepage,
.direct_IO = cifs_direct_io,
- .invalidate_folio = cifs_invalidate_folio,
+ .invalidate_folio = netfs_invalidate_folio,
.launder_folio = cifs_launder_folio,
/*
* TODO: investigate and if useful we could add an cifs_migratePage
@@ -4554,13 +3928,13 @@ const struct address_space_operations cifs_addr_ops = {
* to leave cifs_readpages out of the address space operations.
*/
const struct address_space_operations cifs_addr_ops_smallbuf = {
- .readpage = cifs_readpage,
+ .readpage = netfs_readpage,
.writepage = cifs_writepage,
.writepages = cifs_writepages,
.write_begin = cifs_write_begin,
.write_end = cifs_write_end,
.dirty_folio = cifs_dirty_folio,
- .releasepage = cifs_release_page,
- .invalidate_folio = cifs_invalidate_folio,
+ .releasepage = netfs_releasepage,
+ .invalidate_folio = netfs_invalidate_folio,
.launder_folio = cifs_launder_folio,
};
diff --git a/fs/cifs/fscache.c b/fs/cifs/fscache.c
index a638b29e9062..bb1c3a372de4 100644
--- a/fs/cifs/fscache.c
+++ b/fs/cifs/fscache.c
@@ -134,120 +134,3 @@ void cifs_fscache_release_inode_cookie(struct inode *inode)
cifsi->netfs_ctx.cache = NULL;
}
}
-
-/*
- * Fallback page reading interface.
- */
-static int fscache_fallback_read_page(struct inode *inode, struct page *page)
-{
- struct netfs_cache_resources cres;
- struct fscache_cookie *cookie = cifs_inode_cookie(inode);
- struct iov_iter iter;
- struct bio_vec bvec[1];
- int ret;
-
- memset(&cres, 0, sizeof(cres));
- bvec[0].bv_page = page;
- bvec[0].bv_offset = 0;
- bvec[0].bv_len = PAGE_SIZE;
- iov_iter_bvec(&iter, READ, bvec, ARRAY_SIZE(bvec), PAGE_SIZE);
-
- ret = fscache_begin_read_operation(&cres, cookie);
- if (ret < 0)
- return ret;
-
- ret = fscache_read(&cres, page_offset(page), &iter, NETFS_READ_HOLE_FAIL,
- NULL, NULL);
- fscache_end_operation(&cres);
- return ret;
-}
-
-/*
- * Fallback page writing interface.
- */
-static int fscache_fallback_write_page(struct inode *inode, struct page *page,
- bool no_space_allocated_yet)
-{
- struct netfs_cache_resources cres;
- struct fscache_cookie *cookie = cifs_inode_cookie(inode);
- struct iov_iter iter;
- struct bio_vec bvec[1];
- loff_t start = page_offset(page);
- size_t len = PAGE_SIZE;
- int ret;
-
- memset(&cres, 0, sizeof(cres));
- bvec[0].bv_page = page;
- bvec[0].bv_offset = 0;
- bvec[0].bv_len = PAGE_SIZE;
- iov_iter_bvec(&iter, WRITE, bvec, ARRAY_SIZE(bvec), PAGE_SIZE);
-
- ret = fscache_begin_write_operation(&cres, cookie);
- if (ret < 0)
- return ret;
-
- ret = cres.ops->prepare_write(&cres, &start, &len, i_size_read(inode),
- no_space_allocated_yet);
- if (ret == 0)
- ret = fscache_write(&cres, page_offset(page), &iter, NULL, NULL);
- fscache_end_operation(&cres);
- return ret;
-}
-
-/*
- * Retrieve a page from FS-Cache
- */
-int __cifs_readpage_from_fscache(struct inode *inode, struct page *page)
-{
- int ret;
-
- cifs_dbg(FYI, "%s: (fsc:%p, p:%p, i:0x%p\n",
- __func__, cifs_inode_cookie(inode), page, inode);
-
- ret = fscache_fallback_read_page(inode, page);
- if (ret < 0)
- return ret;
-
- /* Read completed synchronously */
- SetPageUptodate(page);
- return 0;
-}
-
-void __cifs_readpage_to_fscache(struct inode *inode, struct page *page)
-{
- cifs_dbg(FYI, "%s: (fsc: %p, p: %p, i: %p)\n",
- __func__, cifs_inode_cookie(inode), page, inode);
-
- fscache_fallback_write_page(inode, page, true);
-}
-
-/*
- * Query the cache occupancy.
- */
-int __cifs_fscache_query_occupancy(struct inode *inode,
- pgoff_t first, unsigned int nr_pages,
- pgoff_t *_data_first,
- unsigned int *_data_nr_pages)
-{
- struct netfs_cache_resources cres;
- struct fscache_cookie *cookie = cifs_inode_cookie(inode);
- loff_t start, data_start;
- size_t len, data_len;
- int ret;
-
- ret = fscache_begin_read_operation(&cres, cookie);
- if (ret < 0)
- return ret;
-
- start = first * PAGE_SIZE;
- len = nr_pages * PAGE_SIZE;
- ret = cres.ops->query_occupancy(&cres, start, len, PAGE_SIZE,
- &data_start, &data_len);
- if (ret == 0) {
- *_data_first = data_start / PAGE_SIZE;
- *_data_nr_pages = len / PAGE_SIZE;
- }
-
- fscache_end_operation(&cres);
- return ret;
-}
diff --git a/fs/cifs/fscache.h b/fs/cifs/fscache.h
index 52355c0912ae..db863e757f7a 100644
--- a/fs/cifs/fscache.h
+++ b/fs/cifs/fscache.h
@@ -73,52 +73,6 @@ static inline void cifs_invalidate_cache(struct inode *inode, unsigned int flags
i_size_read(inode), flags);
}

-extern int __cifs_fscache_query_occupancy(struct inode *inode,
- pgoff_t first, unsigned int nr_pages,
- pgoff_t *_data_first,
- unsigned int *_data_nr_pages);
-
-static inline int cifs_fscache_query_occupancy(struct inode *inode,
- pgoff_t first, unsigned int nr_pages,
- pgoff_t *_data_first,
- unsigned int *_data_nr_pages)
-{
- if (!cifs_inode_cookie(inode))
- return -ENOBUFS;
- return __cifs_fscache_query_occupancy(inode, first, nr_pages,
- _data_first, _data_nr_pages);
-}
-
-extern int __cifs_readpage_from_fscache(struct inode *pinode, struct page *ppage);
-extern void __cifs_readpage_to_fscache(struct inode *pinode, struct page *ppage);
-
-
-static inline int cifs_readpage_from_fscache(struct inode *inode,
- struct page *page)
-{
- if (cifs_inode_cookie(inode))
- return __cifs_readpage_from_fscache(inode, page);
- return -ENOBUFS;
-}
-
-static inline void cifs_readpage_to_fscache(struct inode *inode,
- struct page *page)
-{
- if (cifs_inode_cookie(inode))
- __cifs_readpage_to_fscache(inode, page);
-}
-
-static inline int cifs_fscache_release_page(struct page *page, gfp_t gfp)
-{
- if (PageFsCache(page)) {
- if (current_is_kswapd() || !(gfp & __GFP_FS))
- return false;
- wait_on_page_fscache(page);
- fscache_note_page_release(cifs_inode_cookie(page->mapping->host));
- }
- return true;
-}
-
#else /* CONFIG_CIFS_FSCACHE */
static inline
void cifs_fscache_fill_coherency(struct inode *inode,
@@ -135,30 +89,6 @@ static inline void cifs_fscache_unuse_inode_cookie(struct inode *inode, bool upd
static inline struct fscache_cookie *cifs_inode_cookie(struct inode *inode) { return NULL; }
static inline void cifs_invalidate_cache(struct inode *inode, unsigned int flags) {}

-static inline int cifs_fscache_query_occupancy(struct inode *inode,
- pgoff_t first, unsigned int nr_pages,
- pgoff_t *_data_first,
- unsigned int *_data_nr_pages)
-{
- *_data_first = ULONG_MAX;
- *_data_nr_pages = 0;
- return -ENOBUFS;
-}
-
-static inline int
-cifs_readpage_from_fscache(struct inode *inode, struct page *page)
-{
- return -ENOBUFS;
-}
-
-static inline
-void cifs_readpage_to_fscache(struct inode *inode, struct page *page) {}
-
-static inline int nfs_fscache_release_page(struct page *page, gfp_t gfp)
-{
- return true; /* May release page */
-}
-
#endif /* CONFIG_CIFS_FSCACHE */

#endif /* _CIFS_FSCACHE_H */
diff --git a/fs/cifs/inode.c b/fs/cifs/inode.c
index 2f9e7d2f81b6..40c6f04f7521 100644
--- a/fs/cifs/inode.c
+++ b/fs/cifs/inode.c
@@ -26,6 +26,18 @@
#include "fs_context.h"
#include "cifs_ioctl.h"

+/*
+ * Set parameters for the netfs library
+ */
+static void cifs_set_netfs_context(struct inode *inode)
+{
+ struct netfs_i_context *ctx = netfs_i_context(inode);
+ struct cifs_sb_info *cifs_sb = CIFS_SB(inode->i_sb);
+
+ netfs_i_context_init(inode, &cifs_req_ops);
+ ctx->rsize = cifs_sb->ctx->rsize;
+}
+
static void cifs_set_ops(struct inode *inode)
{
struct cifs_sb_info *cifs_sb = CIFS_SB(inode->i_sb);
@@ -213,8 +225,10 @@ cifs_fattr_to_inode(struct inode *inode, struct cifs_fattr *fattr)

if (fattr->cf_flags & CIFS_FATTR_DFS_REFERRAL)
inode->i_flags |= S_AUTOMOUNT;
- if (inode->i_state & I_NEW)
+ if (inode->i_state & I_NEW) {
+ cifs_set_netfs_context(inode);
cifs_set_ops(inode);
+ }
return 0;
}

diff --git a/fs/cifs/smb2pdu.c b/fs/cifs/smb2pdu.c
index 6bb9a90b018f..19ad6c89121a 100644
--- a/fs/cifs/smb2pdu.c
+++ b/fs/cifs/smb2pdu.c
@@ -23,6 +23,7 @@
#include <linux/uuid.h>
#include <linux/pagemap.h>
#include <linux/xattr.h>
+#include <linux/netfs.h>
#include "cifsglob.h"
#include "cifsacl.h"
#include "cifsproto.h"
@@ -4182,7 +4183,19 @@ smb2_readv_callback(struct mid_q_entry *mid)
tcon->tid, tcon->ses->Suid,
rdata->offset, rdata->got_bytes);

- queue_work(cifsiod_wq, &rdata->work);
+ if (rdata->result == -ENODATA) {
+ /* We may have got an EOF error because fallocate
+ * failed to enlarge the file.
+ */
+ if (rdata->subreq->start < rdata->subreq->rreq->i_size)
+ rdata->result = 0;
+ }
+ if (rdata->result == 0 || rdata->result == -EAGAIN)
+ iov_iter_advance(&rdata->subreq->iter, rdata->got_bytes);
+ netfs_subreq_terminated(rdata->subreq,
+ (rdata->result == 0 || rdata->result == -EAGAIN) ?
+ rdata->got_bytes : rdata->result, true);
+ kref_put(&rdata->refcount, cifs_readdata_release);
DeleteMidQEntry(mid);
add_credits(server, &credits, 0);
}
diff --git a/fs/netfs/io.c b/fs/netfs/io.c
index aaaafc3e1601..a39f07c3758b 100644
--- a/fs/netfs/io.c
+++ b/fs/netfs/io.c
@@ -324,8 +324,13 @@ static void netfs_rreq_assess_dio(struct netfs_io_request *rreq)
unsigned int i;
size_t transferred = 0;

- for (i = 0; i < rreq->direct_bv_count; i++)
+ for (i = 0; i < rreq->direct_bv_count; i++) {
flush_dcache_page(rreq->direct_bv[i].bv_page);
+ // TODO: cifs marks pages in the destination buffer
+ // dirty under some circumstances after a read. Do we
+ // need to do that too?
+ set_page_dirty(rreq->direct_bv[i].bv_page);
+ }

list_for_each_entry(subreq, &rreq->subrequests, rreq_link) {
if (subreq->error || subreq->transferred == 0)


2022-04-07 02:44:37

by David Howells

[permalink] [raw]
Subject: [PATCH 11/14] cifs: Clamp length according to credits and rsize

A read request can get expanded beyond the capacity of the available
credits and rsize, so use the ->clamp_length() method to cut the request up
into pieces rather than trying to do it in ->issue_read(), at which point
the subrequest size is already determined.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/cifsglob.h | 2 +
fs/cifs/file.c | 71 ++++++++++++++++++++++++++++++++++++----------------
fs/cifs/smb2pdu.c | 5 ++--
3 files changed, 54 insertions(+), 24 deletions(-)

diff --git a/fs/cifs/cifsglob.h b/fs/cifs/cifsglob.h
index e1e77225d634..2b1930a918b0 100644
--- a/fs/cifs/cifsglob.h
+++ b/fs/cifs/cifsglob.h
@@ -1328,8 +1328,10 @@ struct cifs_io_subrequest {
__u64 offset;
ssize_t got_bytes;
unsigned int bytes;
+ unsigned int xid;
pid_t pid;
int result;
+ bool have_credits;
struct kvec iov[2];
struct TCP_Server_Info *server;
#ifdef CONFIG_CIFS_SMB_DIRECT
diff --git a/fs/cifs/file.c b/fs/cifs/file.c
index 45510bd1f702..12663d9d1e51 100644
--- a/fs/cifs/file.c
+++ b/fs/cifs/file.c
@@ -3437,6 +3437,47 @@ int cifs_file_mmap(struct file *file, struct vm_area_struct *vma)
return rc;
}

+/*
+ * Split the read up according to how many credits we can get for each piece.
+ * It's okay to sleep here if we need to wait for more credit to become
+ * available.
+ *
+ * We also choose the server and allocate an operation ID to be cleaned up
+ * later.
+ */
+static bool cifs_clamp_length(struct netfs_io_subrequest *subreq)
+{
+ struct netfs_io_request *rreq = subreq->rreq;
+ struct TCP_Server_Info *server;
+ struct cifs_io_subrequest *rdata = container_of(subreq, struct cifs_io_subrequest, subreq);
+ struct cifs_io_request *req = container_of(subreq->rreq, struct cifs_io_request, rreq);
+ struct cifs_sb_info *cifs_sb = CIFS_SB(rreq->inode->i_sb);
+ unsigned int rsize = 0;
+ int rc;
+
+ rdata->xid = get_xid();
+
+ server = cifs_pick_channel(tlink_tcon(req->cfile->tlink)->ses);
+ rdata->server = server;
+
+ if (cifs_sb->ctx->rsize == 0)
+ cifs_sb->ctx->rsize =
+ server->ops->negotiate_rsize(tlink_tcon(req->cfile->tlink),
+ cifs_sb->ctx);
+
+
+ rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->rsize, &rsize,
+ &rdata->credits);
+ if (rc) {
+ subreq->error = rc;
+ return false;
+ }
+
+ rdata->have_credits = true;
+ subreq->len = min_t(size_t, subreq->len, rsize);
+ return true;
+}
+
/*
* Issue a read operation on behalf of the netfs helper functions. We're asked
* to make a read of a certain size at a point in the file. We are permitted
@@ -3446,24 +3487,17 @@ int cifs_file_mmap(struct file *file, struct vm_area_struct *vma)
static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
{
struct netfs_io_request *rreq = subreq->rreq;
- struct TCP_Server_Info *server;
struct cifs_io_subrequest *rdata = container_of(subreq, struct cifs_io_subrequest, subreq);
struct cifs_io_request *req = container_of(subreq->rreq, struct cifs_io_request, rreq);
struct cifs_sb_info *cifs_sb = CIFS_SB(rreq->inode->i_sb);
- unsigned int xid;
pid_t pid;
int rc = 0;
- unsigned int rsize;
-
- xid = get_xid();

if (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_RWPIDFORWARD)
pid = req->cfile->pid;
else
pid = current->tgid; // Ummm... This may be a workqueue

- server = cifs_pick_channel(tlink_tcon(req->cfile->tlink)->ses);
-
cifs_dbg(FYI, "%s: op=%08x[%x] mapping=%p len=%zu/%zu\n",
__func__, rreq->debug_id, subreq->debug_index, rreq->mapping,
subreq->transferred, subreq->len);
@@ -3476,32 +3510,20 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
goto out;
}

- if (cifs_sb->ctx->rsize == 0)
- cifs_sb->ctx->rsize =
- server->ops->negotiate_rsize(tlink_tcon(req->cfile->tlink),
- cifs_sb->ctx);
-
- rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->rsize, &rsize,
- &rdata->credits);
- if (rc)
- goto out;
-
__set_bit(NETFS_SREQ_CLEAR_TAIL, &subreq->flags);
- rdata->server = server;
rdata->offset = subreq->start + subreq->transferred;
rdata->bytes = subreq->len - subreq->transferred;
rdata->pid = pid;

- rc = adjust_credits(server, &rdata->credits, rdata->bytes);
+ rc = adjust_credits(rdata->server, &rdata->credits, rdata->bytes);
if (!rc) {
if (rdata->req->cfile->invalidHandle)
rc = -EAGAIN;
else
- rc = server->ops->async_readv(rdata);
+ rc = rdata->server->ops->async_readv(rdata);
}

out:
- free_xid(xid);
if (rc)
netfs_subreq_terminated(subreq, rc, false);
}
@@ -3580,6 +3602,7 @@ static void cifs_free_subrequest(struct netfs_io_subrequest *subreq)
{
struct cifs_io_subrequest *rdata =
container_of(subreq, struct cifs_io_subrequest, subreq);
+ int rc;

if (rdata->subreq.source == NETFS_DOWNLOAD_FROM_SERVER) {
#ifdef CONFIG_CIFS_SMB_DIRECT
@@ -3589,7 +3612,10 @@ static void cifs_free_subrequest(struct netfs_io_subrequest *subreq)
}
#endif

- add_credits_and_wake_if(rdata->server, &rdata->credits, 0);
+ if (rdata->have_credits)
+ add_credits_and_wake_if(rdata->server, &rdata->credits, 0);
+ rc = subreq->error;
+ free_xid(rdata->xid);
}
}

@@ -3601,6 +3627,7 @@ const struct netfs_request_ops cifs_req_ops = {
.free_subrequest = cifs_free_subrequest,
.begin_cache_operation = cifs_begin_cache_operation,
.expand_readahead = cifs_expand_readahead,
+ .clamp_length = cifs_clamp_length,
.issue_read = cifs_req_issue_read,
.done = cifs_rreq_done,
};
diff --git a/fs/cifs/smb2pdu.c b/fs/cifs/smb2pdu.c
index 6a8aaa003e54..952f242bee55 100644
--- a/fs/cifs/smb2pdu.c
+++ b/fs/cifs/smb2pdu.c
@@ -4174,7 +4174,7 @@ smb2_readv_callback(struct mid_q_entry *mid)
#endif
if (rdata->result && rdata->result != -ENODATA) {
cifs_stats_fail_inc(tcon, SMB2_READ_HE);
- trace_smb3_read_err(0 /* xid */,
+ trace_smb3_read_err(rdata->xid,
rdata->req->cfile->fid.persistent_fid,
tcon->tid, tcon->ses->Suid, rdata->offset,
rdata->bytes, rdata->result);
@@ -4193,6 +4193,7 @@ smb2_readv_callback(struct mid_q_entry *mid)
}
if (rdata->result == 0 || rdata->result == -EAGAIN)
iov_iter_advance(&rdata->subreq.iter, rdata->got_bytes);
+ rdata->have_credits = false;
netfs_subreq_terminated(&rdata->subreq,
(rdata->result == 0 || rdata->result == -EAGAIN) ?
rdata->got_bytes : rdata->result, true);
@@ -4259,7 +4260,7 @@ smb2_async_readv(struct cifs_io_subrequest *rdata)
&rdata->credits);
if (rc) {
cifs_stats_fail_inc(io_parms.tcon, SMB2_READ_HE);
- trace_smb3_read_err(0 /* xid */, io_parms.persistent_fid,
+ trace_smb3_read_err(rdata->xid, io_parms.persistent_fid,
io_parms.tcon->tid,
io_parms.tcon->ses->Suid,
io_parms.offset, io_parms.length, rc);


2022-04-07 03:07:56

by David Howells

[permalink] [raw]
Subject: [PATCH 02/14] cifs: Add a function to read into an iter from a socket

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/cifsproto.h | 3 +++
fs/cifs/connect.c | 16 ++++++++++++++++
2 files changed, 19 insertions(+)

diff --git a/fs/cifs/cifsproto.h b/fs/cifs/cifsproto.h
index 0df3b24a0bf4..dd4931308ea6 100644
--- a/fs/cifs/cifsproto.h
+++ b/fs/cifs/cifsproto.h
@@ -245,6 +245,9 @@ extern int cifs_read_page_from_socket(struct TCP_Server_Info *server,
unsigned int page_offset,
unsigned int to_read);
extern int cifs_setup_cifs_sb(struct cifs_sb_info *cifs_sb);
+extern int cifs_read_iter_from_socket(struct TCP_Server_Info *server,
+ struct iov_iter *iter,
+ unsigned int to_read);
extern int cifs_match_super(struct super_block *, void *);
extern int cifs_mount(struct cifs_sb_info *cifs_sb, struct smb3_fs_context *ctx);
extern void cifs_umount(struct cifs_sb_info *);
diff --git a/fs/cifs/connect.c b/fs/cifs/connect.c
index ee3b7c15e884..080e19e9ee47 100644
--- a/fs/cifs/connect.c
+++ b/fs/cifs/connect.c
@@ -754,6 +754,22 @@ cifs_read_page_from_socket(struct TCP_Server_Info *server, struct page *page,
return cifs_readv_from_socket(server, &smb_msg);
}

+int
+cifs_read_iter_from_socket(struct TCP_Server_Info *server, struct iov_iter *iter,
+ unsigned int to_read)
+{
+ struct msghdr smb_msg;
+ int ret;
+
+ smb_msg.msg_iter = *iter;
+ if (smb_msg.msg_iter.count > to_read)
+ smb_msg.msg_iter.count = to_read;
+ ret = cifs_readv_from_socket(server, &smb_msg);
+ if (ret > 0)
+ iov_iter_advance(iter, ret);
+ return ret;
+}
+
static bool
is_smb_response(struct TCP_Server_Info *server, unsigned char type)
{


2022-04-07 03:12:15

by David Howells

[permalink] [raw]
Subject: [PATCH 09/14] cifs: Put credits into cifs_io_subrequest, not on the stack

Move the credits into the cifs_io_subrequest struct rather than placing them
on the stack. They are then allocated by netfslib when it allocates its
netfs_io_subrequest.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/file.c | 12 ++++--------
1 file changed, 4 insertions(+), 8 deletions(-)

diff --git a/fs/cifs/file.c b/fs/cifs/file.c
index fb2885134154..fc3a46f7e2cf 100644
--- a/fs/cifs/file.c
+++ b/fs/cifs/file.c
@@ -3450,7 +3450,6 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
struct cifs_io_subrequest *rdata = container_of(subreq, struct cifs_io_subrequest, subreq);
struct cifsFileInfo *open_file = rreq->netfs_priv;
struct cifs_sb_info *cifs_sb = CIFS_SB(rreq->inode->i_sb);
- struct cifs_credits credits_on_stack, *credits = &credits_on_stack;
unsigned int xid;
pid_t pid;
int rc = 0;
@@ -3482,7 +3481,8 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
server->ops->negotiate_rsize(tlink_tcon(open_file->tlink),
cifs_sb->ctx);

- rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->rsize, &rsize, credits);
+ rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->rsize, &rsize,
+ &rdata->credits);
if (rc)
goto out;

@@ -3492,7 +3492,6 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
rdata->offset = subreq->start + subreq->transferred;
rdata->bytes = subreq->len - subreq->transferred;
rdata->pid = pid;
- rdata->credits = credits_on_stack;

rc = adjust_credits(server, &rdata->credits, rdata->bytes);
if (!rc) {
@@ -3502,11 +3501,6 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
rc = server->ops->async_readv(rdata);
}

- if (rc) {
- add_credits_and_wake_if(server, &rdata->credits, 0);
- /* Fallback to the readpage in error/reconnect cases */
- }
-
out:
free_xid(xid);
if (rc)
@@ -3584,6 +3578,8 @@ static void cifs_free_subrequest(struct netfs_io_subrequest *subreq)
rdata->mr = NULL;
}
#endif
+
+ add_credits_and_wake_if(rdata->server, &rdata->credits, 0);
if (rdata->cfile)
cifsFileInfo_put(rdata->cfile);
}


2022-04-07 05:32:20

by David Howells

[permalink] [raw]
Subject: [PATCH 14/14] mm, netfs, fscache: Stop read optimisation when folio removed from pagecache

Fscache has an optimisation by which reads from the cache are skipped until
we know that (a) there's data there to be read and (b) that data isn't
entirely covered by pages resident in the netfs pagecache. This is done
with two flags manipulated by fscache_note_page_release():

if (...
test_bit(FSCACHE_COOKIE_HAVE_DATA, &cookie->flags) &&
test_bit(FSCACHE_COOKIE_NO_DATA_TO_READ, &cookie->flags))
clear_bit(FSCACHE_COOKIE_NO_DATA_TO_READ, &cookie->flags);

where the NO_DATA_TO_READ flag causes cachefiles_prepare_read() to indicate
that netfslib should download from the server or clear the page instead.

The fscache_note_page_release() function is intended to be called from
->releasepage() - but that only gets called if PG_private or PG_private_2
is set - and currently the former is at the discretion of the netfs and the
latter is only set whilst a page is being written to the cache, so
sometimes we miss clearing the optimisation.

Fix this by adding an extra address_space operation, ->removing folio(),
and flag, AS_NOTIFY_REMOVING_FOLIO. The operation is called if the flag is
set when a folio is removed from the pagecache. The flag should be set if
a non-NULL cookie is obtained from fscache and cleared in ->evict_inode()
before truncate_inode_pages_final() is called.

This can be tested by reading some data and then dropping caches. The
number of times the new op is called is counted in /proc/fs/fscache/stats:

RdHelp : DR=0 RA=4100 RP=0 WB=0 WBZ=0 rm=131072 <----

Reported-by: Rohith Surabattula <[email protected]>
Signed-off-by: David Howells <[email protected]>
cc: Matthew Wilcox <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/afs/file.c | 1 +
fs/afs/inode.c | 1 +
fs/afs/internal.h | 2 ++
fs/cifs/cifsfs.c | 1 +
fs/cifs/file.c | 1 +
fs/cifs/fscache.c | 3 +++
fs/netfs/internal.h | 1 +
fs/netfs/misc.c | 13 ++++++++++++-
fs/netfs/stats.c | 9 ++++++---
include/linux/fs.h | 2 ++
include/linux/netfs.h | 1 +
include/linux/pagemap.h | 1 +
mm/filemap.c | 15 +++++++++++++++
13 files changed, 47 insertions(+), 4 deletions(-)

diff --git a/fs/afs/file.c b/fs/afs/file.c
index 1398b3fc6fef..b833f440903d 100644
--- a/fs/afs/file.c
+++ b/fs/afs/file.c
@@ -54,6 +54,7 @@ const struct address_space_operations afs_file_aops = {
.launder_folio = afs_launder_folio,
.releasepage = netfs_releasepage,
.invalidate_folio = netfs_invalidate_folio,
+ .removing_folio = netfs_removing_folio,
.write_begin = afs_write_begin,
.write_end = afs_write_end,
.writepage = afs_writepage,
diff --git a/fs/afs/inode.c b/fs/afs/inode.c
index 12d2438661fc..6b8891402df5 100644
--- a/fs/afs/inode.c
+++ b/fs/afs/inode.c
@@ -790,6 +790,7 @@ void afs_evict_inode(struct inode *inode)

ASSERTCMP(inode->i_ino, ==, vnode->fid.vnode);

+ clear_bit(AS_NOTIFY_REMOVING_FOLIO, &inode->i_mapping->flags);
truncate_inode_pages_final(&inode->i_data);

afs_set_cache_aux(vnode, &aux);
diff --git a/fs/afs/internal.h b/fs/afs/internal.h
index 7b7ef945dc78..7756217dab09 100644
--- a/fs/afs/internal.h
+++ b/fs/afs/internal.h
@@ -686,6 +686,8 @@ static inline void afs_vnode_set_cache(struct afs_vnode *vnode,
{
#ifdef CONFIG_AFS_FSCACHE
vnode->netfs_ctx.cache = cookie;
+ if (cookie)
+ set_bit(AS_NOTIFY_REMOVING_FOLIO, &vnode->vfs_inode.i_mapping->flags);
#endif
}

diff --git a/fs/cifs/cifsfs.c b/fs/cifs/cifsfs.c
index d07cb28652f2..125d303c5038 100644
--- a/fs/cifs/cifsfs.c
+++ b/fs/cifs/cifsfs.c
@@ -404,6 +404,7 @@ cifs_free_inode(struct inode *inode)
static void
cifs_evict_inode(struct inode *inode)
{
+ clear_bit(AS_NOTIFY_REMOVING_FOLIO, &inode->i_mapping->flags);
truncate_inode_pages_final(&inode->i_data);
if (inode->i_state & I_PINNING_FSCACHE_WB)
cifs_fscache_unuse_inode_cookie(inode, true);
diff --git a/fs/cifs/file.c b/fs/cifs/file.c
index 12663d9d1e51..71f96701e602 100644
--- a/fs/cifs/file.c
+++ b/fs/cifs/file.c
@@ -3923,6 +3923,7 @@ const struct address_space_operations cifs_addr_ops = {
.direct_IO = cifs_direct_io,
.invalidate_folio = netfs_invalidate_folio,
.launder_folio = cifs_launder_folio,
+ .removing_folio = netfs_removing_folio,
/*
* TODO: investigate and if useful we could add an cifs_migratePage
* helper (under an CONFIG_MIGRATION) in the future, and also
diff --git a/fs/cifs/fscache.c b/fs/cifs/fscache.c
index bb1c3a372de4..5d8464bed798 100644
--- a/fs/cifs/fscache.c
+++ b/fs/cifs/fscache.c
@@ -108,6 +108,9 @@ void cifs_fscache_get_inode_cookie(struct inode *inode)
&cifsi->uniqueid, sizeof(cifsi->uniqueid),
&cd, sizeof(cd),
i_size_read(&cifsi->vfs_inode));
+ if (cifsi->netfs_ctx.cache)
+ set_bit(AS_NOTIFY_REMOVING_FOLIO, inode->i_mapping->flags);
+
}

void cifs_fscache_unuse_inode_cookie(struct inode *inode, bool update)
diff --git a/fs/netfs/internal.h b/fs/netfs/internal.h
index bc4db520d5c7..f373b9da6258 100644
--- a/fs/netfs/internal.h
+++ b/fs/netfs/internal.h
@@ -114,6 +114,7 @@ extern atomic_t netfs_n_rh_write_begin;
extern atomic_t netfs_n_rh_write_done;
extern atomic_t netfs_n_rh_write_failed;
extern atomic_t netfs_n_rh_write_zskip;
+extern atomic_t netfs_n_rh_remove_folio;


static inline void netfs_stat(atomic_t *stat)
diff --git a/fs/netfs/misc.c b/fs/netfs/misc.c
index a9e8abfa89f8..59ac380c2b08 100644
--- a/fs/netfs/misc.c
+++ b/fs/netfs/misc.c
@@ -178,7 +178,18 @@ int netfs_releasepage(struct page *page, gfp_t gfp)
folio_wait_fscache(folio);
}

- fscache_note_page_release(netfs_i_cookie(folio_inode(folio)));
return true;
}
EXPORT_SYMBOL(netfs_releasepage);
+
+/**
+ * netfs_removing_folio - Notification of a folio about to be removed
+ * @mapping: The pagecache about to be altered
+ * @folio: The folio about to be removed
+ */
+void netfs_removing_folio(struct address_space *mapping, struct folio *folio)
+{
+ netfs_stat(&netfs_n_rh_remove_folio);
+ fscache_note_page_release(netfs_i_cookie(mapping->host));
+}
+EXPORT_SYMBOL(netfs_removing_folio);
diff --git a/fs/netfs/stats.c b/fs/netfs/stats.c
index 85aef51cc6cf..9a0c49540979 100644
--- a/fs/netfs/stats.c
+++ b/fs/netfs/stats.c
@@ -28,17 +28,20 @@ atomic_t netfs_n_rh_write_begin;
atomic_t netfs_n_rh_write_done;
atomic_t netfs_n_rh_write_failed;
atomic_t netfs_n_rh_write_zskip;
+atomic_t netfs_n_rh_remove_folio;

void netfs_stats_show(struct seq_file *m)
{
- seq_printf(m, "RdHelp : DR=%u RA=%u RP=%u WB=%u WBZ=%u rr=%u sr=%u\n",
+ seq_printf(m, "RdHelp : rr=%u sr=%u\n",
+ atomic_read(&netfs_n_rh_rreq),
+ atomic_read(&netfs_n_rh_sreq));
+ seq_printf(m, "RdHelp : DR=%u RA=%u RP=%u WB=%u WBZ=%u rm=%u\n",
atomic_read(&netfs_n_rh_dio_read),
atomic_read(&netfs_n_rh_readahead),
atomic_read(&netfs_n_rh_readpage),
atomic_read(&netfs_n_rh_write_begin),
atomic_read(&netfs_n_rh_write_zskip),
- atomic_read(&netfs_n_rh_rreq),
- atomic_read(&netfs_n_rh_sreq));
+ atomic_read(&netfs_n_rh_remove_folio));
seq_printf(m, "RdHelp : ZR=%u sh=%u sk=%u\n",
atomic_read(&netfs_n_rh_zero),
atomic_read(&netfs_n_rh_short_read),
diff --git a/include/linux/fs.h b/include/linux/fs.h
index bbde95387a23..269327d36bd7 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -360,6 +360,8 @@ struct address_space_operations {
sector_t (*bmap)(struct address_space *, sector_t);
void (*invalidate_folio) (struct folio *, size_t offset, size_t len);
int (*releasepage) (struct page *, gfp_t);
+ void (*removing_folio)(struct address_space *mapping,
+ struct folio *folio);
void (*freepage)(struct page *);
ssize_t (*direct_IO)(struct kiocb *, struct iov_iter *iter);
/*
diff --git a/include/linux/netfs.h b/include/linux/netfs.h
index b503d13061f4..717114531da9 100644
--- a/include/linux/netfs.h
+++ b/include/linux/netfs.h
@@ -316,6 +316,7 @@ extern int netfs_write_begin(struct file *, struct address_space *,
void **);
extern void netfs_invalidate_folio(struct folio *folio, size_t offset, size_t length);
extern int netfs_releasepage(struct page *page, gfp_t gfp_flags);
+extern void netfs_removing_folio(struct address_space *mapping, struct folio *folio);

extern void netfs_subreq_terminated(struct netfs_io_subrequest *, ssize_t, bool);
extern void netfs_get_subrequest(struct netfs_io_subrequest *subreq,
diff --git a/include/linux/pagemap.h b/include/linux/pagemap.h
index 993994cd943a..976b56180700 100644
--- a/include/linux/pagemap.h
+++ b/include/linux/pagemap.h
@@ -199,6 +199,7 @@ enum mapping_flags {
/* writeback related tags are not used */
AS_NO_WRITEBACK_TAGS = 5,
AS_LARGE_FOLIO_SUPPORT = 6,
+ AS_NOTIFY_REMOVING_FOLIO, /* Call aops->removing_folio() */
};

/**
diff --git a/mm/filemap.c b/mm/filemap.c
index 3a5ffb5587cd..1cacf11369d9 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -209,6 +209,20 @@ static void filemap_unaccount_folio(struct address_space *mapping,
folio_account_cleaned(folio, inode_to_wb(mapping->host));
}

+/*
+ * Note that a page is about to be removed from the pagecache. If a page that
+ * is about to be removed had been copied to the cache, then in future fscache
+ * won't be able to skip checking in the cache. We do, however, need to do the
+ * notification before the removal lest we race with the page being brought
+ * back again.
+ */
+static void fscache_notify_removing_page(struct address_space *mapping,
+ struct folio *folio)
+{
+ if (unlikely(test_bit(AS_NOTIFY_REMOVING_FOLIO, &mapping->flags)))
+ mapping->a_ops->removing_folio(mapping, folio);
+}
+
/*
* Delete a page from the page cache and free it. Caller has to make
* sure the page is locked and that nobody else uses it - or that usage
@@ -219,6 +233,7 @@ void __filemap_remove_folio(struct folio *folio, void *shadow)
struct address_space *mapping = folio->mapping;

trace_mm_filemap_delete_from_page_cache(folio);
+ fscache_notify_removing_page(mapping, folio);
filemap_unaccount_folio(mapping, folio);
page_cache_delete(mapping, folio, shadow);
}


2022-04-07 07:39:15

by David Howells

[permalink] [raw]
Subject: [PATCH 01/14] cifs: Add some helper functions

Add some helper functions to manipulate the folio marks by iterating
through a list of folios held in an xarray rather than using a page list.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/cifsfs.h | 3 ++
fs/cifs/cifssmb.c | 85 +++++++++++++++++++++++++++++++++++++++++++++++++++++
2 files changed, 88 insertions(+)

diff --git a/fs/cifs/cifsfs.h b/fs/cifs/cifsfs.h
index 15a5c5db038b..1c77bbc0815f 100644
--- a/fs/cifs/cifsfs.h
+++ b/fs/cifs/cifsfs.h
@@ -110,6 +110,9 @@ extern int cifs_file_strict_mmap(struct file * , struct vm_area_struct *);
extern const struct file_operations cifs_dir_ops;
extern int cifs_dir_open(struct inode *inode, struct file *file);
extern int cifs_readdir(struct file *file, struct dir_context *ctx);
+extern void cifs_pages_written_back(struct inode *inode, loff_t start, unsigned int len);
+extern void cifs_pages_write_failed(struct inode *inode, loff_t start, unsigned int len);
+extern void cifs_pages_write_redirty(struct inode *inode, loff_t start, unsigned int len);

/* Functions related to dir entries */
extern const struct dentry_operations cifs_dentry_ops;
diff --git a/fs/cifs/cifssmb.c b/fs/cifs/cifssmb.c
index 47e927c4ff8d..88e2de74f74f 100644
--- a/fs/cifs/cifssmb.c
+++ b/fs/cifs/cifssmb.c
@@ -27,6 +27,7 @@
#include "cifsglob.h"
#include "cifsacl.h"
#include "cifsproto.h"
+#include "cifsfs.h"
#include "cifs_unicode.h"
#include "cifs_debug.h"
#include "smb2proto.h"
@@ -1928,6 +1929,90 @@ cifs_writedata_release(struct kref *refcount)
kfree(wdata);
}

+/*
+ * Completion of write to server.
+ */
+void cifs_pages_written_back(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(&xas, folio, end) {
+ if (!folio_test_writeback(folio)) {
+ pr_err("bad %x @%llx page %lx %lx\n",
+ len, start, folio_index(folio), end);
+ BUG();
+ }
+
+ folio_detach_private(folio);
+ folio_end_writeback(folio);
+ }
+
+ rcu_read_unlock();
+}
+
+/*
+ * Failure of write to server.
+ */
+void cifs_pages_write_failed(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(&xas, folio, end) {
+ if (!folio_test_writeback(folio)) {
+ pr_err("bad %x @%llx page %lx %lx\n",
+ len, start, folio_index(folio), end);
+ BUG();
+ }
+
+ folio_set_error(folio);
+ folio_end_writeback(folio);
+ }
+
+ rcu_read_unlock();
+}
+
+/*
+ * Redirty pages after a temporary failure.
+ */
+void cifs_pages_write_redirty(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(&xas, folio, end) {
+ if (!folio_test_writeback(folio)) {
+ pr_err("bad %x @%llx page %lx %lx\n",
+ len, start, folio_index(folio), end);
+ BUG();
+ }
+
+ filemap_dirty_folio(folio->mapping, folio);
+ folio_end_writeback(folio);
+ }
+
+ rcu_read_unlock();
+}
+
/*
* Write failed with a retryable error. Resend the write request. It's also
* possible that the page was redirtied so re-clean the page.


2022-04-07 08:31:58

by David Howells

[permalink] [raw]
Subject: [PATCH 10/14] cifs: Hold the open file on netfs_io_request, not netfs_io_subrequest

Hold the file open on the netfs_io_request rather than on the
netfs_io_subrequest struct, thereby sharing the open file with all
subrequests contributing to a request.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/cifsglob.h | 12 ++++++++++--
fs/cifs/cifssmb.c | 6 +++---
fs/cifs/file.c | 32 +++++++++++++++++++++-----------
fs/cifs/smb2pdu.c | 14 +++++++-------
4 files changed, 41 insertions(+), 23 deletions(-)

diff --git a/fs/cifs/cifsglob.h b/fs/cifs/cifsglob.h
index 61b40721178a..e1e77225d634 100644
--- a/fs/cifs/cifsglob.h
+++ b/fs/cifs/cifsglob.h
@@ -1313,10 +1313,18 @@ struct cifs_aio_ctx {
bool direct_io;
};

+struct cifs_io_request {
+ struct netfs_io_request rreq;
+ struct cifsFileInfo *cfile;
+};
+
/* asynchronous read support */
struct cifs_io_subrequest {
- struct netfs_io_subrequest subreq;
- struct cifsFileInfo *cfile;
+ union {
+ struct netfs_io_subrequest subreq;
+ struct netfs_io_request *rreq;
+ struct cifs_io_request *req;
+ };
__u64 offset;
ssize_t got_bytes;
unsigned int bytes;
diff --git a/fs/cifs/cifssmb.c b/fs/cifs/cifssmb.c
index f736493746b4..0c039eee1234 100644
--- a/fs/cifs/cifssmb.c
+++ b/fs/cifs/cifssmb.c
@@ -1570,7 +1570,7 @@ static void
cifs_readv_callback(struct mid_q_entry *mid)
{
struct cifs_io_subrequest *rdata = mid->callback_data;
- struct cifs_tcon *tcon = tlink_tcon(rdata->cfile->tlink);
+ struct cifs_tcon *tcon = tlink_tcon(rdata->req->cfile->tlink);
struct TCP_Server_Info *server = tcon->ses->server;
struct smb_rqst rqst = { .rq_iov = rdata->iov,
.rq_nvec = 2,
@@ -1628,7 +1628,7 @@ cifs_async_readv(struct cifs_io_subrequest *rdata)
int rc;
READ_REQ *smb = NULL;
int wct;
- struct cifs_tcon *tcon = tlink_tcon(rdata->cfile->tlink);
+ struct cifs_tcon *tcon = tlink_tcon(rdata->req->cfile->tlink);
struct smb_rqst rqst = { .rq_iov = rdata->iov,
.rq_nvec = 2 };

@@ -1653,7 +1653,7 @@ cifs_async_readv(struct cifs_io_subrequest *rdata)
smb->hdr.PidHigh = cpu_to_le16((__u16)(rdata->pid >> 16));

smb->AndXCommand = 0xFF; /* none */
- smb->Fid = rdata->cfile->fid.netfid;
+ smb->Fid = rdata->req->cfile->fid.netfid;
smb->OffsetLow = cpu_to_le32(rdata->offset & 0xFFFFFFFF);
if (wct == 12)
smb->OffsetHigh = cpu_to_le32(rdata->offset >> 32);
diff --git a/fs/cifs/file.c b/fs/cifs/file.c
index fc3a46f7e2cf..45510bd1f702 100644
--- a/fs/cifs/file.c
+++ b/fs/cifs/file.c
@@ -3448,7 +3448,7 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
struct netfs_io_request *rreq = subreq->rreq;
struct TCP_Server_Info *server;
struct cifs_io_subrequest *rdata = container_of(subreq, struct cifs_io_subrequest, subreq);
- struct cifsFileInfo *open_file = rreq->netfs_priv;
+ struct cifs_io_request *req = container_of(subreq->rreq, struct cifs_io_request, rreq);
struct cifs_sb_info *cifs_sb = CIFS_SB(rreq->inode->i_sb);
unsigned int xid;
pid_t pid;
@@ -3458,19 +3458,19 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
xid = get_xid();

if (cifs_sb->mnt_cifs_flags & CIFS_MOUNT_RWPIDFORWARD)
- pid = open_file->pid;
+ pid = req->cfile->pid;
else
pid = current->tgid; // Ummm... This may be a workqueue

- server = cifs_pick_channel(tlink_tcon(open_file->tlink)->ses);
+ server = cifs_pick_channel(tlink_tcon(req->cfile->tlink)->ses);

cifs_dbg(FYI, "%s: op=%08x[%x] mapping=%p len=%zu/%zu\n",
__func__, rreq->debug_id, subreq->debug_index, rreq->mapping,
subreq->transferred, subreq->len);

- if (open_file->invalidHandle) {
+ if (req->cfile->invalidHandle) {
do {
- rc = cifs_reopen_file(open_file, true);
+ rc = cifs_reopen_file(req->cfile, true);
} while (rc == -EAGAIN);
if (rc)
goto out;
@@ -3478,7 +3478,7 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)

if (cifs_sb->ctx->rsize == 0)
cifs_sb->ctx->rsize =
- server->ops->negotiate_rsize(tlink_tcon(open_file->tlink),
+ server->ops->negotiate_rsize(tlink_tcon(req->cfile->tlink),
cifs_sb->ctx);

rc = server->ops->wait_mtu_credits(server, cifs_sb->ctx->rsize, &rsize,
@@ -3487,7 +3487,6 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
goto out;

__set_bit(NETFS_SREQ_CLEAR_TAIL, &subreq->flags);
- rdata->cfile = cifsFileInfo_get(open_file);
rdata->server = server;
rdata->offset = subreq->start + subreq->transferred;
rdata->bytes = subreq->len - subreq->transferred;
@@ -3495,7 +3494,7 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)

rc = adjust_credits(server, &rdata->credits, rdata->bytes);
if (!rc) {
- if (rdata->cfile->invalidHandle)
+ if (rdata->req->cfile->invalidHandle)
rc = -EAGAIN;
else
rc = server->ops->async_readv(rdata);
@@ -3509,7 +3508,10 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)

static int cifs_init_request(struct netfs_io_request *rreq, struct file *file)
{
- rreq->netfs_priv = file->private_data;
+ struct cifs_io_request *req = container_of(rreq, struct cifs_io_request, rreq);
+ struct cifsFileInfo *open_file = file->private_data;
+
+ req->cfile = cifsFileInfo_get(open_file);
return 0;
}

@@ -3566,6 +3568,14 @@ static int cifs_begin_cache_operation(struct netfs_io_request *rreq)
#endif
}

+static void cifs_free_request(struct netfs_io_request *rreq)
+{
+ struct cifs_io_request *req = container_of(rreq, struct cifs_io_request, rreq);
+
+ if (req->cfile)
+ cifsFileInfo_put(req->cfile);
+}
+
static void cifs_free_subrequest(struct netfs_io_subrequest *subreq)
{
struct cifs_io_subrequest *rdata =
@@ -3580,14 +3590,14 @@ static void cifs_free_subrequest(struct netfs_io_subrequest *subreq)
#endif

add_credits_and_wake_if(rdata->server, &rdata->credits, 0);
- if (rdata->cfile)
- cifsFileInfo_put(rdata->cfile);
}
}

const struct netfs_request_ops cifs_req_ops = {
+ .io_request_size = sizeof(struct cifs_io_request),
.io_subrequest_size = sizeof(struct cifs_io_subrequest),
.init_request = cifs_init_request,
+ .free_request = cifs_free_request,
.free_subrequest = cifs_free_subrequest,
.begin_cache_operation = cifs_begin_cache_operation,
.expand_readahead = cifs_expand_readahead,
diff --git a/fs/cifs/smb2pdu.c b/fs/cifs/smb2pdu.c
index 696e38da9ae7..6a8aaa003e54 100644
--- a/fs/cifs/smb2pdu.c
+++ b/fs/cifs/smb2pdu.c
@@ -4110,7 +4110,7 @@ static void
smb2_readv_callback(struct mid_q_entry *mid)
{
struct cifs_io_subrequest *rdata = mid->callback_data;
- struct cifs_tcon *tcon = tlink_tcon(rdata->cfile->tlink);
+ struct cifs_tcon *tcon = tlink_tcon(rdata->req->cfile->tlink);
struct TCP_Server_Info *server = rdata->server;
struct smb2_hdr *shdr =
(struct smb2_hdr *)rdata->iov[0].iov_base;
@@ -4175,12 +4175,12 @@ smb2_readv_callback(struct mid_q_entry *mid)
if (rdata->result && rdata->result != -ENODATA) {
cifs_stats_fail_inc(tcon, SMB2_READ_HE);
trace_smb3_read_err(0 /* xid */,
- rdata->cfile->fid.persistent_fid,
+ rdata->req->cfile->fid.persistent_fid,
tcon->tid, tcon->ses->Suid, rdata->offset,
rdata->bytes, rdata->result);
} else
trace_smb3_read_done(0 /* xid */,
- rdata->cfile->fid.persistent_fid,
+ rdata->req->cfile->fid.persistent_fid,
tcon->tid, tcon->ses->Suid,
rdata->offset, rdata->got_bytes);

@@ -4211,7 +4211,7 @@ smb2_async_readv(struct cifs_io_subrequest *rdata)
struct smb_rqst rqst = { .rq_iov = rdata->iov,
.rq_nvec = 1 };
struct TCP_Server_Info *server;
- struct cifs_tcon *tcon = tlink_tcon(rdata->cfile->tlink);
+ struct cifs_tcon *tcon = tlink_tcon(rdata->req->cfile->tlink);
unsigned int total_len;

cifs_dbg(FYI, "%s: offset=%llu bytes=%u\n",
@@ -4220,12 +4220,12 @@ smb2_async_readv(struct cifs_io_subrequest *rdata)
if (!rdata->server)
rdata->server = cifs_pick_channel(tcon->ses);

- io_parms.tcon = tlink_tcon(rdata->cfile->tlink);
+ io_parms.tcon = tlink_tcon(rdata->req->cfile->tlink);
io_parms.server = server = rdata->server;
io_parms.offset = rdata->offset;
io_parms.length = rdata->bytes;
- io_parms.persistent_fid = rdata->cfile->fid.persistent_fid;
- io_parms.volatile_fid = rdata->cfile->fid.volatile_fid;
+ io_parms.persistent_fid = rdata->req->cfile->fid.persistent_fid;
+ io_parms.volatile_fid = rdata->req->cfile->fid.volatile_fid;
io_parms.pid = rdata->pid;

rc = smb2_new_read_req(


2022-04-07 09:24:07

by David Howells

[permalink] [raw]
Subject: [PATCH 12/14] cifs: Expose netfs subrequest debug ID and index in read tracepoints

Expose the netfs request debug ID and subrequest index in the
smb3_read_enter, smb3_read_done and smb3_read_err tracepoints to match the
netfs tracepoints for convenient grepping.

Also expose the xid in some tracepoints where it is now available by virtue
of being stashed in the cifs_io_subrequest struct.

In future, the write tracepoints should be able to expose this also.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/smb2pdu.c | 35 ++++++++-----
fs/cifs/trace.h | 144 ++++++++++++++++++++++++++++++++++++++++++++++-------
2 files changed, 147 insertions(+), 32 deletions(-)

diff --git a/fs/cifs/smb2pdu.c b/fs/cifs/smb2pdu.c
index 952f242bee55..9a828b56d95a 100644
--- a/fs/cifs/smb2pdu.c
+++ b/fs/cifs/smb2pdu.c
@@ -4041,10 +4041,12 @@ smb2_new_read_req(void **buf, unsigned int *total_len,
req->Length = cpu_to_le32(io_parms->length);
req->Offset = cpu_to_le64(io_parms->offset);

- trace_smb3_read_enter(0 /* xid */,
- io_parms->persistent_fid,
- io_parms->tcon->tid, io_parms->tcon->ses->Suid,
- io_parms->offset, io_parms->length);
+ trace_smb3_read_enter(rdata ? rdata->rreq->debug_id : 0,
+ rdata ? rdata->subreq.debug_index : 0,
+ rdata ? rdata->xid : 0,
+ io_parms->persistent_fid,
+ io_parms->tcon->tid, io_parms->tcon->ses->Suid,
+ io_parms->offset, io_parms->length);
#ifdef CONFIG_CIFS_SMB_DIRECT
/*
* If we want to do a RDMA write, fill in and append
@@ -4174,12 +4176,16 @@ smb2_readv_callback(struct mid_q_entry *mid)
#endif
if (rdata->result && rdata->result != -ENODATA) {
cifs_stats_fail_inc(tcon, SMB2_READ_HE);
- trace_smb3_read_err(rdata->xid,
+ trace_smb3_read_err(rdata->rreq->debug_id,
+ rdata->subreq.debug_index,
+ rdata->xid,
rdata->req->cfile->fid.persistent_fid,
tcon->tid, tcon->ses->Suid, rdata->offset,
rdata->bytes, rdata->result);
} else
- trace_smb3_read_done(0 /* xid */,
+ trace_smb3_read_done(rdata->rreq->debug_id,
+ rdata->subreq.debug_index,
+ rdata->xid,
rdata->req->cfile->fid.persistent_fid,
tcon->tid, tcon->ses->Suid,
rdata->offset, rdata->got_bytes);
@@ -4260,7 +4266,9 @@ smb2_async_readv(struct cifs_io_subrequest *rdata)
&rdata->credits);
if (rc) {
cifs_stats_fail_inc(io_parms.tcon, SMB2_READ_HE);
- trace_smb3_read_err(rdata->xid, io_parms.persistent_fid,
+ trace_smb3_read_err(rdata->rreq->debug_id,
+ rdata->subreq.debug_index,
+ rdata->xid, io_parms.persistent_fid,
io_parms.tcon->tid,
io_parms.tcon->ses->Suid,
io_parms.offset, io_parms.length, rc);
@@ -4311,22 +4319,23 @@ SMB2_read(const unsigned int xid, struct cifs_io_parms *io_parms,
if (rc != -ENODATA) {
cifs_stats_fail_inc(io_parms->tcon, SMB2_READ_HE);
cifs_dbg(VFS, "Send error in read = %d\n", rc);
- trace_smb3_read_err(xid,
+ trace_smb3_read_err(0, 0, xid,
req->PersistentFileId,
io_parms->tcon->tid, ses->Suid,
io_parms->offset, io_parms->length,
rc);
} else
- trace_smb3_read_done(xid, req->PersistentFileId, io_parms->tcon->tid,
+ trace_smb3_read_done(0, 0, xid,
+ req->PersistentFileId, io_parms->tcon->tid,
ses->Suid, io_parms->offset, 0);
free_rsp_buf(resp_buftype, rsp_iov.iov_base);
cifs_small_buf_release(req);
return rc == -ENODATA ? 0 : rc;
} else
- trace_smb3_read_done(xid,
- req->PersistentFileId,
- io_parms->tcon->tid, ses->Suid,
- io_parms->offset, io_parms->length);
+ trace_smb3_read_done(0, 0, xid,
+ req->PersistentFileId,
+ io_parms->tcon->tid, ses->Suid,
+ io_parms->offset, io_parms->length);

cifs_small_buf_release(req);

diff --git a/fs/cifs/trace.h b/fs/cifs/trace.h
index 6cecf302dcfd..c85cf0971eee 100644
--- a/fs/cifs/trace.h
+++ b/fs/cifs/trace.h
@@ -21,6 +21,62 @@

/* For logging errors in read or write */
DECLARE_EVENT_CLASS(smb3_rw_err_class,
+ TP_PROTO(unsigned int rreq_debug_id,
+ unsigned int rreq_debug_index,
+ unsigned int xid,
+ __u64 fid,
+ __u32 tid,
+ __u64 sesid,
+ __u64 offset,
+ __u32 len,
+ int rc),
+ TP_ARGS(rreq_debug_id, rreq_debug_index,
+ xid, fid, tid, sesid, offset, len, rc),
+ TP_STRUCT__entry(
+ __field(unsigned int, rreq_debug_id)
+ __field(unsigned int, rreq_debug_index)
+ __field(unsigned int, xid)
+ __field(__u64, fid)
+ __field(__u32, tid)
+ __field(__u64, sesid)
+ __field(__u64, offset)
+ __field(__u32, len)
+ __field(int, rc)
+ ),
+ TP_fast_assign(
+ __entry->rreq_debug_id = rreq_debug_id;
+ __entry->rreq_debug_index = rreq_debug_index;
+ __entry->xid = xid;
+ __entry->fid = fid;
+ __entry->tid = tid;
+ __entry->sesid = sesid;
+ __entry->offset = offset;
+ __entry->len = len;
+ __entry->rc = rc;
+ ),
+ TP_printk("\tR=%08x[%x] xid=%u sid=0x%llx tid=0x%x fid=0x%llx offset=0x%llx len=0x%x rc=%d",
+ __entry->rreq_debug_id, __entry->rreq_debug_index,
+ __entry->xid, __entry->sesid, __entry->tid, __entry->fid,
+ __entry->offset, __entry->len, __entry->rc)
+)
+
+#define DEFINE_SMB3_RW_ERR_EVENT(name) \
+DEFINE_EVENT(smb3_rw_err_class, smb3_##name, \
+ TP_PROTO(unsigned int rreq_debug_id, \
+ unsigned int rreq_debug_index, \
+ unsigned int xid, \
+ __u64 fid, \
+ __u32 tid, \
+ __u64 sesid, \
+ __u64 offset, \
+ __u32 len, \
+ int rc), \
+ TP_ARGS(rreq_debug_id, rreq_debug_index, xid, fid, tid, sesid, offset, len, rc))
+
+DEFINE_SMB3_RW_ERR_EVENT(read_err);
+
+/* For logging errors in other file I/O ops */
+DECLARE_EVENT_CLASS(smb3_other_err_class,
TP_PROTO(unsigned int xid,
__u64 fid,
__u32 tid,
@@ -52,8 +108,8 @@ DECLARE_EVENT_CLASS(smb3_rw_err_class,
__entry->offset, __entry->len, __entry->rc)
)

-#define DEFINE_SMB3_RW_ERR_EVENT(name) \
-DEFINE_EVENT(smb3_rw_err_class, smb3_##name, \
+#define DEFINE_SMB3_OTHER_ERR_EVENT(name) \
+DEFINE_EVENT(smb3_other_err_class, smb3_##name, \
TP_PROTO(unsigned int xid, \
__u64 fid, \
__u32 tid, \
@@ -63,15 +119,67 @@ DEFINE_EVENT(smb3_rw_err_class, smb3_##name, \
int rc), \
TP_ARGS(xid, fid, tid, sesid, offset, len, rc))

-DEFINE_SMB3_RW_ERR_EVENT(write_err);
-DEFINE_SMB3_RW_ERR_EVENT(read_err);
-DEFINE_SMB3_RW_ERR_EVENT(query_dir_err);
-DEFINE_SMB3_RW_ERR_EVENT(zero_err);
-DEFINE_SMB3_RW_ERR_EVENT(falloc_err);
+DEFINE_SMB3_OTHER_ERR_EVENT(write_err);
+DEFINE_SMB3_OTHER_ERR_EVENT(query_dir_err);
+DEFINE_SMB3_OTHER_ERR_EVENT(zero_err);
+DEFINE_SMB3_OTHER_ERR_EVENT(falloc_err);


/* For logging successful read or write */
DECLARE_EVENT_CLASS(smb3_rw_done_class,
+ TP_PROTO(unsigned int rreq_debug_id,
+ unsigned int rreq_debug_index,
+ unsigned int xid,
+ __u64 fid,
+ __u32 tid,
+ __u64 sesid,
+ __u64 offset,
+ __u32 len),
+ TP_ARGS(rreq_debug_id, rreq_debug_index,
+ xid, fid, tid, sesid, offset, len),
+ TP_STRUCT__entry(
+ __field(unsigned int, rreq_debug_id)
+ __field(unsigned int, rreq_debug_index)
+ __field(unsigned int, xid)
+ __field(__u64, fid)
+ __field(__u32, tid)
+ __field(__u64, sesid)
+ __field(__u64, offset)
+ __field(__u32, len)
+ ),
+ TP_fast_assign(
+ __entry->rreq_debug_id = rreq_debug_id;
+ __entry->rreq_debug_index = rreq_debug_index;
+ __entry->xid = xid;
+ __entry->fid = fid;
+ __entry->tid = tid;
+ __entry->sesid = sesid;
+ __entry->offset = offset;
+ __entry->len = len;
+ ),
+ TP_printk("R=%08x[%x] xid=%u sid=0x%llx tid=0x%x fid=0x%llx offset=0x%llx len=0x%x",
+ __entry->rreq_debug_id, __entry->rreq_debug_index,
+ __entry->xid, __entry->sesid, __entry->tid, __entry->fid,
+ __entry->offset, __entry->len)
+)
+
+#define DEFINE_SMB3_RW_DONE_EVENT(name) \
+DEFINE_EVENT(smb3_rw_done_class, smb3_##name, \
+ TP_PROTO(unsigned int rreq_debug_id, \
+ unsigned int rreq_debug_index, \
+ unsigned int xid, \
+ __u64 fid, \
+ __u32 tid, \
+ __u64 sesid, \
+ __u64 offset, \
+ __u32 len), \
+ TP_ARGS(rreq_debug_id, rreq_debug_index, xid, fid, tid, sesid, offset, len))
+
+DEFINE_SMB3_RW_DONE_EVENT(read_enter);
+DEFINE_SMB3_RW_DONE_EVENT(read_done);
+
+/* For logging successful other op */
+DECLARE_EVENT_CLASS(smb3_other_done_class,
TP_PROTO(unsigned int xid,
__u64 fid,
__u32 tid,
@@ -100,8 +208,8 @@ DECLARE_EVENT_CLASS(smb3_rw_done_class,
__entry->offset, __entry->len)
)

-#define DEFINE_SMB3_RW_DONE_EVENT(name) \
-DEFINE_EVENT(smb3_rw_done_class, smb3_##name, \
+#define DEFINE_SMB3_OTHER_DONE_EVENT(name) \
+DEFINE_EVENT(smb3_other_done_class, smb3_##name, \
TP_PROTO(unsigned int xid, \
__u64 fid, \
__u32 tid, \
@@ -110,16 +218,14 @@ DEFINE_EVENT(smb3_rw_done_class, smb3_##name, \
__u32 len), \
TP_ARGS(xid, fid, tid, sesid, offset, len))

-DEFINE_SMB3_RW_DONE_EVENT(write_enter);
-DEFINE_SMB3_RW_DONE_EVENT(read_enter);
-DEFINE_SMB3_RW_DONE_EVENT(query_dir_enter);
-DEFINE_SMB3_RW_DONE_EVENT(zero_enter);
-DEFINE_SMB3_RW_DONE_EVENT(falloc_enter);
-DEFINE_SMB3_RW_DONE_EVENT(write_done);
-DEFINE_SMB3_RW_DONE_EVENT(read_done);
-DEFINE_SMB3_RW_DONE_EVENT(query_dir_done);
-DEFINE_SMB3_RW_DONE_EVENT(zero_done);
-DEFINE_SMB3_RW_DONE_EVENT(falloc_done);
+DEFINE_SMB3_OTHER_DONE_EVENT(write_enter);
+DEFINE_SMB3_OTHER_DONE_EVENT(query_dir_enter);
+DEFINE_SMB3_OTHER_DONE_EVENT(zero_enter);
+DEFINE_SMB3_OTHER_DONE_EVENT(falloc_enter);
+DEFINE_SMB3_OTHER_DONE_EVENT(write_done);
+DEFINE_SMB3_OTHER_DONE_EVENT(query_dir_done);
+DEFINE_SMB3_OTHER_DONE_EVENT(zero_done);
+DEFINE_SMB3_OTHER_DONE_EVENT(falloc_done);

/*
* For handle based calls other than read and write, and get/set info


2022-04-07 11:06:53

by David Howells

[permalink] [raw]
Subject: [PATCH 03/14] cifs: Check the IOCB_DIRECT flag, not O_DIRECT

Use the IOCB_DIRECT indicator flag on the I/O context rather than checking to
see if the file was opened O_DIRECT.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/cifsfs.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fs/cifs/cifsfs.c b/fs/cifs/cifsfs.c
index a47fa44b6d52..fb60b5410789 100644
--- a/fs/cifs/cifsfs.c
+++ b/fs/cifs/cifsfs.c
@@ -944,7 +944,7 @@ cifs_loose_read_iter(struct kiocb *iocb, struct iov_iter *iter)
ssize_t rc;
struct inode *inode = file_inode(iocb->ki_filp);

- if (iocb->ki_filp->f_flags & O_DIRECT)
+ if (iocb->ki_flags & IOCB_DIRECT)
return cifs_user_readv(iocb, iter);

rc = cifs_revalidate_mapping(inode);


2022-04-07 17:38:31

by David Howells

[permalink] [raw]
Subject: [PATCH 13/14] cifs: Split the smb3_add_credits tracepoint

Split the smb3_add_credits tracepoint to make it more obvious when looking
at the logs which line corresponds to what credit change. Also add a
tracepoint for credit overflow when it's being added back.

Note that it might be better to add another field to the tracepoint to store
the information rather than splitting it. It would also be useful to store
the MID potentially, though that isn't available when the credits are first
obtained.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/connect.c | 2 +-
fs/cifs/smb2ops.c | 9 ++++++---
fs/cifs/trace.h | 7 +++++++
fs/cifs/transport.c | 4 ++--
4 files changed, 16 insertions(+), 6 deletions(-)

diff --git a/fs/cifs/connect.c b/fs/cifs/connect.c
index 080e19e9ee47..dab0e9f93708 100644
--- a/fs/cifs/connect.c
+++ b/fs/cifs/connect.c
@@ -1062,7 +1062,7 @@ smb2_add_credits_from_hdr(char *buffer, struct TCP_Server_Info *server)
spin_unlock(&server->req_lock);
wake_up(&server->request_q);

- trace_smb3_add_credits(server->CurrentMid,
+ trace_smb3_hdr_credits(server->CurrentMid,
server->conn_id, server->hostname, scredits,
le16_to_cpu(shdr->CreditRequest), in_flight);
cifs_server_dbg(FYI, "%s: added %u credits total=%d\n",
diff --git a/fs/cifs/smb2ops.c b/fs/cifs/smb2ops.c
index 03bd64933b7b..cf248309528a 100644
--- a/fs/cifs/smb2ops.c
+++ b/fs/cifs/smb2ops.c
@@ -86,6 +86,9 @@ smb2_add_credits(struct TCP_Server_Info *server,
if (*val > 65000) {
*val = 65000; /* Don't get near 64K credits, avoid srv bugs */
pr_warn_once("server overflowed SMB3 credits\n");
+ trace_smb3_overflow_credits(server->CurrentMid,
+ server->conn_id, server->hostname, *val,
+ add, server->in_flight);
}
server->in_flight--;
if (server->in_flight == 0 &&
@@ -251,7 +254,7 @@ smb2_wait_mtu_credits(struct TCP_Server_Info *server, unsigned int size,
in_flight = server->in_flight;
spin_unlock(&server->req_lock);

- trace_smb3_add_credits(server->CurrentMid,
+ trace_smb3_wait_credits(server->CurrentMid,
server->conn_id, server->hostname, scredits, -(credits->value), in_flight);
cifs_dbg(FYI, "%s: removed %u credits total=%d\n",
__func__, credits->value, scredits);
@@ -300,7 +303,7 @@ smb2_adjust_credits(struct TCP_Server_Info *server,
spin_unlock(&server->req_lock);
wake_up(&server->request_q);

- trace_smb3_add_credits(server->CurrentMid,
+ trace_smb3_adj_credits(server->CurrentMid,
server->conn_id, server->hostname, scredits,
credits->value - new_val, in_flight);
cifs_dbg(FYI, "%s: adjust added %u credits total=%d\n",
@@ -2492,7 +2495,7 @@ smb2_is_status_pending(char *buf, struct TCP_Server_Info *server)
spin_unlock(&server->req_lock);
wake_up(&server->request_q);

- trace_smb3_add_credits(server->CurrentMid,
+ trace_smb3_pend_credits(server->CurrentMid,
server->conn_id, server->hostname, scredits,
le16_to_cpu(shdr->CreditRequest), in_flight);
cifs_dbg(FYI, "%s: status pending add %u credits total=%d\n",
diff --git a/fs/cifs/trace.h b/fs/cifs/trace.h
index c85cf0971eee..d783b14c24ef 100644
--- a/fs/cifs/trace.h
+++ b/fs/cifs/trace.h
@@ -1112,6 +1112,13 @@ DEFINE_SMB3_CREDIT_EVENT(credit_timeout);
DEFINE_SMB3_CREDIT_EVENT(insufficient_credits);
DEFINE_SMB3_CREDIT_EVENT(too_many_credits);
DEFINE_SMB3_CREDIT_EVENT(add_credits);
+DEFINE_SMB3_CREDIT_EVENT(adj_credits);
+DEFINE_SMB3_CREDIT_EVENT(hdr_credits);
+DEFINE_SMB3_CREDIT_EVENT(nblk_credits);
+DEFINE_SMB3_CREDIT_EVENT(pend_credits);
+DEFINE_SMB3_CREDIT_EVENT(wait_credits);
+DEFINE_SMB3_CREDIT_EVENT(waitff_credits);
+DEFINE_SMB3_CREDIT_EVENT(overflow_credits);
DEFINE_SMB3_CREDIT_EVENT(set_credits);

#endif /* _CIFS_TRACE_H */
diff --git a/fs/cifs/transport.c b/fs/cifs/transport.c
index c74aa12ecc7c..4e7a3a0ddd8b 100644
--- a/fs/cifs/transport.c
+++ b/fs/cifs/transport.c
@@ -515,7 +515,7 @@ wait_for_free_credits(struct TCP_Server_Info *server, const int num_credits,
in_flight = server->in_flight;
spin_unlock(&server->req_lock);

- trace_smb3_add_credits(server->CurrentMid,
+ trace_smb3_nblk_credits(server->CurrentMid,
server->conn_id, server->hostname, scredits, -1, in_flight);
cifs_dbg(FYI, "%s: remove %u credits total=%d\n",
__func__, 1, scredits);
@@ -621,7 +621,7 @@ wait_for_free_credits(struct TCP_Server_Info *server, const int num_credits,
in_flight = server->in_flight;
spin_unlock(&server->req_lock);

- trace_smb3_add_credits(server->CurrentMid,
+ trace_smb3_waitff_credits(server->CurrentMid,
server->conn_id, server->hostname, scredits,
-(num_credits), in_flight);
cifs_dbg(FYI, "%s: remove %u credits total=%d\n",


2022-04-07 18:59:00

by David Howells

[permalink] [raw]
Subject: [PATCH 08/14] netfs: Allow the netfs to make the io (sub)request alloc larger

Allow the network filesystem to specify extra space to be allocated on the
end of the io (sub)request. This allows cifs, for example, to use this
space rather than allocating its own cifs_readdata struct.

Signed-off-by: David Howells <[email protected]>
cc: Steve French <[email protected]>
cc: Shyam Prasad N <[email protected]>
cc: Rohith Surabattula <[email protected]>
cc: [email protected]
---

fs/cifs/cifsglob.h | 10 +++-----
fs/cifs/cifsproto.h | 2 +-
fs/cifs/cifssmb.c | 22 +++++++-----------
fs/cifs/file.c | 63 ++++++++++++++++-----------------------------------
fs/cifs/smb2ops.c | 6 ++---
fs/cifs/smb2pdu.c | 18 ++++++---------
fs/cifs/smb2proto.h | 2 +-
7 files changed, 46 insertions(+), 77 deletions(-)

diff --git a/fs/cifs/cifsglob.h b/fs/cifs/cifsglob.h
index 49eb86ebbdfb..61b40721178a 100644
--- a/fs/cifs/cifsglob.h
+++ b/fs/cifs/cifsglob.h
@@ -212,7 +212,7 @@ struct dfs_info3_param;
struct cifs_fattr;
struct smb3_fs_context;
struct cifs_fid;
-struct cifs_readdata;
+struct cifs_io_subrequest;
struct cifs_writedata;
struct cifs_io_parms;
struct cifs_search_info;
@@ -378,7 +378,7 @@ struct smb_version_operations {
/* send a flush request to the server */
int (*flush)(const unsigned int, struct cifs_tcon *, struct cifs_fid *);
/* async read from the server */
- int (*async_readv)(struct cifs_readdata *);
+ int (*async_readv)(struct cifs_io_subrequest *);
/* async write to the server */
int (*async_writev)(struct cifs_writedata *,
void (*release)(struct kref *));
@@ -1314,16 +1314,14 @@ struct cifs_aio_ctx {
};

/* asynchronous read support */
-struct cifs_readdata {
- struct netfs_io_subrequest *subreq;
- struct kref refcount;
+struct cifs_io_subrequest {
+ struct netfs_io_subrequest subreq;
struct cifsFileInfo *cfile;
__u64 offset;
ssize_t got_bytes;
unsigned int bytes;
pid_t pid;
int result;
- struct iov_iter iter;
struct kvec iov[2];
struct TCP_Server_Info *server;
#ifdef CONFIG_CIFS_SMB_DIRECT
diff --git a/fs/cifs/cifsproto.h b/fs/cifs/cifsproto.h
index 981b6f779f2e..5090992eaa15 100644
--- a/fs/cifs/cifsproto.h
+++ b/fs/cifs/cifsproto.h
@@ -575,7 +575,7 @@ extern struct cifs_ses *
cifs_get_smb_ses(struct TCP_Server_Info *server, struct smb3_fs_context *ctx);

void cifs_readdata_release(struct kref *refcount);
-int cifs_async_readv(struct cifs_readdata *rdata);
+int cifs_async_readv(struct cifs_io_subrequest *rdata);
int cifs_readv_receive(struct TCP_Server_Info *server, struct mid_q_entry *mid);

int cifs_async_writev(struct cifs_writedata *wdata,
diff --git a/fs/cifs/cifssmb.c b/fs/cifs/cifssmb.c
index 5956caaec0e3..f736493746b4 100644
--- a/fs/cifs/cifssmb.c
+++ b/fs/cifs/cifssmb.c
@@ -24,6 +24,7 @@
#include <linux/task_io_accounting_ops.h>
#include <linux/uaccess.h>
#include <linux/netfs.h>
+#include <trace/events/netfs.h>
#include "cifspdu.h"
#include "cifsfs.h"
#include "cifsglob.h"
@@ -1424,7 +1425,7 @@ __cifs_readv_discard(struct TCP_Server_Info *server, struct mid_q_entry *mid,
static int
cifs_readv_discard(struct TCP_Server_Info *server, struct mid_q_entry *mid)
{
- struct cifs_readdata *rdata = mid->callback_data;
+ struct cifs_io_subrequest *rdata = mid->callback_data;

return __cifs_readv_discard(server, mid, rdata->result);
}
@@ -1434,7 +1435,7 @@ cifs_readv_receive(struct TCP_Server_Info *server, struct mid_q_entry *mid)
{
int length, len;
unsigned int data_offset, data_len;
- struct cifs_readdata *rdata = mid->callback_data;
+ struct cifs_io_subrequest *rdata = mid->callback_data;
char *buf = server->smallbuf;
unsigned int buflen = server->pdu_size +
server->vals->header_preamble_size;
@@ -1546,7 +1547,7 @@ cifs_readv_receive(struct TCP_Server_Info *server, struct mid_q_entry *mid)
length = data_len; /* An RDMA read is already done. */
else
#endif
- length = cifs_read_iter_from_socket(server, &rdata->iter,
+ length = cifs_read_iter_from_socket(server, &rdata->subreq.iter,
data_len);
if (length > 0)
rdata->got_bytes += length;
@@ -1568,12 +1569,12 @@ cifs_readv_receive(struct TCP_Server_Info *server, struct mid_q_entry *mid)
static void
cifs_readv_callback(struct mid_q_entry *mid)
{
- struct cifs_readdata *rdata = mid->callback_data;
+ struct cifs_io_subrequest *rdata = mid->callback_data;
struct cifs_tcon *tcon = tlink_tcon(rdata->cfile->tlink);
struct TCP_Server_Info *server = tcon->ses->server;
struct smb_rqst rqst = { .rq_iov = rdata->iov,
.rq_nvec = 2,
- .rq_iter = rdata->iter };
+ .rq_iter = rdata->subreq.iter };
struct cifs_credits credits = { .value = 1, .instance = 0 };

cifs_dbg(FYI, "%s: mid=%llu state=%d result=%d bytes=%u\n",
@@ -1611,19 +1612,18 @@ cifs_readv_callback(struct mid_q_entry *mid)
}

if (rdata->result == 0 || rdata->result == -EAGAIN)
- iov_iter_advance(&rdata->subreq->iter, rdata->got_bytes);
- netfs_subreq_terminated(rdata->subreq,
+ iov_iter_advance(&rdata->subreq.iter, rdata->got_bytes);
+ netfs_subreq_terminated(&rdata->subreq,
(rdata->result == 0 || rdata->result == -EAGAIN) ?
rdata->got_bytes : rdata->result,
false);
- kref_put(&rdata->refcount, cifs_readdata_release);
DeleteMidQEntry(mid);
add_credits(server, &credits, 0);
}

/* cifs_async_readv - send an async write, and set up mid to handle result */
int
-cifs_async_readv(struct cifs_readdata *rdata)
+cifs_async_readv(struct cifs_io_subrequest *rdata)
{
int rc;
READ_REQ *smb = NULL;
@@ -1675,15 +1675,11 @@ cifs_async_readv(struct cifs_readdata *rdata)
rdata->iov[1].iov_base = (char *)smb + 4;
rdata->iov[1].iov_len = get_rfc1002_length(smb);

- kref_get(&rdata->refcount);
rc = cifs_call_async(tcon->ses->server, &rqst, cifs_readv_receive,
cifs_readv_callback, NULL, rdata, 0, NULL);

if (rc == 0)
cifs_stats_inc(&tcon->stats.cifs_stats.num_reads);
- else
- kref_put(&rdata->refcount, cifs_readdata_release);
-
cifs_small_buf_release(smb);
return rc;
}
diff --git a/fs/cifs/file.c b/fs/cifs/file.c
index fcf92c25bc8f..fb2885134154 100644
--- a/fs/cifs/file.c
+++ b/fs/cifs/file.c
@@ -3322,35 +3322,6 @@ cifs_strict_writev(struct kiocb *iocb, struct iov_iter *from)
return written;
}

-static struct cifs_readdata *cifs_readdata_alloc(work_func_t complete)
-{
- struct cifs_readdata *rdata;
-
- rdata = kzalloc(sizeof(*rdata), GFP_KERNEL);
- if (rdata)
- kref_init(&rdata->refcount);
-
- return rdata;
-}
-
-void
-cifs_readdata_release(struct kref *refcount)
-{
- struct cifs_readdata *rdata = container_of(refcount,
- struct cifs_readdata, refcount);
-
-#ifdef CONFIG_CIFS_SMB_DIRECT
- if (rdata->mr) {
- smbd_deregister_mr(rdata->mr);
- rdata->mr = NULL;
- }
-#endif
- if (rdata->cfile)
- cifsFileInfo_put(rdata->cfile);
-
- kfree(rdata);
-}
-
ssize_t
cifs_strict_readv(struct kiocb *iocb, struct iov_iter *to)
{
@@ -3476,7 +3447,7 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
{
struct netfs_io_request *rreq = subreq->rreq;
struct TCP_Server_Info *server;
- struct cifs_readdata *rdata;
+ struct cifs_io_subrequest *rdata = container_of(subreq, struct cifs_io_subrequest, subreq);
struct cifsFileInfo *open_file = rreq->netfs_priv;
struct cifs_sb_info *cifs_sb = CIFS_SB(rreq->inode->i_sb);
struct cifs_credits credits_on_stack, *credits = &credits_on_stack;
@@ -3515,22 +3486,13 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
if (rc)
goto out;

- rdata = cifs_readdata_alloc(NULL);
- if (!rdata) {
- add_credits_and_wake_if(server, credits, 0);
- rc = -ENOMEM;
- goto out;
- }
-
__set_bit(NETFS_SREQ_CLEAR_TAIL, &subreq->flags);
- rdata->subreq = subreq;
rdata->cfile = cifsFileInfo_get(open_file);
rdata->server = server;
rdata->offset = subreq->start + subreq->transferred;
rdata->bytes = subreq->len - subreq->transferred;
rdata->pid = pid;
rdata->credits = credits_on_stack;
- rdata->iter = subreq->iter;

rc = adjust_credits(server, &rdata->credits, rdata->bytes);
if (!rc) {
@@ -3543,12 +3505,8 @@ static void cifs_req_issue_read(struct netfs_io_subrequest *subreq)
if (rc) {
add_credits_and_wake_if(server, &rdata->credits, 0);
/* Fallback to the readpage in error/reconnect cases */
- kref_put(&rdata->refcount, cifs_readdata_release);
- goto out;
}

- kref_put(&rdata->refcount, cifs_readdata_release);
-
out:
free_xid(xid);
if (rc)
@@ -3614,8 +3572,27 @@ static int cifs_begin_cache_operation(struct netfs_io_request *rreq)
#endif
}

+static void cifs_free_subrequest(struct netfs_io_subrequest *subreq)
+{
+ struct cifs_io_subrequest *rdata =
+ container_of(subreq, struct cifs_io_subrequest, subreq);
+
+ if (rdata->subreq.source == NETFS_DOWNLOAD_FROM_SERVER) {
+#ifdef CONFIG_CIFS_SMB_DIRECT
+ if (rdata->mr) {
+ smbd_deregister_mr(rdata->mr);
+ rdata->mr = NULL;
+ }
+#endif
+ if (rdata->cfile)
+ cifsFileInfo_put(rdata->cfile);
+ }
+}
+
const struct netfs_request_ops cifs_req_ops = {
+ .io_subrequest_size = sizeof(struct cifs_io_subrequest),
.init_request = cifs_init_request,
+ .free_subrequest = cifs_free_subrequest,
.begin_cache_operation = cifs_begin_cache_operation,
.expand_readahead = cifs_expand_readahead,
.issue_read = cifs_req_issue_read,
diff --git a/fs/cifs/smb2ops.c b/fs/cifs/smb2ops.c
index 2caac8696100..03bd64933b7b 100644
--- a/fs/cifs/smb2ops.c
+++ b/fs/cifs/smb2ops.c
@@ -4842,7 +4842,7 @@ handle_read_data(struct TCP_Server_Info *server, struct mid_q_entry *mid,
unsigned int cur_off;
unsigned int cur_page_idx;
unsigned int pad_len;
- struct cifs_readdata *rdata = mid->callback_data;
+ struct cifs_io_subrequest *rdata = mid->callback_data;
struct smb2_hdr *shdr = (struct smb2_hdr *)buf;
int length;
bool use_rdma_mr = false;
@@ -4944,7 +4944,7 @@ handle_read_data(struct TCP_Server_Info *server, struct mid_q_entry *mid,

/* Copy the data to the output I/O iterator. */
rdata->result = cifs_copy_pages_to_iter(pages, pages_len,
- cur_off, &rdata->iter);
+ cur_off, &rdata->subreq.iter);
if (rdata->result != 0) {
if (is_offloaded)
mid->mid_state = MID_RESPONSE_MALFORMED;
@@ -4958,7 +4958,7 @@ handle_read_data(struct TCP_Server_Info *server, struct mid_q_entry *mid,
/* read response payload is in buf */
WARN_ONCE(pages && !xa_empty(pages),
"read data can be either in buf or in pages");
- length = copy_to_iter(buf + data_offset, data_len, &rdata->iter);
+ length = copy_to_iter(buf + data_offset, data_len, &rdata->subreq.iter);
if (length < 0)
return length;
rdata->got_bytes = data_len;
diff --git a/fs/cifs/smb2pdu.c b/fs/cifs/smb2pdu.c
index 19ad6c89121a..696e38da9ae7 100644
--- a/fs/cifs/smb2pdu.c
+++ b/fs/cifs/smb2pdu.c
@@ -24,6 +24,7 @@
#include <linux/pagemap.h>
#include <linux/xattr.h>
#include <linux/netfs.h>
+#include <trace/events/netfs.h>
#include "cifsglob.h"
#include "cifsacl.h"
#include "cifsproto.h"
@@ -4012,7 +4013,7 @@ SMB2_flush(const unsigned int xid, struct cifs_tcon *tcon, u64 persistent_fid,
*/
static int
smb2_new_read_req(void **buf, unsigned int *total_len,
- struct cifs_io_parms *io_parms, struct cifs_readdata *rdata,
+ struct cifs_io_parms *io_parms, struct cifs_io_subrequest *rdata,
unsigned int remaining_bytes, int request_type)
{
int rc = -EACCES;
@@ -4108,7 +4109,7 @@ smb2_new_read_req(void **buf, unsigned int *total_len,
static void
smb2_readv_callback(struct mid_q_entry *mid)
{
- struct cifs_readdata *rdata = mid->callback_data;
+ struct cifs_io_subrequest *rdata = mid->callback_data;
struct cifs_tcon *tcon = tlink_tcon(rdata->cfile->tlink);
struct TCP_Server_Info *server = rdata->server;
struct smb2_hdr *shdr =
@@ -4116,7 +4117,7 @@ smb2_readv_callback(struct mid_q_entry *mid)
struct cifs_credits credits = { .value = 0, .instance = 0 };
struct smb_rqst rqst = { .rq_iov = &rdata->iov[1],
.rq_nvec = 1,
- .rq_iter = rdata->iter };
+ .rq_iter = rdata->subreq.iter };

WARN_ONCE(rdata->server != mid->server,
"rdata server %p != mid server %p",
@@ -4187,22 +4188,21 @@ smb2_readv_callback(struct mid_q_entry *mid)
/* We may have got an EOF error because fallocate
* failed to enlarge the file.
*/
- if (rdata->subreq->start < rdata->subreq->rreq->i_size)
+ if (rdata->subreq.start < rdata->subreq.rreq->i_size)
rdata->result = 0;
}
if (rdata->result == 0 || rdata->result == -EAGAIN)
- iov_iter_advance(&rdata->subreq->iter, rdata->got_bytes);
- netfs_subreq_terminated(rdata->subreq,
+ iov_iter_advance(&rdata->subreq.iter, rdata->got_bytes);
+ netfs_subreq_terminated(&rdata->subreq,
(rdata->result == 0 || rdata->result == -EAGAIN) ?
rdata->got_bytes : rdata->result, true);
- kref_put(&rdata->refcount, cifs_readdata_release);
DeleteMidQEntry(mid);
add_credits(server, &credits, 0);
}

/* smb2_async_readv - send an async read, and set up mid to handle result */
int
-smb2_async_readv(struct cifs_readdata *rdata)
+smb2_async_readv(struct cifs_io_subrequest *rdata)
{
int rc, flags = 0;
char *buf;
@@ -4253,13 +4253,11 @@ smb2_async_readv(struct cifs_readdata *rdata)
flags |= CIFS_HAS_CREDITS;
}

- kref_get(&rdata->refcount);
rc = cifs_call_async(server, &rqst,
cifs_readv_receive, smb2_readv_callback,
smb3_handle_read_data, rdata, flags,
&rdata->credits);
if (rc) {
- kref_put(&rdata->refcount, cifs_readdata_release);
cifs_stats_fail_inc(io_parms.tcon, SMB2_READ_HE);
trace_smb3_read_err(0 /* xid */, io_parms.persistent_fid,
io_parms.tcon->tid,
diff --git a/fs/cifs/smb2proto.h b/fs/cifs/smb2proto.h
index a69f1eed1cfe..fa36bb52c83b 100644
--- a/fs/cifs/smb2proto.h
+++ b/fs/cifs/smb2proto.h
@@ -197,7 +197,7 @@ extern int SMB2_query_acl(const unsigned int xid, struct cifs_tcon *tcon,
extern int SMB2_get_srv_num(const unsigned int xid, struct cifs_tcon *tcon,
u64 persistent_fid, u64 volatile_fid,
__le64 *uniqueid);
-extern int smb2_async_readv(struct cifs_readdata *rdata);
+extern int smb2_async_readv(struct cifs_io_subrequest *rdata);
extern int SMB2_read(const unsigned int xid, struct cifs_io_parms *io_parms,
unsigned int *nbytes, char **buf, int *buf_type);
extern int smb2_async_writev(struct cifs_writedata *wdata,


2022-04-07 19:41:56

by Matthew Wilcox

[permalink] [raw]
Subject: Re: [PATCH 14/14] mm, netfs, fscache: Stop read optimisation when folio removed from pagecache

On Thu, Apr 07, 2022 at 12:05:05AM +0100, David Howells wrote:
> Fix this by adding an extra address_space operation, ->removing folio(),
> and flag, AS_NOTIFY_REMOVING_FOLIO. The operation is called if the flag is
> set when a folio is removed from the pagecache. The flag should be set if
> a non-NULL cookie is obtained from fscache and cleared in ->evict_inode()
> before truncate_inode_pages_final() is called.

What's wrong with ->freepage?

2022-04-07 20:36:21

by David Howells

[permalink] [raw]
Subject: Re: [PATCH 14/14] mm, netfs, fscache: Stop read optimisation when folio removed from pagecache

Matthew Wilcox <[email protected]> wrote:

> On Thu, Apr 07, 2022 at 12:05:05AM +0100, David Howells wrote:
> > Fix this by adding an extra address_space operation, ->removing folio(),
> > and flag, AS_NOTIFY_REMOVING_FOLIO. The operation is called if the flag is
> > set when a folio is removed from the pagecache. The flag should be set if
> > a non-NULL cookie is obtained from fscache and cleared in ->evict_inode()
> > before truncate_inode_pages_final() is called.
>
> What's wrong with ->freepage?

It's too late. The optimisation must be cancelled before there's a chance
that a new page can be allocated and attached to the pagecache - but
->freepage() is called after the folio has been removed. Doing it in
->freepage() would allow ->readahead(), ->readpage() or ->write_begin() to
jump in and start a new read (which gets skipped because the optimisation is
still in play).

Another possibility could be that the FSCACHE_COOKIE_HAVE_DATA and
FSCACHE_COOKIE_NO_DATA_TO_READ flags could be moved from cookie->flags to
mapping->flags and the VM could do the twiddling itself (no aop required) -
except that fscache can't currently then find them (maybe use an aop for
that?).

David

2022-04-07 22:19:54

by Matthew Wilcox

[permalink] [raw]
Subject: Re: [PATCH 14/14] mm, netfs, fscache: Stop read optimisation when folio removed from pagecache

On Thu, Apr 07, 2022 at 07:41:47AM +0100, David Howells wrote:
> Matthew Wilcox <[email protected]> wrote:
>
> > On Thu, Apr 07, 2022 at 12:05:05AM +0100, David Howells wrote:
> > > Fix this by adding an extra address_space operation, ->removing folio(),
> > > and flag, AS_NOTIFY_REMOVING_FOLIO. The operation is called if the flag is
> > > set when a folio is removed from the pagecache. The flag should be set if
> > > a non-NULL cookie is obtained from fscache and cleared in ->evict_inode()
> > > before truncate_inode_pages_final() is called.
> >
> > What's wrong with ->freepage?
>
> It's too late. The optimisation must be cancelled before there's a chance
> that a new page can be allocated and attached to the pagecache - but
> ->freepage() is called after the folio has been removed. Doing it in
> ->freepage() would allow ->readahead(), ->readpage() or ->write_begin() to
> jump in and start a new read (which gets skipped because the optimisation is
> still in play).

OK. You suggested that releasepage was an acceptable place to call it.
How about we have AS_RELEASE_ALL (... or something ...) and then
page_has_private() becomes a bit more complicated ... to the point
where we should probably get rid of it (by embedding it into
filemap_release_folio():

+++ b/mm/filemap.c
@@ -3981,6 +3981,9 @@ bool filemap_release_folio(struct folio *folio, gfp_t gfp)
struct address_space * const mapping = folio->mapping;

BUG_ON(!folio_test_locked(folio));
+ if (!mapping_needs_release(mapping) && !folio_test_private(folio) &&
+ !folio_test_private_2(folio))
+ return false;
if (folio_test_writeback(folio))
return false;



2022-04-25 15:25:07

by David Howells

[permalink] [raw]
Subject: Re: [PATCH 14/14] mm, netfs, fscache: Stop read optimisation when folio removed from pagecache

Matthew Wilcox <[email protected]> wrote:

> OK. You suggested that releasepage was an acceptable place to call it.
> How about we have AS_RELEASE_ALL (... or something ...) and then
> page_has_private() becomes a bit more complicated ... to the point
> where we should probably get rid of it (by embedding it into
> filemap_release_folio():

I'm not sure page_has_private() is quite so easy to get rid of.
shrink_page_list() and collapse_file(), for example, use it to conditionalise
a call to try_to_release_page() plus some other bits.

I think that, for the moment, I would need to add a check for AS_RELEASE_ALL
to page_has_private().

David

2022-04-26 00:16:50

by Matthew Wilcox

[permalink] [raw]
Subject: Re: [PATCH 14/14] mm, netfs, fscache: Stop read optimisation when folio removed from pagecache

On Mon, Apr 25, 2022 at 01:07:41PM +0100, David Howells wrote:
> Matthew Wilcox <[email protected]> wrote:
>
> > OK. You suggested that releasepage was an acceptable place to call it.
> > How about we have AS_RELEASE_ALL (... or something ...) and then
> > page_has_private() becomes a bit more complicated ... to the point
> > where we should probably get rid of it (by embedding it into
> > filemap_release_folio():
>
> I'm not sure page_has_private() is quite so easy to get rid of.
> shrink_page_list() and collapse_file(), for example, use it to conditionalise
> a call to try_to_release_page() plus some other bits.

That's what I was saying. Make the calls to try_to_release_page()
unconditional and delete page_has_private() because it only confuses
people who should actually be using PagePrivate().

> I think that, for the moment, I would need to add a check for AS_RELEASE_ALL
> to page_has_private().
>
> David
>
>

2022-09-02 01:19:14

by David Wysochanski

[permalink] [raw]
Subject: Re: [PATCH 14/14] mm, netfs, fscache: Stop read optimisation when folio removed from pagecache

On Mon, Apr 25, 2022 at 8:30 AM Matthew Wilcox <[email protected]> wrote:
>
> On Mon, Apr 25, 2022 at 01:07:41PM +0100, David Howells wrote:
> > Matthew Wilcox <[email protected]> wrote:
> >
> > > OK. You suggested that releasepage was an acceptable place to call it.
> > > How about we have AS_RELEASE_ALL (... or something ...) and then
> > > page_has_private() becomes a bit more complicated ... to the point
> > > where we should probably get rid of it (by embedding it into
> > > filemap_release_folio():
> >
> > I'm not sure page_has_private() is quite so easy to get rid of.
> > shrink_page_list() and collapse_file(), for example, use it to conditionalise
> > a call to try_to_release_page() plus some other bits.
>
> That's what I was saying. Make the calls to try_to_release_page()
> unconditional and delete page_has_private() because it only confuses
> people who should actually be using PagePrivate().
>
> > I think that, for the moment, I would need to add a check for AS_RELEASE_ALL
> > to page_has_private().
> >
> > David
> >
> >
>

I am not sure what the next steps are here but I wanted to ping about
this patch. NFS also needs this patch or something like it. David are
you planning to submit an updated series with an updated patch?

A partial backport of David's original patch here on top of my v3 NFS
netfs conversion patches [1] resolves one of my unit test failures
where there were extra reads from the network instead of the cache.
Also Daire Byrne indicates that he too was seeing the same thing
and he tested my patches below and it resolved his issue as well.
Note that I needed another netfs patch in this series,
"netfs: Provide invalidatepage and releasepage calls"
on top of this one, to resolve the problem.

[1] https://github.com/DaveWysochanskiRH/kernel/commits/nfs-fscache-netfs-removing-folio
$ git log --oneline | head
ad90bddf6570 NFS: Add usage of new VFS API removing folio
9e2a7c301564 mm,netfs: Add removing_folio() to stop netfs read
optimization (TEST ONLY)
776088910162 netfs: Provide invalidatepage and releasepage calls
8aa1379ceb49 NFS: Convert buffered read paths to use netfs when
fscache is enabled
807808d87040 NFS: Configure support for netfs when NFS fscache is configured
43a41cce491d NFS: Rename readpage_async_filler to nfs_pageio_add_page
b90cb1053190 Linux 6.0-rc3