Linux Audio

Check our new training course

Loading...
Note: File does not exist in v6.2.
  1// SPDX-License-Identifier: GPL-2.0-only
  2/* Network filesystem write subrequest result collection, assessment
  3 * and retrying.
  4 *
  5 * Copyright (C) 2024 Red Hat, Inc. All Rights Reserved.
  6 * Written by David Howells (dhowells@redhat.com)
  7 */
  8
  9#include <linux/export.h>
 10#include <linux/fs.h>
 11#include <linux/mm.h>
 12#include <linux/pagemap.h>
 13#include <linux/slab.h>
 14#include "internal.h"
 15
 16/* Notes made in the collector */
 17#define HIT_PENDING		0x01	/* A front op was still pending */
 18#define NEED_REASSESS		0x02	/* Need to loop round and reassess */
 19#define MADE_PROGRESS		0x04	/* Made progress cleaning up a stream or the folio set */
 20#define BUFFERED		0x08	/* The pagecache needs cleaning up */
 21#define NEED_RETRY		0x10	/* A front op requests retrying */
 22#define SAW_FAILURE		0x20	/* One stream or hit a permanent failure */
 23
 24/*
 25 * Successful completion of write of a folio to the server and/or cache.  Note
 26 * that we are not allowed to lock the folio here on pain of deadlocking with
 27 * truncate.
 28 */
 29int netfs_folio_written_back(struct folio *folio)
 30{
 31	enum netfs_folio_trace why = netfs_folio_trace_clear;
 32	struct netfs_inode *ictx = netfs_inode(folio->mapping->host);
 33	struct netfs_folio *finfo;
 34	struct netfs_group *group = NULL;
 35	int gcount = 0;
 36
 37	if ((finfo = netfs_folio_info(folio))) {
 38		/* Streaming writes cannot be redirtied whilst under writeback,
 39		 * so discard the streaming record.
 40		 */
 41		unsigned long long fend;
 42
 43		fend = folio_pos(folio) + finfo->dirty_offset + finfo->dirty_len;
 44		if (fend > ictx->zero_point)
 45			ictx->zero_point = fend;
 46
 47		folio_detach_private(folio);
 48		group = finfo->netfs_group;
 49		gcount++;
 50		kfree(finfo);
 51		why = netfs_folio_trace_clear_s;
 52		goto end_wb;
 53	}
 54
 55	if ((group = netfs_folio_group(folio))) {
 56		if (group == NETFS_FOLIO_COPY_TO_CACHE) {
 57			why = netfs_folio_trace_clear_cc;
 58			folio_detach_private(folio);
 59			goto end_wb;
 60		}
 61
 62		/* Need to detach the group pointer if the page didn't get
 63		 * redirtied.  If it has been redirtied, then it must be within
 64		 * the same group.
 65		 */
 66		why = netfs_folio_trace_redirtied;
 67		if (!folio_test_dirty(folio)) {
 68			folio_detach_private(folio);
 69			gcount++;
 70			why = netfs_folio_trace_clear_g;
 71		}
 72	}
 73
 74end_wb:
 75	trace_netfs_folio(folio, why);
 76	folio_end_writeback(folio);
 77	return gcount;
 78}
 79
 80/*
 81 * Unlock any folios we've finished with.
 82 */
 83static void netfs_writeback_unlock_folios(struct netfs_io_request *wreq,
 84					  unsigned int *notes)
 85{
 86	struct folio_queue *folioq = wreq->buffer;
 87	unsigned long long collected_to = wreq->collected_to;
 88	unsigned int slot = wreq->buffer_head_slot;
 89
 90	if (wreq->origin == NETFS_PGPRIV2_COPY_TO_CACHE) {
 91		if (netfs_pgpriv2_unlock_copied_folios(wreq))
 92			*notes |= MADE_PROGRESS;
 93		return;
 94	}
 95
 96	if (slot >= folioq_nr_slots(folioq)) {
 97		folioq = netfs_delete_buffer_head(wreq);
 98		slot = 0;
 99	}
100
101	for (;;) {
102		struct folio *folio;
103		struct netfs_folio *finfo;
104		unsigned long long fpos, fend;
105		size_t fsize, flen;
106
107		folio = folioq_folio(folioq, slot);
108		if (WARN_ONCE(!folio_test_writeback(folio),
109			      "R=%08x: folio %lx is not under writeback\n",
110			      wreq->debug_id, folio->index))
111			trace_netfs_folio(folio, netfs_folio_trace_not_under_wback);
112
113		fpos = folio_pos(folio);
114		fsize = folio_size(folio);
115		finfo = netfs_folio_info(folio);
116		flen = finfo ? finfo->dirty_offset + finfo->dirty_len : fsize;
117
118		fend = min_t(unsigned long long, fpos + flen, wreq->i_size);
119
120		trace_netfs_collect_folio(wreq, folio, fend, collected_to);
121
122		/* Unlock any folio we've transferred all of. */
123		if (collected_to < fend)
124			break;
125
126		wreq->nr_group_rel += netfs_folio_written_back(folio);
127		wreq->cleaned_to = fpos + fsize;
128		*notes |= MADE_PROGRESS;
129
130		/* Clean up the head folioq.  If we clear an entire folioq, then
131		 * we can get rid of it provided it's not also the tail folioq
132		 * being filled by the issuer.
133		 */
134		folioq_clear(folioq, slot);
135		slot++;
136		if (slot >= folioq_nr_slots(folioq)) {
137			if (READ_ONCE(wreq->buffer_tail) == folioq)
138				break;
139			folioq = netfs_delete_buffer_head(wreq);
140			slot = 0;
141		}
142
143		if (fpos + fsize >= collected_to)
144			break;
145	}
146
147	wreq->buffer = folioq;
148	wreq->buffer_head_slot = slot;
149}
150
151/*
152 * Perform retries on the streams that need it.
153 */
154static void netfs_retry_write_stream(struct netfs_io_request *wreq,
155				     struct netfs_io_stream *stream)
156{
157	struct list_head *next;
158
159	_enter("R=%x[%x:]", wreq->debug_id, stream->stream_nr);
160
161	if (list_empty(&stream->subrequests))
162		return;
163
164	if (stream->source == NETFS_UPLOAD_TO_SERVER &&
165	    wreq->netfs_ops->retry_request)
166		wreq->netfs_ops->retry_request(wreq, stream);
167
168	if (unlikely(stream->failed))
169		return;
170
171	/* If there's no renegotiation to do, just resend each failed subreq. */
172	if (!stream->prepare_write) {
173		struct netfs_io_subrequest *subreq;
174
175		list_for_each_entry(subreq, &stream->subrequests, rreq_link) {
176			if (test_bit(NETFS_SREQ_FAILED, &subreq->flags))
177				break;
178			if (__test_and_clear_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags)) {
179				struct iov_iter source = subreq->io_iter;
180
181				iov_iter_revert(&source, subreq->len - source.count);
182				netfs_get_subrequest(subreq, netfs_sreq_trace_get_resubmit);
183				netfs_reissue_write(stream, subreq, &source);
184			}
185		}
186		return;
187	}
188
189	next = stream->subrequests.next;
190
191	do {
192		struct netfs_io_subrequest *subreq = NULL, *from, *to, *tmp;
193		struct iov_iter source;
194		unsigned long long start, len;
195		size_t part;
196		bool boundary = false;
197
198		/* Go through the stream and find the next span of contiguous
199		 * data that we then rejig (cifs, for example, needs the wsize
200		 * renegotiating) and reissue.
201		 */
202		from = list_entry(next, struct netfs_io_subrequest, rreq_link);
203		to = from;
204		start = from->start + from->transferred;
205		len   = from->len   - from->transferred;
206
207		if (test_bit(NETFS_SREQ_FAILED, &from->flags) ||
208		    !test_bit(NETFS_SREQ_NEED_RETRY, &from->flags))
209			return;
210
211		list_for_each_continue(next, &stream->subrequests) {
212			subreq = list_entry(next, struct netfs_io_subrequest, rreq_link);
213			if (subreq->start + subreq->transferred != start + len ||
214			    test_bit(NETFS_SREQ_BOUNDARY, &subreq->flags) ||
215			    !test_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags))
216				break;
217			to = subreq;
218			len += to->len;
219		}
220
221		/* Determine the set of buffers we're going to use.  Each
222		 * subreq gets a subset of a single overall contiguous buffer.
223		 */
224		netfs_reset_iter(from);
225		source = from->io_iter;
226		source.count = len;
227
228		/* Work through the sublist. */
229		subreq = from;
230		list_for_each_entry_from(subreq, &stream->subrequests, rreq_link) {
231			if (!len)
232				break;
233			/* Renegotiate max_len (wsize) */
234			trace_netfs_sreq(subreq, netfs_sreq_trace_retry);
235			__clear_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags);
236			subreq->retry_count++;
237			stream->prepare_write(subreq);
238
239			part = min(len, stream->sreq_max_len);
240			subreq->len = part;
241			subreq->start = start;
242			subreq->transferred = 0;
243			len -= part;
244			start += part;
245			if (len && subreq == to &&
246			    __test_and_clear_bit(NETFS_SREQ_BOUNDARY, &to->flags))
247				boundary = true;
248
249			netfs_get_subrequest(subreq, netfs_sreq_trace_get_resubmit);
250			netfs_reissue_write(stream, subreq, &source);
251			if (subreq == to)
252				break;
253		}
254
255		/* If we managed to use fewer subreqs, we can discard the
256		 * excess; if we used the same number, then we're done.
257		 */
258		if (!len) {
259			if (subreq == to)
260				continue;
261			list_for_each_entry_safe_from(subreq, tmp,
262						      &stream->subrequests, rreq_link) {
263				trace_netfs_sreq(subreq, netfs_sreq_trace_discard);
264				list_del(&subreq->rreq_link);
265				netfs_put_subrequest(subreq, false, netfs_sreq_trace_put_done);
266				if (subreq == to)
267					break;
268			}
269			continue;
270		}
271
272		/* We ran out of subrequests, so we need to allocate some more
273		 * and insert them after.
274		 */
275		do {
276			subreq = netfs_alloc_subrequest(wreq);
277			subreq->source		= to->source;
278			subreq->start		= start;
279			subreq->debug_index	= atomic_inc_return(&wreq->subreq_counter);
280			subreq->stream_nr	= to->stream_nr;
281			subreq->retry_count	= 1;
282
283			trace_netfs_sreq_ref(wreq->debug_id, subreq->debug_index,
284					     refcount_read(&subreq->ref),
285					     netfs_sreq_trace_new);
286			netfs_get_subrequest(subreq, netfs_sreq_trace_get_resubmit);
287
288			list_add(&subreq->rreq_link, &to->rreq_link);
289			to = list_next_entry(to, rreq_link);
290			trace_netfs_sreq(subreq, netfs_sreq_trace_retry);
291
292			stream->sreq_max_len	= len;
293			stream->sreq_max_segs	= INT_MAX;
294			switch (stream->source) {
295			case NETFS_UPLOAD_TO_SERVER:
296				netfs_stat(&netfs_n_wh_upload);
297				stream->sreq_max_len = umin(len, wreq->wsize);
298				break;
299			case NETFS_WRITE_TO_CACHE:
300				netfs_stat(&netfs_n_wh_write);
301				break;
302			default:
303				WARN_ON_ONCE(1);
304			}
305
306			stream->prepare_write(subreq);
307
308			part = umin(len, stream->sreq_max_len);
309			subreq->len = subreq->transferred + part;
310			len -= part;
311			start += part;
312			if (!len && boundary) {
313				__set_bit(NETFS_SREQ_BOUNDARY, &to->flags);
314				boundary = false;
315			}
316
317			netfs_reissue_write(stream, subreq, &source);
318			if (!len)
319				break;
320
321		} while (len);
322
323	} while (!list_is_head(next, &stream->subrequests));
324}
325
326/*
327 * Perform retries on the streams that need it.  If we're doing content
328 * encryption and the server copy changed due to a third-party write, we may
329 * need to do an RMW cycle and also rewrite the data to the cache.
330 */
331static void netfs_retry_writes(struct netfs_io_request *wreq)
332{
333	struct netfs_io_subrequest *subreq;
334	struct netfs_io_stream *stream;
335	int s;
336
337	/* Wait for all outstanding I/O to quiesce before performing retries as
338	 * we may need to renegotiate the I/O sizes.
339	 */
340	for (s = 0; s < NR_IO_STREAMS; s++) {
341		stream = &wreq->io_streams[s];
342		if (!stream->active)
343			continue;
344
345		list_for_each_entry(subreq, &stream->subrequests, rreq_link) {
346			wait_on_bit(&subreq->flags, NETFS_SREQ_IN_PROGRESS,
347				    TASK_UNINTERRUPTIBLE);
348		}
349	}
350
351	// TODO: Enc: Fetch changed partial pages
352	// TODO: Enc: Reencrypt content if needed.
353	// TODO: Enc: Wind back transferred point.
354	// TODO: Enc: Mark cache pages for retry.
355
356	for (s = 0; s < NR_IO_STREAMS; s++) {
357		stream = &wreq->io_streams[s];
358		if (stream->need_retry) {
359			stream->need_retry = false;
360			netfs_retry_write_stream(wreq, stream);
361		}
362	}
363}
364
365/*
366 * Collect and assess the results of various write subrequests.  We may need to
367 * retry some of the results - or even do an RMW cycle for content crypto.
368 *
369 * Note that we have a number of parallel, overlapping lists of subrequests,
370 * one to the server and one to the local cache for example, which may not be
371 * the same size or starting position and may not even correspond in boundary
372 * alignment.
373 */
374static void netfs_collect_write_results(struct netfs_io_request *wreq)
375{
376	struct netfs_io_subrequest *front, *remove;
377	struct netfs_io_stream *stream;
378	unsigned long long collected_to, issued_to;
379	unsigned int notes;
380	int s;
381
382	_enter("%llx-%llx", wreq->start, wreq->start + wreq->len);
383	trace_netfs_collect(wreq);
384	trace_netfs_rreq(wreq, netfs_rreq_trace_collect);
385
386reassess_streams:
387	issued_to = atomic64_read(&wreq->issued_to);
388	smp_rmb();
389	collected_to = ULLONG_MAX;
390	if (wreq->origin == NETFS_WRITEBACK ||
391	    wreq->origin == NETFS_WRITETHROUGH ||
392	    wreq->origin == NETFS_PGPRIV2_COPY_TO_CACHE)
393		notes = BUFFERED;
394	else
395		notes = 0;
396
397	/* Remove completed subrequests from the front of the streams and
398	 * advance the completion point on each stream.  We stop when we hit
399	 * something that's in progress.  The issuer thread may be adding stuff
400	 * to the tail whilst we're doing this.
401	 */
402	for (s = 0; s < NR_IO_STREAMS; s++) {
403		stream = &wreq->io_streams[s];
404		/* Read active flag before list pointers */
405		if (!smp_load_acquire(&stream->active))
406			continue;
407
408		front = stream->front;
409		while (front) {
410			trace_netfs_collect_sreq(wreq, front);
411			//_debug("sreq [%x] %llx %zx/%zx",
412			//       front->debug_index, front->start, front->transferred, front->len);
413
414			if (stream->collected_to < front->start) {
415				trace_netfs_collect_gap(wreq, stream, issued_to, 'F');
416				stream->collected_to = front->start;
417			}
418
419			/* Stall if the front is still undergoing I/O. */
420			if (test_bit(NETFS_SREQ_IN_PROGRESS, &front->flags)) {
421				notes |= HIT_PENDING;
422				break;
423			}
424			smp_rmb(); /* Read counters after I-P flag. */
425
426			if (stream->failed) {
427				stream->collected_to = front->start + front->len;
428				notes |= MADE_PROGRESS | SAW_FAILURE;
429				goto cancel;
430			}
431			if (front->start + front->transferred > stream->collected_to) {
432				stream->collected_to = front->start + front->transferred;
433				stream->transferred = stream->collected_to - wreq->start;
434				notes |= MADE_PROGRESS;
435			}
436			if (test_bit(NETFS_SREQ_FAILED, &front->flags)) {
437				stream->failed = true;
438				stream->error = front->error;
439				if (stream->source == NETFS_UPLOAD_TO_SERVER)
440					mapping_set_error(wreq->mapping, front->error);
441				notes |= NEED_REASSESS | SAW_FAILURE;
442				break;
443			}
444			if (front->transferred < front->len) {
445				stream->need_retry = true;
446				notes |= NEED_RETRY | MADE_PROGRESS;
447				break;
448			}
449
450		cancel:
451			/* Remove if completely consumed. */
452			spin_lock_bh(&wreq->lock);
453
454			remove = front;
455			list_del_init(&front->rreq_link);
456			front = list_first_entry_or_null(&stream->subrequests,
457							 struct netfs_io_subrequest, rreq_link);
458			stream->front = front;
459			spin_unlock_bh(&wreq->lock);
460			netfs_put_subrequest(remove, false,
461					     notes & SAW_FAILURE ?
462					     netfs_sreq_trace_put_cancel :
463					     netfs_sreq_trace_put_done);
464		}
465
466		/* If we have an empty stream, we need to jump it forward
467		 * otherwise the collection point will never advance.
468		 */
469		if (!front && issued_to > stream->collected_to) {
470			trace_netfs_collect_gap(wreq, stream, issued_to, 'E');
471			stream->collected_to = issued_to;
472		}
473
474		if (stream->collected_to < collected_to)
475			collected_to = stream->collected_to;
476	}
477
478	if (collected_to != ULLONG_MAX && collected_to > wreq->collected_to)
479		wreq->collected_to = collected_to;
480
481	for (s = 0; s < NR_IO_STREAMS; s++) {
482		stream = &wreq->io_streams[s];
483		if (stream->active)
484			trace_netfs_collect_stream(wreq, stream);
485	}
486
487	trace_netfs_collect_state(wreq, wreq->collected_to, notes);
488
489	/* Unlock any folios that we have now finished with. */
490	if (notes & BUFFERED) {
491		if (wreq->cleaned_to < wreq->collected_to)
492			netfs_writeback_unlock_folios(wreq, &notes);
493	} else {
494		wreq->cleaned_to = wreq->collected_to;
495	}
496
497	// TODO: Discard encryption buffers
498
499	if (notes & NEED_RETRY)
500		goto need_retry;
501	if ((notes & MADE_PROGRESS) && test_bit(NETFS_RREQ_PAUSE, &wreq->flags)) {
502		trace_netfs_rreq(wreq, netfs_rreq_trace_unpause);
503		clear_and_wake_up_bit(NETFS_RREQ_PAUSE, &wreq->flags);
504	}
505
506	if (notes & NEED_REASSESS) {
507		//cond_resched();
508		goto reassess_streams;
509	}
510	if (notes & MADE_PROGRESS) {
511		//cond_resched();
512		goto reassess_streams;
513	}
514
515out:
516	netfs_put_group_many(wreq->group, wreq->nr_group_rel);
517	wreq->nr_group_rel = 0;
518	_leave(" = %x", notes);
519	return;
520
521need_retry:
522	/* Okay...  We're going to have to retry one or both streams.  Note
523	 * that any partially completed op will have had any wholly transferred
524	 * folios removed from it.
525	 */
526	_debug("retry");
527	netfs_retry_writes(wreq);
528	goto out;
529}
530
531/*
532 * Perform the collection of subrequests, folios and encryption buffers.
533 */
534void netfs_write_collection_worker(struct work_struct *work)
535{
536	struct netfs_io_request *wreq = container_of(work, struct netfs_io_request, work);
537	struct netfs_inode *ictx = netfs_inode(wreq->inode);
538	size_t transferred;
539	int s;
540
541	_enter("R=%x", wreq->debug_id);
542
543	netfs_see_request(wreq, netfs_rreq_trace_see_work);
544	if (!test_bit(NETFS_RREQ_IN_PROGRESS, &wreq->flags)) {
545		netfs_put_request(wreq, false, netfs_rreq_trace_put_work);
546		return;
547	}
548
549	netfs_collect_write_results(wreq);
550
551	/* We're done when the app thread has finished posting subreqs and all
552	 * the queues in all the streams are empty.
553	 */
554	if (!test_bit(NETFS_RREQ_ALL_QUEUED, &wreq->flags)) {
555		netfs_put_request(wreq, false, netfs_rreq_trace_put_work);
556		return;
557	}
558	smp_rmb(); /* Read ALL_QUEUED before lists. */
559
560	transferred = LONG_MAX;
561	for (s = 0; s < NR_IO_STREAMS; s++) {
562		struct netfs_io_stream *stream = &wreq->io_streams[s];
563		if (!stream->active)
564			continue;
565		if (!list_empty(&stream->subrequests)) {
566			netfs_put_request(wreq, false, netfs_rreq_trace_put_work);
567			return;
568		}
569		if (stream->transferred < transferred)
570			transferred = stream->transferred;
571	}
572
573	/* Okay, declare that all I/O is complete. */
574	wreq->transferred = transferred;
575	trace_netfs_rreq(wreq, netfs_rreq_trace_write_done);
576
577	if (wreq->io_streams[1].active &&
578	    wreq->io_streams[1].failed) {
579		/* Cache write failure doesn't prevent writeback completion
580		 * unless we're in disconnected mode.
581		 */
582		ictx->ops->invalidate_cache(wreq);
583	}
584
585	if (wreq->cleanup)
586		wreq->cleanup(wreq);
587
588	if (wreq->origin == NETFS_DIO_WRITE &&
589	    wreq->mapping->nrpages) {
590		/* mmap may have got underfoot and we may now have folios
591		 * locally covering the region we just wrote.  Attempt to
592		 * discard the folios, but leave in place any modified locally.
593		 * ->write_iter() is prevented from interfering by the DIO
594		 * counter.
595		 */
596		pgoff_t first = wreq->start >> PAGE_SHIFT;
597		pgoff_t last = (wreq->start + wreq->transferred - 1) >> PAGE_SHIFT;
598		invalidate_inode_pages2_range(wreq->mapping, first, last);
599	}
600
601	if (wreq->origin == NETFS_DIO_WRITE)
602		inode_dio_end(wreq->inode);
603
604	_debug("finished");
605	trace_netfs_rreq(wreq, netfs_rreq_trace_wake_ip);
606	clear_and_wake_up_bit(NETFS_RREQ_IN_PROGRESS, &wreq->flags);
607
608	if (wreq->iocb) {
609		size_t written = min(wreq->transferred, wreq->len);
610		wreq->iocb->ki_pos += written;
611		if (wreq->iocb->ki_complete)
612			wreq->iocb->ki_complete(
613				wreq->iocb, wreq->error ? wreq->error : written);
614		wreq->iocb = VFS_PTR_POISON;
615	}
616
617	netfs_clear_subrequests(wreq, false);
618	netfs_put_request(wreq, false, netfs_rreq_trace_put_work_complete);
619}
620
621/*
622 * Wake the collection work item.
623 */
624void netfs_wake_write_collector(struct netfs_io_request *wreq, bool was_async)
625{
626	if (!work_pending(&wreq->work)) {
627		netfs_get_request(wreq, netfs_rreq_trace_get_work);
628		if (!queue_work(system_unbound_wq, &wreq->work))
629			netfs_put_request(wreq, was_async, netfs_rreq_trace_put_work_nq);
630	}
631}
632
633/**
634 * netfs_write_subrequest_terminated - Note the termination of a write operation.
635 * @_op: The I/O request that has terminated.
636 * @transferred_or_error: The amount of data transferred or an error code.
637 * @was_async: The termination was asynchronous
638 *
639 * This tells the library that a contributory write I/O operation has
640 * terminated, one way or another, and that it should collect the results.
641 *
642 * The caller indicates in @transferred_or_error the outcome of the operation,
643 * supplying a positive value to indicate the number of bytes transferred or a
644 * negative error code.  The library will look after reissuing I/O operations
645 * as appropriate and writing downloaded data to the cache.
646 *
647 * If @was_async is true, the caller might be running in softirq or interrupt
648 * context and we can't sleep.
649 *
650 * When this is called, ownership of the subrequest is transferred back to the
651 * library, along with a ref.
652 *
653 * Note that %_op is a void* so that the function can be passed to
654 * kiocb::term_func without the need for a casting wrapper.
655 */
656void netfs_write_subrequest_terminated(void *_op, ssize_t transferred_or_error,
657				       bool was_async)
658{
659	struct netfs_io_subrequest *subreq = _op;
660	struct netfs_io_request *wreq = subreq->rreq;
661	struct netfs_io_stream *stream = &wreq->io_streams[subreq->stream_nr];
662
663	_enter("%x[%x] %zd", wreq->debug_id, subreq->debug_index, transferred_or_error);
664
665	switch (subreq->source) {
666	case NETFS_UPLOAD_TO_SERVER:
667		netfs_stat(&netfs_n_wh_upload_done);
668		break;
669	case NETFS_WRITE_TO_CACHE:
670		netfs_stat(&netfs_n_wh_write_done);
671		break;
672	case NETFS_INVALID_WRITE:
673		break;
674	default:
675		BUG();
676	}
677
678	if (IS_ERR_VALUE(transferred_or_error)) {
679		subreq->error = transferred_or_error;
680		if (subreq->error == -EAGAIN)
681			set_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags);
682		else
683			set_bit(NETFS_SREQ_FAILED, &subreq->flags);
684		trace_netfs_failure(wreq, subreq, transferred_or_error, netfs_fail_write);
685
686		switch (subreq->source) {
687		case NETFS_WRITE_TO_CACHE:
688			netfs_stat(&netfs_n_wh_write_failed);
689			break;
690		case NETFS_UPLOAD_TO_SERVER:
691			netfs_stat(&netfs_n_wh_upload_failed);
692			break;
693		default:
694			break;
695		}
696		trace_netfs_rreq(wreq, netfs_rreq_trace_set_pause);
697		set_bit(NETFS_RREQ_PAUSE, &wreq->flags);
698	} else {
699		if (WARN(transferred_or_error > subreq->len - subreq->transferred,
700			 "Subreq excess write: R=%x[%x] %zd > %zu - %zu",
701			 wreq->debug_id, subreq->debug_index,
702			 transferred_or_error, subreq->len, subreq->transferred))
703			transferred_or_error = subreq->len - subreq->transferred;
704
705		subreq->error = 0;
706		subreq->transferred += transferred_or_error;
707
708		if (subreq->transferred < subreq->len)
709			set_bit(NETFS_SREQ_NEED_RETRY, &subreq->flags);
710	}
711
712	trace_netfs_sreq(subreq, netfs_sreq_trace_terminated);
713
714	clear_and_wake_up_bit(NETFS_SREQ_IN_PROGRESS, &subreq->flags);
715
716	/* If we are at the head of the queue, wake up the collector,
717	 * transferring a ref to it if we were the ones to do so.
718	 */
719	if (list_is_first(&subreq->rreq_link, &stream->subrequests))
720		netfs_wake_write_collector(wreq, was_async);
721
722	netfs_put_subrequest(subreq, was_async, netfs_sreq_trace_put_terminated);
723}
724EXPORT_SYMBOL(netfs_write_subrequest_terminated);