summaryrefslogtreecommitdiff
path: root/include
diff options
context:
space:
mode:
authorLinus Torvalds <torvalds@linux-foundation.org>2024-01-19 20:10:23 +0300
committerLinus Torvalds <torvalds@linux-foundation.org>2024-01-19 20:10:23 +0300
commit16df6e07d6a88dc3049a5674654ed44dfbc74d81 (patch)
treea20acefcc36e8f7db865b1d5e28e92f43082fb1d /include
parent9d1694dc91ce7b80bc96d6d8eaf1a1eca668d847 (diff)
parent1d5911d43cab5fb99229b02bce173b0c6d9da7d2 (diff)
downloadlinux-16df6e07d6a88dc3049a5674654ed44dfbc74d81.tar.xz
Merge tag 'vfs-6.8.netfs' of gitolite.kernel.org:pub/scm/linux/kernel/git/vfs/vfs
Pull netfs updates from Christian Brauner: "This extends the netfs helper library that network filesystems can use to replace their own implementations. Both afs and 9p are ported. cifs is ready as well but the patches are way bigger and will be routed separately once this is merged. That will remove lots of code as well. The overal goal is to get high-level I/O and knowledge of the page cache and ouf of the filesystem drivers. This includes knowledge about the existence of pages and folios The pull request converts afs and 9p. This removes about 800 lines of code from afs and 300 from 9p. For 9p it is now possible to do writes in larger than a page chunks. Additionally, multipage folio support can be turned on for 9p. Separate patches exist for cifs removing another 2000+ lines. I've included detailed information in the individual pulls I took. Summary: - Add NFS-style (and Ceph-style) locking around DIO vs buffered I/O calls to prevent these from happening at the same time. - Support for direct and unbuffered I/O. - Support for write-through caching in the page cache. - O_*SYNC and RWF_*SYNC writes use write-through rather than writing to the page cache and then flushing afterwards. - Support for write-streaming. - Support for write grouping. - Skip reads for which the server could only return zeros or EOF. - The fscache module is now part of the netfs library and the corresponding maintainer entry is updated. - Some helpers from the fscache subsystem are renamed to mark them as belonging to the netfs library. - Follow-up fixes for the netfs library. - Follow-up fixes for the 9p conversion" * tag 'vfs-6.8.netfs' of gitolite.kernel.org:pub/scm/linux/kernel/git/vfs/vfs: (50 commits) netfs: Fix wrong #ifdef hiding wait cachefiles: Fix signed/unsigned mixup netfs: Fix the loop that unmarks folios after writing to the cache netfs: Fix interaction between write-streaming and cachefiles culling netfs: Count DIO writes netfs: Mark netfs_unbuffered_write_iter_locked() static netfs: Fix proc/fs/fscache symlink to point to "netfs" not "../netfs" netfs: Rearrange netfs_io_subrequest to put request pointer first 9p: Use length of data written to the server in preference to error 9p: Do a couple of cleanups 9p: Fix initialisation of netfs_inode for 9p cachefiles: Fix __cachefiles_prepare_write() 9p: Use netfslib read/write_iter afs: Use the netfs write helpers netfs: Export the netfs_sreq tracepoint netfs: Optimise away reads above the point at which there can be no data netfs: Implement a write-through caching option netfs: Provide a launder_folio implementation netfs: Provide a writepages implementation netfs, cachefiles: Pass upper bound length to allow expansion ...
Diffstat (limited to 'include')
-rw-r--r--include/linux/fs.h2
-rw-r--r--include/linux/fscache-cache.h3
-rw-r--r--include/linux/fscache.h45
-rw-r--r--include/linux/netfs.h181
-rw-r--r--include/linux/writeback.h2
-rw-r--r--include/trace/events/afs.h31
-rw-r--r--include/trace/events/netfs.h155
7 files changed, 320 insertions, 99 deletions
diff --git a/include/linux/fs.h b/include/linux/fs.h
index e6ba0cc6f2ee..ed5966a70495 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -2371,7 +2371,7 @@ static inline void kiocb_clone(struct kiocb *kiocb, struct kiocb *kiocb_src,
#define I_CREATING (1 << 15)
#define I_DONTCACHE (1 << 16)
#define I_SYNC_QUEUED (1 << 17)
-#define I_PINNING_FSCACHE_WB (1 << 18)
+#define I_PINNING_NETFS_WB (1 << 18)
#define I_DIRTY_INODE (I_DIRTY_SYNC | I_DIRTY_DATASYNC)
#define I_DIRTY (I_DIRTY_INODE | I_DIRTY_PAGES)
diff --git a/include/linux/fscache-cache.h b/include/linux/fscache-cache.h
index a174cedf4d90..bdf7f3eddf0a 100644
--- a/include/linux/fscache-cache.h
+++ b/include/linux/fscache-cache.h
@@ -189,17 +189,20 @@ extern atomic_t fscache_n_write;
extern atomic_t fscache_n_no_write_space;
extern atomic_t fscache_n_no_create_space;
extern atomic_t fscache_n_culled;
+extern atomic_t fscache_n_dio_misfit;
#define fscache_count_read() atomic_inc(&fscache_n_read)
#define fscache_count_write() atomic_inc(&fscache_n_write)
#define fscache_count_no_write_space() atomic_inc(&fscache_n_no_write_space)
#define fscache_count_no_create_space() atomic_inc(&fscache_n_no_create_space)
#define fscache_count_culled() atomic_inc(&fscache_n_culled)
+#define fscache_count_dio_misfit() atomic_inc(&fscache_n_dio_misfit)
#else
#define fscache_count_read() do {} while(0)
#define fscache_count_write() do {} while(0)
#define fscache_count_no_write_space() do {} while(0)
#define fscache_count_no_create_space() do {} while(0)
#define fscache_count_culled() do {} while(0)
+#define fscache_count_dio_misfit() do {} while(0)
#endif
#endif /* _LINUX_FSCACHE_CACHE_H */
diff --git a/include/linux/fscache.h b/include/linux/fscache.h
index 8e312c8323a8..6e8562cbcc43 100644
--- a/include/linux/fscache.h
+++ b/include/linux/fscache.h
@@ -437,9 +437,6 @@ const struct netfs_cache_ops *fscache_operation_valid(const struct netfs_cache_r
* indicates the cache resources to which the operation state should be
* attached; @cookie indicates the cache object that will be accessed.
*
- * This is intended to be called from the ->begin_cache_operation() netfs lib
- * operation as implemented by the network filesystem.
- *
* @cres->inval_counter is set from @cookie->inval_counter for comparison at
* the end of the operation. This allows invalidation during the operation to
* be detected by the caller.
@@ -629,48 +626,6 @@ static inline void fscache_write_to_cache(struct fscache_cookie *cookie,
}
-#if __fscache_available
-bool fscache_dirty_folio(struct address_space *mapping, struct folio *folio,
- struct fscache_cookie *cookie);
-#else
-#define fscache_dirty_folio(MAPPING, FOLIO, COOKIE) \
- filemap_dirty_folio(MAPPING, FOLIO)
-#endif
-
-/**
- * fscache_unpin_writeback - Unpin writeback resources
- * @wbc: The writeback control
- * @cookie: The cookie referring to the cache object
- *
- * Unpin the writeback resources pinned by fscache_dirty_folio(). This is
- * intended to be called by the netfs's ->write_inode() method.
- */
-static inline void fscache_unpin_writeback(struct writeback_control *wbc,
- struct fscache_cookie *cookie)
-{
- if (wbc->unpinned_fscache_wb)
- fscache_unuse_cookie(cookie, NULL, NULL);
-}
-
-/**
- * fscache_clear_inode_writeback - Clear writeback resources pinned by an inode
- * @cookie: The cookie referring to the cache object
- * @inode: The inode to clean up
- * @aux: Auxiliary data to apply to the inode
- *
- * Clear any writeback resources held by an inode when the inode is evicted.
- * This must be called before clear_inode() is called.
- */
-static inline void fscache_clear_inode_writeback(struct fscache_cookie *cookie,
- struct inode *inode,
- const void *aux)
-{
- if (inode->i_state & I_PINNING_FSCACHE_WB) {
- loff_t i_size = i_size_read(inode);
- fscache_unuse_cookie(cookie, aux, &i_size);
- }
-}
-
/**
* fscache_note_page_release - Note that a netfs page got released
* @cookie: The cookie corresponding to the file
diff --git a/include/linux/netfs.h b/include/linux/netfs.h
index b11a84f6c32b..100cbb261269 100644
--- a/include/linux/netfs.h
+++ b/include/linux/netfs.h
@@ -109,11 +109,18 @@ static inline int wait_on_page_fscache_killable(struct page *page)
return folio_wait_private_2_killable(page_folio(page));
}
+/* Marks used on xarray-based buffers */
+#define NETFS_BUF_PUT_MARK XA_MARK_0 /* - Page needs putting */
+#define NETFS_BUF_PAGECACHE_MARK XA_MARK_1 /* - Page needs wb/dirty flag wrangling */
+
enum netfs_io_source {
NETFS_FILL_WITH_ZEROES,
NETFS_DOWNLOAD_FROM_SERVER,
NETFS_READ_FROM_CACHE,
NETFS_INVALID_READ,
+ NETFS_UPLOAD_TO_SERVER,
+ NETFS_WRITE_TO_CACHE,
+ NETFS_INVALID_WRITE,
} __mode(byte);
typedef void (*netfs_io_terminated_t)(void *priv, ssize_t transferred_or_error,
@@ -129,9 +136,57 @@ struct netfs_inode {
struct fscache_cookie *cache;
#endif
loff_t remote_i_size; /* Size of the remote file */
+ loff_t zero_point; /* Size after which we assume there's no data
+ * on the server */
+ unsigned long flags;
+#define NETFS_ICTX_ODIRECT 0 /* The file has DIO in progress */
+#define NETFS_ICTX_UNBUFFERED 1 /* I/O should not use the pagecache */
+#define NETFS_ICTX_WRITETHROUGH 2 /* Write-through caching */
+#define NETFS_ICTX_NO_WRITE_STREAMING 3 /* Don't engage in write-streaming */
+};
+
+/*
+ * A netfs group - for instance a ceph snap. This is marked on dirty pages and
+ * pages marked with a group must be flushed before they can be written under
+ * the domain of another group.
+ */
+struct netfs_group {
+ refcount_t ref;
+ void (*free)(struct netfs_group *netfs_group);
};
/*
+ * Information about a dirty page (attached only if necessary).
+ * folio->private
+ */
+struct netfs_folio {
+ struct netfs_group *netfs_group; /* Filesystem's grouping marker (or NULL). */
+ unsigned int dirty_offset; /* Write-streaming dirty data offset */
+ unsigned int dirty_len; /* Write-streaming dirty data length */
+};
+#define NETFS_FOLIO_INFO 0x1UL /* OR'd with folio->private. */
+
+static inline struct netfs_folio *netfs_folio_info(struct folio *folio)
+{
+ void *priv = folio_get_private(folio);
+
+ if ((unsigned long)priv & NETFS_FOLIO_INFO)
+ return (struct netfs_folio *)((unsigned long)priv & ~NETFS_FOLIO_INFO);
+ return NULL;
+}
+
+static inline struct netfs_group *netfs_folio_group(struct folio *folio)
+{
+ struct netfs_folio *finfo;
+ void *priv = folio_get_private(folio);
+
+ finfo = netfs_folio_info(folio);
+ if (finfo)
+ return finfo->netfs_group;
+ return priv;
+}
+
+/*
* Resources required to do operations on a cache.
*/
struct netfs_cache_resources {
@@ -143,17 +198,24 @@ struct netfs_cache_resources {
};
/*
- * Descriptor for a single component subrequest.
+ * Descriptor for a single component subrequest. Each operation represents an
+ * individual read/write from/to a server, a cache, a journal, etc..
+ *
+ * The buffer iterator is persistent for the life of the subrequest struct and
+ * the pages it points to can be relied on to exist for the duration.
*/
struct netfs_io_subrequest {
struct netfs_io_request *rreq; /* Supervising I/O request */
+ struct work_struct work;
struct list_head rreq_link; /* Link in rreq->subrequests */
+ struct iov_iter io_iter; /* Iterator for this subrequest */
loff_t start; /* Where to start the I/O */
size_t len; /* Size of the I/O */
size_t transferred; /* Amount of data transferred */
refcount_t ref;
short error; /* 0 or error that occurred */
unsigned short debug_index; /* Index in list (for debugging output) */
+ unsigned int max_nr_segs; /* 0 or max number of segments in an iterator */
enum netfs_io_source source; /* Where to read from/write to */
unsigned long flags;
#define NETFS_SREQ_COPY_TO_CACHE 0 /* Set if should copy the data to the cache */
@@ -168,6 +230,13 @@ enum netfs_io_origin {
NETFS_READAHEAD, /* This read was triggered by readahead */
NETFS_READPAGE, /* This read is a synchronous read */
NETFS_READ_FOR_WRITE, /* This read is to prepare a write */
+ NETFS_WRITEBACK, /* This write was triggered by writepages */
+ NETFS_WRITETHROUGH, /* This write was made by netfs_perform_write() */
+ NETFS_LAUNDER_WRITE, /* This is triggered by ->launder_folio() */
+ NETFS_UNBUFFERED_WRITE, /* This is an unbuffered write */
+ NETFS_DIO_READ, /* This is a direct I/O read */
+ NETFS_DIO_WRITE, /* This is a direct I/O write */
+ nr__netfs_io_origin
} __mode(byte);
/*
@@ -175,19 +244,34 @@ enum netfs_io_origin {
* operations to a variety of data stores and then stitch the result together.
*/
struct netfs_io_request {
- struct work_struct work;
+ union {
+ struct work_struct work;
+ struct rcu_head rcu;
+ };
struct inode *inode; /* The file being accessed */
struct address_space *mapping; /* The mapping being accessed */
+ struct kiocb *iocb; /* AIO completion vector */
struct netfs_cache_resources cache_resources;
+ struct list_head proc_link; /* Link in netfs_iorequests */
struct list_head subrequests; /* Contributory I/O operations */
+ struct iov_iter iter; /* Unencrypted-side iterator */
+ struct iov_iter io_iter; /* I/O (Encrypted-side) iterator */
void *netfs_priv; /* Private data for the netfs */
+ struct bio_vec *direct_bv; /* DIO buffer list (when handling iovec-iter) */
+ unsigned int direct_bv_count; /* Number of elements in direct_bv[] */
unsigned int debug_id;
+ unsigned int rsize; /* Maximum read size (0 for none) */
+ unsigned int wsize; /* Maximum write size (0 for none) */
+ unsigned int subreq_counter; /* Next subreq->debug_index */
atomic_t nr_outstanding; /* Number of ops in progress */
atomic_t nr_copy_ops; /* Number of copy-to-cache ops in progress */
size_t submitted; /* Amount submitted for I/O so far */
size_t len; /* Length of the request */
+ size_t upper_len; /* Length can be extended to here */
+ size_t transferred; /* Amount to be indicated as transferred */
short error; /* 0 or error that occurred */
enum netfs_io_origin origin; /* Origin of the request */
+ bool direct_bv_unpin; /* T if direct_bv[] must be unpinned */
loff_t i_size; /* Size of the file */
loff_t start; /* Start position */
pgoff_t no_unlock_folio; /* Don't unlock this folio after read */
@@ -199,17 +283,25 @@ struct netfs_io_request {
#define NETFS_RREQ_DONT_UNLOCK_FOLIOS 3 /* Don't unlock the folios on completion */
#define NETFS_RREQ_FAILED 4 /* The request failed */
#define NETFS_RREQ_IN_PROGRESS 5 /* Unlocked when the request completes */
+#define NETFS_RREQ_WRITE_TO_CACHE 7 /* Need to write to the cache */
+#define NETFS_RREQ_UPLOAD_TO_SERVER 8 /* Need to write to the server */
+#define NETFS_RREQ_NONBLOCK 9 /* Don't block if possible (O_NONBLOCK) */
+#define NETFS_RREQ_BLOCKED 10 /* We blocked */
const struct netfs_request_ops *netfs_ops;
+ void (*cleanup)(struct netfs_io_request *req);
};
/*
* Operations the network filesystem can/must provide to the helpers.
*/
struct netfs_request_ops {
+ unsigned int io_request_size; /* Alloc size for netfs_io_request struct */
+ unsigned int io_subrequest_size; /* Alloc size for netfs_io_subrequest struct */
int (*init_request)(struct netfs_io_request *rreq, struct file *file);
void (*free_request)(struct netfs_io_request *rreq);
- int (*begin_cache_operation)(struct netfs_io_request *rreq);
+ void (*free_subrequest)(struct netfs_io_subrequest *rreq);
+ /* Read request handling */
void (*expand_readahead)(struct netfs_io_request *rreq);
bool (*clamp_length)(struct netfs_io_subrequest *subreq);
void (*issue_read)(struct netfs_io_subrequest *subreq);
@@ -217,6 +309,14 @@ struct netfs_request_ops {
int (*check_write_begin)(struct file *file, loff_t pos, unsigned len,
struct folio **foliop, void **_fsdata);
void (*done)(struct netfs_io_request *rreq);
+
+ /* Modification handling */
+ void (*update_i_size)(struct inode *inode, loff_t i_size);
+
+ /* Write request handling */
+ void (*create_write_requests)(struct netfs_io_request *wreq,
+ loff_t start, size_t len);
+ void (*invalidate_cache)(struct netfs_io_request *wreq);
};
/*
@@ -229,8 +329,7 @@ enum netfs_read_from_hole {
};
/*
- * Table of operations for access to a cache. This is obtained by
- * rreq->ops->begin_cache_operation().
+ * Table of operations for access to a cache.
*/
struct netfs_cache_ops {
/* End an operation */
@@ -265,8 +364,8 @@ struct netfs_cache_ops {
* actually do.
*/
int (*prepare_write)(struct netfs_cache_resources *cres,
- loff_t *_start, size_t *_len, loff_t i_size,
- bool no_space_allocated_yet);
+ loff_t *_start, size_t *_len, size_t upper_len,
+ loff_t i_size, bool no_space_allocated_yet);
/* Prepare an on-demand read operation, shortening it to a cached/uncached
* boundary as appropriate.
@@ -284,22 +383,62 @@ struct netfs_cache_ops {
loff_t *_data_start, size_t *_data_len);
};
+/* High-level read API. */
+ssize_t netfs_unbuffered_read_iter(struct kiocb *iocb, struct iov_iter *iter);
+ssize_t netfs_buffered_read_iter(struct kiocb *iocb, struct iov_iter *iter);
+ssize_t netfs_file_read_iter(struct kiocb *iocb, struct iov_iter *iter);
+
+/* High-level write API */
+ssize_t netfs_perform_write(struct kiocb *iocb, struct iov_iter *iter,
+ struct netfs_group *netfs_group);
+ssize_t netfs_buffered_write_iter_locked(struct kiocb *iocb, struct iov_iter *from,
+ struct netfs_group *netfs_group);
+ssize_t netfs_unbuffered_write_iter(struct kiocb *iocb, struct iov_iter *from);
+ssize_t netfs_file_write_iter(struct kiocb *iocb, struct iov_iter *from);
+
+/* Address operations API */
struct readahead_control;
void netfs_readahead(struct readahead_control *);
int netfs_read_folio(struct file *, struct folio *);
int netfs_write_begin(struct netfs_inode *, struct file *,
- struct address_space *, loff_t pos, unsigned int len,
- struct folio **, void **fsdata);
-
+ struct address_space *, loff_t pos, unsigned int len,
+ struct folio **, void **fsdata);
+int netfs_writepages(struct address_space *mapping,
+ struct writeback_control *wbc);
+bool netfs_dirty_folio(struct address_space *mapping, struct folio *folio);
+int netfs_unpin_writeback(struct inode *inode, struct writeback_control *wbc);
+void netfs_clear_inode_writeback(struct inode *inode, const void *aux);
+void netfs_invalidate_folio(struct folio *folio, size_t offset, size_t length);
+bool netfs_release_folio(struct folio *folio, gfp_t gfp);
+int netfs_launder_folio(struct folio *folio);
+
+/* VMA operations API. */
+vm_fault_t netfs_page_mkwrite(struct vm_fault *vmf, struct netfs_group *netfs_group);
+
+/* (Sub)request management API. */
void netfs_subreq_terminated(struct netfs_io_subrequest *, ssize_t, bool);
void netfs_get_subrequest(struct netfs_io_subrequest *subreq,
enum netfs_sreq_ref_trace what);
void netfs_put_subrequest(struct netfs_io_subrequest *subreq,
bool was_async, enum netfs_sreq_ref_trace what);
-void netfs_stats_show(struct seq_file *);
ssize_t netfs_extract_user_iter(struct iov_iter *orig, size_t orig_len,
struct iov_iter *new,
iov_iter_extraction_t extraction_flags);
+size_t netfs_limit_iter(const struct iov_iter *iter, size_t start_offset,
+ size_t max_size, size_t max_segs);
+struct netfs_io_subrequest *netfs_create_write_request(
+ struct netfs_io_request *wreq, enum netfs_io_source dest,
+ loff_t start, size_t len, work_func_t worker);
+void netfs_write_subrequest_terminated(void *_op, ssize_t transferred_or_error,
+ bool was_async);
+void netfs_queue_write_request(struct netfs_io_subrequest *subreq);
+
+int netfs_start_io_read(struct inode *inode);
+void netfs_end_io_read(struct inode *inode);
+int netfs_start_io_write(struct inode *inode);
+void netfs_end_io_write(struct inode *inode);
+int netfs_start_io_direct(struct inode *inode);
+void netfs_end_io_direct(struct inode *inode);
/**
* netfs_inode - Get the netfs inode context from the inode
@@ -317,30 +456,44 @@ static inline struct netfs_inode *netfs_inode(struct inode *inode)
* netfs_inode_init - Initialise a netfslib inode context
* @ctx: The netfs inode to initialise
* @ops: The netfs's operations list
+ * @use_zero_point: True to use the zero_point read optimisation
*
* Initialise the netfs library context struct. This is expected to follow on
* directly from the VFS inode struct.
*/
static inline void netfs_inode_init(struct netfs_inode *ctx,
- const struct netfs_request_ops *ops)
+ const struct netfs_request_ops *ops,
+ bool use_zero_point)
{
ctx->ops = ops;
ctx->remote_i_size = i_size_read(&ctx->inode);
+ ctx->zero_point = LLONG_MAX;
+ ctx->flags = 0;
#if IS_ENABLED(CONFIG_FSCACHE)
ctx->cache = NULL;
#endif
+ /* ->releasepage() drives zero_point */
+ if (use_zero_point) {
+ ctx->zero_point = ctx->remote_i_size;
+ mapping_set_release_always(ctx->inode.i_mapping);
+ }
}
/**
* netfs_resize_file - Note that a file got resized
* @ctx: The netfs inode being resized
* @new_i_size: The new file size
+ * @changed_on_server: The change was applied to the server
*
* Inform the netfs lib that a file got resized so that it can adjust its state.
*/
-static inline void netfs_resize_file(struct netfs_inode *ctx, loff_t new_i_size)
+static inline void netfs_resize_file(struct netfs_inode *ctx, loff_t new_i_size,
+ bool changed_on_server)
{
- ctx->remote_i_size = new_i_size;
+ if (changed_on_server)
+ ctx->remote_i_size = new_i_size;
+ if (new_i_size < ctx->zero_point)
+ ctx->zero_point = new_i_size;
}
/**
diff --git a/include/linux/writeback.h b/include/linux/writeback.h
index 6d0a14f7019d..453736fd1d23 100644
--- a/include/linux/writeback.h
+++ b/include/linux/writeback.h
@@ -60,7 +60,7 @@ struct writeback_control {
unsigned for_reclaim:1; /* Invoked from the page allocator */
unsigned range_cyclic:1; /* range_start is cyclic */
unsigned for_sync:1; /* sync(2) WB_SYNC_ALL writeback */
- unsigned unpinned_fscache_wb:1; /* Cleared I_PINNING_FSCACHE_WB */
+ unsigned unpinned_netfs_wb:1; /* Cleared I_PINNING_NETFS_WB */
/*
* When writeback IOs are bounced through async layers, only the
diff --git a/include/trace/events/afs.h b/include/trace/events/afs.h
index 5194b7e6dc8d..8d73171cb9f0 100644
--- a/include/trace/events/afs.h
+++ b/include/trace/events/afs.h
@@ -902,37 +902,6 @@ TRACE_EVENT(afs_dir_check_failed,
__entry->vnode, __entry->off, __entry->i_size)
);
-TRACE_EVENT(afs_folio_dirty,
- TP_PROTO(struct afs_vnode *vnode, const char *where, struct folio *folio),
-
- TP_ARGS(vnode, where, folio),
-
- TP_STRUCT__entry(
- __field(struct afs_vnode *, vnode)
- __field(const char *, where)
- __field(pgoff_t, index)
- __field(unsigned long, from)
- __field(unsigned long, to)
- ),
-
- TP_fast_assign(
- unsigned long priv = (unsigned long)folio_get_private(folio);
- __entry->vnode = vnode;
- __entry->where = where;
- __entry->index = folio_index(folio);
- __entry->from = afs_folio_dirty_from(folio, priv);
- __entry->to = afs_folio_dirty_to(folio, priv);
- __entry->to |= (afs_is_folio_dirty_mmapped(priv) ?
- (1UL << (BITS_PER_LONG - 1)) : 0);
- ),
-
- TP_printk("vn=%p %lx %s %lx-%lx%s",
- __entry->vnode, __entry->index, __entry->where,
- __entry->from,
- __entry->to & ~(1UL << (BITS_PER_LONG - 1)),
- __entry->to & (1UL << (BITS_PER_LONG - 1)) ? " M" : "")
- );
-
TRACE_EVENT(afs_call_state,
TP_PROTO(struct afs_call *call,
enum afs_call_state from,
diff --git a/include/trace/events/netfs.h b/include/trace/events/netfs.h
index beec534cbaab..447a8c21cf57 100644
--- a/include/trace/events/netfs.h
+++ b/include/trace/events/netfs.h
@@ -16,34 +16,57 @@
* Define enums for tracing information.
*/
#define netfs_read_traces \
+ EM(netfs_read_trace_dio_read, "DIO-READ ") \
EM(netfs_read_trace_expanded, "EXPANDED ") \
EM(netfs_read_trace_readahead, "READAHEAD") \
EM(netfs_read_trace_readpage, "READPAGE ") \
+ EM(netfs_read_trace_prefetch_for_write, "PREFETCHW") \
E_(netfs_read_trace_write_begin, "WRITEBEGN")
+#define netfs_write_traces \
+ EM(netfs_write_trace_dio_write, "DIO-WRITE") \
+ EM(netfs_write_trace_launder, "LAUNDER ") \
+ EM(netfs_write_trace_unbuffered_write, "UNB-WRITE") \
+ EM(netfs_write_trace_writeback, "WRITEBACK") \
+ E_(netfs_write_trace_writethrough, "WRITETHRU")
+
#define netfs_rreq_origins \
EM(NETFS_READAHEAD, "RA") \
EM(NETFS_READPAGE, "RP") \
- E_(NETFS_READ_FOR_WRITE, "RW")
+ EM(NETFS_READ_FOR_WRITE, "RW") \
+ EM(NETFS_WRITEBACK, "WB") \
+ EM(NETFS_WRITETHROUGH, "WT") \
+ EM(NETFS_LAUNDER_WRITE, "LW") \
+ EM(NETFS_UNBUFFERED_WRITE, "UW") \
+ EM(NETFS_DIO_READ, "DR") \
+ E_(NETFS_DIO_WRITE, "DW")
#define netfs_rreq_traces \
EM(netfs_rreq_trace_assess, "ASSESS ") \
EM(netfs_rreq_trace_copy, "COPY ") \
EM(netfs_rreq_trace_done, "DONE ") \
EM(netfs_rreq_trace_free, "FREE ") \
+ EM(netfs_rreq_trace_redirty, "REDIRTY") \
EM(netfs_rreq_trace_resubmit, "RESUBMT") \
EM(netfs_rreq_trace_unlock, "UNLOCK ") \
- E_(netfs_rreq_trace_unmark, "UNMARK ")
+ EM(netfs_rreq_trace_unmark, "UNMARK ") \
+ EM(netfs_rreq_trace_wait_ip, "WAIT-IP") \
+ EM(netfs_rreq_trace_wake_ip, "WAKE-IP") \
+ E_(netfs_rreq_trace_write_done, "WR-DONE")
#define netfs_sreq_sources \
EM(NETFS_FILL_WITH_ZEROES, "ZERO") \
EM(NETFS_DOWNLOAD_FROM_SERVER, "DOWN") \
EM(NETFS_READ_FROM_CACHE, "READ") \
- E_(NETFS_INVALID_READ, "INVL") \
+ EM(NETFS_INVALID_READ, "INVL") \
+ EM(NETFS_UPLOAD_TO_SERVER, "UPLD") \
+ EM(NETFS_WRITE_TO_CACHE, "WRIT") \
+ E_(NETFS_INVALID_WRITE, "INVL")
#define netfs_sreq_traces \
EM(netfs_sreq_trace_download_instead, "RDOWN") \
EM(netfs_sreq_trace_free, "FREE ") \
+ EM(netfs_sreq_trace_limited, "LIMIT") \
EM(netfs_sreq_trace_prepare, "PREP ") \
EM(netfs_sreq_trace_resubmit_short, "SHORT") \
EM(netfs_sreq_trace_submit, "SUBMT") \
@@ -55,19 +78,24 @@
#define netfs_failures \
EM(netfs_fail_check_write_begin, "check-write-begin") \
EM(netfs_fail_copy_to_cache, "copy-to-cache") \
+ EM(netfs_fail_dio_read_short, "dio-read-short") \
+ EM(netfs_fail_dio_read_zero, "dio-read-zero") \
EM(netfs_fail_read, "read") \
EM(netfs_fail_short_read, "short-read") \
- E_(netfs_fail_prepare_write, "prep-write")
+ EM(netfs_fail_prepare_write, "prep-write") \
+ E_(netfs_fail_write, "write")
#define netfs_rreq_ref_traces \
- EM(netfs_rreq_trace_get_hold, "GET HOLD ") \
+ EM(netfs_rreq_trace_get_for_outstanding,"GET OUTSTND") \
EM(netfs_rreq_trace_get_subreq, "GET SUBREQ ") \
EM(netfs_rreq_trace_put_complete, "PUT COMPLT ") \
EM(netfs_rreq_trace_put_discard, "PUT DISCARD") \
EM(netfs_rreq_trace_put_failed, "PUT FAILED ") \
- EM(netfs_rreq_trace_put_hold, "PUT HOLD ") \
+ EM(netfs_rreq_trace_put_no_submit, "PUT NO-SUBM") \
+ EM(netfs_rreq_trace_put_return, "PUT RETURN ") \
EM(netfs_rreq_trace_put_subreq, "PUT SUBREQ ") \
- EM(netfs_rreq_trace_put_zero_len, "PUT ZEROLEN") \
+ EM(netfs_rreq_trace_put_work, "PUT WORK ") \
+ EM(netfs_rreq_trace_see_work, "SEE WORK ") \
E_(netfs_rreq_trace_new, "NEW ")
#define netfs_sreq_ref_traces \
@@ -76,11 +104,44 @@
EM(netfs_sreq_trace_get_short_read, "GET SHORTRD") \
EM(netfs_sreq_trace_new, "NEW ") \
EM(netfs_sreq_trace_put_clear, "PUT CLEAR ") \
+ EM(netfs_sreq_trace_put_discard, "PUT DISCARD") \
EM(netfs_sreq_trace_put_failed, "PUT FAILED ") \
EM(netfs_sreq_trace_put_merged, "PUT MERGED ") \
EM(netfs_sreq_trace_put_no_copy, "PUT NO COPY") \
+ EM(netfs_sreq_trace_put_wip, "PUT WIP ") \
+ EM(netfs_sreq_trace_put_work, "PUT WORK ") \
E_(netfs_sreq_trace_put_terminated, "PUT TERM ")
+#define netfs_folio_traces \
+ /* The first few correspond to enum netfs_how_to_modify */ \
+ EM(netfs_folio_is_uptodate, "mod-uptodate") \
+ EM(netfs_just_prefetch, "mod-prefetch") \
+ EM(netfs_whole_folio_modify, "mod-whole-f") \
+ EM(netfs_modify_and_clear, "mod-n-clear") \
+ EM(netfs_streaming_write, "mod-streamw") \
+ EM(netfs_streaming_write_cont, "mod-streamw+") \
+ EM(netfs_flush_content, "flush") \
+ EM(netfs_streaming_filled_page, "mod-streamw-f") \
+ EM(netfs_streaming_cont_filled_page, "mod-streamw-f+") \
+ /* The rest are for writeback */ \
+ EM(netfs_folio_trace_clear, "clear") \
+ EM(netfs_folio_trace_clear_s, "clear-s") \
+ EM(netfs_folio_trace_clear_g, "clear-g") \
+ EM(netfs_folio_trace_copy_to_cache, "copy") \
+ EM(netfs_folio_trace_end_copy, "end-copy") \
+ EM(netfs_folio_trace_filled_gaps, "filled-gaps") \
+ EM(netfs_folio_trace_kill, "kill") \
+ EM(netfs_folio_trace_launder, "launder") \
+ EM(netfs_folio_trace_mkwrite, "mkwrite") \
+ EM(netfs_folio_trace_mkwrite_plus, "mkwrite+") \
+ EM(netfs_folio_trace_read_gaps, "read-gaps") \
+ EM(netfs_folio_trace_redirty, "redirty") \
+ EM(netfs_folio_trace_redirtied, "redirtied") \
+ EM(netfs_folio_trace_store, "store") \
+ EM(netfs_folio_trace_store_plus, "store+") \
+ EM(netfs_folio_trace_wthru, "wthru") \
+ E_(netfs_folio_trace_wthru_plus, "wthru+")
+
#ifndef __NETFS_DECLARE_TRACE_ENUMS_ONCE_ONLY
#define __NETFS_DECLARE_TRACE_ENUMS_ONCE_ONLY
@@ -90,11 +151,13 @@
#define E_(a, b) a
enum netfs_read_trace { netfs_read_traces } __mode(byte);
+enum netfs_write_trace { netfs_write_traces } __mode(byte);
enum netfs_rreq_trace { netfs_rreq_traces } __mode(byte);
enum netfs_sreq_trace { netfs_sreq_traces } __mode(byte);
enum netfs_failure { netfs_failures } __mode(byte);
enum netfs_rreq_ref_trace { netfs_rreq_ref_traces } __mode(byte);
enum netfs_sreq_ref_trace { netfs_sreq_ref_traces } __mode(byte);
+enum netfs_folio_trace { netfs_folio_traces } __mode(byte);
#endif
@@ -107,6 +170,7 @@ enum netfs_sreq_ref_trace { netfs_sreq_ref_traces } __mode(byte);
#define E_(a, b) TRACE_DEFINE_ENUM(a);
netfs_read_traces;
+netfs_write_traces;
netfs_rreq_origins;
netfs_rreq_traces;
netfs_sreq_sources;
@@ -114,6 +178,7 @@ netfs_sreq_traces;
netfs_failures;
netfs_rreq_ref_traces;
netfs_sreq_ref_traces;
+netfs_folio_traces;
/*
* Now redefine the EM() and E_() macros to map the enums to the strings that
@@ -314,6 +379,82 @@ TRACE_EVENT(netfs_sreq_ref,
__entry->ref)
);
+TRACE_EVENT(netfs_folio,
+ TP_PROTO(struct folio *folio, enum netfs_folio_trace why),
+
+ TP_ARGS(folio, why),
+
+ TP_STRUCT__entry(
+ __field(ino_t, ino)
+ __field(pgoff_t, index)
+ __field(unsigned int, nr)
+ __field(enum netfs_folio_trace, why)
+ ),
+
+ TP_fast_assign(
+ __entry->ino = folio->mapping->host->i_ino;
+ __entry->why = why;
+ __entry->index = folio_index(folio);
+ __entry->nr = folio_nr_pages(folio);
+ ),
+
+ TP_printk("i=%05lx ix=%05lx-%05lx %s",
+ __entry->ino, __entry->index, __entry->index + __entry->nr - 1,
+ __print_symbolic(__entry->why, netfs_folio_traces))
+ );
+
+TRACE_EVENT(netfs_write_iter,
+ TP_PROTO(const struct kiocb *iocb, const struct iov_iter *from),
+
+ TP_ARGS(iocb, from),
+
+ TP_STRUCT__entry(
+ __field(unsigned long long, start )
+ __field(size_t, len )
+ __field(unsigned int, flags )
+ ),
+
+ TP_fast_assign(
+ __entry->start = iocb->ki_pos;
+ __entry->len = iov_iter_count(from);
+ __entry->flags = iocb->ki_flags;
+ ),
+
+ TP_printk("WRITE-ITER s=%llx l=%zx f=%x",
+ __entry->start, __entry->len, __entry->flags)
+ );
+
+TRACE_EVENT(netfs_write,
+ TP_PROTO(const struct netfs_io_request *wreq,
+ enum netfs_write_trace what),
+
+ TP_ARGS(wreq, what),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, wreq )
+ __field(unsigned int, cookie )
+ __field(enum netfs_write_trace, what )
+ __field(unsigned long long, start )
+ __field(size_t, len )
+ ),
+
+ TP_fast_assign(
+ struct netfs_inode *__ctx = netfs_inode(wreq->inode);
+ struct fscache_cookie *__cookie = netfs_i_cookie(__ctx);
+ __entry->wreq = wreq->debug_id;
+ __entry->cookie = __cookie ? __cookie->debug_id : 0;
+ __entry->what = what;
+ __entry->start = wreq->start;
+ __entry->len = wreq->len;
+ ),
+
+ TP_printk("R=%08x %s c=%08x by=%llx-%llx",
+ __entry->wreq,
+ __print_symbolic(__entry->what, netfs_write_traces),
+ __entry->cookie,
+ __entry->start, __entry->start + __entry->len - 1)
+ );
+
#undef EM
#undef E_
#endif /* _TRACE_NETFS_H */