summaryrefslogtreecommitdiff
path: root/fs/netfs/read_collect.c
diff options
context:
space:
mode:
Diffstat (limited to 'fs/netfs/read_collect.c')
-rw-r--r--fs/netfs/read_collect.c764
1 files changed, 459 insertions, 305 deletions
diff --git a/fs/netfs/read_collect.c b/fs/netfs/read_collect.c
index 8878b46589ff..23c75755ad4e 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,239 +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, bool was_async)
+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;
-
-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: orders=%02x%02x%02x%02x\n",
- folioq->orders[0], folioq->orders[1],
- folioq->orders[2], folioq->orders[3]);
- 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_bh(&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_bh(&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_bh(&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_bh(&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_bh(&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_bh(&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);
+
+ 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);
- /* Advance the rolling buffer to the next folio. */
+ /* 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];
- if (!was_async)
- 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_bh(&rreq->lock);
+ front = READ_ONCE(stream->front);
+ while (front) {
+ size_t transferred;
- if (subreq->prev_donated != prev_donated ||
- subreq->next_donated != next_donated) {
- spin_unlock_bh(&rreq->lock);
- cond_resched();
- goto donation_changed;
- }
+ trace_netfs_collect_sreq(rreq, front);
+ _debug("sreq [%x] %llx %zx/%zx",
+ front->debug_index, front->start, front->transferred, front->len);
- /* 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)) {
- prev = list_prev_entry(subreq, rreq_link);
- if (subreq->start == prev->start + prev->len) {
- 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;
+ 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);
+ }
+
+ 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);
+ netfs_put_subrequest(remove, false,
+ notes & ABANDON_SREQ ?
+ netfs_sreq_trace_put_abandon :
+ netfs_sreq_trace_put_done);
}
- /* If we can't donate down the chain, donate up the chain instead. */
- excess = subreq->len - subreq->consumed + next_donated;
+ trace_netfs_collect_stream(rreq, stream);
+ trace_netfs_collect_state(rreq, rreq->collected_to, notes);
- if (!subreq->consumed)
- excess += prev_donated;
+ if (!(notes & BUFFERED))
+ rreq->cleaned_to = rreq->collected_to;
- 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 & 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 (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_bh(&rreq->lock);
-remove_subreq_locked:
- subreq->consumed = subreq->len;
- list_del(&subreq->rreq_link);
- spin_unlock_bh(&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;
}
/*
@@ -322,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 ||
@@ -360,25 +382,67 @@ static void netfs_rreq_assess_dio(struct netfs_io_request *rreq)
}
/*
- * Assess the state of a read request and decide what to do next.
+ * Do processing after reading a monolithic single object.
+ */
+static void netfs_rreq_assess_single(struct netfs_io_request *rreq)
+{
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
+
+ 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) {
+ rreq->iocb->ki_pos += rreq->transferred;
+ if (rreq->iocb->ki_complete)
+ rreq->iocb->ki_complete(
+ rreq->iocb, rreq->error ? rreq->error : rreq->transferred);
+ }
+ if (rreq->netfs_ops->done)
+ rreq->netfs_ops->done(rreq);
+}
+
+/*
+ * Perform the collection of subrequests and folios.
*
* Note that we're in normal kernel thread context at this point, possibly
* running on a workqueue.
*/
-static void netfs_rreq_assess(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);
- if (rreq->origin == NETFS_DIO_READ ||
- rreq->origin == NETFS_READ_GAPS)
+ //netfs_rreq_is_still_valid(rreq);
+
+ switch (rreq->origin) {
+ case NETFS_DIO_READ:
+ case NETFS_READ_GAPS:
netfs_rreq_assess_dio(rreq);
+ break;
+ case NETFS_READ_SINGLE:
+ netfs_rreq_assess_single(rreq);
+ break;
+ default:
+ break;
+ }
task_io_account_read(rreq->transferred);
trace_netfs_rreq(rreq, netfs_rreq_trace_wake_ip);
@@ -387,61 +451,67 @@ static void netfs_rreq_assess(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_termination_worker(struct work_struct *work)
+void netfs_read_collection_worker(struct work_struct *work)
{
- struct netfs_io_request *rreq =
- container_of(work, struct netfs_io_request, work);
+ struct netfs_io_request *rreq = container_of(work, struct netfs_io_request, work);
+
netfs_see_request(rreq, netfs_rreq_trace_see_work);
- netfs_rreq_assess(rreq);
- netfs_put_request(rreq, false, netfs_rreq_trace_put_work_complete);
+ if (test_bit(NETFS_RREQ_IN_PROGRESS, &rreq->flags))
+ netfs_read_collection(rreq);
+ netfs_put_request(rreq, false, netfs_rreq_trace_put_work);
}
/*
- * Handle the completion of all outstanding I/O operations on a read request.
- * We inherit a ref from the caller.
+ * Wake the collection work item.
*/
-void netfs_rreq_terminated(struct netfs_io_request *rreq, bool was_async)
+void netfs_wake_read_collector(struct netfs_io_request *rreq)
{
- if (!was_async)
- return netfs_rreq_assess(rreq);
- 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, was_async, netfs_rreq_trace_put_work_nq);
+ if (test_bit(NETFS_RREQ_OFFLOAD_COLLECTION, &rreq->flags) &&
+ !test_bit(NETFS_RREQ_RETRYING, &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);
}
}
/**
* netfs_read_subreq_progress - Note progress of a read operation.
* @subreq: The read request that has terminated.
- * @was_async: True if we're in an asynchronous context.
*
* This tells the read side of netfs lib that a contributory I/O operation has
* made some progress and that it may be possible to unlock some folios.
*
* Before calling, the filesystem should update subreq->transferred to track
* the amount of data copied into the output buffer.
- *
- * If @was_async is true, the caller might be running in softirq or interrupt
- * context and we can't sleep.
*/
-void netfs_read_subreq_progress(struct netfs_io_subrequest *subreq,
- bool was_async)
+void netfs_read_subreq_progress(struct netfs_io_subrequest *subreq)
{
struct netfs_io_request *rreq = subreq->rreq;
+ 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, was_async);
+ 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);
@@ -449,27 +519,23 @@ EXPORT_SYMBOL(netfs_read_subreq_progress);
/**
* netfs_read_subreq_terminated - Note the termination of an I/O operation.
* @subreq: The I/O request that has terminated.
- * @error: Error code indicating type of completion.
- * @was_async: The termination was asynchronous
*
* This tells the read helper that a contributory I/O operation has terminated,
* one way or another, and that it should integrate the results.
*
- * The caller indicates the outcome of the operation through @error, supplying
- * 0 to indicate a successful or retryable transfer (if NETFS_SREQ_NEED_RETRY
- * is set) or a negative error code. The helper will look after reissuing I/O
- * operations as appropriate and writing downloaded data to the cache.
+ * The caller indicates the outcome of the operation through @subreq->error,
+ * supplying 0 to indicate a successful or retryable transfer (if
+ * NETFS_SREQ_NEED_RETRY is set) or a negative error code. The helper will
+ * look after reissuing I/O operations as appropriate and writing downloaded
+ * data to the cache.
*
* Before calling, the filesystem should update subreq->transferred to track
* the amount of data copied into the output buffer.
- *
- * If @was_async is true, the caller might be running in softirq or interrupt
- * context and we can't sleep.
*/
-void netfs_read_subreq_terminated(struct netfs_io_subrequest *subreq,
- int error, bool was_async)
+void netfs_read_subreq_terminated(struct netfs_io_subrequest *subreq)
{
struct netfs_io_request *rreq = subreq->rreq;
+ struct netfs_io_stream *stream = &rreq->io_streams[0];
switch (subreq->source) {
case NETFS_READ_FROM_CACHE:
@@ -482,71 +548,159 @@ 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 (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, was_async);
- __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 (!error && subreq->transferred < subreq->len) {
+ 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);
- error = -ENODATA;
- }
}
}
- subreq->error = error;
- trace_netfs_sreq(subreq, netfs_sreq_trace_terminated);
-
- if (unlikely(error < 0)) {
- trace_netfs_failure(rreq, subreq, error, netfs_fail_read);
+ 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, was_async);
+ 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, was_async, 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) ||
+ test_bit(NETFS_RREQ_RETRYING, &rreq->flags))
+ netfs_wake_read_collector(rreq);
+
+ netfs_put_subrequest(subreq, true, netfs_sreq_trace_put_terminated);
}
EXPORT_SYMBOL(netfs_read_subreq_terminated);
+
+/*
+ * Handle termination of a read from the cache.
+ */
+void netfs_cache_read_terminated(void *priv, ssize_t transferred_or_error, bool was_async)
+{
+ 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);
+}
+
+/*
+ * 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);
+
+ if (!test_bit(NETFS_RREQ_OFFLOAD_COLLECTION, &rreq->flags)) {
+ 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);
+}