~ [ source navigation ] ~ [ diff markup ] ~ [ identifier search ] ~

TOMOYO Linux Cross Reference
Linux/fs/netfs/write_issue.c

Version: ~ [ linux-6.11.5 ] ~ [ linux-6.10.14 ] ~ [ linux-6.9.12 ] ~ [ linux-6.8.12 ] ~ [ linux-6.7.12 ] ~ [ linux-6.6.58 ] ~ [ linux-6.5.13 ] ~ [ linux-6.4.16 ] ~ [ linux-6.3.13 ] ~ [ linux-6.2.16 ] ~ [ linux-6.1.114 ] ~ [ linux-6.0.19 ] ~ [ linux-5.19.17 ] ~ [ linux-5.18.19 ] ~ [ linux-5.17.15 ] ~ [ linux-5.16.20 ] ~ [ linux-5.15.169 ] ~ [ linux-5.14.21 ] ~ [ linux-5.13.19 ] ~ [ linux-5.12.19 ] ~ [ linux-5.11.22 ] ~ [ linux-5.10.228 ] ~ [ linux-5.9.16 ] ~ [ linux-5.8.18 ] ~ [ linux-5.7.19 ] ~ [ linux-5.6.19 ] ~ [ linux-5.5.19 ] ~ [ linux-5.4.284 ] ~ [ linux-5.3.18 ] ~ [ linux-5.2.21 ] ~ [ linux-5.1.21 ] ~ [ linux-5.0.21 ] ~ [ linux-4.20.17 ] ~ [ linux-4.19.322 ] ~ [ linux-4.18.20 ] ~ [ linux-4.17.19 ] ~ [ linux-4.16.18 ] ~ [ linux-4.15.18 ] ~ [ linux-4.14.336 ] ~ [ linux-4.13.16 ] ~ [ linux-4.12.14 ] ~ [ linux-4.11.12 ] ~ [ linux-4.10.17 ] ~ [ linux-4.9.337 ] ~ [ linux-4.4.302 ] ~ [ linux-3.10.108 ] ~ [ linux-2.6.32.71 ] ~ [ linux-2.6.0 ] ~ [ linux-2.4.37.11 ] ~ [ unix-v6-master ] ~ [ ccs-tools-1.8.9 ] ~ [ policy-sample ] ~
Architecture: ~ [ i386 ] ~ [ alpha ] ~ [ m68k ] ~ [ mips ] ~ [ ppc ] ~ [ sparc ] ~ [ sparc64 ] ~

  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  */
 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  */
 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_buffered = (origin == NETFS_WRITEBACK ||
 98                             origin == NETFS_WRITETHROUGH);
 99 
