summaryrefslogtreecommitdiff
path: root/include
diff options
context:
space:
mode:
Diffstat (limited to 'include')
-rw-r--r--include/linux/fscache.h22
-rw-r--r--include/linux/netfs.h197
-rw-r--r--include/linux/pagemap.h2
-rw-r--r--include/net/9p/client.h2
-rw-r--r--include/trace/events/netfs.h250
5 files changed, 355 insertions, 118 deletions
diff --git a/include/linux/fscache.h b/include/linux/fscache.h
index 6e8562cbcc43..9de27643607f 100644
--- a/include/linux/fscache.h
+++ b/include/linux/fscache.h
@@ -172,9 +172,12 @@ extern void __fscache_invalidate(struct fscache_cookie *, const void *, loff_t,
extern int __fscache_begin_read_operation(struct netfs_cache_resources *, struct fscache_cookie *);
extern int __fscache_begin_write_operation(struct netfs_cache_resources *, struct fscache_cookie *);
-extern void __fscache_write_to_cache(struct fscache_cookie *, struct address_space *,
- loff_t, size_t, loff_t, netfs_io_terminated_t, void *,
- bool);
+void __fscache_write_to_cache(struct fscache_cookie *cookie,
+ struct address_space *mapping,
+ loff_t start, size_t len, loff_t i_size,
+ netfs_io_terminated_t term_func,
+ void *term_func_priv,
+ bool using_pgpriv2, bool cond);
extern void __fscache_clear_page_bits(struct address_space *, loff_t, size_t);
/**
@@ -597,7 +600,8 @@ static inline void fscache_clear_page_bits(struct address_space *mapping,
* @i_size: The new size of the inode
* @term_func: The function to call upon completion
* @term_func_priv: The private data for @term_func
- * @caching: If PG_fscache has been set
+ * @using_pgpriv2: If we're using PG_private_2 to mark in-progress write
+ * @caching: If we actually want to do the caching
*
* Helper function for a netfs to write dirty data from an inode into the cache
* object that's backing it.
@@ -608,19 +612,21 @@ static inline void fscache_clear_page_bits(struct address_space *mapping,
* marked with PG_fscache.
*
* If given, @term_func will be called upon completion and supplied with
- * @term_func_priv. Note that the PG_fscache flags will have been cleared by
- * this point, so the netfs must retain its own pin on the mapping.
+ * @term_func_priv. Note that if @using_pgpriv2 is set, the PG_private_2 flags
+ * will have been cleared by this point, so the netfs must retain its own pin
+ * on the mapping.
*/
static inline void fscache_write_to_cache(struct fscache_cookie *cookie,
struct address_space *mapping,
loff_t start, size_t len, loff_t i_size,
netfs_io_terminated_t term_func,
void *term_func_priv,
- bool caching)
+ bool using_pgpriv2, bool caching)
{
if (caching)
__fscache_write_to_cache(cookie, mapping, start, len, i_size,
- term_func, term_func_priv, caching);
+ term_func, term_func_priv,
+ using_pgpriv2, caching);
else if (term_func)
term_func(term_func_priv, -ENOBUFS, false);
diff --git a/include/linux/netfs.h b/include/linux/netfs.h
index 100cbb261269..f45d06284f2f 100644
--- a/include/linux/netfs.h
+++ b/include/linux/netfs.h
@@ -20,95 +20,24 @@
#include <linux/uio.h>
enum netfs_sreq_ref_trace;
-
-/*
- * Overload PG_private_2 to give us PG_fscache - this is used to indicate that
- * a page is currently backed by a local disk cache
- */
-#define folio_test_fscache(folio) folio_test_private_2(folio)
-#define PageFsCache(page) PagePrivate2((page))
-#define SetPageFsCache(page) SetPagePrivate2((page))
-#define ClearPageFsCache(page) ClearPagePrivate2((page))
-#define TestSetPageFsCache(page) TestSetPagePrivate2((page))
-#define TestClearPageFsCache(page) TestClearPagePrivate2((page))
+typedef struct mempool_s mempool_t;
/**
- * folio_start_fscache - Start an fscache write on a folio.
+ * folio_start_private_2 - Start an fscache write on a folio. [DEPRECATED]
* @folio: The folio.
*
* Call this function before writing a folio to a local cache. Starting a
* second write before the first one finishes is not allowed.
+ *
+ * Note that this should no longer be used.
*/
-static inline void folio_start_fscache(struct folio *folio)
+static inline void folio_start_private_2(struct folio *folio)
{
VM_BUG_ON_FOLIO(folio_test_private_2(folio), folio);
folio_get(folio);
folio_set_private_2(folio);
}
-/**
- * folio_end_fscache - End an fscache write on a folio.
- * @folio: The folio.
- *
- * Call this function after the folio has been written to the local cache.
- * This will wake any sleepers waiting on this folio.
- */
-static inline void folio_end_fscache(struct folio *folio)
-{
- folio_end_private_2(folio);
-}
-
-/**
- * folio_wait_fscache - Wait for an fscache write on this folio to end.
- * @folio: The folio.
- *
- * If this folio is currently being written to a local cache, wait for
- * the write to finish. Another write may start after this one finishes,
- * unless the caller holds the folio lock.
- */
-static inline void folio_wait_fscache(struct folio *folio)
-{
- folio_wait_private_2(folio);
-}
-
-/**
- * folio_wait_fscache_killable - Wait for an fscache write on this folio to end.
- * @folio: The folio.
- *
- * If this folio is currently being written to a local cache, wait
- * for the write to finish or for a fatal signal to be received.
- * Another write may start after this one finishes, unless the caller
- * holds the folio lock.
- *
- * Return:
- * - 0 if successful.
- * - -EINTR if a fatal signal was encountered.
- */
-static inline int folio_wait_fscache_killable(struct folio *folio)
-{
- return folio_wait_private_2_killable(folio);
-}
-
-static inline void set_page_fscache(struct page *page)
-{
- folio_start_fscache(page_folio(page));
-}
-
-static inline void end_page_fscache(struct page *page)
-{
- folio_end_private_2(page_folio(page));
-}
-
-static inline void wait_on_page_fscache(struct page *page)
-{
- folio_wait_private_2(page_folio(page));
-}
-
-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 */
@@ -135,6 +64,7 @@ struct netfs_inode {
#if IS_ENABLED(CONFIG_FSCACHE)
struct fscache_cookie *cache;
#endif
+ struct mutex wb_lock; /* Writeback serialisation */
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 */
@@ -142,7 +72,8 @@ struct netfs_inode {
#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 */
+#define NETFS_ICTX_USE_PGPRIV2 31 /* [DEPRECATED] Use PG_private_2 to mark
+ * write to cache on read */
};
/*
@@ -165,16 +96,25 @@ struct netfs_folio {
unsigned int dirty_len; /* Write-streaming dirty data length */
};
#define NETFS_FOLIO_INFO 0x1UL /* OR'd with folio->private. */
+#define NETFS_FOLIO_COPY_TO_CACHE ((struct netfs_group *)0x356UL) /* Write to the cache only */
-static inline struct netfs_folio *netfs_folio_info(struct folio *folio)
+static inline bool netfs_is_folio_info(const void *priv)
{
- void *priv = folio_get_private(folio);
+ return (unsigned long)priv & NETFS_FOLIO_INFO;
+}
- if ((unsigned long)priv & NETFS_FOLIO_INFO)
+static inline struct netfs_folio *__netfs_folio_info(const void *priv)
+{
+ if (netfs_is_folio_info(priv))
return (struct netfs_folio *)((unsigned long)priv & ~NETFS_FOLIO_INFO);
return NULL;
}
+static inline struct netfs_folio *netfs_folio_info(struct folio *folio)
+{
+ return __netfs_folio_info(folio_get_private(folio));
+}
+
static inline struct netfs_group *netfs_folio_group(struct folio *folio)
{
struct netfs_folio *finfo;
@@ -187,6 +127,33 @@ static inline struct netfs_group *netfs_folio_group(struct folio *folio)
}
/*
+ * Stream of I/O subrequests going to a particular destination, such as the
+ * server or the local cache. This is mainly intended for writing where we may
+ * have to write to multiple destinations concurrently.
+ */
+struct netfs_io_stream {
+ /* Submission tracking */
+ struct netfs_io_subrequest *construct; /* Op being constructed */
+ unsigned int submit_off; /* Folio offset we're submitting from */
+ unsigned int submit_len; /* Amount of data left to submit */
+ unsigned int submit_max_len; /* Amount I/O can be rounded up to */
+ void (*prepare_write)(struct netfs_io_subrequest *subreq);
+ void (*issue_write)(struct netfs_io_subrequest *subreq);
+ /* Collection tracking */
+ struct list_head subrequests; /* Contributory I/O operations */
+ struct netfs_io_subrequest *front; /* Op being collected */
+ unsigned long long collected_to; /* Position we've collected results to */
+ size_t transferred; /* The amount transferred from this stream */
+ enum netfs_io_source source; /* Where to read from/write to */
+ unsigned short error; /* Aggregate error for the stream */
+ unsigned char stream_nr; /* Index of stream in parent table */
+ bool avail; /* T if stream is available */
+ bool active; /* T if stream is active */
+ bool need_retry; /* T if this stream needs retrying */
+ bool failed; /* T if this stream failed */
+};
+
+/*
* Resources required to do operations on a cache.
*/
struct netfs_cache_resources {
@@ -209,14 +176,17 @@ struct netfs_io_subrequest {
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 */
+ unsigned long long start; /* Where to start the I/O */
+ size_t max_len; /* Maximum size of 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 nr_segs; /* Number of segs in io_iter */
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 char stream_nr; /* I/O stream this belongs to */
unsigned long flags;
#define NETFS_SREQ_COPY_TO_CACHE 0 /* Set if should copy the data to the cache */
#define NETFS_SREQ_CLEAR_TAIL 1 /* Set if the rest of the read should be cleared */
@@ -224,15 +194,20 @@ struct netfs_io_subrequest {
#define NETFS_SREQ_SEEK_DATA_READ 3 /* Set if ->read() should SEEK_DATA first */
#define NETFS_SREQ_NO_PROGRESS 4 /* Set if we didn't manage to read any data */
#define NETFS_SREQ_ONDEMAND 5 /* Set if it's from on-demand read mode */
+#define NETFS_SREQ_BOUNDARY 6 /* Set if ends on hard boundary (eg. ceph object) */
+#define NETFS_SREQ_IN_PROGRESS 8 /* Unlocked when the subrequest completes */
+#define NETFS_SREQ_NEED_RETRY 9 /* Set if the filesystem requests a retry */
+#define NETFS_SREQ_RETRYING 10 /* Set if we're retrying */
+#define NETFS_SREQ_FAILED 11 /* Set if the subreq failed unretryably */
};
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_COPY_TO_CACHE, /* This write is to copy a read to the cache */
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 */
@@ -254,26 +229,36 @@ struct netfs_io_request {
struct netfs_cache_resources cache_resources;
struct list_head proc_link; /* Link in netfs_iorequests */
struct list_head subrequests; /* Contributory I/O operations */
+ struct netfs_io_stream io_streams[2]; /* Streams of parallel I/O operations */
+#define NR_IO_STREAMS 2 //wreq->nr_io_streams
+ struct netfs_group *group; /* Writeback group being written back */
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 */
+ void *netfs_priv2; /* 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 subreq_counter; /* Next subreq->debug_index */
+ unsigned int nr_group_rel; /* Number of refs to release on ->group */
+ spinlock_t lock; /* Lock for queuing subreqs */
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 */
+ unsigned long long submitted; /* Amount submitted for I/O so far */
+ unsigned long long len; /* Length of the request */
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 */
+ unsigned long long i_size; /* Size of the file */
+ unsigned long long start; /* Start position */
+ atomic64_t issued_to; /* Write issuer folio cursor */
+ unsigned long long contiguity; /* Tracking for gaps in the writeback sequence */
+ unsigned long long collected_to; /* Point we've collected to */
+ unsigned long long cleaned_to; /* Position we've cleaned folios to */
pgoff_t no_unlock_folio; /* Don't unlock this folio after read */
refcount_t ref;
unsigned long flags;
@@ -287,6 +272,11 @@ struct netfs_io_request {
#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 */
+#define NETFS_RREQ_PAUSE 11 /* Pause subrequest generation */
+#define NETFS_RREQ_USE_IO_ITER 12 /* Use ->io_iter rather than ->i_pages */
+#define NETFS_RREQ_ALL_QUEUED 13 /* All subreqs are now queued */
+#define NETFS_RREQ_USE_PGPRIV2 31 /* [DEPRECATED] Use PG_private_2 to mark
+ * write to cache on read */
const struct netfs_request_ops *netfs_ops;
void (*cleanup)(struct netfs_io_request *req);
};
@@ -295,8 +285,8 @@ struct netfs_io_request {
* 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 */
+ mempool_t *request_pool;
+ mempool_t *subrequest_pool;
int (*init_request)(struct netfs_io_request *rreq, struct file *file);
void (*free_request)(struct netfs_io_request *rreq);
void (*free_subrequest)(struct netfs_io_subrequest *rreq);
@@ -312,10 +302,13 @@ struct netfs_request_ops {
/* Modification handling */
void (*update_i_size)(struct inode *inode, loff_t i_size);
+ void (*post_modify)(struct inode *inode);
/* Write request handling */
- void (*create_write_requests)(struct netfs_io_request *wreq,
- loff_t start, size_t len);
+ void (*begin_writeback)(struct netfs_io_request *wreq);
+ void (*prepare_write)(struct netfs_io_subrequest *subreq);
+ void (*issue_write)(struct netfs_io_subrequest *subreq);
+ void (*retry_request)(struct netfs_io_request *wreq, struct netfs_io_stream *stream);
void (*invalidate_cache)(struct netfs_io_request *wreq);
};
@@ -350,15 +343,27 @@ struct netfs_cache_ops {
netfs_io_terminated_t term_func,
void *term_func_priv);
+ /* Write data to the cache from a netfs subrequest. */
+ void (*issue_write)(struct netfs_io_subrequest *subreq);
+
/* Expand readahead request */
void (*expand_readahead)(struct netfs_cache_resources *cres,
- loff_t *_start, size_t *_len, loff_t i_size);
+ unsigned long long *_start,
+ unsigned long long *_len,
+ unsigned long long i_size);
/* Prepare a read operation, shortening it to a cached/uncached
* boundary as appropriate.
*/
enum netfs_io_source (*prepare_read)(struct netfs_io_subrequest *subreq,
- loff_t i_size);
+ unsigned long long i_size);
+
+ /* Prepare a write subrequest, working out if we're allowed to do it
+ * and finding out the maximum amount of data to gather before
+ * attempting to submit. If we're not permitted to do it, the
+ * subrequest should be marked failed.
+ */
+ void (*prepare_write_subreq)(struct netfs_io_subrequest *subreq);
/* Prepare a write operation, working out what part of the write we can
* actually do.
@@ -410,7 +415,6 @@ 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);
@@ -426,9 +430,7 @@ ssize_t netfs_extract_user_iter(struct iov_iter *orig, size_t orig_len,
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_prepare_write_failed(struct netfs_io_subrequest *subreq);
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);
@@ -472,6 +474,7 @@ static inline void netfs_inode_init(struct netfs_inode *ctx,
#if IS_ENABLED(CONFIG_FSCACHE)
ctx->cache = NULL;
#endif
+ mutex_init(&ctx->wb_lock);
/* ->releasepage() drives zero_point */
if (use_zero_point) {
ctx->zero_point = ctx->remote_i_size;
diff --git a/include/linux/pagemap.h b/include/linux/pagemap.h
index 2df35e65557d..c5e33e2ca48a 100644
--- a/include/linux/pagemap.h
+++ b/include/linux/pagemap.h
@@ -40,6 +40,8 @@ int filemap_fdatawait_keep_errors(struct address_space *mapping);
int filemap_fdatawait_range(struct address_space *, loff_t lstart, loff_t lend);
int filemap_fdatawait_range_keep_errors(struct address_space *mapping,
loff_t start_byte, loff_t end_byte);
+int filemap_invalidate_inode(struct inode *inode, bool flush,
+ loff_t start, loff_t end);
static inline int filemap_fdatawait(struct address_space *mapping)
{
diff --git a/include/net/9p/client.h b/include/net/9p/client.h
index 78ebcf782ce5..4f785098c67a 100644
--- a/include/net/9p/client.h
+++ b/include/net/9p/client.h
@@ -207,6 +207,8 @@ int p9_client_read(struct p9_fid *fid, u64 offset, struct iov_iter *to, int *err
int p9_client_read_once(struct p9_fid *fid, u64 offset, struct iov_iter *to,
int *err);
int p9_client_write(struct p9_fid *fid, u64 offset, struct iov_iter *from, int *err);
+struct netfs_io_subrequest;
+void p9_client_write_subreq(struct netfs_io_subrequest *subreq);
int p9_client_readdir(struct p9_fid *fid, char *data, u32 count, u64 offset);
int p9dirent_read(struct p9_client *clnt, char *buf, int len,
struct p9_dirent *dirent);
diff --git a/include/trace/events/netfs.h b/include/trace/events/netfs.h
index 447a8c21cf57..da23484268df 100644
--- a/include/trace/events/netfs.h
+++ b/include/trace/events/netfs.h
@@ -24,8 +24,8 @@
E_(netfs_read_trace_write_begin, "WRITEBEGN")
#define netfs_write_traces \
+ EM(netfs_write_trace_copy_to_cache, "COPY2CACH") \
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")
@@ -34,9 +34,9 @@
EM(NETFS_READAHEAD, "RA") \
EM(NETFS_READPAGE, "RP") \
EM(NETFS_READ_FOR_WRITE, "RW") \
+ EM(NETFS_COPY_TO_CACHE, "CC") \
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")
@@ -44,14 +44,18 @@
#define netfs_rreq_traces \
EM(netfs_rreq_trace_assess, "ASSESS ") \
EM(netfs_rreq_trace_copy, "COPY ") \
+ EM(netfs_rreq_trace_collect, "COLLECT") \
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_set_pause, "PAUSE ") \
EM(netfs_rreq_trace_unlock, "UNLOCK ") \
EM(netfs_rreq_trace_unmark, "UNMARK ") \
EM(netfs_rreq_trace_wait_ip, "WAIT-IP") \
+ EM(netfs_rreq_trace_wait_pause, "WT-PAUS") \
EM(netfs_rreq_trace_wake_ip, "WAKE-IP") \
+ EM(netfs_rreq_trace_unpause, "UNPAUSE") \
E_(netfs_rreq_trace_write_done, "WR-DONE")
#define netfs_sreq_sources \
@@ -64,11 +68,15 @@
E_(NETFS_INVALID_WRITE, "INVL")
#define netfs_sreq_traces \
+ EM(netfs_sreq_trace_discard, "DSCRD") \
EM(netfs_sreq_trace_download_instead, "RDOWN") \
+ EM(netfs_sreq_trace_fail, "FAIL ") \
EM(netfs_sreq_trace_free, "FREE ") \
EM(netfs_sreq_trace_limited, "LIMIT") \
EM(netfs_sreq_trace_prepare, "PREP ") \
+ EM(netfs_sreq_trace_prep_failed, "PRPFL") \
EM(netfs_sreq_trace_resubmit_short, "SHORT") \
+ EM(netfs_sreq_trace_retry, "RETRY") \
EM(netfs_sreq_trace_submit, "SUBMT") \
EM(netfs_sreq_trace_terminated, "TERM ") \
EM(netfs_sreq_trace_write, "WRITE") \
@@ -88,6 +96,7 @@
#define netfs_rreq_ref_traces \
EM(netfs_rreq_trace_get_for_outstanding,"GET OUTSTND") \
EM(netfs_rreq_trace_get_subreq, "GET SUBREQ ") \
+ EM(netfs_rreq_trace_get_work, "GET WORK ") \
EM(netfs_rreq_trace_put_complete, "PUT COMPLT ") \
EM(netfs_rreq_trace_put_discard, "PUT DISCARD") \
EM(netfs_rreq_trace_put_failed, "PUT FAILED ") \
@@ -95,19 +104,25 @@
EM(netfs_rreq_trace_put_return, "PUT RETURN ") \
EM(netfs_rreq_trace_put_subreq, "PUT SUBREQ ") \
EM(netfs_rreq_trace_put_work, "PUT WORK ") \
+ EM(netfs_rreq_trace_put_work_complete, "PUT WORK CP") \
+ EM(netfs_rreq_trace_put_work_nq, "PUT WORK NQ") \
EM(netfs_rreq_trace_see_work, "SEE WORK ") \
E_(netfs_rreq_trace_new, "NEW ")
#define netfs_sreq_ref_traces \
EM(netfs_sreq_trace_get_copy_to_cache, "GET COPY2C ") \
EM(netfs_sreq_trace_get_resubmit, "GET RESUBMIT") \
+ EM(netfs_sreq_trace_get_submit, "GET SUBMIT") \
EM(netfs_sreq_trace_get_short_read, "GET SHORTRD") \
EM(netfs_sreq_trace_new, "NEW ") \
+ EM(netfs_sreq_trace_put_cancel, "PUT CANCEL ") \
EM(netfs_sreq_trace_put_clear, "PUT CLEAR ") \
EM(netfs_sreq_trace_put_discard, "PUT DISCARD") \
+ EM(netfs_sreq_trace_put_done, "PUT DONE ") \
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_oom, "PUT OOM ") \
EM(netfs_sreq_trace_put_wip, "PUT WIP ") \
EM(netfs_sreq_trace_put_work, "PUT WORK ") \
E_(netfs_sreq_trace_put_terminated, "PUT TERM ")
@@ -124,24 +139,33 @@
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_cancel_copy, "cancel-copy") \
EM(netfs_folio_trace_clear, "clear") \
- EM(netfs_folio_trace_clear_s, "clear-s") \
+ EM(netfs_folio_trace_clear_cc, "clear-cc") \
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_clear_s, "clear-s") \
+ EM(netfs_folio_trace_copy_to_cache, "mark-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_kill_cc, "kill-cc") \
+ EM(netfs_folio_trace_kill_g, "kill-g") \
+ EM(netfs_folio_trace_kill_s, "kill-s") \
EM(netfs_folio_trace_mkwrite, "mkwrite") \
EM(netfs_folio_trace_mkwrite_plus, "mkwrite+") \
+ EM(netfs_folio_trace_not_under_wback, "!wback") \
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_copy, "store-copy") \
EM(netfs_folio_trace_store_plus, "store+") \
EM(netfs_folio_trace_wthru, "wthru") \
E_(netfs_folio_trace_wthru_plus, "wthru+")
+#define netfs_collect_contig_traces \
+ EM(netfs_contig_trace_collect, "Collect") \
+ EM(netfs_contig_trace_jump, "-->JUMP-->") \
+ E_(netfs_contig_trace_unlock, "Unlock")
+
#ifndef __NETFS_DECLARE_TRACE_ENUMS_ONCE_ONLY
#define __NETFS_DECLARE_TRACE_ENUMS_ONCE_ONLY
@@ -158,6 +182,7 @@ 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);
+enum netfs_collect_contig_trace { netfs_collect_contig_traces } __mode(byte);
#endif
@@ -179,6 +204,7 @@ netfs_failures;
netfs_rreq_ref_traces;
netfs_sreq_ref_traces;
netfs_folio_traces;
+netfs_collect_contig_traces;
/*
* Now redefine the EM() and E_() macros to map the enums to the strings that
@@ -279,7 +305,7 @@ TRACE_EVENT(netfs_sreq,
__entry->start = sreq->start;
),
- TP_printk("R=%08x[%u] %s %s f=%02x s=%llx %zx/%zx e=%d",
+ TP_printk("R=%08x[%x] %s %s f=%02x s=%llx %zx/%zx e=%d",
__entry->rreq, __entry->index,
__print_symbolic(__entry->source, netfs_sreq_sources),
__print_symbolic(__entry->what, netfs_sreq_traces),
@@ -319,7 +345,7 @@ TRACE_EVENT(netfs_failure,
__entry->start = sreq ? sreq->start : 0;
),
- TP_printk("R=%08x[%d] %s f=%02x s=%llx %zx/%zx %s e=%d",
+ TP_printk("R=%08x[%x] %s f=%02x s=%llx %zx/%zx %s e=%d",
__entry->rreq, __entry->index,
__print_symbolic(__entry->source, netfs_sreq_sources),
__entry->flags,
@@ -412,16 +438,18 @@ TRACE_EVENT(netfs_write_iter,
__field(unsigned long long, start )
__field(size_t, len )
__field(unsigned int, flags )
+ __field(unsigned int, ino )
),
TP_fast_assign(
__entry->start = iocb->ki_pos;
__entry->len = iov_iter_count(from);
+ __entry->ino = iocb->ki_filp->f_inode->i_ino;
__entry->flags = iocb->ki_flags;
),
- TP_printk("WRITE-ITER s=%llx l=%zx f=%x",
- __entry->start, __entry->len, __entry->flags)
+ TP_printk("WRITE-ITER i=%x s=%llx l=%zx f=%x",
+ __entry->ino, __entry->start, __entry->len, __entry->flags)
);
TRACE_EVENT(netfs_write,
@@ -433,9 +461,10 @@ TRACE_EVENT(netfs_write,
TP_STRUCT__entry(
__field(unsigned int, wreq )
__field(unsigned int, cookie )
+ __field(unsigned int, ino )
__field(enum netfs_write_trace, what )
__field(unsigned long long, start )
- __field(size_t, len )
+ __field(unsigned long long, len )
),
TP_fast_assign(
@@ -443,18 +472,213 @@ TRACE_EVENT(netfs_write,
struct fscache_cookie *__cookie = netfs_i_cookie(__ctx);
__entry->wreq = wreq->debug_id;
__entry->cookie = __cookie ? __cookie->debug_id : 0;
+ __entry->ino = wreq->inode->i_ino;
__entry->what = what;
__entry->start = wreq->start;
__entry->len = wreq->len;
),
- TP_printk("R=%08x %s c=%08x by=%llx-%llx",
+ TP_printk("R=%08x %s c=%08x i=%x by=%llx-%llx",
__entry->wreq,
__print_symbolic(__entry->what, netfs_write_traces),
__entry->cookie,
+ __entry->ino,
__entry->start, __entry->start + __entry->len - 1)
);
+TRACE_EVENT(netfs_collect,
+ TP_PROTO(const struct netfs_io_request *wreq),
+
+ TP_ARGS(wreq),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, wreq )
+ __field(unsigned int, len )
+ __field(unsigned long long, transferred )
+ __field(unsigned long long, start )
+ ),
+
+ TP_fast_assign(
+ __entry->wreq = wreq->debug_id;
+ __entry->start = wreq->start;
+ __entry->len = wreq->len;
+ __entry->transferred = wreq->transferred;
+ ),
+
+ TP_printk("R=%08x s=%llx-%llx",
+ __entry->wreq,
+ __entry->start + __entry->transferred,
+ __entry->start + __entry->len)
+ );
+
+TRACE_EVENT(netfs_collect_contig,
+ TP_PROTO(const struct netfs_io_request *wreq, unsigned long long to,
+ enum netfs_collect_contig_trace type),
+
+ TP_ARGS(wreq, to, type),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, wreq)
+ __field(enum netfs_collect_contig_trace, type)
+ __field(unsigned long long, contiguity)
+ __field(unsigned long long, to)
+ ),
+
+ TP_fast_assign(
+ __entry->wreq = wreq->debug_id;
+ __entry->type = type;
+ __entry->contiguity = wreq->contiguity;
+ __entry->to = to;
+ ),
+
+ TP_printk("R=%08x %llx -> %llx %s",
+ __entry->wreq,
+ __entry->contiguity,
+ __entry->to,
+ __print_symbolic(__entry->type, netfs_collect_contig_traces))
+ );
+
+TRACE_EVENT(netfs_collect_sreq,
+ TP_PROTO(const struct netfs_io_request *wreq,
+ const struct netfs_io_subrequest *subreq),
+
+ TP_ARGS(wreq, subreq),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, wreq )
+ __field(unsigned int, subreq )
+ __field(unsigned int, stream )
+ __field(unsigned int, len )
+ __field(unsigned int, transferred )
+ __field(unsigned long long, start )
+ ),
+
+ TP_fast_assign(
+ __entry->wreq = wreq->debug_id;
+ __entry->subreq = subreq->debug_index;
+ __entry->stream = subreq->stream_nr;
+ __entry->start = subreq->start;
+ __entry->len = subreq->len;
+ __entry->transferred = subreq->transferred;
+ ),
+
+ TP_printk("R=%08x[%u:%02x] s=%llx t=%x/%x",
+ __entry->wreq, __entry->stream, __entry->subreq,
+ __entry->start, __entry->transferred, __entry->len)
+ );
+
+TRACE_EVENT(netfs_collect_folio,
+ TP_PROTO(const struct netfs_io_request *wreq,
+ const struct folio *folio,
+ unsigned long long fend,
+ unsigned long long collected_to),
+
+ TP_ARGS(wreq, folio, fend, collected_to),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, wreq )
+ __field(unsigned long, index )
+ __field(unsigned long long, fend )
+ __field(unsigned long long, cleaned_to )
+ __field(unsigned long long, collected_to )
+ ),
+
+ TP_fast_assign(
+ __entry->wreq = wreq->debug_id;
+ __entry->index = folio->index;
+ __entry->fend = fend;
+ __entry->cleaned_to = wreq->cleaned_to;
+ __entry->collected_to = collected_to;
+ ),
+
+ TP_printk("R=%08x ix=%05lx r=%llx-%llx t=%llx/%llx",
+ __entry->wreq, __entry->index,
+ (unsigned long long)__entry->index * PAGE_SIZE, __entry->fend,
+ __entry->cleaned_to, __entry->collected_to)
+ );
+
+TRACE_EVENT(netfs_collect_state,
+ TP_PROTO(const struct netfs_io_request *wreq,
+ unsigned long long collected_to,
+ unsigned int notes),
+
+ TP_ARGS(wreq, collected_to, notes),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, wreq )
+ __field(unsigned int, notes )
+ __field(unsigned long long, collected_to )
+ __field(unsigned long long, cleaned_to )
+ __field(unsigned long long, contiguity )
+ ),
+
+ TP_fast_assign(
+ __entry->wreq = wreq->debug_id;
+ __entry->notes = notes;
+ __entry->collected_to = collected_to;
+ __entry->cleaned_to = wreq->cleaned_to;
+ __entry->contiguity = wreq->contiguity;
+ ),
+
+ TP_printk("R=%08x cto=%llx fto=%llx ctg=%llx n=%x",
+ __entry->wreq, __entry->collected_to,
+ __entry->cleaned_to, __entry->contiguity,
+ __entry->notes)
+ );
+
+TRACE_EVENT(netfs_collect_gap,
+ TP_PROTO(const struct netfs_io_request *wreq,
+ const struct netfs_io_stream *stream,
+ unsigned long long jump_to, char type),
+
+ TP_ARGS(wreq, stream, jump_to, type),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, wreq)
+ __field(unsigned char, stream)
+ __field(unsigned char, type)
+ __field(unsigned long long, from)
+ __field(unsigned long long, to)
+ ),
+
+ TP_fast_assign(
+ __entry->wreq = wreq->debug_id;
+ __entry->stream = stream->stream_nr;
+ __entry->from = stream->collected_to;
+ __entry->to = jump_to;
+ __entry->type = type;
+ ),
+
+ TP_printk("R=%08x[%x:] %llx->%llx %c",
+ __entry->wreq, __entry->stream,
+ __entry->from, __entry->to, __entry->type)
+ );
+
+TRACE_EVENT(netfs_collect_stream,
+ TP_PROTO(const struct netfs_io_request *wreq,
+ const struct netfs_io_stream *stream),
+
+ TP_ARGS(wreq, stream),
+
+ TP_STRUCT__entry(
+ __field(unsigned int, wreq)
+ __field(unsigned char, stream)
+ __field(unsigned long long, collected_to)
+ __field(unsigned long long, front)
+ ),
+
+ TP_fast_assign(
+ __entry->wreq = wreq->debug_id;
+ __entry->stream = stream->stream_nr;
+ __entry->collected_to = stream->collected_to;
+ __entry->front = stream->front ? stream->front->start : UINT_MAX;
+ ),
+
+ TP_printk("R=%08x[%x:] cto=%llx frn=%llx",
+ __entry->wreq, __entry->stream,
+ __entry->collected_to, __entry->front)
+ );
+
#undef EM
#undef E_
#endif /* _TRACE_NETFS_H */