summaryrefslogtreecommitdiff
path: root/fs/netfs/read_collect.c
diff options
context:
space:
mode:
authorDavid Howells <dhowells@redhat.com>2024-12-16 20:41:17 +0000
committerChristian Brauner <brauner@kernel.org>2024-12-20 22:34:08 +0100
commite2d46f2ec332533816417b60933954173f602121 (patch)
tree53b798c049ae403a978b3a81739a3aa13345b075 /fs/netfs/read_collect.c
parenteddf51f2bb2c28b082199c6f5fd95611ca511135 (diff)
netfs: Change the read result collector to only use one work item
Change the way netfslib collects read results to do all the collection for a particular read request using a single work item that walks along the subrequest queue as subrequests make progress or complete, unlocking folios progressively rather than doing the unlock in parallel as parallel requests come in. The code is remodelled to be more like the write-side code, though only using a single stream. This makes it more directly comparable and thus easier to duplicate fixes between the two sides. This has a number of advantages: (1) It's simpler. There doesn't need to be a complex donation mechanism to handle mismatches between the size and alignment of subrequests and folios. The collector unlocks folios as the subrequests covering each complete. (2) It should cause less scheduler overhead as there's a single work item in play unlocking pages in parallel when a read gets split up into a lot of subrequests instead of one per subrequest. Whilst the parallellism is nice in theory, in practice, the vast majority of loads are sequential reads of the whole file, so committing a bunch of threads to unlocking folios out of order doesn't help in those cases. (3) It should make it easier to implement content decryption. A folio cannot be decrypted until all the requests that contribute to it have completed - and, again, most loads are sequential and so, most of the time, we want to begin decryption sequentially (though it's great if the decryption can happen in parallel). There is a disadvantage in that we're losing the ability to decrypt and unlock things on an as-things-arrive basis which may affect some applications. Signed-off-by: David Howells <dhowells@redhat.com> Link: https://lore.kernel.org/r/20241216204124.3752367-28-dhowells@redhat.com cc: Jeff Layton <jlayton@kernel.org> cc: netfs@lists.linux.dev cc: linux-fsdevel@vger.kernel.org Signed-off-by: Christian Brauner <brauner@kernel.org>
Diffstat (limited to 'fs/netfs/read_collect.c')
-rw-r--r--fs/netfs/read_collect.c716
1 files changed, 419 insertions, 297 deletions
diff --git a/fs/netfs/read_collect.c b/fs/netfs/read_collect.c
index 2e9291ab1d62..f65affa5a9e4 100644
--- a/fs/netfs/read_collect.c
+++ b/fs/netfs/read_collect.c
@@ -14,6 +14,14 @@
#include <linux/task_io_accounting_ops.h>
#include "internal.h"
+/* Notes made in the collector */
+#define HIT_PENDING 0x01 /* A front op was still pending */
+#define MADE_PROGRESS 0x04 /* Made progress cleaning up a stream or the folio set */
+#define BUFFERED 0x08 /* The pagecache needs cleaning up */
+#define NEED_RETRY 0x10 /* A front op requests retrying */
+#define COPY_TO_CACHE 0x40 /* Need to copy subrequest to cache */
+#define ABANDON_SREQ 0x80 /* Need to abandon untransferred part of subrequest */
+
/*
* Clear the unread part of an I/O request.
*/
@@ -31,14 +39,18 @@ static void netfs_clear_unread(struct netfs_io_subrequest *subreq)
* cache the folio, we set the group to NETFS_FOLIO_COPY_TO_CACHE, mark it
* dirty and let writeback handle it.
*/
-static void netfs_unlock_read_folio(struct netfs_io_subrequest *subreq,
- struct netfs_io_request *rreq,
+static void netfs_unlock_read_folio(struct netfs_io_request *rreq,
struct folio_queue *folioq,
int slot)
{
struct netfs_folio *finfo;
struct folio *folio = folioq_folio(folioq, slot);
+ if (unlikely(folio_pos(folio) < rreq->abandon_to)) {
+ trace_netfs_folio(folio, netfs_folio_trace_abandon);
+ goto just_unlock;
+ }
+
flush_dcache_folio(folio);
folio_mark_uptodate(folio);
@@ -53,7 +65,7 @@ static void netfs_unlock_read_folio(struct netfs_io_subrequest *subreq,
kfree(finfo);
}
- if (test_bit(NETFS_SREQ_COPY_TO_CACHE, &subreq->flags)) {
+ if (test_bit(NETFS_RREQ_FOLIO_COPY_TO_CACHE, &rreq->flags)) {
if (!WARN_ON_ONCE(folio_get_private(folio) != NULL)) {
trace_netfs_folio(folio, netfs_folio_trace_copy_to_cache);
folio_attach_private(folio, NETFS_FOLIO_COPY_TO_CACHE);
@@ -66,12 +78,11 @@ static void netfs_unlock_read_folio(struct netfs_io_subrequest *subreq,
folioq_clear(folioq, slot);
} else {
// TODO: Use of PG_private_2 is deprecated.
- if (test_bit(NETFS_SREQ_COPY_TO_CACHE, &subreq->flags))
- netfs_pgpriv2_mark_copy_to_cache(subreq, rreq, folioq, slot);
- else
- folioq_clear(folioq, slot);
+ if (test_bit(NETFS_RREQ_FOLIO_COPY_TO_CACHE, &rreq->flags))
+ netfs_pgpriv2_copy_to_cache(rreq, folio);
}
+just_unlock:
if (!test_bit(NETFS_RREQ_DONT_UNLOCK_FOLIOS, &rreq->flags)) {
if (folio->index == rreq->no_unlock_folio &&
test_bit(NETFS_RREQ_NO_UNLOCK_FOLIO, &rreq->flags)) {
@@ -81,241 +92,249 @@ static void netfs_unlock_read_folio(struct netfs_io_subrequest *subreq,
folio_unlock(folio);
}
}
+
+ folioq_clear(folioq, slot);
}
/*
- * Unlock any folios that are now completely read. Returns true if the
- * subrequest is removed from the list.
+ * Unlock any folios we've finished with.
*/
-static bool netfs_consume_read_data(struct netfs_io_subrequest *subreq)
+static void netfs_read_unlock_folios(struct netfs_io_request *rreq,
+ unsigned int *notes)
{
- struct netfs_io_subrequest *prev, *next;
- struct netfs_io_request *rreq = subreq->rreq;
- struct folio_queue *folioq = subreq->curr_folioq;
- size_t avail, prev_donated, next_donated, fsize, part, excess;
- loff_t fpos, start;
- loff_t fend;
- int slot = subreq->curr_folioq_slot;
-
- if (WARN(subreq->transferred > subreq->len,
- "Subreq overread: R%x[%x] %zu > %zu",
- rreq->debug_id, subreq->debug_index,
- subreq->transferred, subreq->len))
- subreq->transferred = subreq->len;
-
- trace_netfs_folioq(folioq, netfs_trace_folioq_read_progress);
-next_folio:
- fsize = PAGE_SIZE << subreq->curr_folio_order;
- fpos = round_down(subreq->start + subreq->consumed, fsize);
- fend = fpos + fsize;
-
- if (WARN_ON_ONCE(!folioq) ||
- WARN_ON_ONCE(!folioq_folio(folioq, slot)) ||
- WARN_ON_ONCE(folioq_folio(folioq, slot)->index != fpos / PAGE_SIZE)) {
- pr_err("R=%08x[%x] s=%llx-%llx ctl=%zx/%zx/%zx sl=%u\n",
- rreq->debug_id, subreq->debug_index,
- subreq->start, subreq->start + subreq->transferred - 1,
- subreq->consumed, subreq->transferred, subreq->len,
- slot);
- if (folioq) {
- struct folio *folio = folioq_folio(folioq, slot);
-
- pr_err("folioq: fq=%x orders=%02x%02x%02x%02x %px\n",
- folioq->debug_id,
- folioq->orders[0], folioq->orders[1],
- folioq->orders[2], folioq->orders[3],
- folioq);
- if (folio)
- pr_err("folio: %llx-%llx ix=%llx o=%u qo=%u\n",
- fpos, fend - 1, folio_pos(folio), folio_order(folio),
- folioq_folio_order(folioq, slot));
- }
- }
+ struct folio_queue *folioq = rreq->buffer.tail;
+ unsigned long long collected_to = rreq->collected_to;
+ unsigned int slot = rreq->buffer.first_tail_slot;
-donation_changed:
- /* Try to consume the current folio if we've hit or passed the end of
- * it. There's a possibility that this subreq doesn't start at the
- * beginning of the folio, in which case we need to donate to/from the
- * preceding subreq.
- *
- * We also need to include any potential donation back from the
- * following subreq.
- */
- prev_donated = READ_ONCE(subreq->prev_donated);
- next_donated = READ_ONCE(subreq->next_donated);
- if (prev_donated || next_donated) {
- spin_lock(&rreq->lock);
- prev_donated = subreq->prev_donated;
- next_donated = subreq->next_donated;
- subreq->start -= prev_donated;
- subreq->len += prev_donated;
- subreq->transferred += prev_donated;
- prev_donated = subreq->prev_donated = 0;
- if (subreq->transferred == subreq->len) {
- subreq->len += next_donated;
- subreq->transferred += next_donated;
- next_donated = subreq->next_donated = 0;
+ if (rreq->cleaned_to >= rreq->collected_to)
+ return;
+
+ // TODO: Begin decryption
+
+ if (slot >= folioq_nr_slots(folioq)) {
+ folioq = rolling_buffer_delete_spent(&rreq->buffer);
+ if (!folioq) {
+ rreq->front_folio_order = 0;
+ return;
}
- trace_netfs_sreq(subreq, netfs_sreq_trace_add_donations);
- spin_unlock(&rreq->lock);
+ slot = 0;
}
- avail = subreq->transferred;
- if (avail == subreq->len)
- avail += next_donated;
- start = subreq->start;
- if (subreq->consumed == 0) {
- start -= prev_donated;
- avail += prev_donated;
- } else {
- start += subreq->consumed;
- avail -= subreq->consumed;
- }
- part = umin(avail, fsize);
-
- trace_netfs_progress(subreq, start, avail, part);
-
- if (start + avail >= fend) {
- if (fpos == start) {
- /* Flush, unlock and mark for caching any folio we've just read. */
- subreq->consumed = fend - subreq->start;
- netfs_unlock_read_folio(subreq, rreq, folioq, slot);
- folioq_mark2(folioq, slot);
- if (subreq->consumed >= subreq->len)
- goto remove_subreq;
- } else if (fpos < start) {
- excess = fend - subreq->start;
-
- spin_lock(&rreq->lock);
- /* If we complete first on a folio split with the
- * preceding subreq, donate to that subreq - otherwise
- * we get the responsibility.
- */
- if (subreq->prev_donated != prev_donated) {
- spin_unlock(&rreq->lock);
- goto donation_changed;
- }
+ for (;;) {
+ struct folio *folio;
+ unsigned long long fpos, fend;
+ unsigned int order;
+ size_t fsize;
- if (list_is_first(&subreq->rreq_link, &rreq->subrequests)) {
- spin_unlock(&rreq->lock);
- pr_err("Can't donate prior to front\n");
- goto bad;
- }
+ if (*notes & COPY_TO_CACHE)
+ set_bit(NETFS_RREQ_FOLIO_COPY_TO_CACHE, &rreq->flags);
- prev = list_prev_entry(subreq, rreq_link);
- WRITE_ONCE(prev->next_donated, prev->next_donated + excess);
- subreq->start += excess;
- subreq->len -= excess;
- subreq->transferred -= excess;
- trace_netfs_donate(rreq, subreq, prev, excess,
- netfs_trace_donate_tail_to_prev);
- trace_netfs_sreq(subreq, netfs_sreq_trace_donate_to_prev);
-
- if (subreq->consumed >= subreq->len)
- goto remove_subreq_locked;
- spin_unlock(&rreq->lock);
- } else {
- pr_err("fpos > start\n");
- goto bad;
- }
+ folio = folioq_folio(folioq, slot);
+ if (WARN_ONCE(!folio_test_locked(folio),
+ "R=%08x: folio %lx is not locked\n",
+ rreq->debug_id, folio->index))
+ trace_netfs_folio(folio, netfs_folio_trace_not_locked);
- /* Advance the rolling buffer to the next folio. */
+ order = folioq_folio_order(folioq, slot);
+ rreq->front_folio_order = order;
+ fsize = PAGE_SIZE << order;
+ fpos = folio_pos(folio);
+ fend = umin(fpos + fsize, rreq->i_size);
+
+ trace_netfs_collect_folio(rreq, folio, fend, collected_to);
+
+ /* Unlock any folio we've transferred all of. */
+ if (collected_to < fend)
+ break;
+
+ netfs_unlock_read_folio(rreq, folioq, slot);
+ WRITE_ONCE(rreq->cleaned_to, fpos + fsize);
+ *notes |= MADE_PROGRESS;
+
+ clear_bit(NETFS_RREQ_FOLIO_COPY_TO_CACHE, &rreq->flags);
+
+ /* Clean up the head folioq. If we clear an entire folioq, then
+ * we can get rid of it provided it's not also the tail folioq
+ * being filled by the issuer.
+ */
+ folioq_clear(folioq, slot);
slot++;
if (slot >= folioq_nr_slots(folioq)) {
+ folioq = rolling_buffer_delete_spent(&rreq->buffer);
+ if (!folioq)
+ goto done;
slot = 0;
- folioq = folioq->next;
- subreq->curr_folioq = folioq;
trace_netfs_folioq(folioq, netfs_trace_folioq_read_progress);
}
- subreq->curr_folioq_slot = slot;
- if (folioq && folioq_folio(folioq, slot))
- subreq->curr_folio_order = folioq->orders[slot];
- cond_resched();
- goto next_folio;
+
+ if (fpos + fsize >= collected_to)
+ break;
}
- /* Deal with partial progress. */
- if (subreq->transferred < subreq->len)
- return false;
+ rreq->buffer.tail = folioq;
+done:
+ rreq->buffer.first_tail_slot = slot;
+}
- /* Donate the remaining downloaded data to one of the neighbouring
- * subrequests. Note that we may race with them doing the same thing.
+/*
+ * Collect and assess the results of various read subrequests. We may need to
+ * retry some of the results.
+ *
+ * Note that we have a sequence of subrequests, which may be drawing on
+ * different sources and may or may not be the same size or starting position
+ * and may not even correspond in boundary alignment.
+ */
+static void netfs_collect_read_results(struct netfs_io_request *rreq)
+{
+ struct netfs_io_subrequest *front, *remove;
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
+ unsigned int notes;
+
+ _enter("%llx-%llx", rreq->start, rreq->start + rreq->len);
+ trace_netfs_rreq(rreq, netfs_rreq_trace_collect);
+ trace_netfs_collect(rreq);
+
+reassess:
+ if (rreq->origin == NETFS_READAHEAD ||
+ rreq->origin == NETFS_READPAGE ||
+ rreq->origin == NETFS_READ_FOR_WRITE)
+ notes = BUFFERED;
+ else
+ notes = 0;
+
+ /* Remove completed subrequests from the front of the stream and
+ * advance the completion point. We stop when we hit something that's
+ * in progress. The issuer thread may be adding stuff to the tail
+ * whilst we're doing this.
*/
- spin_lock(&rreq->lock);
+ front = READ_ONCE(stream->front);
+ while (front) {
+ size_t transferred;
+
+ trace_netfs_collect_sreq(rreq, front);
+ _debug("sreq [%x] %llx %zx/%zx",
+ front->debug_index, front->start, front->transferred, front->len);
+
+ if (stream->collected_to < front->start) {
+ trace_netfs_collect_gap(rreq, stream, front->start, 'F');
+ stream->collected_to = front->start;
+ }
+
+ if (test_bit(NETFS_SREQ_IN_PROGRESS, &front->flags))
+ notes |= HIT_PENDING;
+ smp_rmb(); /* Read counters after IN_PROGRESS flag. */
+ transferred = READ_ONCE(front->transferred);
+
+ /* If we can now collect the next folio, do so. We don't want
+ * to defer this as we have to decide whether we need to copy
+ * to the cache or not, and that may differ between adjacent
+ * subreqs.
+ */
+ if (notes & BUFFERED) {
+ size_t fsize = PAGE_SIZE << rreq->front_folio_order;
+
+ /* Clear the tail of a short read. */
+ if (!(notes & HIT_PENDING) &&
+ front->error == 0 &&
+ transferred < front->len &&
+ (test_bit(NETFS_SREQ_HIT_EOF, &front->flags) ||
+ test_bit(NETFS_SREQ_CLEAR_TAIL, &front->flags))) {
+ netfs_clear_unread(front);
+ transferred = front->transferred = front->len;
+ trace_netfs_sreq(front, netfs_sreq_trace_clear);
+ }
- if (subreq->prev_donated != prev_donated ||
- subreq->next_donated != next_donated) {
+ stream->collected_to = front->start + transferred;
+ rreq->collected_to = stream->collected_to;
+
+ if (test_bit(NETFS_SREQ_COPY_TO_CACHE, &front->flags))
+ notes |= COPY_TO_CACHE;
+
+ if (test_bit(NETFS_SREQ_FAILED, &front->flags)) {
+ rreq->abandon_to = front->start + front->len;
+ front->transferred = front->len;
+ transferred = front->len;
+ trace_netfs_rreq(rreq, netfs_rreq_trace_set_abandon);
+ }
+ if (front->start + transferred >= rreq->cleaned_to + fsize ||
+ test_bit(NETFS_SREQ_HIT_EOF, &front->flags))
+ netfs_read_unlock_folios(rreq, &notes);
+ } else {
+ stream->collected_to = front->start + transferred;
+ rreq->collected_to = stream->collected_to;
+ }
+
+ /* Stall if the front is still undergoing I/O. */
+ if (notes & HIT_PENDING)
+ break;
+
+ if (test_bit(NETFS_SREQ_FAILED, &front->flags)) {
+ if (!stream->failed) {
+ stream->error = front->error;
+ rreq->error = front->error;
+ set_bit(NETFS_RREQ_FAILED, &rreq->flags);
+ stream->failed = true;
+ }
+ notes |= MADE_PROGRESS | ABANDON_SREQ;
+ } else if (test_bit(NETFS_SREQ_NEED_RETRY, &front->flags)) {
+ stream->need_retry = true;
+ notes |= NEED_RETRY | MADE_PROGRESS;
+ break;
+ } else {
+ if (!stream->failed)
+ stream->transferred = stream->collected_to - rreq->start;
+ notes |= MADE_PROGRESS;
+ }
+
+ /* Remove if completely consumed. */
+ stream->source = front->source;
+ spin_lock(&rreq->lock);
+
+ remove = front;
+ trace_netfs_sreq(front, netfs_sreq_trace_discard);
+ list_del_init(&front->rreq_link);
+ front = list_first_entry_or_null(&stream->subrequests,
+ struct netfs_io_subrequest, rreq_link);
+ stream->front = front;
spin_unlock(&rreq->lock);
- cond_resched();
- goto donation_changed;
+ netfs_put_subrequest(remove, false,
+ notes & ABANDON_SREQ ?
+ netfs_sreq_trace_put_abandon :
+ netfs_sreq_trace_put_done);
}
- /* Deal with the trickiest case: that this subreq is in the middle of a
- * folio, not touching either edge, but finishes first. In such a
- * case, we donate to the previous subreq, if there is one and if it is
- * contiguous, so that the donation is only handled when that completes
- * - and remove this subreq from the list.
- *
- * If the previous subreq finished first, we will have acquired their
- * donation and should be able to unlock folios and/or donate nextwards.
- */
- if (!subreq->consumed &&
- !prev_donated &&
- !list_is_first(&subreq->rreq_link, &rreq->subrequests) &&
- subreq->start == prev->start + prev->len) {
- prev = list_prev_entry(subreq, rreq_link);
- WRITE_ONCE(prev->next_donated, prev->next_donated + subreq->len);
- subreq->start += subreq->len;
- subreq->len = 0;
- subreq->transferred = 0;
- trace_netfs_donate(rreq, subreq, prev, subreq->len,
- netfs_trace_donate_to_prev);
- trace_netfs_sreq(subreq, netfs_sreq_trace_donate_to_prev);
- goto remove_subreq_locked;
- }
+ trace_netfs_collect_stream(rreq, stream);
+ trace_netfs_collect_state(rreq, rreq->collected_to, notes);
- /* If we can't donate down the chain, donate up the chain instead. */
- excess = subreq->len - subreq->consumed + next_donated;
+ if (!(notes & BUFFERED))
+ rreq->cleaned_to = rreq->collected_to;
- if (!subreq->consumed)
- excess += prev_donated;
+ if (notes & NEED_RETRY)
+ goto need_retry;
+ if ((notes & MADE_PROGRESS) && test_bit(NETFS_RREQ_PAUSE, &rreq->flags)) {
+ trace_netfs_rreq(rreq, netfs_rreq_trace_unpause);
+ clear_bit_unlock(NETFS_RREQ_PAUSE, &rreq->flags);
+ smp_mb__after_atomic(); /* Set PAUSE before task state */
+ wake_up(&rreq->waitq);
+ }
- if (list_is_last(&subreq->rreq_link, &rreq->subrequests)) {
- rreq->prev_donated = excess;
- trace_netfs_donate(rreq, subreq, NULL, excess,
- netfs_trace_donate_to_deferred_next);
- } else {
- next = list_next_entry(subreq, rreq_link);
- WRITE_ONCE(next->prev_donated, excess);
- trace_netfs_donate(rreq, subreq, next, excess,
- netfs_trace_donate_to_next);
+ if (notes & MADE_PROGRESS) {
+ //cond_resched();
+ goto reassess;
}
- trace_netfs_sreq(subreq, netfs_sreq_trace_donate_to_next);
- subreq->len = subreq->consumed;
- subreq->transferred = subreq->consumed;
- goto remove_subreq_locked;
-
-remove_subreq:
- spin_lock(&rreq->lock);
-remove_subreq_locked:
- subreq->consumed = subreq->len;
- list_del(&subreq->rreq_link);
- spin_unlock(&rreq->lock);
- netfs_put_subrequest(subreq, false, netfs_sreq_trace_put_consumed);
- return true;
-
-bad:
- /* Errr... prev and next both donated to us, but insufficient to finish
- * the folio.
+
+out:
+ _leave(" = %x", notes);
+ return;
+
+need_retry:
+ /* Okay... We're going to have to retry parts of the stream. Note
+ * that any partially completed op will have had any wholly transferred
+ * folios removed from it.
*/
- printk("R=%08x[%x] s=%llx-%llx %zx/%zx/%zx\n",
- rreq->debug_id, subreq->debug_index,
- subreq->start, subreq->start + subreq->transferred - 1,
- subreq->consumed, subreq->transferred, subreq->len);
- printk("folio: %llx-%llx\n", fpos, fend - 1);
- printk("donated: prev=%zx next=%zx\n", prev_donated, next_donated);
- printk("s=%llx av=%zx part=%zx\n", start, avail, part);
- BUG();
+ _debug("retry");
+ netfs_retry_reads(rreq);
+ goto out;
}
/*
@@ -324,12 +343,13 @@ bad:
static void netfs_rreq_assess_dio(struct netfs_io_request *rreq)
{
struct netfs_io_subrequest *subreq;
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
unsigned int i;
/* Collect unbuffered reads and direct reads, adding up the transfer
* sizes until we find the first short or failed subrequest.
*/
- list_for_each_entry(subreq, &rreq->subrequests, rreq_link) {
+ list_for_each_entry(subreq, &stream->subrequests, rreq_link) {
rreq->transferred += subreq->transferred;
if (subreq->transferred < subreq->len ||
@@ -366,22 +386,12 @@ static void netfs_rreq_assess_dio(struct netfs_io_request *rreq)
*/
static void netfs_rreq_assess_single(struct netfs_io_request *rreq)
{
- struct netfs_io_subrequest *subreq;
struct netfs_io_stream *stream = &rreq->io_streams[0];
- subreq = list_first_entry_or_null(&stream->subrequests,
- struct netfs_io_subrequest, rreq_link);
- if (subreq) {
- if (test_bit(NETFS_SREQ_FAILED, &subreq->flags))
- rreq->error = subreq->error;
- else
- rreq->transferred = subreq->transferred;
-
- if (!rreq->error && subreq->source == NETFS_DOWNLOAD_FROM_SERVER &&
- fscache_resources_valid(&rreq->cache_resources)) {
- trace_netfs_rreq(rreq, netfs_rreq_trace_dirty);
- netfs_single_mark_inode_dirty(rreq->inode);
- }
+ if (!rreq->error && stream->source == NETFS_DOWNLOAD_FROM_SERVER &&
+ fscache_resources_valid(&rreq->cache_resources)) {
+ trace_netfs_rreq(rreq, netfs_rreq_trace_dirty);
+ netfs_single_mark_inode_dirty(rreq->inode);
}
if (rreq->iocb) {
@@ -395,21 +405,32 @@ static void netfs_rreq_assess_single(struct netfs_io_request *rreq)
}
/*
- * Assess the state of a read request and decide what to do next.
+ * Perform the collection of subrequests and folios.
*
* Note that we're in normal kernel thread context at this point, possibly
* running on a workqueue.
*/
-void netfs_rreq_terminated(struct netfs_io_request *rreq)
+static void netfs_read_collection(struct netfs_io_request *rreq)
{
- trace_netfs_rreq(rreq, netfs_rreq_trace_assess);
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
- //netfs_rreq_is_still_valid(rreq);
+ netfs_collect_read_results(rreq);
- if (test_and_clear_bit(NETFS_RREQ_NEED_RETRY, &rreq->flags)) {
- netfs_retry_reads(rreq);
+ /* We're done when the app thread has finished posting subreqs and the
+ * queue is empty.
+ */
+ if (!test_bit(NETFS_RREQ_ALL_QUEUED, &rreq->flags))
+ return;
+ smp_rmb(); /* Read ALL_QUEUED before subreq lists. */
+
+ if (!list_empty(&stream->subrequests))
return;
- }
+
+ /* Okay, declare that all I/O is complete. */
+ rreq->transferred = stream->transferred;
+ trace_netfs_rreq(rreq, netfs_rreq_trace_complete);
+
+ //netfs_rreq_is_still_valid(rreq);
switch (rreq->origin) {
case NETFS_DIO_READ:
@@ -430,8 +451,35 @@ void netfs_rreq_terminated(struct netfs_io_request *rreq)
trace_netfs_rreq(rreq, netfs_rreq_trace_done);
netfs_clear_subrequests(rreq, false);
netfs_unlock_abandoned_read_pages(rreq);
- if (unlikely(test_bit(NETFS_RREQ_USE_PGPRIV2, &rreq->flags)))
- netfs_pgpriv2_write_to_the_cache(rreq);
+ if (unlikely(rreq->copy_to_cache))
+ netfs_pgpriv2_end_copy_to_cache(rreq);
+}
+
+void netfs_read_collection_worker(struct work_struct *work)
+{
+ struct netfs_io_request *rreq = container_of(work, struct netfs_io_request, work);
+
+ netfs_see_request(rreq, netfs_rreq_trace_see_work);
+ if (test_bit(NETFS_RREQ_IN_PROGRESS, &rreq->flags))
+ netfs_read_collection(rreq);
+ netfs_put_request(rreq, false, netfs_rreq_trace_put_work);
+}
+
+/*
+ * Wake the collection work item.
+ */
+void netfs_wake_read_collector(struct netfs_io_request *rreq)
+{
+ if (test_bit(NETFS_RREQ_OFFLOAD_COLLECTION, &rreq->flags)) {
+ if (!work_pending(&rreq->work)) {
+ netfs_get_request(rreq, netfs_rreq_trace_get_work);
+ if (!queue_work(system_unbound_wq, &rreq->work))
+ netfs_put_request(rreq, true, netfs_rreq_trace_put_work_nq);
+ }
+ } else {
+ trace_netfs_rreq(rreq, netfs_rreq_trace_wake_queue);
+ wake_up(&rreq->waitq);
+ }
}
/**
@@ -447,17 +495,22 @@ void netfs_rreq_terminated(struct netfs_io_request *rreq)
void netfs_read_subreq_progress(struct netfs_io_subrequest *subreq)
{
struct netfs_io_request *rreq = subreq->rreq;
-
- might_sleep();
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
+ size_t fsize = PAGE_SIZE << rreq->front_folio_order;
trace_netfs_sreq(subreq, netfs_sreq_trace_progress);
- if (subreq->transferred > subreq->consumed &&
+ /* If we are at the head of the queue, wake up the collector,
+ * getting a ref to it if we were the ones to do so.
+ */
+ if (subreq->start + subreq->transferred > rreq->cleaned_to + fsize &&
(rreq->origin == NETFS_READAHEAD ||
rreq->origin == NETFS_READPAGE ||
- rreq->origin == NETFS_READ_FOR_WRITE)) {
- netfs_consume_read_data(subreq);
+ rreq->origin == NETFS_READ_FOR_WRITE) &&
+ list_is_first(&subreq->rreq_link, &stream->subrequests)
+ ) {
__set_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags);
+ netfs_wake_read_collector(rreq);
}
}
EXPORT_SYMBOL(netfs_read_subreq_progress);
@@ -481,8 +534,7 @@ EXPORT_SYMBOL(netfs_read_subreq_progress);
void netfs_read_subreq_terminated(struct netfs_io_subrequest *subreq)
{
struct netfs_io_request *rreq = subreq->rreq;
-
- might_sleep();
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
switch (subreq->source) {
case NETFS_READ_FROM_CACHE:
@@ -495,86 +547,156 @@ void netfs_read_subreq_terminated(struct netfs_io_subrequest *subreq)
break;
}
- if (rreq->origin != NETFS_DIO_READ) {
- /* Collect buffered reads.
- *
- * If the read completed validly short, then we can clear the
- * tail before going on to unlock the folios.
- */
- if (subreq->error == 0 && subreq->transferred < subreq->len &&
- (test_bit(NETFS_SREQ_HIT_EOF, &subreq->flags) ||
- test_bit(NETFS_SREQ_CLEAR_TAIL, &subreq->flags))) {
- netfs_clear_unread(subreq);
- subreq->transferred = subreq->len;
- trace_netfs_sreq(subreq, netfs_sreq_trace_clear);
- }
- if (subreq->transferred > subreq->consumed &&
- (rreq->origin == NETFS_READAHEAD ||
- rreq->origin == NETFS_READPAGE ||
- rreq->origin == NETFS_READ_FOR_WRITE)) {
- netfs_consume_read_data(subreq);
- __set_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags);
- }
- rreq->transferred += subreq->transferred;
- }
-
/* Deal with retry requests, short reads and errors. If we retry
* but don't make progress, we abandon the attempt.
*/
if (!subreq->error && subreq->transferred < subreq->len) {
if (test_bit(NETFS_SREQ_HIT_EOF, &subreq->flags)) {
trace_netfs_sreq(subreq, netfs_sreq_trace_hit_eof);
+ } else if (test_bit(NETFS_SREQ_CLEAR_TAIL, &subreq->flags)) {
+ trace_netfs_sreq(subreq, netfs_sreq_trace_need_clear);
+ } else if (test_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags)) {
+ trace_netfs_sreq(subreq, netfs_sreq_trace_need_retry);
+ } else if (test_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags)) {
+ __set_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags);
+ trace_netfs_sreq(subreq, netfs_sreq_trace_partial_read);
} else {
+ __set_bit(NETFS_SREQ_FAILED, &subreq->flags);
+ subreq->error = -ENODATA;
trace_netfs_sreq(subreq, netfs_sreq_trace_short);
- if (subreq->transferred > subreq->consumed) {
- /* If we didn't read new data, abandon retry. */
- if (subreq->retry_count &&
- test_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags)) {
- __set_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags);
- set_bit(NETFS_RREQ_NEED_RETRY, &rreq->flags);
- }
- } else if (test_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags)) {
- __set_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags);
- set_bit(NETFS_RREQ_NEED_RETRY, &rreq->flags);
- } else {
- __set_bit(NETFS_SREQ_FAILED, &subreq->flags);
- subreq->error = -ENODATA;
- }
}
}
- trace_netfs_sreq(subreq, netfs_sreq_trace_terminated);
-
if (unlikely(subreq->error < 0)) {
trace_netfs_failure(rreq, subreq, subreq->error, netfs_fail_read);
if (subreq->source == NETFS_READ_FROM_CACHE) {
netfs_stat(&netfs_n_rh_read_failed);
+ __set_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags);
} else {
netfs_stat(&netfs_n_rh_download_failed);
- set_bit(NETFS_RREQ_FAILED, &rreq->flags);
- rreq->error = subreq->error;
+ __set_bit(NETFS_SREQ_FAILED, &subreq->flags);
}
+ trace_netfs_rreq(rreq, netfs_rreq_trace_set_pause);
+ set_bit(NETFS_RREQ_PAUSE, &rreq->flags);
}
- if (atomic_dec_and_test(&rreq->nr_outstanding))
- netfs_rreq_terminated(rreq);
+ trace_netfs_sreq(subreq, netfs_sreq_trace_terminated);
+
+ clear_bit_unlock(NETFS_SREQ_IN_PROGRESS, &subreq->flags);
+ smp_mb__after_atomic(); /* Clear IN_PROGRESS before task state */
- netfs_put_subrequest(subreq, false, netfs_sreq_trace_put_terminated);
+ /* If we are at the head of the queue, wake up the collector. */
+ if (list_is_first(&subreq->rreq_link, &stream->subrequests))
+ netfs_wake_read_collector(rreq);
+
+ netfs_put_subrequest(subreq, true, netfs_sreq_trace_put_terminated);
}
EXPORT_SYMBOL(netfs_read_subreq_terminated);
-/**
- * netfs_read_subreq_termination_worker - Workqueue helper for read termination
- * @work: The subreq->work in the I/O request that has been terminated.
- *
- * Helper function to jump to netfs_read_subreq_terminated() from the
- * subrequest work item.
+/*
+ * Handle termination of a read from the cache.
*/
-void netfs_read_subreq_termination_worker(struct work_struct *work)
+void netfs_cache_read_terminated(void *priv, ssize_t transferred_or_error, bool was_async)
{
- struct netfs_io_subrequest *subreq =
- container_of(work, struct netfs_io_subrequest, work);
+ struct netfs_io_subrequest *subreq = priv;
+ if (transferred_or_error > 0) {
+ subreq->error = 0;
+ if (transferred_or_error > 0) {
+ subreq->transferred += transferred_or_error;
+ __set_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags);
+ }
+ } else {
+ subreq->error = transferred_or_error;
+ }
netfs_read_subreq_terminated(subreq);
}
-EXPORT_SYMBOL(netfs_read_subreq_termination_worker);
+
+/*
+ * Wait for the read operation to complete, successfully or otherwise.
+ */
+ssize_t netfs_wait_for_read(struct netfs_io_request *rreq)
+{
+ struct netfs_io_subrequest *subreq;
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
+ DEFINE_WAIT(myself);
+ ssize_t ret;
+
+ for (;;) {
+ trace_netfs_rreq(rreq, netfs_rreq_trace_wait_queue);
+ prepare_to_wait(&rreq->waitq, &myself, TASK_UNINTERRUPTIBLE);
+
+ subreq = list_first_entry_or_null(&stream->subrequests,
+ struct netfs_io_subrequest, rreq_link);
+ if (subreq &&
+ (!test_bit(NETFS_SREQ_IN_PROGRESS, &subreq->flags) ||
+ test_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags))) {
+ __set_current_state(TASK_RUNNING);
+ netfs_read_collection(rreq);
+ continue;
+ }
+
+ if (!test_bit(NETFS_RREQ_IN_PROGRESS, &rreq->flags))
+ break;
+
+ schedule();
+ trace_netfs_rreq(rreq, netfs_rreq_trace_woke_queue);
+ }
+
+ finish_wait(&rreq->waitq, &myself);
+
+ ret = rreq->error;
+ if (ret == 0) {
+ ret = rreq->transferred;
+ switch (rreq->origin) {
+ case NETFS_DIO_READ:
+ case NETFS_READ_SINGLE:
+ ret = rreq->transferred;
+ break;
+ default:
+ if (rreq->submitted < rreq->len) {
+ trace_netfs_failure(rreq, NULL, ret, netfs_fail_short_read);
+ ret = -EIO;
+ }
+ break;
+ }
+ }
+
+ return ret;
+}
+
+/*
+ * Wait for a paused read operation to unpause or complete in some manner.
+ */
+void netfs_wait_for_pause(struct netfs_io_request *rreq)
+{
+ struct netfs_io_subrequest *subreq;
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
+ DEFINE_WAIT(myself);
+
+ trace_netfs_rreq(rreq, netfs_rreq_trace_wait_pause);
+
+ for (;;) {
+ trace_netfs_rreq(rreq, netfs_rreq_trace_wait_queue);
+ prepare_to_wait(&rreq->waitq, &myself, TASK_UNINTERRUPTIBLE);
+
+ subreq = list_first_entry_or_null(&stream->subrequests,
+ struct netfs_io_subrequest, rreq_link);
+ if (subreq &&
+ (!test_bit(NETFS_SREQ_IN_PROGRESS, &subreq->flags) ||
+ test_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags))) {
+ __set_current_state(TASK_RUNNING);
+ netfs_read_collection(rreq);
+ continue;
+ }
+
+ if (!test_bit(NETFS_RREQ_IN_PROGRESS, &rreq->flags) ||
+ !test_bit(NETFS_RREQ_PAUSE, &rreq->flags))
+ break;
+
+ schedule();
+ trace_netfs_rreq(rreq, netfs_rreq_trace_woke_queue);
+ }
+
+ finish_wait(&rreq->waitq, &myself);
+}