summaryrefslogtreecommitdiff
path: root/include/linux
diff options
context:
space:
mode:
authorLinus Torvalds <torvalds@linux-foundation.org>2024-09-16 13:13:31 +0300
committerLinus Torvalds <torvalds@linux-foundation.org>2024-09-16 13:13:31 +0300
commit35219bc5c71f4197c8bd10297597de797c1eece5 (patch)
tree2448156135b78f54cd341a8457ccd84a371ddac7 /include/linux
parent9020d0d844ad58a051f90b1e5b82ba34123925b9 (diff)
parent4b40d43d9f951d87ae8dc414c2ef5ae50303a266 (diff)
downloadlinux-35219bc5c71f4197c8bd10297597de797c1eece5.tar.xz
Merge tag 'vfs-6.12.netfs' of git://git.kernel.org/pub/scm/linux/kernel/git/vfs/vfs
Pull netfs updates from Christian Brauner: "This contains the work to improve read/write performance for the new netfs library. The main performance enhancing changes are: - Define a structure, struct folio_queue, and a new iterator type, ITER_FOLIOQ, to hold a buffer as a replacement for ITER_XARRAY. See that patch for questions about naming and form. ITER_FOLIOQ is provided as a replacement for ITER_XARRAY. The problem with an xarray is that accessing it requires the use of a lock (typically the RCU read lock) - and this means that we can't supply iterate_and_advance() with a step function that might sleep (crypto for example) without having to drop the lock between pages. ITER_FOLIOQ is the iterator for a chain of folio_queue structs, where each folio_queue holds a small list of folios. A folio_queue struct is a simpler structure than xarray and is not subject to concurrent manipulation by the VM. folio_queue is used rather than a bvec[] as it can form lists of indefinite size, adding to one end and removing from the other on the fly. - Provide a copy_folio_from_iter() wrapper. - Make cifs RDMA support ITER_FOLIOQ. - Use folio queues in the write-side helpers instead of xarrays. - Add a function to reset the iterator in a subrequest. - Simplify the write-side helpers to use sheaves to skip gaps rather than trying to work out where gaps are. - In afs, make the read subrequests asynchronous, putting them into work items to allow the next patch to do progressive unlocking/reading. - Overhaul the read-side helpers to improve performance. - Fix the caching of a partial block at the end of a file. - Allow a store to be cancelled. Then some changes for cifs to make it use folio queues instead of xarrays for crypto bufferage: - Use raw iteration functions rather than manually coding iteration when hashing data. - Switch to using folio_queue for crypto buffers. - Remove the xarray bits. Make some adjustments to the /proc/fs/netfs/stats file such that: - All the netfs stats lines begin 'Netfs:' but change this to something a bit more useful. - Add a couple of stats counters to track the numbers of skips and waits on the per-inode writeback serialisation lock to make it easier to check for this as a source of performance loss. Miscellaneous work: - Ensure that the sb_writers lock is taken around vfs_{set,remove}xattr() in the cachefiles code. - Reduce the number of conditional branches in netfs_perform_write(). - Move the CIFS_INO_MODIFIED_ATTR flag to the netfs_inode struct and remove cifs_post_modify(). - Move the max_len/max_nr_segs members from netfs_io_subrequest to netfs_io_request as they're only needed for one subreq at a time. - Add an 'unknown' source value for tracing purposes. - Remove NETFS_COPY_TO_CACHE as it's no longer used. - Set the request work function up front at allocation time. - Use bh-disabling spinlocks for rreq->lock as cachefiles completion may be run from block-filesystem DIO completion in softirq context. - Remove fs/netfs/io.c" * tag 'vfs-6.12.netfs' of git://git.kernel.org/pub/scm/linux/kernel/git/vfs/vfs: (25 commits) docs: filesystems: corrected grammar of netfs page cifs: Don't support ITER_XARRAY cifs: Switch crypto buffer to use a folio_queue rather than an xarray cifs: Use iterate_and_advance*() routines directly for hashing netfs: Cancel dirty folios that have no storage destination cachefiles, netfs: Fix write to partial block at EOF netfs: Remove fs/netfs/io.c netfs: Speed up buffered reading afs: Make read subreqs async netfs: Simplify the writeback code netfs: Provide an iterator-reset function netfs: Use new folio_queue data type and iterator instead of xarray iter cifs: Provide the capability to extract from ITER_FOLIOQ to RDMA SGEs iov_iter: Provide copy_folio_from_iter() mm: Define struct folio_queue and ITER_FOLIOQ to handle a sequence of folios netfs: Use bh-disabling spinlocks for rreq->lock netfs: Set the request work function upon allocation netfs: Remove NETFS_COPY_TO_CACHE netfs: Reserve netfs_sreq_source 0 as unset/unknown netfs: Move max_len/max_nr_segs from netfs_io_subrequest to netfs_io_stream ...
Diffstat (limited to 'include/linux')
-rw-r--r--include/linux/folio_queue.h156
-rw-r--r--include/linux/iov_iter.h104
-rw-r--r--include/linux/netfs.h46
-rw-r--r--include/linux/uio.h18
4 files changed, 306 insertions, 18 deletions
diff --git a/include/linux/folio_queue.h b/include/linux/folio_queue.h
new file mode 100644
index 000000000000..955680c3bb5f
--- /dev/null
+++ b/include/linux/folio_queue.h
@@ -0,0 +1,156 @@
+/* SPDX-License-Identifier: GPL-2.0-or-later */
+/* Queue of folios definitions
+ *
+ * Copyright (C) 2024 Red Hat, Inc. All Rights Reserved.
+ * Written by David Howells (dhowells@redhat.com)
+ */
+
+#ifndef _LINUX_FOLIO_QUEUE_H
+#define _LINUX_FOLIO_QUEUE_H
+
+#include <linux/pagevec.h>
+
+/*
+ * Segment in a queue of running buffers. Each segment can hold a number of
+ * folios and a portion of the queue can be referenced with the ITER_FOLIOQ
+ * iterator. The possibility exists of inserting non-folio elements into the
+ * queue (such as gaps).
+ *
+ * Explicit prev and next pointers are used instead of a list_head to make it
+ * easier to add segments to tail and remove them from the head without the
+ * need for a lock.
+ */
+struct folio_queue {
+ struct folio_batch vec; /* Folios in the queue segment */
+ u8 orders[PAGEVEC_SIZE]; /* Order of each folio */
+ struct folio_queue *next; /* Next queue segment or NULL */
+ struct folio_queue *prev; /* Previous queue segment of NULL */
+ unsigned long marks; /* 1-bit mark per folio */
+ unsigned long marks2; /* Second 1-bit mark per folio */
+ unsigned long marks3; /* Third 1-bit mark per folio */
+#if PAGEVEC_SIZE > BITS_PER_LONG
+#error marks is not big enough
+#endif
+};
+
+static inline void folioq_init(struct folio_queue *folioq)
+{
+ folio_batch_init(&folioq->vec);
+ folioq->next = NULL;
+ folioq->prev = NULL;
+ folioq->marks = 0;
+ folioq->marks2 = 0;
+ folioq->marks3 = 0;
+}
+
+static inline unsigned int folioq_nr_slots(const struct folio_queue *folioq)
+{
+ return PAGEVEC_SIZE;
+}
+
+static inline unsigned int folioq_count(struct folio_queue *folioq)
+{
+ return folio_batch_count(&folioq->vec);
+}
+
+static inline bool folioq_full(struct folio_queue *folioq)
+{
+ //return !folio_batch_space(&folioq->vec);
+ return folioq_count(folioq) >= folioq_nr_slots(folioq);
+}
+
+static inline bool folioq_is_marked(const struct folio_queue *folioq, unsigned int slot)
+{
+ return test_bit(slot, &folioq->marks);
+}
+
+static inline void folioq_mark(struct folio_queue *folioq, unsigned int slot)
+{
+ set_bit(slot, &folioq->marks);
+}
+
+static inline void folioq_unmark(struct folio_queue *folioq, unsigned int slot)
+{
+ clear_bit(slot, &folioq->marks);
+}
+
+static inline bool folioq_is_marked2(const struct folio_queue *folioq, unsigned int slot)
+{
+ return test_bit(slot, &folioq->marks2);
+}
+
+static inline void folioq_mark2(struct folio_queue *folioq, unsigned int slot)
+{
+ set_bit(slot, &folioq->marks2);
+}
+
+static inline void folioq_unmark2(struct folio_queue *folioq, unsigned int slot)
+{
+ clear_bit(slot, &folioq->marks2);
+}
+
+static inline bool folioq_is_marked3(const struct folio_queue *folioq, unsigned int slot)
+{
+ return test_bit(slot, &folioq->marks3);
+}
+
+static inline void folioq_mark3(struct folio_queue *folioq, unsigned int slot)
+{
+ set_bit(slot, &folioq->marks3);
+}
+
+static inline void folioq_unmark3(struct folio_queue *folioq, unsigned int slot)
+{
+ clear_bit(slot, &folioq->marks3);
+}
+
+static inline unsigned int __folio_order(struct folio *folio)
+{
+ if (!folio_test_large(folio))
+ return 0;
+ return folio->_flags_1 & 0xff;
+}
+
+static inline unsigned int folioq_append(struct folio_queue *folioq, struct folio *folio)
+{
+ unsigned int slot = folioq->vec.nr++;
+
+ folioq->vec.folios[slot] = folio;
+ folioq->orders[slot] = __folio_order(folio);
+ return slot;
+}
+
+static inline unsigned int folioq_append_mark(struct folio_queue *folioq, struct folio *folio)
+{
+ unsigned int slot = folioq->vec.nr++;
+
+ folioq->vec.folios[slot] = folio;
+ folioq->orders[slot] = __folio_order(folio);
+ folioq_mark(folioq, slot);
+ return slot;
+}
+
+static inline struct folio *folioq_folio(const struct folio_queue *folioq, unsigned int slot)
+{
+ return folioq->vec.folios[slot];
+}
+
+static inline unsigned int folioq_folio_order(const struct folio_queue *folioq, unsigned int slot)
+{
+ return folioq->orders[slot];
+}
+
+static inline size_t folioq_folio_size(const struct folio_queue *folioq, unsigned int slot)
+{
+ return PAGE_SIZE << folioq_folio_order(folioq, slot);
+}
+
+static inline void folioq_clear(struct folio_queue *folioq, unsigned int slot)
+{
+ folioq->vec.folios[slot] = NULL;
+ folioq_unmark(folioq, slot);
+ folioq_unmark2(folioq, slot);
+ folioq_unmark3(folioq, slot);
+}
+
+#endif /* _LINUX_FOLIO_QUEUE_H */
diff --git a/include/linux/iov_iter.h b/include/linux/iov_iter.h
index 270454a6703d..c4aa58032faf 100644
--- a/include/linux/iov_iter.h
+++ b/include/linux/iov_iter.h
@@ -10,6 +10,7 @@
#include <linux/uio.h>
#include <linux/bvec.h>
+#include <linux/folio_queue.h>
typedef size_t (*iov_step_f)(void *iter_base, size_t progress, size_t len,
void *priv, void *priv2);
@@ -141,6 +142,60 @@ size_t iterate_bvec(struct iov_iter *iter, size_t len, void *priv, void *priv2,
}
/*
+ * Handle ITER_FOLIOQ.
+ */
+static __always_inline
+size_t iterate_folioq(struct iov_iter *iter, size_t len, void *priv, void *priv2,
+ iov_step_f step)
+{
+ const struct folio_queue *folioq = iter->folioq;
+ unsigned int slot = iter->folioq_slot;
+ size_t progress = 0, skip = iter->iov_offset;
+
+ if (slot == folioq_nr_slots(folioq)) {
+ /* The iterator may have been extended. */
+ folioq = folioq->next;
+ slot = 0;
+ }
+
+ do {
+ struct folio *folio = folioq_folio(folioq, slot);
+ size_t part, remain, consumed;
+ size_t fsize;
+ void *base;
+
+ if (!folio)
+ break;
+
+ fsize = folioq_folio_size(folioq, slot);
+ base = kmap_local_folio(folio, skip);
+ part = umin(len, PAGE_SIZE - skip % PAGE_SIZE);
+ remain = step(base, progress, part, priv, priv2);
+ kunmap_local(base);
+ consumed = part - remain;
+ len -= consumed;
+ progress += consumed;
+ skip += consumed;
+ if (skip >= fsize) {
+ skip = 0;
+ slot++;
+ if (slot == folioq_nr_slots(folioq) && folioq->next) {
+ folioq = folioq->next;
+ slot = 0;
+ }
+ }
+ if (remain)
+ break;
+ } while (len);
+
+ iter->folioq_slot = slot;
+ iter->folioq = folioq;
+ iter->iov_offset = skip;
+ iter->count -= progress;
+ return progress;
+}
+
+/*
* Handle ITER_XARRAY.
*/
static __always_inline
@@ -249,6 +304,8 @@ size_t iterate_and_advance2(struct iov_iter *iter, size_t len, void *priv,
return iterate_bvec(iter, len, priv, priv2, step);
if (iov_iter_is_kvec(iter))
return iterate_kvec(iter, len, priv, priv2, step);
+ if (iov_iter_is_folioq(iter))
+ return iterate_folioq(iter, len, priv, priv2, step);
if (iov_iter_is_xarray(iter))
return iterate_xarray(iter, len, priv, priv2, step);
return iterate_discard(iter, len, priv, priv2, step);
@@ -271,4 +328,51 @@ size_t iterate_and_advance(struct iov_iter *iter, size_t len, void *priv,
return iterate_and_advance2(iter, len, priv, NULL, ustep, step);
}
+/**
+ * iterate_and_advance_kernel - Iterate over a kernel-internal iterator
+ * @iter: The iterator to iterate over.
+ * @len: The amount to iterate over.
+ * @priv: Data for the step functions.
+ * @priv2: More data for the step functions.
+ * @step: Function for other iterators; given kernel addresses.
+ *
+ * Iterate over the next part of an iterator, up to the specified length. The
+ * buffer is presented in segments, which for kernel iteration are broken up by
+ * physical pages and mapped, with the mapped address being presented.
+ *
+ * [!] Note This will only handle BVEC, KVEC, FOLIOQ, XARRAY and DISCARD-type
+ * iterators; it will not handle UBUF or IOVEC-type iterators.
+ *
+ * A step functions, @step, must be provided, one for handling mapped kernel
+ * addresses and the other is given user addresses which have the potential to
+ * fault since no pinning is performed.
+ *
+ * The step functions are passed the address and length of the segment, @priv,
+ * @priv2 and the amount of data so far iterated over (which can, for example,
+ * be added to @priv to point to the right part of a second buffer). The step
+ * functions should return the amount of the segment they didn't process (ie. 0
+ * indicates complete processsing).
+ *
+ * This function returns the amount of data processed (ie. 0 means nothing was
+ * processed and the value of @len means processes to completion).
+ */
+static __always_inline
+size_t iterate_and_advance_kernel(struct iov_iter *iter, size_t len, void *priv,
+ void *priv2, iov_step_f step)
+{
+ if (unlikely(iter->count < len))
+ len = iter->count;
+ if (unlikely(!len))
+ return 0;
+ if (iov_iter_is_bvec(iter))
+ return iterate_bvec(iter, len, priv, priv2, step);
+ if (iov_iter_is_kvec(iter))
+ return iterate_kvec(iter, len, priv, priv2, step);
+ if (iov_iter_is_folioq(iter))
+ return iterate_folioq(iter, len, priv, priv2, step);
+ if (iov_iter_is_xarray(iter))
+ return iterate_xarray(iter, len, priv, priv2, step);
+ return iterate_discard(iter, len, priv, priv2, step);
+}
+
#endif /* _LINUX_IOV_ITER_H */
diff --git a/include/linux/netfs.h b/include/linux/netfs.h
index c47443e7a97e..5eaceef41e6c 100644
--- a/include/linux/netfs.h
+++ b/include/linux/netfs.h
@@ -38,11 +38,8 @@ static inline void folio_start_private_2(struct folio *folio)
folio_set_private_2(folio);
}
-/* Marks used on xarray-based buffers */
-#define NETFS_BUF_PUT_MARK XA_MARK_0 /* - Page needs putting */
-#define NETFS_BUF_PAGECACHE_MARK XA_MARK_1 /* - Page needs wb/dirty flag wrangling */
-
enum netfs_io_source {
+ NETFS_SOURCE_UNKNOWN,
NETFS_FILL_WITH_ZEROES,
NETFS_DOWNLOAD_FROM_SERVER,
NETFS_READ_FROM_CACHE,
@@ -73,6 +70,7 @@ struct netfs_inode {
#define NETFS_ICTX_ODIRECT 0 /* The file has DIO in progress */
#define NETFS_ICTX_UNBUFFERED 1 /* I/O should not use the pagecache */
#define NETFS_ICTX_WRITETHROUGH 2 /* Write-through caching */
+#define NETFS_ICTX_MODIFIED_ATTR 3 /* Indicate change in mtime/ctime */
};
/*
@@ -133,9 +131,11 @@ static inline struct netfs_group *netfs_folio_group(struct folio *folio)
struct netfs_io_stream {
/* Submission tracking */
struct netfs_io_subrequest *construct; /* Op being constructed */
+ size_t sreq_max_len; /* Maximum size of a subrequest */
+ unsigned int sreq_max_segs; /* 0 or max number of segments in an iterator */
unsigned int submit_off; /* Folio offset we're submitting from */
unsigned int submit_len; /* Amount of data left to submit */
- unsigned int submit_max_len; /* Amount I/O can be rounded up to */
+ unsigned int submit_extendable_to; /* Amount I/O can be rounded up to */
void (*prepare_write)(struct netfs_io_subrequest *subreq);
void (*issue_write)(struct netfs_io_subrequest *subreq);
/* Collection tracking */
@@ -176,41 +176,45 @@ struct netfs_io_subrequest {
struct list_head rreq_link; /* Link in rreq->subrequests */
struct iov_iter io_iter; /* Iterator for this subrequest */
unsigned long long start; /* Where to start the I/O */
- size_t max_len; /* Maximum size of the I/O */
size_t len; /* Size of the I/O */
size_t transferred; /* Amount of data transferred */
+ size_t consumed; /* Amount of read data consumed */
+ size_t prev_donated; /* Amount of data donated from previous subreq */
+ size_t next_donated; /* Amount of data donated from next subreq */
refcount_t ref;
short error; /* 0 or error that occurred */
unsigned short debug_index; /* Index in list (for debugging output) */
unsigned int nr_segs; /* Number of segs in io_iter */
- unsigned int max_nr_segs; /* 0 or max number of segments in an iterator */
enum netfs_io_source source; /* Where to read from/write to */
unsigned char stream_nr; /* I/O stream this belongs to */
+ unsigned char curr_folioq_slot; /* Folio currently being read */
+ unsigned char curr_folio_order; /* Order of folio */
+ struct folio_queue *curr_folioq; /* Queue segment in which current folio resides */
unsigned long flags;
#define NETFS_SREQ_COPY_TO_CACHE 0 /* Set if should copy the data to the cache */
#define NETFS_SREQ_CLEAR_TAIL 1 /* Set if the rest of the read should be cleared */
-#define NETFS_SREQ_SHORT_IO 2 /* Set if the I/O was short */
#define NETFS_SREQ_SEEK_DATA_READ 3 /* Set if ->read() should SEEK_DATA first */
#define NETFS_SREQ_NO_PROGRESS 4 /* Set if we didn't manage to read any data */
#define NETFS_SREQ_ONDEMAND 5 /* Set if it's from on-demand read mode */
#define NETFS_SREQ_BOUNDARY 6 /* Set if ends on hard boundary (eg. ceph object) */
+#define NETFS_SREQ_HIT_EOF 7 /* Set if short due to EOF */
#define NETFS_SREQ_IN_PROGRESS 8 /* Unlocked when the subrequest completes */
#define NETFS_SREQ_NEED_RETRY 9 /* Set if the filesystem requests a retry */
#define NETFS_SREQ_RETRYING 10 /* Set if we're retrying */
#define NETFS_SREQ_FAILED 11 /* Set if the subreq failed unretryably */
-#define NETFS_SREQ_HIT_EOF 12 /* Set if we hit the EOF */
};
enum netfs_io_origin {
NETFS_READAHEAD, /* This read was triggered by readahead */
NETFS_READPAGE, /* This read is a synchronous read */
+ NETFS_READ_GAPS, /* This read is a synchronous read to fill gaps */
NETFS_READ_FOR_WRITE, /* This read is to prepare a write */
- NETFS_COPY_TO_CACHE, /* This write is to copy a read to the cache */
+ NETFS_DIO_READ, /* This is a direct I/O read */
NETFS_WRITEBACK, /* This write was triggered by writepages */
NETFS_WRITETHROUGH, /* This write was made by netfs_perform_write() */
NETFS_UNBUFFERED_WRITE, /* This is an unbuffered write */
- NETFS_DIO_READ, /* This is a direct I/O read */
NETFS_DIO_WRITE, /* This is a direct I/O write */
+ NETFS_PGPRIV2_COPY_TO_CACHE, /* [DEPRECATED] This is writing read data to the cache */
nr__netfs_io_origin
} __mode(byte);
@@ -227,11 +231,14 @@ struct netfs_io_request {
struct address_space *mapping; /* The mapping being accessed */
struct kiocb *iocb; /* AIO completion vector */
struct netfs_cache_resources cache_resources;
+ struct readahead_control *ractl; /* Readahead descriptor */
struct list_head proc_link; /* Link in netfs_iorequests */
struct list_head subrequests; /* Contributory I/O operations */
struct netfs_io_stream io_streams[2]; /* Streams of parallel I/O operations */
#define NR_IO_STREAMS 2 //wreq->nr_io_streams
struct netfs_group *group; /* Writeback group being written back */
+ struct folio_queue *buffer; /* Head of I/O buffer */
+ struct folio_queue *buffer_tail; /* Tail of I/O buffer */
struct iov_iter iter; /* Unencrypted-side iterator */
struct iov_iter io_iter; /* I/O (Encrypted-side) iterator */
void *netfs_priv; /* Private data for the netfs */
@@ -245,24 +252,23 @@ struct netfs_io_request {
unsigned int nr_group_rel; /* Number of refs to release on ->group */
spinlock_t lock; /* Lock for queuing subreqs */
atomic_t nr_outstanding; /* Number of ops in progress */
- atomic_t nr_copy_ops; /* Number of copy-to-cache ops in progress */
- size_t upper_len; /* Length can be extended to here */
unsigned long long submitted; /* Amount submitted for I/O so far */
unsigned long long len; /* Length of the request */
size_t transferred; /* Amount to be indicated as transferred */
- short error; /* 0 or error that occurred */
+ long error; /* 0 or error that occurred */
enum netfs_io_origin origin; /* Origin of the request */
bool direct_bv_unpin; /* T if direct_bv[] must be unpinned */
+ u8 buffer_head_slot; /* First slot in ->buffer */
+ u8 buffer_tail_slot; /* Next slot in ->buffer_tail */
unsigned long long i_size; /* Size of the file */
unsigned long long start; /* Start position */
atomic64_t issued_to; /* Write issuer folio cursor */
- unsigned long long contiguity; /* Tracking for gaps in the writeback sequence */
unsigned long long collected_to; /* Point we've collected to */
unsigned long long cleaned_to; /* Position we've cleaned folios to */
pgoff_t no_unlock_folio; /* Don't unlock this folio after read */
+ size_t prev_donated; /* Fallback for subreq->prev_donated */
refcount_t ref;
unsigned long flags;
-#define NETFS_RREQ_INCOMPLETE_IO 0 /* Some ioreqs terminated short or with error */
#define NETFS_RREQ_COPY_TO_CACHE 1 /* Need to write to the cache */
#define NETFS_RREQ_NO_UNLOCK_FOLIO 2 /* Don't unlock no_unlock_folio on completion */
#define NETFS_RREQ_DONT_UNLOCK_FOLIOS 3 /* Don't unlock the folios on completion */
@@ -274,6 +280,7 @@ struct netfs_io_request {
#define NETFS_RREQ_PAUSE 11 /* Pause subrequest generation */
#define NETFS_RREQ_USE_IO_ITER 12 /* Use ->io_iter rather than ->i_pages */
#define NETFS_RREQ_ALL_QUEUED 13 /* All subreqs are now queued */
+#define NETFS_RREQ_NEED_RETRY 14 /* Need to try retrying */
#define NETFS_RREQ_USE_PGPRIV2 31 /* [DEPRECATED] Use PG_private_2 to mark
* write to cache on read */
const struct netfs_request_ops *netfs_ops;
@@ -292,7 +299,7 @@ struct netfs_request_ops {
/* Read request handling */
void (*expand_readahead)(struct netfs_io_request *rreq);
- bool (*clamp_length)(struct netfs_io_subrequest *subreq);
+ int (*prepare_read)(struct netfs_io_subrequest *subreq);
void (*issue_read)(struct netfs_io_subrequest *subreq);
bool (*is_still_valid)(struct netfs_io_request *rreq);
int (*check_write_begin)(struct file *file, loff_t pos, unsigned len,
@@ -422,7 +429,10 @@ bool netfs_release_folio(struct folio *folio, gfp_t gfp);
vm_fault_t netfs_page_mkwrite(struct vm_fault *vmf, struct netfs_group *netfs_group);
/* (Sub)request management API. */
-void netfs_subreq_terminated(struct netfs_io_subrequest *, ssize_t, bool);
+void netfs_read_subreq_progress(struct netfs_io_subrequest *subreq,
+ bool was_async);
+void netfs_read_subreq_terminated(struct netfs_io_subrequest *subreq,
+ int error, bool was_async);
void netfs_get_subrequest(struct netfs_io_subrequest *subreq,
enum netfs_sreq_ref_trace what);
void netfs_put_subrequest(struct netfs_io_subrequest *subreq,
diff --git a/include/linux/uio.h b/include/linux/uio.h
index 7020adedfa08..853f9de5aa05 100644
--- a/include/linux/uio.h
+++ b/include/linux/uio.h
@@ -11,6 +11,7 @@
#include <uapi/linux/uio.h>
struct page;
+struct folio_queue;
typedef unsigned int __bitwise iov_iter_extraction_t;
@@ -25,6 +26,7 @@ enum iter_type {
ITER_IOVEC,
ITER_BVEC,
ITER_KVEC,
+ ITER_FOLIOQ,
ITER_XARRAY,
ITER_DISCARD,
};
@@ -66,6 +68,7 @@ struct iov_iter {
const struct iovec *__iov;
const struct kvec *kvec;
const struct bio_vec *bvec;
+ const struct folio_queue *folioq;
struct xarray *xarray;
void __user *ubuf;
};
@@ -74,6 +77,7 @@ struct iov_iter {
};
union {
unsigned long nr_segs;
+ u8 folioq_slot;
loff_t xarray_start;
};
};
@@ -126,6 +130,11 @@ static inline bool iov_iter_is_discard(const struct iov_iter *i)
return iov_iter_type(i) == ITER_DISCARD;
}
+static inline bool iov_iter_is_folioq(const struct iov_iter *i)
+{
+ return iov_iter_type(i) == ITER_FOLIOQ;
+}
+
static inline bool iov_iter_is_xarray(const struct iov_iter *i)
{
return iov_iter_type(i) == ITER_XARRAY;
@@ -180,6 +189,12 @@ static inline size_t copy_folio_to_iter(struct folio *folio, size_t offset,
return copy_page_to_iter(&folio->page, offset, bytes, i);
}
+static inline size_t copy_folio_from_iter(struct folio *folio, size_t offset,
+ size_t bytes, struct iov_iter *i)
+{
+ return copy_page_from_iter(&folio->page, offset, bytes, i);
+}
+
static inline size_t copy_folio_from_iter_atomic(struct folio *folio,
size_t offset, size_t bytes, struct iov_iter *i)
{
@@ -273,6 +288,9 @@ void iov_iter_kvec(struct iov_iter *i, unsigned int direction, const struct kvec
void iov_iter_bvec(struct iov_iter *i, unsigned int direction, const struct bio_vec *bvec,
unsigned long nr_segs, size_t count);
void iov_iter_discard(struct iov_iter *i, unsigned int direction, size_t count);
+void iov_iter_folio_queue(struct iov_iter *i, unsigned int direction,
+ const struct folio_queue *folioq,
+ unsigned int first_slot, unsigned int offset, size_t count);
void iov_iter_xarray(struct iov_iter *i, unsigned int direction, struct xarray *xarray,
loff_t start, size_t count);
ssize_t iov_iter_get_pages2(struct iov_iter *i, struct page **pages,