100         wreq = netfs_alloc_request(mapping, file, start, 0, origin);
101         if (IS_ERR(wreq))
102                 return wreq;
103 
104         _enter("R=%x", wreq->debug_id);
105 
106         ictx = netfs_inode(wreq->inode);
107         if (is_buffered && netfs_is_cache_enabled(ictx))
108                 fscache_begin_write_operation(&wreq->cache_resources, netfs_i_cookie(ictx));
109 
110         wreq->contiguity = wreq->start;
111         wreq->cleaned_to = wreq->start;
112         INIT_WORK(&wreq->work, netfs_write_collection_worker);
113 
114         wreq->io_streams[0].stream_nr           = 0;
115         wreq->io_streams[0].source              = NETFS_UPLOAD_TO_SERVER;
116         wreq->io_streams[0].prepare_write       = ictx->ops->prepare_write;
117         wreq->io_streams[0].issue_write         = ictx->ops->issue_write;
118         wreq->io_streams[0].collected_to        = start;
119         wreq->io_streams[0].transferred         = LONG_MAX;
120 
121         wreq->io_streams[1].stream_nr           = 1;
122         wreq->io_streams[1].source              = NETFS_WRITE_TO_CACHE;
123         wreq->io_streams[1].collected_to        = start;
124         wreq->io_streams[1].transferred         = LONG_MAX;
125         if (fscache_resources_valid(&wreq->cache_resources)) {
126                 wreq->io_streams[1].avail       = true;
127                 wreq->io_streams[1].active      = true;
128                 wreq->io_streams[1].prepare_write = wreq->cache_resources.ops->prepare_write_subreq;
129                 wreq->io_streams[1].issue_write = wreq->cache_resources.ops->issue_write;
130         }
131 
132         return wreq;
133 }
134 
135 /**
136  * netfs_prepare_write_failed - Note write preparation failed
137  * @subreq: The subrequest to mark
138  *
139  * Mark a subrequest to note that preparation for write failed.
140  */
141 void netfs_prepare_write_failed(struct netfs_io_subrequest *subreq)
142 {
143         __set_bit(NETFS_SREQ_FAILED, &subreq->flags);
144         trace_netfs_sreq(subreq, netfs_sreq_trace_prep_failed);
145 }
146 EXPORT_SYMBOL(netfs_prepare_write_failed);
147 
148 /*
149  * Prepare a write subrequest.  We need to allocate a new subrequest
150  * if we don't have one.
151  */
152 static void netfs_prepare_write(struct netfs_io_request *wreq,
153                                 struct netfs_io_stream *stream,
154                                 loff_t start)
155 {
156         struct netfs_io_subrequest *subreq;
157 
158         subreq = netfs_alloc_subrequest(wreq);
159         subreq->source          = stream->source;
160         subreq->start           = start;
161         subreq->max_len         = ULONG_MAX;
162         subreq->max_nr_segs     = INT_MAX;
163         subreq->stream_nr       = stream->stream_nr;
164 
165         _enter("R=%x[%x]", wreq->debug_id, subreq->debug_index);
166 
167         trace_netfs_sreq_ref(wreq->debug_id, subreq->debug_index,
168                              refcount_read(&subreq->ref),
169                              netfs_sreq_trace_new);
170 
171         trace_netfs_sreq(subreq, netfs_sreq_trace_prepare);
172 
173         switch (stream->source) {
174         case NETFS_UPLOAD_TO_SERVER:
175                 netfs_stat(&netfs_n_wh_upload);
176                 subreq->max_len = wreq->wsize;
177                 break;
178         case NETFS_WRITE_TO_CACHE:
179                 netfs_stat(&netfs_n_wh_write);
180                 break;
181         default:
182                 WARN_ON_ONCE(1);
183                 break;
184         }
185 
186         if (stream->prepare_write)
187                 stream->prepare_write(subreq);
188 
189         __set_bit(NETFS_SREQ_IN_PROGRESS, &subreq->flags);
190 
191         /* We add to the end of the list whilst the collector may be walking
192          * the list.  The collector only goes nextwards and uses the lock to
193          * remove entries off of the front.
194          */
195         spin_lock(&wreq->lock);
196         list_add_tail(&subreq->rreq_link, &stream->subrequests);
197         if (list_is_first(&subreq->rreq_link, &stream->subrequests)) {
198                 stream->front = subreq;
199                 if (!stream->active) {
200                         stream->collected_to = stream->front->start;
201                         /* Write list pointers before active flag */
202                         smp_store_release(&stream->active, true);
203                 }
204         }
205 
206         spin_unlock(&wreq->lock);
207 
208         stream->construct = subreq;
209 }
210 
211 /*
212  * Set the I/O iterator for the filesystem/cache to use and dispatch the I/O
213  * operation.  The operation may be asynchronous and should call
214  * netfs_write_subrequest_terminated() when complete.
215  */
216 static void netfs_do_issue_write(struct netfs_io_stream *stream,
217                                  struct netfs_io_subrequest *subreq)
218 {
219         struct netfs_io_request *wreq = subreq->rreq;
220 
221         _enter("R=%x[%x],%zx", wreq->debug_id, subreq->debug_index, subreq->len);
222 
223         if (test_bit(NETFS_SREQ_FAILED, &subreq->flags))
224                 return netfs_write_subrequest_terminated(subreq, subreq->error, false);
225 
226         // TODO: Use encrypted buffer
227         if (test_bit(NETFS_RREQ_USE_IO_ITER, &wreq->flags)) {
228                 subreq->io_iter = wreq->io_iter;
229                 iov_iter_advance(&subreq->io_iter,
230                                  subreq->start + subreq->transferred - wreq->start);
231                 iov_iter_truncate(&subreq->io_iter,
232                                  subreq->len - subreq->transferred);
233         } else {
234                 iov_iter_xarray(&subreq->io_iter, ITER_SOURCE, &wreq->mapping->i_pages,
235                                 subreq->start + subreq->transferred,
236                                 subreq->len   - subreq->transferred);
237         }
238 
239         trace_netfs_sreq(subreq, netfs_sreq_trace_submit);
240         stream->issue_write(subreq);
241 }
242 
243 void netfs_reissue_write(struct netfs_io_stream *stream,
244                          struct netfs_io_subrequest *subreq)
245 {
246         __set_bit(NETFS_SREQ_IN_PROGRESS, &subreq->flags);
247         netfs_do_issue_write(stream, subreq);
248 }
249 
250 static void netfs_issue_write(struct netfs_io_request *wreq,
251                               struct netfs_io_stream *stream)
252 {
253         struct netfs_io_subrequest *subreq = stream->construct;
254 
255         if (!subreq)
256                 return;
257         stream->construct = NULL;
258 
259         if (subreq->start + subreq->len > wreq->start + wreq->submitted)
260                 WRITE_ONCE(wreq->submitted, subreq->start + subreq->len - wreq->start);
261         netfs_do_issue_write(stream, subreq);
262 }
263 
264 /*
265  * Add data to the write subrequest, dispatching each as we fill it up or if it
266  * is discontiguous with the previous.  We only fill one part at a time so that
267  * we can avoid overrunning the credits obtained (cifs) and try to parallelise
268  * content-crypto preparation with network writes.
269  */
270 int netfs_advance_write(struct netfs_io_request *wreq,
271                         struct netfs_io_stream *stream,
272                         loff_t start, size_t len, bool to_eof)
273 {
274         struct netfs_io_subrequest *subreq = stream->construct;
275         size_t part;
276 
277         if (!stream->avail) {
278                 _leave("no write");
279                 return len;
280         }
281 
282         _enter("R=%x[%x]", wreq->debug_id, subreq ? subreq->debug_index : 0);
283 
284         if (subreq && start != subreq->start + subreq->len) {
285                 netfs_issue_write(wreq, stream);
286                 subreq = NULL;
287         }
288 
289         if (!stream->construct)
290                 netfs_prepare_write(wreq, stream, start);
291         subreq = stream->construct;
292 
293         part = min(subreq->max_len - subreq->len, len);
294         _debug("part %zx/%zx %zx/%zx", subreq->len, subreq->max_len, part, len);
295         subreq->len += part;
296         subreq->nr_segs++;
297 
298         if (subreq->len >= subreq->max_len ||
299             subreq->nr_segs >= subreq->max_nr_segs ||
300             to_eof) {
301                 netfs_issue_write(wreq, stream);
302                 subreq = NULL;
303         }
304 
305         return part;
306 }
307 
308 /*
309  * Write some of a pending folio data back to the server.
310  */
311 static int netfs_write_folio(struct netfs_io_request *wreq,
312                              struct writeback_control *wbc,
313                              struct folio *folio)
314 {
315         struct netfs_io_stream *upload = &wreq->io_streams[0];
316         struct netfs_io_stream *cache  = &wreq->io_streams[1];
317         struct netfs_io_stream *stream;
318         struct netfs_group *fgroup; /* TODO: Use this with ceph */
319         struct netfs_folio *finfo;
320         size_t fsize = folio_size(folio), flen = fsize, foff = 0;
321         loff_t fpos = folio_pos(folio), i_size;
322         bool to_eof = false, streamw = false;
323         bool debug = false;
324 
325         _enter("");
326 
327         /* netfs_perform_write() may shift i_size around the page or from out
328          * of the page to beyond it, but cannot move i_size into or through the
329          * page since we have it locked.
330          */
331         i_size = i_size_read(wreq->inode);
332 
333         if (fpos >= i_size) {
334                 /* mmap beyond eof. */
335                 _debug("beyond eof");
336                 folio_start_writeback(folio);
337                 folio_unlock(folio);
338                 wreq->nr_group_rel += netfs_folio_written_back(folio);
339                 netfs_put_group_many(wreq->group, wreq->nr_group_rel);
340                 wreq->nr_group_rel = 0;
341                 return 0;
342         }
343 
344         if (fpos + fsize > wreq->i_size)
345                 wreq->i_size = i_size;
346 
347         fgroup = netfs_folio_group(folio);
348         finfo = netfs_folio_info(folio);
349         if (finfo) {
350                 foff = finfo->dirty_offset;
351                 flen = foff + finfo->dirty_len;
352                 streamw = true;
353         }
354 
355         if (wreq->origin == NETFS_WRITETHROUGH) {
356                 to_eof = false;
357                 if (flen > i_size - fpos)
358                         flen = i_size - fpos;
359         } else if (flen > i_size - fpos) {
360                 flen = i_size - fpos;
361                 if (!streamw)
362                         folio_zero_segment(folio, flen, fsize);
363                 to_eof = true;
364         } else if (flen == i_size - fpos) {
365                 to_eof = true;
366         }
367         flen -= foff;
368 
369         _debug("folio %zx %zx %zx", foff, flen, fsize);
370 
371         /* Deal with discontinuities in the stream of dirty pages.  These can
372          * arise from a number of sources:
373          *
374          * (1) Intervening non-dirty pages from random-access writes, multiple
375          *     flushers writing back different parts simultaneously and manual
376          *     syncing.
377          *
378          * (2) Partially-written pages from write-streaming.
379          *
380          * (3) Pages that belong to a different write-back group (eg.  Ceph
381          *     snapshots).
382          *
383          * (4) Actually-clean pages that were marked for write to the cache
384          *     when they were read.  Note that these appear as a special
385          *     write-back group.
386          */
387         if (fgroup == NETFS_FOLIO_COPY_TO_CACHE) {
388                 netfs_issue_write(wreq, upload);
389         } else if (fgroup != wreq->group) {
390                 /* We can't write this page to the server yet. */
391                 kdebug("wrong group");
392                 folio_redirty_for_writepage(wbc, folio);
393                 folio_unlock(folio);
394                 netfs_issue_write(wreq, upload);
395                 netfs_issue_write(wreq, cache);
396                 return 0;
397         }
398 
399         if (foff > 0)
400                 netfs_issue_write(wreq, upload);
401         if (streamw)
402                 netfs_issue_write(wreq, cache);
403 
404         /* Flip the page to the writeback state and unlock.  If we're called
405          * from write-through, then the page has already been put into the wb
406          * state.
407          */
408         if (wreq->origin == NETFS_WRITEBACK)
409                 folio_start_writeback(folio);
410         folio_unlock(folio);
411 
412         if (fgroup == NETFS_FOLIO_COPY_TO_CACHE) {
413                 if (!cache->avail) {
414                         trace_netfs_folio(folio, netfs_folio_trace_cancel_copy);
415                         netfs_issue_write(wreq, upload);
416                         netfs_folio_written_back(folio);
417                         return 0;
418                 }
419                 trace_netfs_folio(folio, netfs_folio_trace_store_copy);
420         } else if (!upload->avail && !cache->avail) {
421                 trace_netfs_folio(folio, netfs_folio_trace_cancel_store);
422                 netfs_folio_written_back(folio);
423                 return 0;
424         } else if (!upload->construct) {
425                 trace_netfs_folio(folio, netfs_folio_trace_store);
426         } else {
427                 trace_netfs_folio(folio, netfs_folio_trace_store_plus);
428         }
429 
430         /* Move the submission point forward to allow for write-streaming data
431          * not starting at the front of the page.  We don't do write-streaming
432          * with the cache as the cache requires DIO alignment.
433          *
434          * Also skip uploading for data that's been read and just needs copying
435          * to the cache.
436          */
437         for (int s = 0; s < NR_IO_STREAMS; s++) {
438                 stream = &wreq->io_streams[s];
439                 stream->submit_max_len = fsize;
440                 stream->submit_off = foff;
441                 stream->submit_len = flen;
442                 if ((stream->source == NETFS_WRITE_TO_CACHE && streamw) ||
443                     (stream->source == NETFS_UPLOAD_TO_SERVER &&
444                      fgroup == NETFS_FOLIO_COPY_TO_CACHE)) {
445                         stream->submit_off = UINT_MAX;
446                         stream->submit_len = 0;
447                         stream->submit_max_len = 0;
448                 }
449         }
450 
451         /* Attach the folio to one or more subrequests.  For a big folio, we
452          * could end up with thousands of subrequests if the wsize is small -
453          * but we might need to wait during the creation of subrequests for
454          * network resources (eg. SMB credits).
455          */
456         for (;;) {
457                 ssize_t part;
458                 size_t lowest_off = ULONG_MAX;
459                 int choose_s = -1;
460 
461                 /* Always add to the lowest-submitted stream first. */
462                 for (int s = 0; s < NR_IO_STREAMS; s++) {
463                         stream = &wreq->io_streams[s];
464                         if (stream->submit_len > 0 &&
465                             stream->submit_off < lowest_off) {
466                                 lowest_off = stream->submit_off;
467                                 choose_s = s;
468                         }
469                 }
470 
471                 if (choose_s < 0)
472                         break;
473                 stream = &wreq->io_streams[choose_s];
474 
475                 part = netfs_advance_write(wreq, stream, fpos + stream->submit_off,
476                                            stream->submit_len, to_eof);
477                 atomic64_set(&wreq->issued_to, fpos + stream->submit_off);
478                 stream->submit_off += part;
479                 stream->submit_max_len -= part;
480                 if (part > stream->submit_len)
481                         stream->submit_len = 0;
482                 else
483                         stream->submit_len -= part;
484                 if (part > 0)
485                         debug = true;
486         }
487 
488         atomic64_set(&wreq->issued_to, fpos + fsize);
489 
490         if (!debug)
491                 kdebug("R=%x: No submit", wreq->debug_id);
492 
493         if (foff + flen < fsize)
494                 for (int s = 0; s < NR_IO_STREAMS; s++)
495                         netfs_issue_write(wreq, &wreq->io_streams[s]);
496 
497         _leave(" = 0");
498         return 0;
499 }
500 
501 /*
502  * End the issuing of writes, letting the collector know we're done.
503  */
504 static void netfs_end_issue_write(struct netfs_io_request *wreq)
505 {
506         bool needs_poke = true;
507 
508         smp_wmb(); /* Write subreq lists before ALL_QUEUED. */
509         set_bit(NETFS_RREQ_ALL_QUEUED, &wreq->flags);
510 
511         for (int s = 0; s < NR_IO_STREAMS; s++) {
512                 struct netfs_io_stream *stream = &wreq->io_streams[s];
513 
514                 if (!stream->active)
515                         continue;
516                 if (!list_empty(&stream->subrequests))
517                         needs_poke = false;
518                 netfs_issue_write(wreq, stream);
519         }
520 
521         if (needs_poke)
522                 netfs_wake_write_collector(wreq, false);
523 }
524 
525 /*
526  * Write some of the pending data back to the server
527  */
528 int netfs_writepages(struct address_space *mapping,
529                      struct writeback_control *wbc)
530 {
531         struct netfs_inode *ictx = netfs_inode(mapping->host);
532         struct netfs_io_request *wreq = NULL;
533         struct folio *folio;
534         int error = 0;
535 
536         if (wbc->sync_mode == WB_SYNC_ALL)
537                 mutex_lock(&ictx->wb_lock);
538         else if (!mutex_trylock(&ictx->wb_lock))
539                 return 0;
540 
541         /* Need the first folio to be able to set up the op. */
542         folio = writeback_iter(mapping, wbc, NULL, &error);
543         if (!folio)
544                 goto out;
545 
546         wreq = netfs_create_write_req(mapping, NULL, folio_pos(folio), NETFS_WRITEBACK);
547         if (IS_ERR(wreq)) {
548                 error = PTR_ERR(wreq);
549                 goto couldnt_start;
550         }
551 
552         trace_netfs_write(wreq, netfs_write_trace_writeback);
553         netfs_stat(&netfs_n_wh_writepages);
554 
555         do {
556                 _debug("wbiter %lx %llx", folio->index, wreq->start + wreq->submitted);
557 
558                 /* It appears we don't have to handle cyclic writeback wrapping. */
559                 WARN_ON_ONCE(wreq && folio_pos(folio) < wreq->start + wreq->submitted);
560 
561                 if (netfs_folio_group(folio) != NETFS_FOLIO_COPY_TO_CACHE &&
562                     unlikely(!test_bit(NETFS_RREQ_UPLOAD_TO_SERVER, &wreq->flags))) {
563                         set_bit(NETFS_RREQ_UPLOAD_TO_SERVER, &wreq->flags);
564                         wreq->netfs_ops->begin_writeback(wreq);
565                 }
566 
567                 error = netfs_write_folio(wreq, wbc, folio);
568                 if (error < 0)
569                         break;
570         } while ((folio = writeback_iter(mapping, wbc, folio, &error)));
571 
572         netfs_end_issue_write(wreq);
573 
574         mutex_unlock(&ictx->wb_lock);
575 
576         netfs_put_request(wreq, false, netfs_rreq_trace_put_return);
577         _leave(" = %d", error);
578         return error;
579 
580 couldnt_start:
581         netfs_kill_dirty_pages(mapping, wbc, folio);
582 out:
583         mutex_unlock(&ictx->wb_lock);
584         _leave(" = %d", error);
585         return error;
586 }
587 EXPORT_SYMBOL(netfs_writepages);
588 
589 /*
590  * Begin a write operation for writing through the pagecache.
591  */
592 struct netfs_io_request *netfs_begin_writethrough(struct kiocb *iocb, size_t len)
593 {
594         struct netfs_io_request *wreq = NULL;
595         struct netfs_inode *ictx = netfs_inode(file_inode(iocb->ki_filp));
596 
597         mutex_lock(&ictx->wb_lock);
598 
599         wreq = netfs_create_write_req(iocb->ki_filp->f_mapping, iocb->ki_filp,
600                                       iocb->ki_pos, NETFS_WRITETHROUGH);
601         if (IS_ERR(wreq)) {
602                 mutex_unlock(&ictx->wb_lock);
603                 return wreq;
604         }
605 
606         wreq->io_streams[0].avail = true;
607         trace_netfs_write(wreq, netfs_write_trace_writethrough);
608         return wreq;
609 }
610 
611 /*
612  * Advance the state of the write operation used when writing through the
613  * pagecache.  Data has been copied into the pagecache that we need to append
614  * to the request.  If we've added more than wsize then we need to create a new
615  * subrequest.
616  */
617 int netfs_advance_writethrough(struct netfs_io_request *wreq, struct writeback_control *wbc,
618                                struct folio *folio, size_t copied, bool to_page_end,
619                                struct folio **writethrough_cache)
620 {
621         _enter("R=%x ic=%zu ws=%u cp=%zu tp=%u",
622                wreq->debug_id, wreq->iter.count, wreq->wsize, copied, to_page_end);
623 
624         if (!*writethrough_cache) {
625                 if (folio_test_dirty(folio))
626                         /* Sigh.  mmap. */
627                         folio_clear_dirty_for_io(folio);
628 
629                 /* We can make multiple writes to the folio... */
630                 folio_start_writeback(folio);
631                 if (wreq->len == 0)
632                         trace_netfs_folio(folio, netfs_folio_trace_wthru);
633                 else
634                         trace_netfs_folio(folio, netfs_folio_trace_wthru_plus);
635                 *writethrough_cache = folio;
636         }
637 
638         wreq->len += copied;
639         if (!to_page_end)
640                 return 0;
641 
642         *writethrough_cache = NULL;
643         return netfs_write_folio(wreq, wbc, folio);
644 }
645 
646 /*
647  * End a write operation used when writing through the pagecache.
648  */
649 int netfs_end_writethrough(struct netfs_io_request *wreq, struct writeback_control *wbc,
650                            struct folio *writethrough_cache)
651 {
652         struct netfs_inode *ictx = netfs_inode(wreq->inode);
653         int ret;
654 
655         _enter("R=%x", wreq->debug_id);
656 
657         if (writethrough_cache)
658                 netfs_write_folio(wreq, wbc, writethrough_cache);
659 
660         netfs_end_issue_write(wreq);
661 
662         mutex_unlock(&ictx->wb_lock);
663 
664         if (wreq->iocb) {
665                 ret = -EIOCBQUEUED;
666         } else {
667                 wait_on_bit(&wreq->flags, NETFS_RREQ_IN_PROGRESS, TASK_UNINTERRUPTIBLE);
668                 ret = wreq->error;
669         }
670         netfs_put_request(wreq, false, netfs_rreq_trace_put_return);
671         return ret;
672 }
673 
674 /*
675  * Write data to the server without going through the pagecache and without
676  * writing it to the local cache.
677  */
678 int netfs_unbuffered_write(struct netfs_io_request *wreq, bool may_wait, size_t len)
679 {
680         struct netfs_io_stream *upload = &wreq->io_streams[0];
681         ssize_t part;
682         loff_t start = wreq->start;
683         int error = 0;
684 
685         _enter("%zx", len);
686 
687         if (wreq->origin == NETFS_DIO_WRITE)
688                 inode_dio_begin(wreq->inode);
689 
690         while (len) {
691                 // TODO: Prepare content encryption
692 
693                 _debug("unbuffered %zx", len);
694                 part = netfs_advance_write(wreq, upload, start, len, false);
695                 start += part;
696                 len -= part;
697                 if (test_bit(NETFS_RREQ_PAUSE, &wreq->flags)) {
698                         trace_netfs_rreq(wreq, netfs_rreq_trace_wait_pause);
699                         wait_on_bit(&wreq->flags, NETFS_RREQ_PAUSE, TASK_UNINTERRUPTIBLE);
700                 }
701                 if (test_bit(NETFS_RREQ_FAILED, &wreq->flags))
702                         break;
703         }
704 
705         netfs_end_issue_write(wreq);
706         _leave(" = %d", error);
707         return error;
708 }
709 

~ [ source navigation ] ~ [ diff markup ] ~ [ identifier search ] ~

kernel.org | git.kernel.org | LWN.net | Project Home | SVN repository | Mail admin

Linux® is a registered trademark of Linus Torvalds in the United States and other countries.
TOMOYO® is a registered trademark of NTT DATA CORPORATION.

sflogo.php