xref: /linux/fs/netfs/write_issue.c (revision 2408a807bfc3f738850ef5ad5e3fd59d66168996)
1  // SPDX-License-Identifier: GPL-2.0-only
2  /* Network filesystem high-level (buffered) writeback.
3   *
4   * Copyright (C) 2024 Red Hat, Inc. All Rights Reserved.
5   * Written by David Howells (dhowells@redhat.com)
6   *
7   *
8   * To support network filesystems with local caching, we manage a situation
9   * that can be envisioned like the following:
10   *
11   *               +---+---+-----+-----+---+----------+
12   *    Folios:    |   |   |     |     |   |          |
13   *               +---+---+-----+-----+---+----------+
14   *
15   *                 +------+------+     +----+----+
16   *    Upload:      |      |      |.....|    |    |
17   *  (Stream 0)     +------+------+     +----+----+
18   *
19   *               +------+------+------+------+------+
20   *    Cache:     |      |      |      |      |      |
21   *  (Stream 1)   +------+------+------+------+------+
22   *
23   * Where we have a sequence of folios of varying sizes that we need to overlay
24   * with multiple parallel streams of I/O requests, where the I/O requests in a
25   * stream may also be of various sizes (in cifs, for example, the sizes are
26   * negotiated with the server; in something like ceph, they may represent the
27   * sizes of storage objects).
28   *
29   * The sequence in each stream may contain gaps and noncontiguous subrequests
30   * may be glued together into single vectored write RPCs.
31   */
32  
33  #include <linux/export.h>
34  #include <linux/fs.h>
35  #include <linux/mm.h>
36  #include <linux/pagemap.h>
37  #include "internal.h"
38  
39  /*
40   * Kill all dirty folios in the event of an unrecoverable error, starting with
41   * a locked folio we've already obtained from writeback_iter().
42   */
netfs_kill_dirty_pages(struct address_space * mapping,struct writeback_control * wbc,struct folio * folio)43  static void netfs_kill_dirty_pages(struct address_space *mapping,
44  				   struct writeback_control *wbc,
45  				   struct folio *folio)
46  {
47  	int error = 0;
48  
49  	do {
50  		enum netfs_folio_trace why = netfs_folio_trace_kill;
51  		struct netfs_group *group = NULL;
52  		struct netfs_folio *finfo = NULL;
53  		void *priv;
54  
55  		priv = folio_detach_private(folio);
56  		if (priv) {
57  			finfo = __netfs_folio_info(priv);
58  			if (finfo) {
59  				/* Kill folio from streaming write. */
60  				group = finfo->netfs_group;
61  				why = netfs_folio_trace_kill_s;
62  			} else {
63  				group = priv;
64  				if (group == NETFS_FOLIO_COPY_TO_CACHE) {
65  					/* Kill copy-to-cache folio */
66  					why = netfs_folio_trace_kill_cc;
67  					group = NULL;
68  				} else {
69  					/* Kill folio with group */
70  					why = netfs_folio_trace_kill_g;
71  				}
72  			}
73  		}
74  
75  		trace_netfs_folio(folio, why);
76  
77  		folio_start_writeback(folio);
78  		folio_unlock(folio);
79  		folio_end_writeback(folio);
80  
81  		netfs_put_group(group);
82  		kfree(finfo);
83  
84  	} while ((folio = writeback_iter(mapping, wbc, folio, &error)));
85  }
86  
87  /*
88   * Create a write request and set it up appropriately for the origin type.
89   */
netfs_create_write_req(struct address_space * mapping,struct file * file,loff_t start,enum netfs_io_origin origin)90  struct netfs_io_request *netfs_create_write_req(struct address_space *mapping,
91  						struct file *file,
92  						loff_t start,
93  						enum netfs_io_origin origin)
94  {
95  	struct netfs_io_request *wreq;
96  	struct netfs_inode *ictx;
97  	bool is_cacheable = (origin == NETFS_WRITEBACK ||
98  			     origin == NETFS_WRITEBACK_SINGLE ||
99  			     origin == NETFS_WRITETHROUGH ||
100  			     origin == NETFS_PGPRIV2_COPY_TO_CACHE);
101  
102  	wreq = netfs_alloc_request(mapping, file, start, 0, origin);
103  	if (IS_ERR(wreq))
104  		return wreq;
105  
106  	_enter("R=%x", wreq->debug_id);
107  
108  	ictx = netfs_inode(wreq->inode);
109  	if (is_cacheable && netfs_is_cache_enabled(ictx))
110  		fscache_begin_write_operation(&wreq->cache_resources, netfs_i_cookie(ictx));
111  	if (rolling_buffer_init(&wreq->buffer, wreq->debug_id, ITER_SOURCE) < 0)
112  		goto nomem;
113  
114  	wreq->cleaned_to = wreq->start;
115  
116  	wreq->io_streams[0].stream_nr		= 0;
117  	wreq->io_streams[0].source		= NETFS_UPLOAD_TO_SERVER;
118  	wreq->io_streams[0].prepare_write	= ictx->ops->prepare_write;
119  	wreq->io_streams[0].issue_write		= ictx->ops->issue_write;
120  	wreq->io_streams[0].collected_to	= start;
121  	wreq->io_streams[0].transferred		= LONG_MAX;
122  
123  	wreq->io_streams[1].stream_nr		= 1;
124  	wreq->io_streams[1].source		= NETFS_WRITE_TO_CACHE;
125  	wreq->io_streams[1].collected_to	= start;
126  	wreq->io_streams[1].transferred		= LONG_MAX;
127  	if (fscache_resources_valid(&wreq->cache_resources)) {
128  		wreq->io_streams[1].avail	= true;
129  		wreq->io_streams[1].active	= true;
130  		wreq->io_streams[1].prepare_write = wreq->cache_resources.ops->prepare_write_subreq;
131  		wreq->io_streams[1].issue_write = wreq->cache_resources.ops->issue_write;
132  	}
133  
134  	return wreq;
135  nomem:
136  	wreq->error = -ENOMEM;
137  	netfs_put_request(wreq, false, netfs_rreq_trace_put_failed);
138  	return ERR_PTR(-ENOMEM);
139  }
140  
141  /**
142   * netfs_prepare_write_failed - Note write preparation failed
143   * @subreq: The subrequest to mark
144   *
145   * Mark a subrequest to note that preparation for write failed.
146   */
netfs_prepare_write_failed(struct netfs_io_subrequest * subreq)147  void netfs_prepare_write_failed(struct netfs_io_subrequest *subreq)
148  {
149  	__set_bit(NETFS_SREQ_FAILED, &subreq->flags);
150  	trace_netfs_sreq(subreq, netfs_sreq_trace_prep_failed);
151  }
152  EXPORT_SYMBOL(netfs_prepare_write_failed);
153  
154  /*
155   * Prepare a write subrequest.  We need to allocate a new subrequest
156   * if we don't have one.
157   */
netfs_prepare_write(struct netfs_io_request * wreq,struct netfs_io_stream * stream,loff_t start)158  static void netfs_prepare_write(struct netfs_io_request *wreq,
159  				struct netfs_io_stream *stream,
160  				loff_t start)
161  {
162  	struct netfs_io_subrequest *subreq;
163  	struct iov_iter *wreq_iter = &wreq->buffer.iter;
164  
165  	/* Make sure we don't point the iterator at a used-up folio_queue
166  	 * struct being used as a placeholder to prevent the queue from
167  	 * collapsing.  In such a case, extend the queue.
168  	 */
169  	if (iov_iter_is_folioq(wreq_iter) &&
170  	    wreq_iter->folioq_slot >= folioq_nr_slots(wreq_iter->folioq))
171  		rolling_buffer_make_space(&wreq->buffer);
172  
173  	subreq = netfs_alloc_subrequest(wreq);
174  	subreq->source		= stream->source;
175  	subreq->start		= start;
176  	subreq->stream_nr	= stream->stream_nr;
177  	subreq->io_iter		= *wreq_iter;
178  
179  	_enter("R=%x[%x]", wreq->debug_id, subreq->debug_index);
180  
181  	trace_netfs_sreq(subreq, netfs_sreq_trace_prepare);
182  
183  	stream->sreq_max_len	= UINT_MAX;
184  	stream->sreq_max_segs	= INT_MAX;
185  	switch (stream->source) {
186  	case NETFS_UPLOAD_TO_SERVER:
187  		netfs_stat(&netfs_n_wh_upload);
188  		stream->sreq_max_len = wreq->wsize;
189  		break;
190  	case NETFS_WRITE_TO_CACHE:
191  		netfs_stat(&netfs_n_wh_write);
192  		break;
193  	default:
194  		WARN_ON_ONCE(1);
195  		break;
196  	}
197  
198  	if (stream->prepare_write)
199  		stream->prepare_write(subreq);
200  
201  	__set_bit(NETFS_SREQ_IN_PROGRESS, &subreq->flags);
202  
203  	/* We add to the end of the list whilst the collector may be walking
204  	 * the list.  The collector only goes nextwards and uses the lock to
205  	 * remove entries off of the front.
206  	 */
207  	spin_lock(&wreq->lock);
208  	list_add_tail(&subreq->rreq_link, &stream->subrequests);
209  	if (list_is_first(&subreq->rreq_link, &stream->subrequests)) {
210  		stream->front = subreq;
211  		if (!stream->active) {
212  			stream->collected_to = stream->front->start;
213  			/* Write list pointers before active flag */
214  			smp_store_release(&stream->active, true);
215  		}
216  	}
217  
218  	spin_unlock(&wreq->lock);
219  
220  	stream->construct = subreq;
221  }
222  
223  /*
224   * Set the I/O iterator for the filesystem/cache to use and dispatch the I/O
225   * operation.  The operation may be asynchronous and should call
226   * netfs_write_subrequest_terminated() when complete.
227   */
netfs_do_issue_write(struct netfs_io_stream * stream,struct netfs_io_subrequest * subreq)228  static void netfs_do_issue_write(struct netfs_io_stream *stream,
229  				 struct netfs_io_subrequest *subreq)
230  {
231  	struct netfs_io_request *wreq = subreq->rreq;
232  
233  	_enter("R=%x[%x],%zx", wreq->debug_id, subreq->debug_index, subreq->len);
234  
235  	if (test_bit(NETFS_SREQ_FAILED, &subreq->flags))
236  		return netfs_write_subrequest_terminated(subreq, subreq->error, false);
237  
238  	trace_netfs_sreq(subreq, netfs_sreq_trace_submit);
239  	stream->issue_write(subreq);
240  }
241  
netfs_reissue_write(struct netfs_io_stream * stream,struct netfs_io_subrequest * subreq,struct iov_iter * source)242  void netfs_reissue_write(struct netfs_io_stream *stream,
243  			 struct netfs_io_subrequest *subreq,
244  			 struct iov_iter *source)
245  {
246  	size_t size = subreq->len - subreq->transferred;
247  
248  	// TODO: Use encrypted buffer
249  	subreq->io_iter = *source;
250  	iov_iter_advance(source, size);
251  	iov_iter_truncate(&subreq->io_iter, size);
252  
253  	subreq->retry_count++;
254  	__clear_bit(NETFS_SREQ_MADE_PROGRESS, &subreq->flags);
255  	__set_bit(NETFS_SREQ_IN_PROGRESS, &subreq->flags);
256  	netfs_stat(&netfs_n_wh_retry_write_subreq);
257  	netfs_do_issue_write(stream, subreq);
258  }
259  
netfs_issue_write(struct netfs_io_request * wreq,struct netfs_io_stream * stream)260  void netfs_issue_write(struct netfs_io_request *wreq,
261  		       struct netfs_io_stream *stream)
262  {
263  	struct netfs_io_subrequest *subreq = stream->construct;
264  
265  	if (!subreq)
266  		return;
267  	stream->construct = NULL;
268  	subreq->io_iter.count = subreq->len;
269  	netfs_do_issue_write(stream, subreq);
270  }
271  
272  /*
273   * Add data to the write subrequest, dispatching each as we fill it up or if it
274   * is discontiguous with the previous.  We only fill one part at a time so that
275   * we can avoid overrunning the credits obtained (cifs) and try to parallelise
276   * content-crypto preparation with network writes.
277   */
netfs_advance_write(struct netfs_io_request * wreq,struct netfs_io_stream * stream,loff_t start,size_t len,bool to_eof)278  size_t netfs_advance_write(struct netfs_io_request *wreq,
279  			   struct netfs_io_stream *stream,
280  			   loff_t start, size_t len, bool to_eof)
281  {
282  	struct netfs_io_subrequest *subreq = stream->construct;
283  	size_t part;
284  
285  	if (!stream->avail) {
286  		_leave("no write");
287  		return len;
288  	}
289  
290  	_enter("R=%x[%x]", wreq->debug_id, subreq ? subreq->debug_index : 0);
291  
292  	if (subreq && start != subreq->start + subreq->len) {
293  		netfs_issue_write(wreq, stream);
294  		subreq = NULL;
295  	}
296  
297  	if (!stream->construct)
298  		netfs_prepare_write(wreq, stream, start);
299  	subreq = stream->construct;
300  
301  	part = umin(stream->sreq_max_len - subreq->len, len);
302  	_debug("part %zx/%zx %zx/%zx", subreq->len, stream->sreq_max_len, part, len);
303  	subreq->len += part;
304  	subreq->nr_segs++;
305  	stream->submit_extendable_to -= part;
306  
307  	if (subreq->len >= stream->sreq_max_len ||
308  	    subreq->nr_segs >= stream->sreq_max_segs ||
309  	    to_eof) {
310  		netfs_issue_write(wreq, stream);
311  		subreq = NULL;
312  	}
313  
314  	return part;
315  }
316  
317  /*
318   * Write some of a pending folio data back to the server.
319   */
netfs_write_folio(struct netfs_io_request * wreq,struct writeback_control * wbc,struct folio * folio)320  static int netfs_write_folio(struct netfs_io_request *wreq,
321  			     struct writeback_control *wbc,
322  			     struct folio *folio)
323  {
324  	struct netfs_io_stream *upload = &wreq->io_streams[0];
325  	struct netfs_io_stream *cache  = &wreq->io_streams[1];
326  	struct netfs_io_stream *stream;
327  	struct netfs_group *fgroup; /* TODO: Use this with ceph */
328  	struct netfs_folio *finfo;
329  	size_t iter_off = 0;
330  	size_t fsize = folio_size(folio), flen = fsize, foff = 0;
331  	loff_t fpos = folio_pos(folio), i_size;
332  	bool to_eof = false, streamw = false;
333  	bool debug = false;
334  
335  	_enter("");
336  
337  	if (rolling_buffer_make_space(&wreq->buffer) < 0)
338  		return -ENOMEM;
339  
340  	/* netfs_perform_write() may shift i_size around the page or from out
341  	 * of the page to beyond it, but cannot move i_size into or through the
342  	 * page since we have it locked.
343  	 */
344  	i_size = i_size_read(wreq->inode);
345  
346  	if (fpos >= i_size) {
347  		/* mmap beyond eof. */
348  		_debug("beyond eof");
349  		folio_start_writeback(folio);
350  		folio_unlock(folio);
351  		wreq->nr_group_rel += netfs_folio_written_back(folio);
352  		netfs_put_group_many(wreq->group, wreq->nr_group_rel);
353  		wreq->nr_group_rel = 0;
354  		return 0;
355  	}
356  
357  	if (fpos + fsize > wreq->i_size)
358  		wreq->i_size = i_size;
359  
360  	fgroup = netfs_folio_group(folio);
361  	finfo = netfs_folio_info(folio);
362  	if (finfo) {
363  		foff = finfo->dirty_offset;
364  		flen = foff + finfo->dirty_len;
365  		streamw = true;
366  	}
367  
368  	if (wreq->origin == NETFS_WRITETHROUGH) {
369  		to_eof = false;
370  		if (flen > i_size - fpos)
371  			flen = i_size - fpos;
372  	} else if (flen > i_size - fpos) {
373  		flen = i_size - fpos;
374  		if (!streamw)
375  			folio_zero_segment(folio, flen, fsize);
376  		to_eof = true;
377  	} else if (flen == i_size - fpos) {
378  		to_eof = true;
379  	}
380  	flen -= foff;
381  
382  	_debug("folio %zx %zx %zx", foff, flen, fsize);
383  
384  	/* Deal with discontinuities in the stream of dirty pages.  These can
385  	 * arise from a number of sources:
386  	 *
387  	 * (1) Intervening non-dirty pages from random-access writes, multiple
388  	 *     flushers writing back different parts simultaneously and manual
389  	 *     syncing.
390  	 *
391  	 * (2) Partially-written pages from write-streaming.
392  	 *
393  	 * (3) Pages that belong to a different write-back group (eg.  Ceph
394  	 *     snapshots).
395  	 *
396  	 * (4) Actually-clean pages that were marked for write to the cache
397  	 *     when they were read.  Note that these appear as a special
398  	 *     write-back group.
399  	 */
400  	if (fgroup == NETFS_FOLIO_COPY_TO_CACHE) {
401  		netfs_issue_write(wreq, upload);
402  	} else if (fgroup != wreq->group) {
403  		/* We can't write this page to the server yet. */
404  		kdebug("wrong group");
405  		folio_redirty_for_writepage(wbc, folio);
406  		folio_unlock(folio);
407  		netfs_issue_write(wreq, upload);
408  		netfs_issue_write(wreq, cache);
409  		return 0;
410  	}
411  
412  	if (foff > 0)
413  		netfs_issue_write(wreq, upload);
414  	if (streamw)
415  		netfs_issue_write(wreq, cache);
416  
417  	/* Flip the page to the writeback state and unlock.  If we're called
418  	 * from write-through, then the page has already been put into the wb
419  	 * state.
420  	 */
421  	if (wreq->origin == NETFS_WRITEBACK)
422  		folio_start_writeback(folio);
423  	folio_unlock(folio);
424  
425  	if (fgroup == NETFS_FOLIO_COPY_TO_CACHE) {
426  		if (!cache->avail) {
427  			trace_netfs_folio(folio, netfs_folio_trace_cancel_copy);
428  			netfs_issue_write(wreq, upload);
429  			netfs_folio_written_back(folio);
430  			return 0;
431  		}
432  		trace_netfs_folio(folio, netfs_folio_trace_store_copy);
433  	} else if (!upload->avail && !cache->avail) {
434  		trace_netfs_folio(folio, netfs_folio_trace_cancel_store);
435  		netfs_folio_written_back(folio);
436  		return 0;
437  	} else if (!upload->construct) {
438  		trace_netfs_folio(folio, netfs_folio_trace_store);
439  	} else {
440  		trace_netfs_folio(folio, netfs_folio_trace_store_plus);
441  	}
442  
443  	/* Attach the folio to the rolling buffer. */
444  	rolling_buffer_append(&wreq->buffer, folio, 0);
445  
446  	/* Move the submission point forward to allow for write-streaming data
447  	 * not starting at the front of the page.  We don't do write-streaming
448  	 * with the cache as the cache requires DIO alignment.
449  	 *
450  	 * Also skip uploading for data that's been read and just needs copying
451  	 * to the cache.
452  	 */
453  	for (int s = 0; s < NR_IO_STREAMS; s++) {
454  		stream = &wreq->io_streams[s];
455  		stream->submit_off = foff;
456  		stream->submit_len = flen;
457  		if (!stream->avail ||
458  		    (stream->source == NETFS_WRITE_TO_CACHE && streamw) ||
459  		    (stream->source == NETFS_UPLOAD_TO_SERVER &&
460  		     fgroup == NETFS_FOLIO_COPY_TO_CACHE)) {
461  			stream->submit_off = UINT_MAX;
462  			stream->submit_len = 0;
463  		}
464  	}
465  
466  	/* Attach the folio to one or more subrequests.  For a big folio, we
467  	 * could end up with thousands of subrequests if the wsize is small -
468  	 * but we might need to wait during the creation of subrequests for
469  	 * network resources (eg. SMB credits).
470  	 */
471  	for (;;) {
472  		ssize_t part;
473  		size_t lowest_off = ULONG_MAX;
474  		int choose_s = -1;
475  
476  		/* Always add to the lowest-submitted stream first. */
477  		for (int s = 0; s < NR_IO_STREAMS; s++) {
478  			stream = &wreq->io_streams[s];
479  			if (stream->submit_len > 0 &&
480  			    stream->submit_off < lowest_off) {
481  				lowest_off = stream->submit_off;
482  				choose_s = s;
483  			}
484  		}
485  
486  		if (choose_s < 0)
487  			break;
488  		stream = &wreq->io_streams[choose_s];
489  
490  		/* Advance the iterator(s). */
491  		if (stream->submit_off > iter_off) {
492  			rolling_buffer_advance(&wreq->buffer, stream->submit_off - iter_off);
493  			iter_off = stream->submit_off;
494  		}
495  
496  		atomic64_set(&wreq->issued_to, fpos + stream->submit_off);
497  		stream->submit_extendable_to = fsize - stream->submit_off;
498  		part = netfs_advance_write(wreq, stream, fpos + stream->submit_off,
499  					   stream->submit_len, to_eof);
500  		stream->submit_off += part;
501  		if (part > stream->submit_len)
502  			stream->submit_len = 0;
503  		else
504  			stream->submit_len -= part;
505  		if (part > 0)
506  			debug = true;
507  	}
508  
509  	if (fsize > iter_off)
510  		rolling_buffer_advance(&wreq->buffer, fsize - iter_off);
511  	atomic64_set(&wreq->issued_to, fpos + fsize);
512  
513  	if (!debug)
514  		kdebug("R=%x: No submit", wreq->debug_id);
515  
516  	if (foff + flen < fsize)
517  		for (int s = 0; s < NR_IO_STREAMS; s++)
518  			netfs_issue_write(wreq, &wreq->io_streams[s]);
519  
520  	_leave(" = 0");
521  	return 0;
522  }
523  
524  /*
525   * End the issuing of writes, letting the collector know we're done.
526   */
netfs_end_issue_write(struct netfs_io_request * wreq)527  static void netfs_end_issue_write(struct netfs_io_request *wreq)
528  {
529  	bool needs_poke = true;
530  
531  	smp_wmb(); /* Write subreq lists before ALL_QUEUED. */
532  	set_bit(NETFS_RREQ_ALL_QUEUED, &wreq->flags);
533  
534  	for (int s = 0; s < NR_IO_STREAMS; s++) {
535  		struct netfs_io_stream *stream = &wreq->io_streams[s];
536  
537  		if (!stream->active)
538  			continue;
539  		if (!list_empty(&stream->subrequests))
540  			needs_poke = false;
541  		netfs_issue_write(wreq, stream);
542  	}
543  
544  	if (needs_poke)
545  		netfs_wake_write_collector(wreq, false);
546  }
547  
548  /*
549   * Write some of the pending data back to the server
550   */
netfs_writepages(struct address_space * mapping,struct writeback_control * wbc)551  int netfs_writepages(struct address_space *mapping,
552  		     struct writeback_control *wbc)
553  {
554  	struct netfs_inode *ictx = netfs_inode(mapping->host);
555  	struct netfs_io_request *wreq = NULL;
556  	struct folio *folio;
557  	int error = 0;
558  
559  	if (!mutex_trylock(&ictx->wb_lock)) {
560  		if (wbc->sync_mode == WB_SYNC_NONE) {
561  			netfs_stat(&netfs_n_wb_lock_skip);
562  			return 0;
563  		}
564  		netfs_stat(&netfs_n_wb_lock_wait);
565  		mutex_lock(&ictx->wb_lock);
566  	}
567  
568  	/* Need the first folio to be able to set up the op. */
569  	folio = writeback_iter(mapping, wbc, NULL, &error);
570  	if (!folio)
571  		goto out;
572  
573  	wreq = netfs_create_write_req(mapping, NULL, folio_pos(folio), NETFS_WRITEBACK);
574  	if (IS_ERR(wreq)) {
575  		error = PTR_ERR(wreq);
576  		goto couldnt_start;
577  	}
578  
579  	trace_netfs_write(wreq, netfs_write_trace_writeback);
580  	netfs_stat(&netfs_n_wh_writepages);
581  
582  	do {
583  		_debug("wbiter %lx %llx", folio->index, atomic64_read(&wreq->issued_to));
584  
585  		/* It appears we don't have to handle cyclic writeback wrapping. */
586  		WARN_ON_ONCE(wreq && folio_pos(folio) < atomic64_read(&wreq->issued_to));
587  
588  		if (netfs_folio_group(folio) != NETFS_FOLIO_COPY_TO_CACHE &&
589  		    unlikely(!test_bit(NETFS_RREQ_UPLOAD_TO_SERVER, &wreq->flags))) {
590  			set_bit(NETFS_RREQ_UPLOAD_TO_SERVER, &wreq->flags);
591  			wreq->netfs_ops->begin_writeback(wreq);
592  		}
593  
594  		error = netfs_write_folio(wreq, wbc, folio);
595  		if (error < 0)
596  			break;
597  	} while ((folio = writeback_iter(mapping, wbc, folio, &error)));
598  
599  	netfs_end_issue_write(wreq);
600  
601  	mutex_unlock(&ictx->wb_lock);
602  
603  	netfs_put_request(wreq, false, netfs_rreq_trace_put_return);
604  	_leave(" = %d", error);
605  	return error;
606  
607  couldnt_start:
608  	netfs_kill_dirty_pages(mapping, wbc, folio);
609  out:
610  	mutex_unlock(&ictx->wb_lock);
611  	_leave(" = %d", error);
612  	return error;
613  }
614  EXPORT_SYMBOL(netfs_writepages);
615  
616  /*
617   * Begin a write operation for writing through the pagecache.
618   */
netfs_begin_writethrough(struct kiocb * iocb,size_t len)619  struct netfs_io_request *netfs_begin_writethrough(struct kiocb *iocb, size_t len)
620  {
621  	struct netfs_io_request *wreq = NULL;
622  	struct netfs_inode *ictx = netfs_inode(file_inode(iocb->ki_filp));
623  
624  	mutex_lock(&ictx->wb_lock);
625  
626  	wreq = netfs_create_write_req(iocb->ki_filp->f_mapping, iocb->ki_filp,
627  				      iocb->ki_pos, NETFS_WRITETHROUGH);
628  	if (IS_ERR(wreq)) {
629  		mutex_unlock(&ictx->wb_lock);
630  		return wreq;
631  	}
632  
633  	wreq->io_streams[0].avail = true;
634  	trace_netfs_write(wreq, netfs_write_trace_writethrough);
635  	return wreq;
636  }
637  
638  /*
639   * Advance the state of the write operation used when writing through the
640   * pagecache.  Data has been copied into the pagecache that we need to append
641   * to the request.  If we've added more than wsize then we need to create a new
642   * subrequest.
643   */
netfs_advance_writethrough(struct netfs_io_request * wreq,struct writeback_control * wbc,struct folio * folio,size_t copied,bool to_page_end,struct folio ** writethrough_cache)644  int netfs_advance_writethrough(struct netfs_io_request *wreq, struct writeback_control *wbc,
645  			       struct folio *folio, size_t copied, bool to_page_end,
646  			       struct folio **writethrough_cache)
647  {
648  	_enter("R=%x ic=%zu ws=%u cp=%zu tp=%u",
649  	       wreq->debug_id, wreq->buffer.iter.count, wreq->wsize, copied, to_page_end);
650  
651  	if (!*writethrough_cache) {
652  		if (folio_test_dirty(folio))
653  			/* Sigh.  mmap. */
654  			folio_clear_dirty_for_io(folio);
655  
656  		/* We can make multiple writes to the folio... */
657  		folio_start_writeback(folio);
658  		if (wreq->len == 0)
659  			trace_netfs_folio(folio, netfs_folio_trace_wthru);
660  		else
661  			trace_netfs_folio(folio, netfs_folio_trace_wthru_plus);
662  		*writethrough_cache = folio;
663  	}
664  
665  	wreq->len += copied;
666  	if (!to_page_end)
667  		return 0;
668  
669  	*writethrough_cache = NULL;
670  	return netfs_write_folio(wreq, wbc, folio);
671  }
672  
673  /*
674   * End a write operation used when writing through the pagecache.
675   */
netfs_end_writethrough(struct netfs_io_request * wreq,struct writeback_control * wbc,struct folio * writethrough_cache)676  int netfs_end_writethrough(struct netfs_io_request *wreq, struct writeback_control *wbc,
677  			   struct folio *writethrough_cache)
678  {
679  	struct netfs_inode *ictx = netfs_inode(wreq->inode);
680  	int ret;
681  
682  	_enter("R=%x", wreq->debug_id);
683  
684  	if (writethrough_cache)
685  		netfs_write_folio(wreq, wbc, writethrough_cache);
686  
687  	netfs_end_issue_write(wreq);
688  
689  	mutex_unlock(&ictx->wb_lock);
690  
691  	if (wreq->iocb) {
692  		ret = -EIOCBQUEUED;
693  	} else {
694  		wait_on_bit(&wreq->flags, NETFS_RREQ_IN_PROGRESS, TASK_UNINTERRUPTIBLE);
695  		ret = wreq->error;
696  	}
697  	netfs_put_request(wreq, false, netfs_rreq_trace_put_return);
698  	return ret;
699  }
700  
701  /*
702   * Write data to the server without going through the pagecache and without
703   * writing it to the local cache.
704   */
netfs_unbuffered_write(struct netfs_io_request * wreq,bool may_wait,size_t len)705  int netfs_unbuffered_write(struct netfs_io_request *wreq, bool may_wait, size_t len)
706  {
707  	struct netfs_io_stream *upload = &wreq->io_streams[0];
708  	ssize_t part;
709  	loff_t start = wreq->start;
710  	int error = 0;
711  
712  	_enter("%zx", len);
713  
714  	if (wreq->origin == NETFS_DIO_WRITE)
715  		inode_dio_begin(wreq->inode);
716  
717  	while (len) {
718  		// TODO: Prepare content encryption
719  
720  		_debug("unbuffered %zx", len);
721  		part = netfs_advance_write(wreq, upload, start, len, false);
722  		start += part;
723  		len -= part;
724  		rolling_buffer_advance(&wreq->buffer, part);
725  		if (test_bit(NETFS_RREQ_PAUSE, &wreq->flags)) {
726  			trace_netfs_rreq(wreq, netfs_rreq_trace_wait_pause);
727  			wait_event(wreq->waitq, !test_bit(NETFS_RREQ_PAUSE, &wreq->flags));
728  		}
729  		if (test_bit(NETFS_RREQ_FAILED, &wreq->flags))
730  			break;
731  	}
732  
733  	netfs_end_issue_write(wreq);
734  	_leave(" = %d", error);
735  	return error;
736  }
737  
738  /*
739   * Write some of a pending folio data back to the server and/or the cache.
740   */
netfs_write_folio_single(struct netfs_io_request * wreq,struct folio * folio)741  static int netfs_write_folio_single(struct netfs_io_request *wreq,
742  				    struct folio *folio)
743  {
744  	struct netfs_io_stream *upload = &wreq->io_streams[0];
745  	struct netfs_io_stream *cache  = &wreq->io_streams[1];
746  	struct netfs_io_stream *stream;
747  	size_t iter_off = 0;
748  	size_t fsize = folio_size(folio), flen;
749  	loff_t fpos = folio_pos(folio);
750  	bool to_eof = false;
751  	bool no_debug = false;
752  
753  	_enter("");
754  
755  	flen = folio_size(folio);
756  	if (flen > wreq->i_size - fpos) {
757  		flen = wreq->i_size - fpos;
758  		folio_zero_segment(folio, flen, fsize);
759  		to_eof = true;
760  	} else if (flen == wreq->i_size - fpos) {
761  		to_eof = true;
762  	}
763  
764  	_debug("folio %zx/%zx", flen, fsize);
765  
766  	if (!upload->avail && !cache->avail) {
767  		trace_netfs_folio(folio, netfs_folio_trace_cancel_store);
768  		return 0;
769  	}
770  
771  	if (!upload->construct)
772  		trace_netfs_folio(folio, netfs_folio_trace_store);
773  	else
774  		trace_netfs_folio(folio, netfs_folio_trace_store_plus);
775  
776  	/* Attach the folio to the rolling buffer. */
777  	folio_get(folio);
778  	rolling_buffer_append(&wreq->buffer, folio, NETFS_ROLLBUF_PUT_MARK);
779  
780  	/* Move the submission point forward to allow for write-streaming data
781  	 * not starting at the front of the page.  We don't do write-streaming
782  	 * with the cache as the cache requires DIO alignment.
783  	 *
784  	 * Also skip uploading for data that's been read and just needs copying
785  	 * to the cache.
786  	 */
787  	for (int s = 0; s < NR_IO_STREAMS; s++) {
788  		stream = &wreq->io_streams[s];
789  		stream->submit_off = 0;
790  		stream->submit_len = flen;
791  		if (!stream->avail) {
792  			stream->submit_off = UINT_MAX;
793  			stream->submit_len = 0;
794  		}
795  	}
796  
797  	/* Attach the folio to one or more subrequests.  For a big folio, we
798  	 * could end up with thousands of subrequests if the wsize is small -
799  	 * but we might need to wait during the creation of subrequests for
800  	 * network resources (eg. SMB credits).
801  	 */
802  	for (;;) {
803  		ssize_t part;
804  		size_t lowest_off = ULONG_MAX;
805  		int choose_s = -1;
806  
807  		/* Always add to the lowest-submitted stream first. */
808  		for (int s = 0; s < NR_IO_STREAMS; s++) {
809  			stream = &wreq->io_streams[s];
810  			if (stream->submit_len > 0 &&
811  			    stream->submit_off < lowest_off) {
812  				lowest_off = stream->submit_off;
813  				choose_s = s;
814  			}
815  		}
816  
817  		if (choose_s < 0)
818  			break;
819  		stream = &wreq->io_streams[choose_s];
820  
821  		/* Advance the iterator(s). */
822  		if (stream->submit_off > iter_off) {
823  			rolling_buffer_advance(&wreq->buffer, stream->submit_off - iter_off);
824  			iter_off = stream->submit_off;
825  		}
826  
827  		atomic64_set(&wreq->issued_to, fpos + stream->submit_off);
828  		stream->submit_extendable_to = fsize - stream->submit_off;
829  		part = netfs_advance_write(wreq, stream, fpos + stream->submit_off,
830  					   stream->submit_len, to_eof);
831  		stream->submit_off += part;
832  		if (part > stream->submit_len)
833  			stream->submit_len = 0;
834  		else
835  			stream->submit_len -= part;
836  		if (part > 0)
837  			no_debug = true;
838  	}
839  
840  	wreq->buffer.iter.iov_offset = 0;
841  	if (fsize > iter_off)
842  		rolling_buffer_advance(&wreq->buffer, fsize - iter_off);
843  	atomic64_set(&wreq->issued_to, fpos + fsize);
844  
845  	if (!no_debug)
846  		kdebug("R=%x: No submit", wreq->debug_id);
847  	_leave(" = 0");
848  	return 0;
849  }
850  
851  /**
852   * netfs_writeback_single - Write back a monolithic payload
853   * @mapping: The mapping to write from
854   * @wbc: Hints from the VM
855   * @iter: Data to write, must be ITER_FOLIOQ.
856   *
857   * Write a monolithic, non-pagecache object back to the server and/or
858   * the cache.
859   */
netfs_writeback_single(struct address_space * mapping,struct writeback_control * wbc,struct iov_iter * iter)860  int netfs_writeback_single(struct address_space *mapping,
861  			   struct writeback_control *wbc,
862  			   struct iov_iter *iter)
863  {
864  	struct netfs_io_request *wreq;
865  	struct netfs_inode *ictx = netfs_inode(mapping->host);
866  	struct folio_queue *fq;
867  	size_t size = iov_iter_count(iter);
868  	int ret;
869  
870  	if (WARN_ON_ONCE(!iov_iter_is_folioq(iter)))
871  		return -EIO;
872  
873  	if (!mutex_trylock(&ictx->wb_lock)) {
874  		if (wbc->sync_mode == WB_SYNC_NONE) {
875  			netfs_stat(&netfs_n_wb_lock_skip);
876  			return 0;
877  		}
878  		netfs_stat(&netfs_n_wb_lock_wait);
879  		mutex_lock(&ictx->wb_lock);
880  	}
881  
882  	wreq = netfs_create_write_req(mapping, NULL, 0, NETFS_WRITEBACK_SINGLE);
883  	if (IS_ERR(wreq)) {
884  		ret = PTR_ERR(wreq);
885  		goto couldnt_start;
886  	}
887  
888  	trace_netfs_write(wreq, netfs_write_trace_writeback);
889  	netfs_stat(&netfs_n_wh_writepages);
890  
891  	if (__test_and_set_bit(NETFS_RREQ_UPLOAD_TO_SERVER, &wreq->flags))
892  		wreq->netfs_ops->begin_writeback(wreq);
893  
894  	for (fq = (struct folio_queue *)iter->folioq; fq; fq = fq->next) {
895  		for (int slot = 0; slot < folioq_count(fq); slot++) {
896  			struct folio *folio = folioq_folio(fq, slot);
897  			size_t part = umin(folioq_folio_size(fq, slot), size);
898  
899  			_debug("wbiter %lx %llx", folio->index, atomic64_read(&wreq->issued_to));
900  
901  			ret = netfs_write_folio_single(wreq, folio);
902  			if (ret < 0)
903  				goto stop;
904  			size -= part;
905  			if (size <= 0)
906  				goto stop;
907  		}
908  	}
909  
910  stop:
911  	for (int s = 0; s < NR_IO_STREAMS; s++)
912  		netfs_issue_write(wreq, &wreq->io_streams[s]);
913  	smp_wmb(); /* Write lists before ALL_QUEUED. */
914  	set_bit(NETFS_RREQ_ALL_QUEUED, &wreq->flags);
915  
916  	mutex_unlock(&ictx->wb_lock);
917  
918  	netfs_put_request(wreq, false, netfs_rreq_trace_put_return);
919  	_leave(" = %d", ret);
920  	return ret;
921  
922  couldnt_start:
923  	mutex_unlock(&ictx->wb_lock);
924  	_leave(" = %d", ret);
925  	return ret;
926  }
927  EXPORT_SYMBOL(netfs_writeback_single);
928