2020-05-26 23:20:37

by Jens Axboe

[permalink] [raw]
Subject: [PATCHSET v5 0/12] Add support for async buffered reads

We technically support this already through io_uring, but it's
implemented with a thread backend to support cases where we would
block. This isn't ideal.

After a few prep patches, the core of this patchset is adding support
for async callbacks on page unlock. With this primitive, we can simply
retry the IO operation. With io_uring, this works a lot like poll based
retry for files that support it. If a page is currently locked and
needed, -EIOCBQUEUED is returned with a callback armed. The callers
callback is responsible for restarting the operation.

With this callback primitive, we can add support for
generic_file_buffered_read(), which is what most file systems end up
using for buffered reads. XFS/ext4/btrfs/bdev is wired up, but probably
trivial to add more.

The file flags support for this by setting FMODE_BUF_RASYNC, similar
to what we do for FMODE_NOWAIT. Open to suggestions here if this is
the preferred method or not.

In terms of results, I wrote a small test app that randomly reads 4G
of data in 4K chunks from a file hosted by ext4. The app uses a queue
depth of 32. If you want to test yourself, you can just use buffered=1
with ioengine=io_uring with fio. No application changes are needed to
use the more optimized buffered async read.

preadv for comparison:
real 1m13.821s
user 0m0.558s
sys 0m11.125s
CPU ~13%

Mainline:
real 0m12.054s
user 0m0.111s
sys 0m5.659s
CPU ~32% + ~50% == ~82%

This patchset:
real 0m9.283s
user 0m0.147s
sys 0m4.619s
CPU ~52%

The CPU numbers are just a rough estimate. For the mainline io_uring
run, this includes the app itself and all the threads doing IO on its
behalf (32% for the app, ~1.6% per worker and 32 of them). Context
switch rate is much smaller with the patchset, since we only have the
one task performing IO.

Also ran a simple fio based test case, varying the queue depth from 1
to 16, doubling every time:

[buf-test]
filename=/data/file
direct=0
ioengine=io_uring
norandommap
rw=randread
bs=4k
iodepth=${QD}
randseed=89
runtime=10s

QD/Test Patchset IOPS Mainline IOPS
1 9046 8294
2 19.8k 18.9k
4 39.2k 28.5k
8 64.4k 31.4k
16 65.7k 37.8k

Outside of my usual environment, so this is just running on a virtualized
NVMe device in qemu, using ext4 as the file system. NVMe isn't very
efficient virtualized, so we run out of steam at ~65K which is why we
flatline on the patched side (nvme_submit_cmd() eats ~75% of the test app
CPU). Before that happens, it's a linear increase. Not shown is context
switch rate, which is massively lower with the new code. The old thread
offload adds a blocking thread per pending IO, so context rate quickly
goes through the roof.

The goal here is efficiency. Async thread offload adds latency, and
it also adds noticable overhead on items such as adding pages to the
page cache. By allowing proper async buffered read support, we don't
have X threads hammering on the same inode page cache, we have just
the single app actually doing IO.

Been beating on this and it's solid for me, and I'm now pretty happy
with how it all turned out. Not aware of any missing bits/pieces or
code cleanups that need doing.

Series can also be found here:

https://git.kernel.dk/cgit/linux-block/log/?h=async-buffered.5

or pull from:

git://git.kernel.dk/linux-block async-buffered.5

fs/block_dev.c | 2 +-
fs/btrfs/file.c | 2 +-
fs/ext4/file.c | 2 +-
fs/io_uring.c | 130 ++++++++++++++++++++++++++++++++++++--
fs/xfs/xfs_file.c | 2 +-
include/linux/blk_types.h | 3 +-
include/linux/fs.h | 10 ++-
include/linux/pagemap.h | 67 ++++++++++++++++++++
mm/filemap.c | 111 ++++++++++++++++++++------------
9 files changed, 279 insertions(+), 50 deletions(-)

Changes since v5:
- Correct commit message, iocb->private -> iocb->ki_waitq
- Get rid of io_uring goto, use an iter read helper
Changes since v3:
- io_uring: don't retry if REQ_F_NOWAIT is set
- io_uring: alloc req->io if the request type didn't already
- Add iocb->ki_waitq instead of (ab)using iocb->private
Changes since v2:
- Get rid of unnecessary wait_page_async struct, just use wait_page_async
- Add another prep handler, adding wake_page_match()
- Use wake_page_match() in both callers
Changes since v1:
- Fix an issue with inline page locking
- Fix a potential race with __wait_on_page_locked_async()
- Fix a hang related to not setting page_match, thus missing a wakeup

--
Jens Axboe



2020-05-26 23:21:02

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 11/12] mm: add kiocb_wait_page_queue_init() helper

Checks if the file supports it, and initializes the values that we need.
Caller passes in 'data' pointer, if any, and the callback function to
be used.

Signed-off-by: Jens Axboe <[email protected]>
---
include/linux/pagemap.h | 21 +++++++++++++++++++++
1 file changed, 21 insertions(+)

diff --git a/include/linux/pagemap.h b/include/linux/pagemap.h
index d3e63c9c61ae..8b65420410ee 100644
--- a/include/linux/pagemap.h
+++ b/include/linux/pagemap.h
@@ -493,6 +493,27 @@ static inline int wake_page_match(struct wait_page_queue *wait_page,
return 1;
}

+static inline int kiocb_wait_page_queue_init(struct kiocb *kiocb,
+ struct wait_page_queue *wait,
+ wait_queue_func_t func,
+ void *data)
+{
+ /* Can't support async wakeup with polled IO */
+ if (kiocb->ki_flags & IOCB_HIPRI)
+ return -EINVAL;
+ if (kiocb->ki_filp->f_mode & FMODE_BUF_RASYNC) {
+ wait->wait.func = func;
+ wait->wait.private = data;
+ wait->wait.flags = 0;
+ INIT_LIST_HEAD(&wait->wait.entry);
+ kiocb->ki_flags |= IOCB_WAITQ;
+ kiocb->ki_waitq = wait;
+ return 0;
+ }
+
+ return -EOPNOTSUPP;
+}
+
extern void __lock_page(struct page *page);
extern int __lock_page_killable(struct page *page);
extern int __lock_page_async(struct page *page, struct wait_page_queue *wait);
--
2.26.2

2020-05-26 23:21:20

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 08/12] block: flag block devices as supporting IOCB_WAITQ

Signed-off-by: Jens Axboe <[email protected]>
---
fs/block_dev.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fs/block_dev.c b/fs/block_dev.c
index d1e08bba925a..980cfce01c9a 100644
--- a/fs/block_dev.c
+++ b/fs/block_dev.c
@@ -1851,7 +1851,7 @@ static int blkdev_open(struct inode * inode, struct file * filp)
*/
filp->f_flags |= O_LARGEFILE;

- filp->f_mode |= FMODE_NOWAIT;
+ filp->f_mode |= FMODE_NOWAIT | FMODE_BUF_RASYNC;

if (filp->f_flags & O_NDELAY)
filp->f_mode |= FMODE_NDELAY;
--
2.26.2

2020-05-26 23:21:21

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 07/12] ext4: flag as supporting buffered async reads

Signed-off-by: Jens Axboe <[email protected]>
---
fs/ext4/file.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fs/ext4/file.c b/fs/ext4/file.c
index 0d624250a62b..9f7d9bf427b4 100644
--- a/fs/ext4/file.c
+++ b/fs/ext4/file.c
@@ -826,7 +826,7 @@ static int ext4_file_open(struct inode * inode, struct file * filp)
return ret;
}

- filp->f_mode |= FMODE_NOWAIT;
+ filp->f_mode |= FMODE_NOWAIT | FMODE_BUF_RASYNC;
return dquot_file_open(inode, filp);
}

--
2.26.2

2020-05-26 23:21:25

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 02/12] mm: allow read-ahead with IOCB_NOWAIT set

The read-ahead shouldn't block, so allow it to be done even if
IOCB_NOWAIT is set in the kiocb.

Signed-off-by: Jens Axboe <[email protected]>
---
mm/filemap.c | 2 --
1 file changed, 2 deletions(-)

diff --git a/mm/filemap.c b/mm/filemap.c
index 23a051a7ef0f..80747f1377d5 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -2031,8 +2031,6 @@ static ssize_t generic_file_buffered_read(struct kiocb *iocb,

page = find_get_page(mapping, index);
if (!page) {
- if (iocb->ki_flags & IOCB_NOWAIT)
- goto would_block;
page_cache_sync_readahead(mapping,
ra, filp,
index, last_index - index);
--
2.26.2

2020-05-26 23:21:34

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 05/12] mm: support async buffered reads in generic_file_buffered_read()

Use the async page locking infrastructure, if IOCB_WAITQ is set in the
passed in iocb. The caller must expect an -EIOCBQUEUED return value,
which means that IO is started but not done yet. This is similar to how
O_DIRECT signals the same operation. Once the callback is received by
the caller for IO completion, the caller must retry the operation.

Signed-off-by: Jens Axboe <[email protected]>
---
mm/filemap.c | 33 ++++++++++++++++++++++++++-------
1 file changed, 26 insertions(+), 7 deletions(-)

diff --git a/mm/filemap.c b/mm/filemap.c
index c746541b1d49..18022de7dc33 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -1219,6 +1219,14 @@ static int __wait_on_page_locked_async(struct page *page,
return ret;
}

+static int wait_on_page_locked_async(struct page *page,
+ struct wait_page_queue *wait)
+{
+ if (!PageLocked(page))
+ return 0;
+ return __wait_on_page_locked_async(compound_head(page), wait, false);
+}
+
/**
* put_and_wait_on_page_locked - Drop a reference and wait for it to be unlocked
* @page: The page to wait for.
@@ -2058,17 +2066,25 @@ static ssize_t generic_file_buffered_read(struct kiocb *iocb,
index, last_index - index);
}
if (!PageUptodate(page)) {
- if (iocb->ki_flags & IOCB_NOWAIT) {
- put_page(page);
- goto would_block;
- }
-
/*
* See comment in do_read_cache_page on why
* wait_on_page_locked is used to avoid unnecessarily
* serialisations and why it's safe.
*/
- error = wait_on_page_locked_killable(page);
+ if (iocb->ki_flags & IOCB_WAITQ) {
+ if (written) {
+ put_page(page);
+ goto out;
+ }
+ error = wait_on_page_locked_async(page,
+ iocb->ki_waitq);
+ } else {
+ if (iocb->ki_flags & IOCB_NOWAIT) {
+ put_page(page);
+ goto would_block;
+ }
+ error = wait_on_page_locked_killable(page);
+ }
if (unlikely(error))
goto readpage_error;
if (PageUptodate(page))
@@ -2156,7 +2172,10 @@ static ssize_t generic_file_buffered_read(struct kiocb *iocb,

page_not_up_to_date:
/* Get exclusive access to the page ... */
- error = lock_page_killable(page);
+ if (iocb->ki_flags & IOCB_WAITQ)
+ error = lock_page_async(page, iocb->ki_waitq);
+ else
+ error = lock_page_killable(page);
if (unlikely(error))
goto readpage_error;

--
2.26.2

2020-05-26 23:21:36

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 06/12] fs: add FMODE_BUF_RASYNC

If set, this indicates that the file system supports IOCB_WAITQ for
buffered reads.

Signed-off-by: Jens Axboe <[email protected]>
---
include/linux/fs.h | 3 +++
1 file changed, 3 insertions(+)

diff --git a/include/linux/fs.h b/include/linux/fs.h
index ba1fff0e7bca..5ffc6d236b01 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -175,6 +175,9 @@ typedef int (dio_iodone_t)(struct kiocb *iocb, loff_t offset,
/* File does not contribute to nr_files count */
#define FMODE_NOACCOUNT ((__force fmode_t)0x20000000)

+/* File supports async buffered reads */
+#define FMODE_BUF_RASYNC ((__force fmode_t)0x40000000)
+
/*
* Flag for rw_copy_check_uvector and compat_rw_copy_check_uvector
* that indicates that they should check the contents of the iovec are
--
2.26.2

2020-05-26 23:22:02

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 09/12] xfs: flag files as supporting buffered async reads

XFS uses generic_file_read_iter(), which already supports this.

Signed-off-by: Jens Axboe <[email protected]>
---
fs/xfs/xfs_file.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fs/xfs/xfs_file.c b/fs/xfs/xfs_file.c
index 4b8bdecc3863..97f44fbf17f2 100644
--- a/fs/xfs/xfs_file.c
+++ b/fs/xfs/xfs_file.c
@@ -1080,7 +1080,7 @@ xfs_file_open(
return -EFBIG;
if (XFS_FORCED_SHUTDOWN(XFS_M(inode->i_sb)))
return -EIO;
- file->f_mode |= FMODE_NOWAIT;
+ file->f_mode |= FMODE_NOWAIT | FMODE_BUF_RASYNC;
return 0;
}

--
2.26.2

2020-05-26 23:43:12

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 03/12] mm: abstract out wake_page_match() from wake_page_function()

No functional changes in this patch, just in preparation for allowing
more callers.

Signed-off-by: Jens Axboe <[email protected]>
---
include/linux/pagemap.h | 37 +++++++++++++++++++++++++++++++++++++
mm/filemap.c | 35 ++++-------------------------------
2 files changed, 41 insertions(+), 31 deletions(-)

diff --git a/include/linux/pagemap.h b/include/linux/pagemap.h
index a8f7bd8ea1c6..53d980f2208d 100644
--- a/include/linux/pagemap.h
+++ b/include/linux/pagemap.h
@@ -456,6 +456,43 @@ static inline pgoff_t linear_page_index(struct vm_area_struct *vma,
return pgoff;
}

+/* This has the same layout as wait_bit_key - see fs/cachefiles/rdwr.c */
+struct wait_page_key {
+ struct page *page;
+ int bit_nr;
+ int page_match;
+};
+
+struct wait_page_queue {
+ struct page *page;
+ int bit_nr;
+ wait_queue_entry_t wait;
+};
+
+static inline int wake_page_match(struct wait_page_queue *wait_page,
+ struct wait_page_key *key)
+{
+ if (wait_page->page != key->page)
+ return 0;
+ key->page_match = 1;
+
+ if (wait_page->bit_nr != key->bit_nr)
+ return 0;
+
+ /*
+ * Stop walking if it's locked.
+ * Is this safe if put_and_wait_on_page_locked() is in use?
+ * Yes: the waker must hold a reference to this page, and if PG_locked
+ * has now already been set by another task, that task must also hold
+ * a reference to the *same usage* of this page; so there is no need
+ * to walk on to wake even the put_and_wait_on_page_locked() callers.
+ */
+ if (test_bit(key->bit_nr, &key->page->flags))
+ return -1;
+
+ return 1;
+}
+
extern void __lock_page(struct page *page);
extern int __lock_page_killable(struct page *page);
extern int __lock_page_or_retry(struct page *page, struct mm_struct *mm,
diff --git a/mm/filemap.c b/mm/filemap.c
index 80747f1377d5..e891b5bee8fd 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -990,43 +990,16 @@ void __init pagecache_init(void)
page_writeback_init();
}

-/* This has the same layout as wait_bit_key - see fs/cachefiles/rdwr.c */
-struct wait_page_key {
- struct page *page;
- int bit_nr;
- int page_match;
-};
-
-struct wait_page_queue {
- struct page *page;
- int bit_nr;
- wait_queue_entry_t wait;
-};
-
static int wake_page_function(wait_queue_entry_t *wait, unsigned mode, int sync, void *arg)
{
struct wait_page_key *key = arg;
struct wait_page_queue *wait_page
= container_of(wait, struct wait_page_queue, wait);
+ int ret;

- if (wait_page->page != key->page)
- return 0;
- key->page_match = 1;
-
- if (wait_page->bit_nr != key->bit_nr)
- return 0;
-
- /*
- * Stop walking if it's locked.
- * Is this safe if put_and_wait_on_page_locked() is in use?
- * Yes: the waker must hold a reference to this page, and if PG_locked
- * has now already been set by another task, that task must also hold
- * a reference to the *same usage* of this page; so there is no need
- * to walk on to wake even the put_and_wait_on_page_locked() callers.
- */
- if (test_bit(key->bit_nr, &key->page->flags))
- return -1;
-
+ ret = wake_page_match(wait_page, key);
+ if (ret != 1)
+ return ret;
return autoremove_wake_function(wait, mode, sync, key);
}

--
2.26.2

2020-05-26 23:43:12

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 12/12] io_uring: support true async buffered reads, if file provides it

If the file is flagged with FMODE_BUF_RASYNC, then we don't have to punt
the buffered read to an io-wq worker. Instead we can rely on page
unlocking callbacks to support retry based async IO. This is a lot more
efficient than doing async thread offload.

The retry is done similarly to how we handle poll based retry. From
the unlock callback, we simply queue the retry to a task_work based
handler.

Signed-off-by: Jens Axboe <[email protected]>
---
fs/io_uring.c | 130 ++++++++++++++++++++++++++++++++++++++++++++++++--
1 file changed, 126 insertions(+), 4 deletions(-)

diff --git a/fs/io_uring.c b/fs/io_uring.c
index e6865afa8467..95df63b0b2ce 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -79,6 +79,7 @@
#include <linux/fs_struct.h>
#include <linux/splice.h>
#include <linux/task_work.h>
+#include <linux/pagemap.h>

#define CREATE_TRACE_POINTS
#include <trace/events/io_uring.h>
@@ -498,6 +499,8 @@ struct io_async_rw {
struct iovec *iov;
ssize_t nr_segs;
ssize_t size;
+ struct wait_page_queue wpq;
+ struct callback_head task_work;
};

struct io_async_ctx {
@@ -2568,6 +2571,119 @@ static int io_read_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe,
return 0;
}

+static void io_async_buf_cancel(struct callback_head *cb)
+{
+ struct io_async_rw *rw;
+ struct io_ring_ctx *ctx;
+ struct io_kiocb *req;
+
+ rw = container_of(cb, struct io_async_rw, task_work);
+ req = rw->wpq.wait.private;
+ ctx = req->ctx;
+
+ spin_lock_irq(&ctx->completion_lock);
+ io_cqring_fill_event(req, -ECANCELED);
+ io_commit_cqring(ctx);
+ spin_unlock_irq(&ctx->completion_lock);
+
+ io_cqring_ev_posted(ctx);
+ req_set_fail_links(req);
+ io_double_put_req(req);
+}
+
+static void io_async_buf_retry(struct callback_head *cb)
+{
+ struct io_async_rw *rw;
+ struct io_ring_ctx *ctx;
+ struct io_kiocb *req;
+
+ rw = container_of(cb, struct io_async_rw, task_work);
+ req = rw->wpq.wait.private;
+ ctx = req->ctx;
+
+ __set_current_state(TASK_RUNNING);
+ mutex_lock(&ctx->uring_lock);
+ __io_queue_sqe(req, NULL);
+ mutex_unlock(&ctx->uring_lock);
+}
+
+static int io_async_buf_func(struct wait_queue_entry *wait, unsigned mode,
+ int sync, void *arg)
+{
+ struct wait_page_queue *wpq;
+ struct io_kiocb *req = wait->private;
+ struct io_async_rw *rw = &req->io->rw;
+ struct wait_page_key *key = arg;
+ struct task_struct *tsk;
+ int ret;
+
+ wpq = container_of(wait, struct wait_page_queue, wait);
+
+ ret = wake_page_match(wpq, key);
+ if (ret != 1)
+ return ret;
+
+ list_del_init(&wait->entry);
+
+ init_task_work(&rw->task_work, io_async_buf_retry);
+ /* submit ref gets dropped, acquire a new one */
+ refcount_inc(&req->refs);
+ tsk = req->task;
+ ret = task_work_add(tsk, &rw->task_work, true);
+ if (unlikely(ret)) {
+ /* queue just for cancelation */
+ init_task_work(&rw->task_work, io_async_buf_cancel);
+ tsk = io_wq_get_task(req->ctx->io_wq);
+ task_work_add(tsk, &rw->task_work, true);
+ }
+ wake_up_process(tsk);
+ return 1;
+}
+
+static bool io_rw_should_retry(struct io_kiocb *req)
+{
+ struct kiocb *kiocb = &req->rw.kiocb;
+ int ret;
+
+ /* never retry for NOWAIT, we just complete with -EAGAIN */
+ if (req->flags & REQ_F_NOWAIT)
+ return false;
+
+ /* already tried, or we're doing O_DIRECT */
+ if (kiocb->ki_flags & (IOCB_DIRECT | IOCB_WAITQ))
+ return false;
+ /*
+ * just use poll if we can, and don't attempt if the fs doesn't
+ * support callback based unlocks
+ */
+ if (file_can_poll(req->file) || !(req->file->f_mode & FMODE_BUF_RASYNC))
+ return false;
+
+ /*
+ * If request type doesn't require req->io to defer in general,
+ * we need to allocate it here
+ */
+ if (!req->io && __io_alloc_async_ctx(req))
+ return false;
+
+ ret = kiocb_wait_page_queue_init(kiocb, &req->io->rw.wpq,
+ io_async_buf_func, req);
+ if (!ret) {
+ get_task_struct(current);
+ req->task = current;
+ return true;
+ }
+
+ return false;
+}
+
+static int io_iter_do_read(struct io_kiocb *req, struct iov_iter *iter)
+{
+ if (req->file->f_op->read_iter)
+ return call_read_iter(req->file, &req->rw.kiocb, iter);
+ return loop_rw_iter(READ, req->file, &req->rw.kiocb, iter);
+}
+
static int io_read(struct io_kiocb *req, bool force_nonblock)
{
struct iovec inline_vecs[UIO_FASTIOV], *iovec = inline_vecs;
@@ -2601,10 +2717,7 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
if (!ret) {
ssize_t ret2;

- if (req->file->f_op->read_iter)
- ret2 = call_read_iter(req->file, kiocb, &iter);
- else
- ret2 = loop_rw_iter(READ, req->file, kiocb, &iter);
+ ret2 = io_iter_do_read(req, &iter);

/* Catch -EAGAIN return for forced non-blocking submission */
if (!force_nonblock || ret2 != -EAGAIN) {
@@ -2619,6 +2732,15 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
if (!(req->flags & REQ_F_NOWAIT) &&
!file_can_poll(req->file))
req->flags |= REQ_F_MUST_PUNT;
+ /* if we can retry, do so with the callbacks armed */
+ if (io_rw_should_retry(req)) {
+ ret2 = io_iter_do_read(req, &iter);
+ if (ret2 != -EAGAIN) {
+ kiocb_done(kiocb, ret2);
+ goto out_free;
+ }
+ }
+ kiocb->ki_flags &= ~IOCB_WAITQ;
return -EAGAIN;
}
}
--
2.26.2

2020-05-26 23:43:13

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 10/12] btrfs: flag files as supporting buffered async reads

btrfs uses generic_file_read_iter(), which already supports this.

Signed-off-by: Jens Axboe <[email protected]>
---
fs/btrfs/file.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fs/btrfs/file.c b/fs/btrfs/file.c
index 719e68ab552c..c933b6a1b4a8 100644
--- a/fs/btrfs/file.c
+++ b/fs/btrfs/file.c
@@ -3480,7 +3480,7 @@ static loff_t btrfs_file_llseek(struct file *file, loff_t offset, int whence)

static int btrfs_file_open(struct inode *inode, struct file *filp)
{
- filp->f_mode |= FMODE_NOWAIT;
+ filp->f_mode |= FMODE_NOWAIT | FMODE_BUF_RASYNC;
return generic_file_open(inode, filp);
}

--
2.26.2

2020-05-26 23:43:18

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 04/12] mm: add support for async page locking

Normally waiting for a page to become unlocked, or locking the page,
requires waiting for IO to complete. Add support for lock_page_async()
and wait_on_page_locked_async(), which are callback based instead. This
allows a caller to get notified when a page becomes unlocked, rather
than wait for it.

We add a new iocb field, ki_waitq, to pass in the necessary data for this
to happen. We can unionize this with ki_cookie, since that is only used
for polled IO. Polled IO can never co-exist with async callbacks, as it is
(by definition) polled completions. struct wait_page_key is made public,
and we define struct wait_page_async as the interface between the caller
and the core.

Signed-off-by: Jens Axboe <[email protected]>
---
include/linux/fs.h | 7 ++++++-
include/linux/pagemap.h | 9 +++++++++
mm/filemap.c | 41 +++++++++++++++++++++++++++++++++++++++++
3 files changed, 56 insertions(+), 1 deletion(-)

diff --git a/include/linux/fs.h b/include/linux/fs.h
index d3ebb49189df..ba1fff0e7bca 100644
--- a/include/linux/fs.h
+++ b/include/linux/fs.h
@@ -314,6 +314,8 @@ enum rw_hint {
#define IOCB_SYNC (1 << 5)
#define IOCB_WRITE (1 << 6)
#define IOCB_NOWAIT (1 << 7)
+/* iocb->ki_waitq is valid */
+#define IOCB_WAITQ (1 << 8)

struct kiocb {
struct file *ki_filp;
@@ -327,7 +329,10 @@ struct kiocb {
int ki_flags;
u16 ki_hint;
u16 ki_ioprio; /* See linux/ioprio.h */
- unsigned int ki_cookie; /* for ->iopoll */
+ union {
+ unsigned int ki_cookie; /* for ->iopoll */
+ struct wait_page_queue *ki_waitq; /* for async buffered IO */
+ };

randomized_struct_fields_end
};
diff --git a/include/linux/pagemap.h b/include/linux/pagemap.h
index 53d980f2208d..d3e63c9c61ae 100644
--- a/include/linux/pagemap.h
+++ b/include/linux/pagemap.h
@@ -495,6 +495,7 @@ static inline int wake_page_match(struct wait_page_queue *wait_page,

extern void __lock_page(struct page *page);
extern int __lock_page_killable(struct page *page);
+extern int __lock_page_async(struct page *page, struct wait_page_queue *wait);
extern int __lock_page_or_retry(struct page *page, struct mm_struct *mm,
unsigned int flags);
extern void unlock_page(struct page *page);
@@ -531,6 +532,14 @@ static inline int lock_page_killable(struct page *page)
return 0;
}

+static inline int lock_page_async(struct page *page,
+ struct wait_page_queue *wait)
+{
+ if (!trylock_page(page))
+ return __lock_page_async(page, wait);
+ return 0;
+}
+
/*
* lock_page_or_retry - Lock the page, unless this would block and the
* caller indicated that it can handle a retry.
diff --git a/mm/filemap.c b/mm/filemap.c
index e891b5bee8fd..c746541b1d49 100644
--- a/mm/filemap.c
+++ b/mm/filemap.c
@@ -1183,6 +1183,42 @@ int wait_on_page_bit_killable(struct page *page, int bit_nr)
}
EXPORT_SYMBOL(wait_on_page_bit_killable);

+static int __wait_on_page_locked_async(struct page *page,
+ struct wait_page_queue *wait, bool set)
+{
+ struct wait_queue_head *q = page_waitqueue(page);
+ int ret = 0;
+
+ wait->page = page;
+ wait->bit_nr = PG_locked;
+
+ spin_lock_irq(&q->lock);
+ if (set)
+ ret = !trylock_page(page);
+ else
+ ret = PageLocked(page);
+ if (ret) {
+ __add_wait_queue_entry_tail(q, &wait->wait);
+ SetPageWaiters(page);
+ if (set)
+ ret = !trylock_page(page);
+ else
+ ret = PageLocked(page);
+ /*
+ * If we were succesful now, we know we're still on the
+ * waitqueue as we're still under the lock. This means it's
+ * safe to remove and return success, we know the callback
+ * isn't going to trigger.
+ */
+ if (!ret)
+ __remove_wait_queue(q, &wait->wait);
+ else
+ ret = -EIOCBQUEUED;
+ }
+ spin_unlock_irq(&q->lock);
+ return ret;
+}
+
/**
* put_and_wait_on_page_locked - Drop a reference and wait for it to be unlocked
* @page: The page to wait for.
@@ -1345,6 +1381,11 @@ int __lock_page_killable(struct page *__page)
}
EXPORT_SYMBOL_GPL(__lock_page_killable);

+int __lock_page_async(struct page *page, struct wait_page_queue *wait)
+{
+ return __wait_on_page_locked_async(page, wait, true);
+}
+
/*
* Return values:
* 1 - page is locked; mmap_sem is still held.
--
2.26.2

2020-05-27 00:04:15

by Jens Axboe

[permalink] [raw]
Subject: [PATCH 01/12] block: read-ahead submission should imply no-wait as well

As read-ahead is opportunistic, don't block for request allocation.

Signed-off-by: Jens Axboe <[email protected]>
---
include/linux/blk_types.h | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/include/linux/blk_types.h b/include/linux/blk_types.h
index ccb895f911b1..c296463c15eb 100644
--- a/include/linux/blk_types.h
+++ b/include/linux/blk_types.h
@@ -374,7 +374,8 @@ enum req_flag_bits {
#define REQ_INTEGRITY (1ULL << __REQ_INTEGRITY)
#define REQ_FUA (1ULL << __REQ_FUA)
#define REQ_PREFLUSH (1ULL << __REQ_PREFLUSH)
-#define REQ_RAHEAD (1ULL << __REQ_RAHEAD)
+#define REQ_RAHEAD \
+ ((1ULL << __REQ_RAHEAD) | (1ULL << __REQ_NOWAIT))
#define REQ_BACKGROUND (1ULL << __REQ_BACKGROUND)
#define REQ_NOWAIT (1ULL << __REQ_NOWAIT)
#define REQ_CGROUP_PUNT (1ULL << __REQ_CGROUP_PUNT)
--
2.26.2

2020-05-27 00:14:13

by Johannes Weiner

[permalink] [raw]
Subject: Re: [PATCH 02/12] mm: allow read-ahead with IOCB_NOWAIT set

On Tue, May 26, 2020 at 01:51:13PM -0600, Jens Axboe wrote:
> The read-ahead shouldn't block, so allow it to be done even if
> IOCB_NOWAIT is set in the kiocb.
>
> Signed-off-by: Jens Axboe <[email protected]>

Acked-by: Johannes Weiner <[email protected]>

Looks reasonable. Especially after patch 1 - although it seems that
even before that, IOCB_NOWAIT could have occasionally ended up in
page_cache_async_readahead(), which isn't too different from the sync
variant except for the range calculations, and may have blocked on IO
submission in the past.

2020-05-27 01:02:09

by Johannes Weiner

[permalink] [raw]
Subject: Re: [PATCH 04/12] mm: add support for async page locking

On Tue, May 26, 2020 at 01:51:15PM -0600, Jens Axboe wrote:
> Normally waiting for a page to become unlocked, or locking the page,
> requires waiting for IO to complete. Add support for lock_page_async()
> and wait_on_page_locked_async(), which are callback based instead. This

wait_on_page_locked_async() is actually in the next patch, requiring
some back and forth to review. I wonder if this and the next patch
could be merged to have the new API and callers introduced together?

> allows a caller to get notified when a page becomes unlocked, rather
> than wait for it.
>
> We add a new iocb field, ki_waitq, to pass in the necessary data for this
> to happen. We can unionize this with ki_cookie, since that is only used
> for polled IO. Polled IO can never co-exist with async callbacks, as it is
> (by definition) polled completions. struct wait_page_key is made public,
> and we define struct wait_page_async as the interface between the caller
> and the core.
>
> Signed-off-by: Jens Axboe <[email protected]>

Acked-by: Johannes Weiner <[email protected]>

2020-05-27 03:50:27

by Johannes Weiner

[permalink] [raw]
Subject: Re: [PATCH 03/12] mm: abstract out wake_page_match() from wake_page_function()

On Tue, May 26, 2020 at 01:51:14PM -0600, Jens Axboe wrote:
> No functional changes in this patch, just in preparation for allowing
> more callers.
>
> Signed-off-by: Jens Axboe <[email protected]>

Acked-by: Johannes Weiner <[email protected]>

2020-05-27 04:07:47

by Johannes Weiner

[permalink] [raw]
Subject: Re: [PATCH 05/12] mm: support async buffered reads in generic_file_buffered_read()

On Tue, May 26, 2020 at 01:51:16PM -0600, Jens Axboe wrote:
> Use the async page locking infrastructure, if IOCB_WAITQ is set in the
> passed in iocb. The caller must expect an -EIOCBQUEUED return value,
> which means that IO is started but not done yet. This is similar to how
> O_DIRECT signals the same operation. Once the callback is received by
> the caller for IO completion, the caller must retry the operation.
>
> Signed-off-by: Jens Axboe <[email protected]>

Acked-by: Johannes Weiner <[email protected]>

2020-05-27 04:07:47

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCH 04/12] mm: add support for async page locking

On 5/26/20 3:59 PM, Johannes Weiner wrote:
> On Tue, May 26, 2020 at 01:51:15PM -0600, Jens Axboe wrote:
>> Normally waiting for a page to become unlocked, or locking the page,
>> requires waiting for IO to complete. Add support for lock_page_async()
>> and wait_on_page_locked_async(), which are callback based instead. This
>
> wait_on_page_locked_async() is actually in the next patch, requiring
> some back and forth to review. I wonder if this and the next patch
> could be merged to have the new API and callers introduced together?

I'm fine with that, if that is preferable. Don't feel strongly about
that at all, just tried to do it as piecemeal as possible to make
it easier to review.

--
Jens Axboe

2020-05-27 04:08:12

by Johannes Weiner

[permalink] [raw]
Subject: Re: [PATCH 11/12] mm: add kiocb_wait_page_queue_init() helper

On Tue, May 26, 2020 at 01:51:22PM -0600, Jens Axboe wrote:
> Checks if the file supports it, and initializes the values that we need.
> Caller passes in 'data' pointer, if any, and the callback function to
> be used.
>
> Signed-off-by: Jens Axboe <[email protected]>

Acked-by: Johannes Weiner <[email protected]>

2020-05-27 19:06:23

by Johannes Weiner

[permalink] [raw]
Subject: Re: [PATCH 04/12] mm: add support for async page locking

On Tue, May 26, 2020 at 04:01:07PM -0600, Jens Axboe wrote:
> On 5/26/20 3:59 PM, Johannes Weiner wrote:
> > On Tue, May 26, 2020 at 01:51:15PM -0600, Jens Axboe wrote:
> >> Normally waiting for a page to become unlocked, or locking the page,
> >> requires waiting for IO to complete. Add support for lock_page_async()
> >> and wait_on_page_locked_async(), which are callback based instead. This
> >
> > wait_on_page_locked_async() is actually in the next patch, requiring
> > some back and forth to review. I wonder if this and the next patch
> > could be merged to have the new API and callers introduced together?
>
> I'm fine with that, if that is preferable. Don't feel strongly about
> that at all, just tried to do it as piecemeal as possible to make
> it easier to review.

Not worth sending a new iteration over, IMO.

2020-05-28 17:05:51

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Tue, May 26, 2020 at 10:59 PM Jens Axboe <[email protected]> wrote:
>
> We technically support this already through io_uring, but it's
> implemented with a thread backend to support cases where we would
> block. This isn't ideal.
>
> After a few prep patches, the core of this patchset is adding support
> for async callbacks on page unlock. With this primitive, we can simply
> retry the IO operation. With io_uring, this works a lot like poll based
> retry for files that support it. If a page is currently locked and
> needed, -EIOCBQUEUED is returned with a callback armed. The callers
> callback is responsible for restarting the operation.
>
> With this callback primitive, we can add support for
> generic_file_buffered_read(), which is what most file systems end up
> using for buffered reads. XFS/ext4/btrfs/bdev is wired up, but probably
> trivial to add more.
>
> The file flags support for this by setting FMODE_BUF_RASYNC, similar
> to what we do for FMODE_NOWAIT. Open to suggestions here if this is
> the preferred method or not.
>
> In terms of results, I wrote a small test app that randomly reads 4G
> of data in 4K chunks from a file hosted by ext4. The app uses a queue
> depth of 32. If you want to test yourself, you can just use buffered=1
> with ioengine=io_uring with fio. No application changes are needed to
> use the more optimized buffered async read.
>
> preadv for comparison:
> real 1m13.821s
> user 0m0.558s
> sys 0m11.125s
> CPU ~13%
>
> Mainline:
> real 0m12.054s
> user 0m0.111s
> sys 0m5.659s
> CPU ~32% + ~50% == ~82%
>
> This patchset:
> real 0m9.283s
> user 0m0.147s
> sys 0m4.619s
> CPU ~52%
>
> The CPU numbers are just a rough estimate. For the mainline io_uring
> run, this includes the app itself and all the threads doing IO on its
> behalf (32% for the app, ~1.6% per worker and 32 of them). Context
> switch rate is much smaller with the patchset, since we only have the
> one task performing IO.
>
> Also ran a simple fio based test case, varying the queue depth from 1
> to 16, doubling every time:
>
> [buf-test]
> filename=/data/file
> direct=0
> ioengine=io_uring
> norandommap
> rw=randread
> bs=4k
> iodepth=${QD}
> randseed=89
> runtime=10s
>
> QD/Test Patchset IOPS Mainline IOPS
> 1 9046 8294
> 2 19.8k 18.9k
> 4 39.2k 28.5k
> 8 64.4k 31.4k
> 16 65.7k 37.8k
>
> Outside of my usual environment, so this is just running on a virtualized
> NVMe device in qemu, using ext4 as the file system. NVMe isn't very
> efficient virtualized, so we run out of steam at ~65K which is why we
> flatline on the patched side (nvme_submit_cmd() eats ~75% of the test app
> CPU). Before that happens, it's a linear increase. Not shown is context
> switch rate, which is massively lower with the new code. The old thread
> offload adds a blocking thread per pending IO, so context rate quickly
> goes through the roof.
>
> The goal here is efficiency. Async thread offload adds latency, and
> it also adds noticable overhead on items such as adding pages to the
> page cache. By allowing proper async buffered read support, we don't
> have X threads hammering on the same inode page cache, we have just
> the single app actually doing IO.
>
> Been beating on this and it's solid for me, and I'm now pretty happy
> with how it all turned out. Not aware of any missing bits/pieces or
> code cleanups that need doing.
>
> Series can also be found here:
>
> https://git.kernel.dk/cgit/linux-block/log/?h=async-buffered.5
>
> or pull from:
>
> git://git.kernel.dk/linux-block async-buffered.5
>

Hi Jens,

I have pulled linux-block.git#async-buffered.5 on top of Linux v5.7-rc7.

From first feelings:
The booting into the system (until sddm display-login-manager) took a
bit longer.
The same after login and booting into KDE/Plasma.

I am building/linking with LLVM/Clang/LLD v10.0.1-rc1 on Debian/testing AMD64.

Here I have an internal HDD (SATA) and my Debian-system is on an
external HDD connected via USB-3.0.
Primarily, I use Ext4-FS.

As said above is the "emotional" side, but I need some technical instructions.

How can I see Async Buffer Reads is active on a Ext4-FS-formatted partition?

Do I need a special boot-parameter (GRUB line)?

Do I need to activate some cool variables via sysfs?

Do I need to pass an option via fstab entry?

Are any Async Buffer Reads related linux-kconfig options not set?
Which make sense?

I am asking all this before doing some FIO testing.

Attached are my linux-config and dmesg-output files.

Thanks.

Regards,
- Sedat -


> fs/block_dev.c | 2 +-
> fs/btrfs/file.c | 2 +-
> fs/ext4/file.c | 2 +-
> fs/io_uring.c | 130 ++++++++++++++++++++++++++++++++++++--
> fs/xfs/xfs_file.c | 2 +-
> include/linux/blk_types.h | 3 +-
> include/linux/fs.h | 10 ++-
> include/linux/pagemap.h | 67 ++++++++++++++++++++
> mm/filemap.c | 111 ++++++++++++++++++++------------
> 9 files changed, 279 insertions(+), 50 deletions(-)
>
> Changes since v5:
> - Correct commit message, iocb->private -> iocb->ki_waitq
> - Get rid of io_uring goto, use an iter read helper
> Changes since v3:
> - io_uring: don't retry if REQ_F_NOWAIT is set
> - io_uring: alloc req->io if the request type didn't already
> - Add iocb->ki_waitq instead of (ab)using iocb->private
> Changes since v2:
> - Get rid of unnecessary wait_page_async struct, just use wait_page_async
> - Add another prep handler, adding wake_page_match()
> - Use wake_page_match() in both callers
> Changes since v1:
> - Fix an issue with inline page locking
> - Fix a potential race with __wait_on_page_locked_async()
> - Fix a hang related to not setting page_match, thus missing a wakeup
>
> --
> Jens Axboe
>
>


Attachments:
dmesg-T_5.7.0-rc7-4-amd64-clang.txt (69.06 kB)
config-5.7.0-rc7-4-amd64-clang (224.12 kB)
Download all attachments

2020-05-28 17:09:32

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 5/28/20 11:02 AM, Sedat Dilek wrote:
> On Tue, May 26, 2020 at 10:59 PM Jens Axboe <[email protected]> wrote:
>>
>> We technically support this already through io_uring, but it's
>> implemented with a thread backend to support cases where we would
>> block. This isn't ideal.
>>
>> After a few prep patches, the core of this patchset is adding support
>> for async callbacks on page unlock. With this primitive, we can simply
>> retry the IO operation. With io_uring, this works a lot like poll based
>> retry for files that support it. If a page is currently locked and
>> needed, -EIOCBQUEUED is returned with a callback armed. The callers
>> callback is responsible for restarting the operation.
>>
>> With this callback primitive, we can add support for
>> generic_file_buffered_read(), which is what most file systems end up
>> using for buffered reads. XFS/ext4/btrfs/bdev is wired up, but probably
>> trivial to add more.
>>
>> The file flags support for this by setting FMODE_BUF_RASYNC, similar
>> to what we do for FMODE_NOWAIT. Open to suggestions here if this is
>> the preferred method or not.
>>
>> In terms of results, I wrote a small test app that randomly reads 4G
>> of data in 4K chunks from a file hosted by ext4. The app uses a queue
>> depth of 32. If you want to test yourself, you can just use buffered=1
>> with ioengine=io_uring with fio. No application changes are needed to
>> use the more optimized buffered async read.
>>
>> preadv for comparison:
>> real 1m13.821s
>> user 0m0.558s
>> sys 0m11.125s
>> CPU ~13%
>>
>> Mainline:
>> real 0m12.054s
>> user 0m0.111s
>> sys 0m5.659s
>> CPU ~32% + ~50% == ~82%
>>
>> This patchset:
>> real 0m9.283s
>> user 0m0.147s
>> sys 0m4.619s
>> CPU ~52%
>>
>> The CPU numbers are just a rough estimate. For the mainline io_uring
>> run, this includes the app itself and all the threads doing IO on its
>> behalf (32% for the app, ~1.6% per worker and 32 of them). Context
>> switch rate is much smaller with the patchset, since we only have the
>> one task performing IO.
>>
>> Also ran a simple fio based test case, varying the queue depth from 1
>> to 16, doubling every time:
>>
>> [buf-test]
>> filename=/data/file
>> direct=0
>> ioengine=io_uring
>> norandommap
>> rw=randread
>> bs=4k
>> iodepth=${QD}
>> randseed=89
>> runtime=10s
>>
>> QD/Test Patchset IOPS Mainline IOPS
>> 1 9046 8294
>> 2 19.8k 18.9k
>> 4 39.2k 28.5k
>> 8 64.4k 31.4k
>> 16 65.7k 37.8k
>>
>> Outside of my usual environment, so this is just running on a virtualized
>> NVMe device in qemu, using ext4 as the file system. NVMe isn't very
>> efficient virtualized, so we run out of steam at ~65K which is why we
>> flatline on the patched side (nvme_submit_cmd() eats ~75% of the test app
>> CPU). Before that happens, it's a linear increase. Not shown is context
>> switch rate, which is massively lower with the new code. The old thread
>> offload adds a blocking thread per pending IO, so context rate quickly
>> goes through the roof.
>>
>> The goal here is efficiency. Async thread offload adds latency, and
>> it also adds noticable overhead on items such as adding pages to the
>> page cache. By allowing proper async buffered read support, we don't
>> have X threads hammering on the same inode page cache, we have just
>> the single app actually doing IO.
>>
>> Been beating on this and it's solid for me, and I'm now pretty happy
>> with how it all turned out. Not aware of any missing bits/pieces or
>> code cleanups that need doing.
>>
>> Series can also be found here:
>>
>> https://git.kernel.dk/cgit/linux-block/log/?h=async-buffered.5
>>
>> or pull from:
>>
>> git://git.kernel.dk/linux-block async-buffered.5
>>
>
> Hi Jens,
>
> I have pulled linux-block.git#async-buffered.5 on top of Linux v5.7-rc7.
>
> From first feelings:
> The booting into the system (until sddm display-login-manager) took a
> bit longer.
> The same after login and booting into KDE/Plasma.

There is no difference for "regular" use cases, only io_uring with
buffered reads will behave differently. So I don't think you have longer
boot times due to this.

> I am building/linking with LLVM/Clang/LLD v10.0.1-rc1 on Debian/testing AMD64.
>
> Here I have an internal HDD (SATA) and my Debian-system is on an
> external HDD connected via USB-3.0.
> Primarily, I use Ext4-FS.
>
> As said above is the "emotional" side, but I need some technical instructions.
>
> How can I see Async Buffer Reads is active on a Ext4-FS-formatted partition?

You can't see that. It'll always be available on ext4 with this series,
and you can watch io_uring instances to see if anyone is using it.

> Do I need a special boot-parameter (GRUB line)?
>
> Do I need to activate some cool variables via sysfs?
>
> Do I need to pass an option via fstab entry?

No to all of these, you don't need anything to activate it. You need the
program to use io_uring to do buffered reads.

> Are any Async Buffer Reads related linux-kconfig options not set?
> Which make sense?

No kconfig options are needed.

--
Jens Axboe

2020-05-28 17:15:03

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Thu, May 28, 2020 at 7:06 PM Jens Axboe <[email protected]> wrote:
>
> On 5/28/20 11:02 AM, Sedat Dilek wrote:
> > On Tue, May 26, 2020 at 10:59 PM Jens Axboe <[email protected]> wrote:
> >>
> >> We technically support this already through io_uring, but it's
> >> implemented with a thread backend to support cases where we would
> >> block. This isn't ideal.
> >>
> >> After a few prep patches, the core of this patchset is adding support
> >> for async callbacks on page unlock. With this primitive, we can simply
> >> retry the IO operation. With io_uring, this works a lot like poll based
> >> retry for files that support it. If a page is currently locked and
> >> needed, -EIOCBQUEUED is returned with a callback armed. The callers
> >> callback is responsible for restarting the operation.
> >>
> >> With this callback primitive, we can add support for
> >> generic_file_buffered_read(), which is what most file systems end up
> >> using for buffered reads. XFS/ext4/btrfs/bdev is wired up, but probably
> >> trivial to add more.
> >>
> >> The file flags support for this by setting FMODE_BUF_RASYNC, similar
> >> to what we do for FMODE_NOWAIT. Open to suggestions here if this is
> >> the preferred method or not.
> >>
> >> In terms of results, I wrote a small test app that randomly reads 4G
> >> of data in 4K chunks from a file hosted by ext4. The app uses a queue
> >> depth of 32. If you want to test yourself, you can just use buffered=1
> >> with ioengine=io_uring with fio. No application changes are needed to
> >> use the more optimized buffered async read.
> >>
> >> preadv for comparison:
> >> real 1m13.821s
> >> user 0m0.558s
> >> sys 0m11.125s
> >> CPU ~13%
> >>
> >> Mainline:
> >> real 0m12.054s
> >> user 0m0.111s
> >> sys 0m5.659s
> >> CPU ~32% + ~50% == ~82%
> >>
> >> This patchset:
> >> real 0m9.283s
> >> user 0m0.147s
> >> sys 0m4.619s
> >> CPU ~52%
> >>
> >> The CPU numbers are just a rough estimate. For the mainline io_uring
> >> run, this includes the app itself and all the threads doing IO on its
> >> behalf (32% for the app, ~1.6% per worker and 32 of them). Context
> >> switch rate is much smaller with the patchset, since we only have the
> >> one task performing IO.
> >>
> >> Also ran a simple fio based test case, varying the queue depth from 1
> >> to 16, doubling every time:
> >>
> >> [buf-test]
> >> filename=/data/file
> >> direct=0
> >> ioengine=io_uring
> >> norandommap
> >> rw=randread
> >> bs=4k
> >> iodepth=${QD}
> >> randseed=89
> >> runtime=10s
> >>
> >> QD/Test Patchset IOPS Mainline IOPS
> >> 1 9046 8294
> >> 2 19.8k 18.9k
> >> 4 39.2k 28.5k
> >> 8 64.4k 31.4k
> >> 16 65.7k 37.8k
> >>
> >> Outside of my usual environment, so this is just running on a virtualized
> >> NVMe device in qemu, using ext4 as the file system. NVMe isn't very
> >> efficient virtualized, so we run out of steam at ~65K which is why we
> >> flatline on the patched side (nvme_submit_cmd() eats ~75% of the test app
> >> CPU). Before that happens, it's a linear increase. Not shown is context
> >> switch rate, which is massively lower with the new code. The old thread
> >> offload adds a blocking thread per pending IO, so context rate quickly
> >> goes through the roof.
> >>
> >> The goal here is efficiency. Async thread offload adds latency, and
> >> it also adds noticable overhead on items such as adding pages to the
> >> page cache. By allowing proper async buffered read support, we don't
> >> have X threads hammering on the same inode page cache, we have just
> >> the single app actually doing IO.
> >>
> >> Been beating on this and it's solid for me, and I'm now pretty happy
> >> with how it all turned out. Not aware of any missing bits/pieces or
> >> code cleanups that need doing.
> >>
> >> Series can also be found here:
> >>
> >> https://git.kernel.dk/cgit/linux-block/log/?h=async-buffered.5
> >>
> >> or pull from:
> >>
> >> git://git.kernel.dk/linux-block async-buffered.5
> >>
> >
> > Hi Jens,
> >
> > I have pulled linux-block.git#async-buffered.5 on top of Linux v5.7-rc7.
> >
> > From first feelings:
> > The booting into the system (until sddm display-login-manager) took a
> > bit longer.
> > The same after login and booting into KDE/Plasma.
>
> There is no difference for "regular" use cases, only io_uring with
> buffered reads will behave differently. So I don't think you have longer
> boot times due to this.
>
> > I am building/linking with LLVM/Clang/LLD v10.0.1-rc1 on Debian/testing AMD64.
> >
> > Here I have an internal HDD (SATA) and my Debian-system is on an
> > external HDD connected via USB-3.0.
> > Primarily, I use Ext4-FS.
> >
> > As said above is the "emotional" side, but I need some technical instructions.
> >
> > How can I see Async Buffer Reads is active on a Ext4-FS-formatted partition?
>
> You can't see that. It'll always be available on ext4 with this series,
> and you can watch io_uring instances to see if anyone is using it.
>

Thanks for answering my questions.

How can I "watch io_uring instances"?

FIO?
Debian has fio version 3.19-2 in its apt repositories.
Version OK?

- Sedat -

> > Do I need a special boot-parameter (GRUB line)?
> >
> > Do I need to activate some cool variables via sysfs?
> >
> > Do I need to pass an option via fstab entry?
>
> No to all of these, you don't need anything to activate it. You need the
> program to use io_uring to do buffered reads.
>
> > Are any Async Buffer Reads related linux-kconfig options not set?
> > Which make sense?
>
> No kconfig options are needed.
>
> --
> Jens Axboe
>

2020-05-28 17:17:08

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 5/28/20 11:12 AM, Sedat Dilek wrote:
> On Thu, May 28, 2020 at 7:06 PM Jens Axboe <[email protected]> wrote:
>>
>> On 5/28/20 11:02 AM, Sedat Dilek wrote:
>>> On Tue, May 26, 2020 at 10:59 PM Jens Axboe <[email protected]> wrote:
>>>>
>>>> We technically support this already through io_uring, but it's
>>>> implemented with a thread backend to support cases where we would
>>>> block. This isn't ideal.
>>>>
>>>> After a few prep patches, the core of this patchset is adding support
>>>> for async callbacks on page unlock. With this primitive, we can simply
>>>> retry the IO operation. With io_uring, this works a lot like poll based
>>>> retry for files that support it. If a page is currently locked and
>>>> needed, -EIOCBQUEUED is returned with a callback armed. The callers
>>>> callback is responsible for restarting the operation.
>>>>
>>>> With this callback primitive, we can add support for
>>>> generic_file_buffered_read(), which is what most file systems end up
>>>> using for buffered reads. XFS/ext4/btrfs/bdev is wired up, but probably
>>>> trivial to add more.
>>>>
>>>> The file flags support for this by setting FMODE_BUF_RASYNC, similar
>>>> to what we do for FMODE_NOWAIT. Open to suggestions here if this is
>>>> the preferred method or not.
>>>>
>>>> In terms of results, I wrote a small test app that randomly reads 4G
>>>> of data in 4K chunks from a file hosted by ext4. The app uses a queue
>>>> depth of 32. If you want to test yourself, you can just use buffered=1
>>>> with ioengine=io_uring with fio. No application changes are needed to
>>>> use the more optimized buffered async read.
>>>>
>>>> preadv for comparison:
>>>> real 1m13.821s
>>>> user 0m0.558s
>>>> sys 0m11.125s
>>>> CPU ~13%
>>>>
>>>> Mainline:
>>>> real 0m12.054s
>>>> user 0m0.111s
>>>> sys 0m5.659s
>>>> CPU ~32% + ~50% == ~82%
>>>>
>>>> This patchset:
>>>> real 0m9.283s
>>>> user 0m0.147s
>>>> sys 0m4.619s
>>>> CPU ~52%
>>>>
>>>> The CPU numbers are just a rough estimate. For the mainline io_uring
>>>> run, this includes the app itself and all the threads doing IO on its
>>>> behalf (32% for the app, ~1.6% per worker and 32 of them). Context
>>>> switch rate is much smaller with the patchset, since we only have the
>>>> one task performing IO.
>>>>
>>>> Also ran a simple fio based test case, varying the queue depth from 1
>>>> to 16, doubling every time:
>>>>
>>>> [buf-test]
>>>> filename=/data/file
>>>> direct=0
>>>> ioengine=io_uring
>>>> norandommap
>>>> rw=randread
>>>> bs=4k
>>>> iodepth=${QD}
>>>> randseed=89
>>>> runtime=10s
>>>>
>>>> QD/Test Patchset IOPS Mainline IOPS
>>>> 1 9046 8294
>>>> 2 19.8k 18.9k
>>>> 4 39.2k 28.5k
>>>> 8 64.4k 31.4k
>>>> 16 65.7k 37.8k
>>>>
>>>> Outside of my usual environment, so this is just running on a virtualized
>>>> NVMe device in qemu, using ext4 as the file system. NVMe isn't very
>>>> efficient virtualized, so we run out of steam at ~65K which is why we
>>>> flatline on the patched side (nvme_submit_cmd() eats ~75% of the test app
>>>> CPU). Before that happens, it's a linear increase. Not shown is context
>>>> switch rate, which is massively lower with the new code. The old thread
>>>> offload adds a blocking thread per pending IO, so context rate quickly
>>>> goes through the roof.
>>>>
>>>> The goal here is efficiency. Async thread offload adds latency, and
>>>> it also adds noticable overhead on items such as adding pages to the
>>>> page cache. By allowing proper async buffered read support, we don't
>>>> have X threads hammering on the same inode page cache, we have just
>>>> the single app actually doing IO.
>>>>
>>>> Been beating on this and it's solid for me, and I'm now pretty happy
>>>> with how it all turned out. Not aware of any missing bits/pieces or
>>>> code cleanups that need doing.
>>>>
>>>> Series can also be found here:
>>>>
>>>> https://git.kernel.dk/cgit/linux-block/log/?h=async-buffered.5
>>>>
>>>> or pull from:
>>>>
>>>> git://git.kernel.dk/linux-block async-buffered.5
>>>>
>>>
>>> Hi Jens,
>>>
>>> I have pulled linux-block.git#async-buffered.5 on top of Linux v5.7-rc7.
>>>
>>> From first feelings:
>>> The booting into the system (until sddm display-login-manager) took a
>>> bit longer.
>>> The same after login and booting into KDE/Plasma.
>>
>> There is no difference for "regular" use cases, only io_uring with
>> buffered reads will behave differently. So I don't think you have longer
>> boot times due to this.
>>
>>> I am building/linking with LLVM/Clang/LLD v10.0.1-rc1 on Debian/testing AMD64.
>>>
>>> Here I have an internal HDD (SATA) and my Debian-system is on an
>>> external HDD connected via USB-3.0.
>>> Primarily, I use Ext4-FS.
>>>
>>> As said above is the "emotional" side, but I need some technical instructions.
>>>
>>> How can I see Async Buffer Reads is active on a Ext4-FS-formatted partition?
>>
>> You can't see that. It'll always be available on ext4 with this series,
>> and you can watch io_uring instances to see if anyone is using it.
>>
>
> Thanks for answering my questions.
>
> How can I "watch io_uring instances"?

You can enable io_uring tracing:

# echo 1 > /sys/kernel/debug/tracing/events/io_uring/io_uring_create/enable
# tail /sys/kernel/debug/tracing/trace

and see if you get any events for setup. Generally you can also look for
the existence of io_wq_manager processes, these will exist for an
io_uring instance.

> FIO?
> Debian has fio version 3.19-2 in its apt repositories.
> Version OK?

Yeah that should work.

--
Jens Axboe

2020-05-28 17:58:30

by Darrick J. Wong

[permalink] [raw]
Subject: Re: [PATCH 09/12] xfs: flag files as supporting buffered async reads

On Tue, May 26, 2020 at 01:51:20PM -0600, Jens Axboe wrote:
> XFS uses generic_file_read_iter(), which already supports this.
>
> Signed-off-by: Jens Axboe <[email protected]>

Er... I guess that looks ok? Assuming you've done enough qa on
io_uring to be able to tell if this breaks anything, since touching the
mm always feels murky to me:

Acked-by: Darrick J. Wong <[email protected]>

--D

> ---
> fs/xfs/xfs_file.c | 2 +-
> 1 file changed, 1 insertion(+), 1 deletion(-)
>
> diff --git a/fs/xfs/xfs_file.c b/fs/xfs/xfs_file.c
> index 4b8bdecc3863..97f44fbf17f2 100644
> --- a/fs/xfs/xfs_file.c
> +++ b/fs/xfs/xfs_file.c
> @@ -1080,7 +1080,7 @@ xfs_file_open(
> return -EFBIG;
> if (XFS_FORCED_SHUTDOWN(XFS_M(inode->i_sb)))
> return -EIO;
> - file->f_mode |= FMODE_NOWAIT;
> + file->f_mode |= FMODE_NOWAIT | FMODE_BUF_RASYNC;
> return 0;
> }
>
> --
> 2.26.2
>

2020-05-28 18:23:04

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Thu, May 28, 2020 at 7:14 PM Jens Axboe <[email protected]> wrote:
>
> On 5/28/20 11:12 AM, Sedat Dilek wrote:
> > On Thu, May 28, 2020 at 7:06 PM Jens Axboe <[email protected]> wrote:
> >>
> >> On 5/28/20 11:02 AM, Sedat Dilek wrote:
> >>> On Tue, May 26, 2020 at 10:59 PM Jens Axboe <[email protected]> wrote:
> >>>>
> >>>> We technically support this already through io_uring, but it's
> >>>> implemented with a thread backend to support cases where we would
> >>>> block. This isn't ideal.
> >>>>
> >>>> After a few prep patches, the core of this patchset is adding support
> >>>> for async callbacks on page unlock. With this primitive, we can simply
> >>>> retry the IO operation. With io_uring, this works a lot like poll based
> >>>> retry for files that support it. If a page is currently locked and
> >>>> needed, -EIOCBQUEUED is returned with a callback armed. The callers
> >>>> callback is responsible for restarting the operation.
> >>>>
> >>>> With this callback primitive, we can add support for
> >>>> generic_file_buffered_read(), which is what most file systems end up
> >>>> using for buffered reads. XFS/ext4/btrfs/bdev is wired up, but probably
> >>>> trivial to add more.
> >>>>
> >>>> The file flags support for this by setting FMODE_BUF_RASYNC, similar
> >>>> to what we do for FMODE_NOWAIT. Open to suggestions here if this is
> >>>> the preferred method or not.
> >>>>
> >>>> In terms of results, I wrote a small test app that randomly reads 4G
> >>>> of data in 4K chunks from a file hosted by ext4. The app uses a queue
> >>>> depth of 32. If you want to test yourself, you can just use buffered=1
> >>>> with ioengine=io_uring with fio. No application changes are needed to
> >>>> use the more optimized buffered async read.
> >>>>
> >>>> preadv for comparison:
> >>>> real 1m13.821s
> >>>> user 0m0.558s
> >>>> sys 0m11.125s
> >>>> CPU ~13%
> >>>>
> >>>> Mainline:
> >>>> real 0m12.054s
> >>>> user 0m0.111s
> >>>> sys 0m5.659s
> >>>> CPU ~32% + ~50% == ~82%
> >>>>
> >>>> This patchset:
> >>>> real 0m9.283s
> >>>> user 0m0.147s
> >>>> sys 0m4.619s
> >>>> CPU ~52%
> >>>>
> >>>> The CPU numbers are just a rough estimate. For the mainline io_uring
> >>>> run, this includes the app itself and all the threads doing IO on its
> >>>> behalf (32% for the app, ~1.6% per worker and 32 of them). Context
> >>>> switch rate is much smaller with the patchset, since we only have the
> >>>> one task performing IO.
> >>>>
> >>>> Also ran a simple fio based test case, varying the queue depth from 1
> >>>> to 16, doubling every time:
> >>>>
> >>>> [buf-test]
> >>>> filename=/data/file
> >>>> direct=0
> >>>> ioengine=io_uring
> >>>> norandommap
> >>>> rw=randread
> >>>> bs=4k
> >>>> iodepth=${QD}
> >>>> randseed=89
> >>>> runtime=10s
> >>>>
> >>>> QD/Test Patchset IOPS Mainline IOPS
> >>>> 1 9046 8294
> >>>> 2 19.8k 18.9k
> >>>> 4 39.2k 28.5k
> >>>> 8 64.4k 31.4k
> >>>> 16 65.7k 37.8k
> >>>>
> >>>> Outside of my usual environment, so this is just running on a virtualized
> >>>> NVMe device in qemu, using ext4 as the file system. NVMe isn't very
> >>>> efficient virtualized, so we run out of steam at ~65K which is why we
> >>>> flatline on the patched side (nvme_submit_cmd() eats ~75% of the test app
> >>>> CPU). Before that happens, it's a linear increase. Not shown is context
> >>>> switch rate, which is massively lower with the new code. The old thread
> >>>> offload adds a blocking thread per pending IO, so context rate quickly
> >>>> goes through the roof.
> >>>>
> >>>> The goal here is efficiency. Async thread offload adds latency, and
> >>>> it also adds noticable overhead on items such as adding pages to the
> >>>> page cache. By allowing proper async buffered read support, we don't
> >>>> have X threads hammering on the same inode page cache, we have just
> >>>> the single app actually doing IO.
> >>>>
> >>>> Been beating on this and it's solid for me, and I'm now pretty happy
> >>>> with how it all turned out. Not aware of any missing bits/pieces or
> >>>> code cleanups that need doing.
> >>>>
> >>>> Series can also be found here:
> >>>>
> >>>> https://git.kernel.dk/cgit/linux-block/log/?h=async-buffered.5
> >>>>
> >>>> or pull from:
> >>>>
> >>>> git://git.kernel.dk/linux-block async-buffered.5
> >>>>
> >>>
> >>> Hi Jens,
> >>>
> >>> I have pulled linux-block.git#async-buffered.5 on top of Linux v5.7-rc7.
> >>>
> >>> From first feelings:
> >>> The booting into the system (until sddm display-login-manager) took a
> >>> bit longer.
> >>> The same after login and booting into KDE/Plasma.
> >>
> >> There is no difference for "regular" use cases, only io_uring with
> >> buffered reads will behave differently. So I don't think you have longer
> >> boot times due to this.
> >>
> >>> I am building/linking with LLVM/Clang/LLD v10.0.1-rc1 on Debian/testing AMD64.
> >>>
> >>> Here I have an internal HDD (SATA) and my Debian-system is on an
> >>> external HDD connected via USB-3.0.
> >>> Primarily, I use Ext4-FS.
> >>>
> >>> As said above is the "emotional" side, but I need some technical instructions.
> >>>
> >>> How can I see Async Buffer Reads is active on a Ext4-FS-formatted partition?
> >>
> >> You can't see that. It'll always be available on ext4 with this series,
> >> and you can watch io_uring instances to see if anyone is using it.
> >>
> >
> > Thanks for answering my questions.
> >
> > How can I "watch io_uring instances"?
>
> You can enable io_uring tracing:
>
> # echo 1 > /sys/kernel/debug/tracing/events/io_uring/io_uring_create/enable
> # tail /sys/kernel/debug/tracing/trace
>
> and see if you get any events for setup. Generally you can also look for
> the existence of io_wq_manager processes, these will exist for an
> io_uring instance.
>
> > FIO?
> > Debian has fio version 3.19-2 in its apt repositories.
> > Version OK?
>
> Yeah that should work.
>

I did:

# echo 1 > /sys/kernel/debug/tracing/events/io_uring/io_uring_create/enable
# cat /sys/kernel/debug/tracing/events/io_uring/io_uring_create/enable
1

# cat ./buf-test-dileks-min
[buf-test-dileks-min]
filename=/path/to/iso-image-file
buffered=1
ioengine=io_uring

# fio --showcmd ./buf-test-dileks-min
fio --name=buf-test-dileks-min --buffered=1 --ioengine=io_uring
--filename=filename=/path/to/iso-image-file

# fio ./buf-test-dileks-min

# ps -ef | egrep 'f[i]o|i[o]_wq'
root 6695 6066 24 20:13 pts/2 00:00:00 fio ./buf-test-dileks-min
root 6701 6695 22 20:13 ? 00:00:00 fio ./buf-test-dileks-min
root 6702 2 0 20:13 ? 00:00:00 [io_wq_manager]
root 6703 2 0 20:13 ? 00:00:00 [io_wqe_worker-0]

# LC_ALL=C tail -f /sys/kernel/debug/tracing/trace
...
# entries-in-buffer/entries-written: 16/16 #P:4
#
# _-----=> irqs-off
# / _----=> need-resched
# | / _---=> hardirq/softirq
# || / _--=> preempt-depth
# ||| / delay
# TASK-PID CPU# |||| TIMESTAMP FUNCTION
# | | | |||| | |
...
fio-6701 [001] .... 6775.117015: io_uring_create: ring
00000000ef052188, fd 5 sq size 1, cq size 2, flags 0

Looks like this works.

Thanks Jens.

- Sedat -

2020-05-28 19:26:07

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCH 09/12] xfs: flag files as supporting buffered async reads

On 5/28/20 11:53 AM, Darrick J. Wong wrote:
> On Tue, May 26, 2020 at 01:51:20PM -0600, Jens Axboe wrote:
>> XFS uses generic_file_read_iter(), which already supports this.
>>
>> Signed-off-by: Jens Axboe <[email protected]>
>
> Er... I guess that looks ok? Assuming you've done enough qa on
> io_uring to be able to tell if this breaks anything, since touching the
> mm always feels murky to me:
>

The mm bits should be fine, haven't seen anything odd in testing.
And it's not like the mm changes are super complicated, I think
they turned out pretty clean and straight forward.

> Acked-by: Darrick J. Wong <[email protected]>

Thanks!

--
Jens Axboe

2020-05-29 10:07:09

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

[ ... ]

> > Hi Jens,
> >
> > I have pulled linux-block.git#async-buffered.5 on top of Linux v5.7-rc7.
> >
> > From first feelings:
> > The booting into the system (until sddm display-login-manager) took a
> > bit longer.
> > The same after login and booting into KDE/Plasma.
>
> There is no difference for "regular" use cases, only io_uring with
> buffered reads will behave differently. So I don't think you have longer
> boot times due to this.
>

Yupp, you are right.

The previous Linux v5.7-rc7 without your patchset shows the same symptoms.

I did some debugging and optimizing with systemd-analyze boot and time.

I optimized systemd-journald.service and systemd-journal-flush.service...

# cat /etc/systemd/journald.conf.d/00-journal-size.conf
[Journal]
SystemMaxUse=50M

...and reduced the time spent flushing systemd's journal from ~30s
down to 1,6s...

# journalctl -b --unit systemd-journald.service
-- Logs begin at Fri 2020-05-29 00:58:37 CEST, end at Fri 2020-05-29
11:42:18 CEST. --
Mai 29 11:34:52 iniza systemd-journald[281]: Journal started
Mai 29 11:34:52 iniza systemd-journald[281]: Runtime Journal
(/run/log/journal/566abbcb226b405db834b17a26fe4727) is 8.0M, max
78.5M, 70.5M free.
Mai 29 11:34:53 iniza systemd-journald[281]: Time spent on flushing to
/var/log/journal/566abbcb226b405db834b17a26fe4727 is 1.656233s for 765
entries.
Mai 29 11:34:53 iniza systemd-journald[281]: System Journal
(/var/log/journal/566abbcb226b405db834b17a26fe4727) is 56.2M, max
50.0M, 0B free.

Unfortunately, I upgraded some user-space stuff like udisks2 and
libblockdev packages.
Downgrading did not help and disabling the systemd-unit also.

As I saw stallings with e2scrub_reap.service and swap partition
(partly seen in the boot-process and noted the UUID 3f8e).
I disabled e2scrub_reap.service and deactivated swap partition in /etc/fstab.

Doing all above together did not help.

Finally, I checked the health of the HDD where my root-fs is.
smartmontools says everything is OK.

I have not checked the status of the Ext4-FS where my root-fs is.
Such things I do with a linux-live-system - as a Debianist I admit I
use an ArchLinux ISO on USB-stick :-).

Unsure, if I will contact the systemd (and mabye udisks) Debian folks
to hear their opinion.

Thanks Jens and your patchset.
I don't know when I last run systemd-analyze & stuff and investigated
so deeply :-).

A lot of Hygge (I love to write wrong Huegge - see English hugs) to you Jens.

- Sedat -

[1] http://ftp.halifax.rwth-aachen.de/archlinux/iso/

2020-05-29 11:25:15

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Fri, May 29, 2020 at 12:02 PM Sedat Dilek <[email protected]> wrote:

[ ... ]

> As I saw stallings with e2scrub_reap.service and swap partition
> (partly seen in the boot-process and noted the UUID 3f8e).
> I disabled e2scrub_reap.service and deactivated swap partition in /etc/fstab.
>

I switched over from using a swap-partition to a swap-file.

The boot-slowdown is "gone" in sense of 1m30s stalls or better in
sense of boot-time is shorter.

# cat systemd-analyze-time.txt
Startup finished in 6.903s (kernel) + 1min 13.501s (userspace) = 1min 20.404s
graphical.target reached after 1min 13.481s in userspace

# cat systemd-analyze-time_swapfile.txt
Startup finished in 6.721s (kernel) + 1min 9.470s (userspace) = 1min 16.192s
graphical.target reached after 1min 9.451s in userspace

# cat systemd-analyze-blame.txt | head -20
35.943s udisks2.service
32.559s accounts-daemon.service
27.925s smartmontools.service
26.561s NetworkManager.service
24.543s dev-sdc2.device
24.478s polkit.service
20.426s NetworkManager-wait-online.service
19.785s avahi-daemon.service
19.586s switcheroo-control.service
19.185s rtkit-daemon.service
18.661s wpa_supplicant.service
18.269s systemd-logind.service
17.627s rsyslog.service
16.312s gpm.service
14.842s e2scrub_reap.service
14.387s packagekit.service
12.017s ModemManager.service
10.584s alsa-restore.service
8.407s atd.service
6.025s exim4.service

# cat systemd-analyze-blame_swapfile.txt | head -20
29.571s udisks2.service
26.383s accounts-daemon.service
24.564s smartmontools.service
20.735s NetworkManager.service
19.254s NetworkManager-wait-online.service
18.675s polkit.service
15.523s dev-sdc2.device
14.152s avahi-daemon.service
14.006s switcheroo-control.service
13.800s rtkit-daemon.service
13.662s packagekit.service
13.353s wpa_supplicant.service
13.178s rsyslog.service
12.788s systemd-logind.service
12.313s e2scrub_reap.service
11.105s ModemManager.service
11.003s gpm.service
10.018s networking.service
6.608s apparmor.service
5.858s exim4.service

Thanks.

Time to experience with ZRAM :-).

- Sedat -

LINK: https://wiki.debian.org/Swap
LINK: https://help.ubuntu.com/community/SwapFaq

mount -t auto /dev/sdb1 /mnt/sandisk

fallocate -l 8g /mnt/sandisk/swapfile

8 x 1024 x 1024 = 8388608

dd if=/dev/zero bs=1024 count=8388608 of=/mnt/sandisk/swapfile
8388608+0 Datensätze ein
8388608+0 Datensätze aus
8589934592 bytes (8,6 GB, 8,0 GiB) copied, 176,511 s, 48,7 MB/s

# chmod 600 /mnt/sandisk/swapfile

# ll /mnt/sandisk/swapfile
-rw------- 1 root root 8,0G Mai 29 12:23 /mnt/sandisk/swapfile

# mkswap /mnt/sandisk/swapfile
Setting up swapspace version 1, size = 8 GiB (8589930496 bytes)
no label, UUID=d3b72e81-c0fc-49fa-9704-cbbaba3822fc

# swapon /mnt/sandisk/swapfile

# free -h
total used free shared buff/cache available
Mem: 7,7Gi 1,6Gi 4,8Gi 167Mi 1,3Gi 5,7Gi
Swap: 8,0Gi 0B 8,0Gi

- EOT -

2020-05-30 13:41:15

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

> Time to experience with ZRAM :-).

I switched over from swap-file to zramswap.

And I can definitely say, my last kernel w/o your patchset does not
show the symptoms.

# cat systemd-analyze-time_5.7.0-rc7-2-amd64-clang_2nd-try.txt
Startup finished in 6.129s (kernel) + 44.192s (userspace) = 50.322s
graphical.target reached after 44.168s in userspace

# cat systemd-analyze-blame_5.7.0-rc7-2-amd64-clang_2nd-try.txt
24.050s udisks2.service
23.711s accounts-daemon.service
18.615s dev-sdc2.device
17.119s polkit.service
16.980s avahi-daemon.service
16.879s NetworkManager.service
16.112s rtkit-daemon.service
15.126s switcheroo-control.service
15.117s wpa_supplicant.service
15.105s systemd-logind.service
14.475s NetworkManager-wait-online.service
14.258s smartmontools.service
13.161s zramswap.service
9.522s rsyslog.service
8.337s gpm.service
6.026s packagekit.service
5.871s ModemManager.service
5.746s networking.service
5.383s e2scrub_reap.service
3.960s systemd-udevd.service
3.396s apparmor.service
3.231s exim4.service
2.795s systemd-journal-flush.service
2.359s alsa-restore.service
2.186s systemd-rfkill.service
1.878s atd.service
1.164s keyboard-setup.service
1.098s bluetooth.service
1.089s systemd-tmpfiles-setup.service
1.021s pppd-dns.service
968ms systemd-backlight@backlight:intel_backlight.service
964ms upower.service
937ms binfmt-support.service
873ms systemd-modules-load.service
849ms systemd-sysusers.service
845ms systemd-journald.service
683ms systemd-timesyncd.service
676ms [email protected]
641ms systemd-udev-trigger.service
620ms dev-hugepages.mount
618ms dev-mqueue.mount
618ms sys-kernel-debug.mount
617ms sys-kernel-tracing.mount
501ms ifupdown-wait-online.service
434ms systemd-sysctl.service
419ms systemd-random-seed.service
413ms systemd-tmpfiles-setup-dev.service
405ms [email protected]
389ms systemd-remount-fs.service
383ms console-setup.service
301ms kmod-static-nodes.service
181ms proc-sys-fs-binfmt_misc.mount
174ms systemd-update-utmp.service
85ms systemd-user-sessions.service
22ms [email protected]
19ms systemd-update-utmp-runlevel.service
5ms ifupdown-pre.service
4ms sys-fs-fuse-connections.mount

[ /etc/zramswap.conf ]

ZRAM_SIZE_PERCENT=20
# ZSTD support for ZRAM and ZSWAP requires Linux >= 5.7-rc+. -dileks
ZRAM_COMPRESSION_ALGO=zstd

[ /etc/fstab ]

# /etc/fstab: static file system information.
#
# Use 'blkid' to print the universally unique identifier for a
# device; this may be used with UUID= as a more robust way to name devices
# that works even if disks are added and removed. See fstab(5).
#
# <file system> || <mount point> || <type> || <options> || <dump> || <pass>
#
# Root-FS (here: /dev/sdc2)
UUID=<myUUID> / ext4 errors=remount-ro 0 1
# SWAP (here: /dev/zram0)
# Zram-based swap (compressed RAM block devices), for details see
zramswap.service and zramswap.conf

# dmesg | egrep 'zram|zswap'
[ 1.041958] zswap: loaded using pool zstd/zbud
[ 29.569355] zram: Added device: zram0
[ 29.581631] zram0: detected capacity change from 0 to 1647824896
[ 30.562279] Adding 1609200k swap on /dev/zram0. Priority:100
extents:1 across:1609200k SSFS

# cat /sys/devices/virtual/block/zram0/comp_algorithm
lzo lzo-rle lz4 lz4hc [zstd]

# swapon --show
NAME TYPE SIZE USED PRIO
/dev/zram0 partition 1,5G 0B 100

# cat /proc/swaps
Filename Type Size Used Priority
/dev/zram0 partition 1609200 0 100

If you have any ideas let me know.

Ah, I see there is async-buffered.6.

- Sedat -

[1] https://aur.archlinux.org/packages/zramswap/
[2] https://aur.archlinux.org/cgit/aur.git/tree/zramswap.conf?h=zramswap
[3] https://aur.archlinux.org/cgit/aur.git/tree/zramswap.service?h=zramswap

2020-05-30 18:59:28

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Here are the numbers with your patchset:

# cat systemd-analyze-time_5.7.0-rc7-4-amd64-clang_2nd-try.txt
Startup finished in 7.229s (kernel) + 1min 18.304s (userspace) = 1min 25.534s
graphical.target reached after 1min 18.286s in userspace

# cat systemd-analyze-blame_5.7.0-rc7-4-amd64-clang_2nd-try.txt
53.727s udisks2.service
47.043s accounts-daemon.service
40.252s NetworkManager.service
38.916s polkit.service
38.730s smartmontools.service
36.970s zramswap.service
35.556s dev-sdc2.device
30.307s switcheroo-control.service
29.179s wpa_supplicant.service
29.051s avahi-daemon.service
28.817s rtkit-daemon.service
27.697s systemd-logind.service
23.913s rsyslog.service
22.616s NetworkManager-wait-online.service
22.357s e2scrub_reap.service
19.579s gpm.service
14.879s ModemManager.service
14.126s packagekit.service
14.047s networking.service
6.137s alsa-restore.service
5.330s exim4.service
4.331s systemd-udevd.service
3.954s apparmor.service
2.606s atd.service
2.342s [email protected]
2.046s systemd-journal-flush.service
1.519s bluetooth.service
1.459s systemd-journald.service
1.386s systemd-udev-trigger.service
1.271s systemd-modules-load.service
1.210s keyboard-setup.service
1.136s systemd-sysusers.service
930ms upower.service
896ms pppd-dns.service
826ms systemd-tmpfiles-setup.service
807ms dev-hugepages.mount
807ms dev-mqueue.mount
806ms sys-kernel-debug.mount
806ms sys-kernel-tracing.mount
712ms ifupdown-wait-online.service
617ms systemd-remount-fs.service
588ms systemd-timesyncd.service
509ms binfmt-support.service
506ms systemd-backlight@backlight:intel_backlight.service
497ms systemd-random-seed.service
490ms systemd-rfkill.service
376ms [email protected]
343ms systemd-tmpfiles-setup-dev.service
305ms console-setup.service
303ms systemd-update-utmp.service
300ms systemd-user-sessions.service
295ms kmod-static-nodes.service
267ms systemd-sysctl.service
71ms proc-sys-fs-binfmt_misc.mount
19ms [email protected]
13ms systemd-update-utmp-runlevel.service
5ms sys-fs-fuse-connections.mount
4ms ifupdown-pre.service

[ diffconfig ]

$ scripts/diffconfig /boot/config-5.7.0-rc7-2-amd64-clang
/boot/config-5.7.0-rc7-4-amd64-clang
BUILD_SALT "5.7.0-rc7-2-amd64-clang" -> "5.7.0-rc7-4-amd64-clang"
+BLK_INLINE_ENCRYPTION n

- Sedat -


- Sedat -

2020-05-31 02:01:59

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 5/30/20 12:57 PM, Sedat Dilek wrote:
> Here are the numbers with your patchset:
>
> # cat systemd-analyze-time_5.7.0-rc7-4-amd64-clang_2nd-try.txt
> Startup finished in 7.229s (kernel) + 1min 18.304s (userspace) = 1min 25.534s
> graphical.target reached after 1min 18.286s in userspace

Can you see if this makes a difference?

diff --git a/include/linux/blk_types.h b/include/linux/blk_types.h
index c296463c15eb..ccb895f911b1 100644
--- a/include/linux/blk_types.h
+++ b/include/linux/blk_types.h
@@ -374,8 +374,7 @@ enum req_flag_bits {
#define REQ_INTEGRITY (1ULL << __REQ_INTEGRITY)
#define REQ_FUA (1ULL << __REQ_FUA)
#define REQ_PREFLUSH (1ULL << __REQ_PREFLUSH)
-#define REQ_RAHEAD \
- ((1ULL << __REQ_RAHEAD) | (1ULL << __REQ_NOWAIT))
+#define REQ_RAHEAD (1ULL << __REQ_RAHEAD)
#define REQ_BACKGROUND (1ULL << __REQ_BACKGROUND)
#define REQ_NOWAIT (1ULL << __REQ_NOWAIT)
#define REQ_CGROUP_PUNT (1ULL << __REQ_CGROUP_PUNT)

--
Jens Axboe

2020-05-31 07:07:26

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Sun, May 31, 2020 at 3:57 AM Jens Axboe <[email protected]> wrote:
>
> On 5/30/20 12:57 PM, Sedat Dilek wrote:
> > Here are the numbers with your patchset:
> >
> > # cat systemd-analyze-time_5.7.0-rc7-4-amd64-clang_2nd-try.txt
> > Startup finished in 7.229s (kernel) + 1min 18.304s (userspace) = 1min 25.534s
> > graphical.target reached after 1min 18.286s in userspace
>
> Can you see if this makes a difference?
>
> diff --git a/include/linux/blk_types.h b/include/linux/blk_types.h
> index c296463c15eb..ccb895f911b1 100644
> --- a/include/linux/blk_types.h
> +++ b/include/linux/blk_types.h
> @@ -374,8 +374,7 @@ enum req_flag_bits {
> #define REQ_INTEGRITY (1ULL << __REQ_INTEGRITY)
> #define REQ_FUA (1ULL << __REQ_FUA)
> #define REQ_PREFLUSH (1ULL << __REQ_PREFLUSH)
> -#define REQ_RAHEAD \
> - ((1ULL << __REQ_RAHEAD) | (1ULL << __REQ_NOWAIT))
> +#define REQ_RAHEAD (1ULL << __REQ_RAHEAD)
> #define REQ_BACKGROUND (1ULL << __REQ_BACKGROUND)
> #define REQ_NOWAIT (1ULL << __REQ_NOWAIT)
> #define REQ_CGROUP_PUNT (1ULL << __REQ_CGROUP_PUNT)
>

Looks good!

With your patch I now get...

# cat systemd-analyze-time_5.7.0-rc7-6-amd64-clang.txt
Startup finished in 6.199s (kernel) + 45.143s (userspace) = 51.343s
graphical.target reached after 45.123s in userspace

# cat systemd-analyze-blame_5.7.0-rc7-6-amd64-clang.txt
25.029s udisks2.service
24.582s accounts-daemon.service
19.257s dev-sdc2.device
18.016s polkit.service
17.857s avahi-daemon.service
17.730s NetworkManager.service
17.433s rtkit-daemon.service
16.558s switcheroo-control.service
16.547s wpa_supplicant.service
16.538s systemd-logind.service
16.078s smartmontools.service
14.982s fwupd-refresh.service
14.580s NetworkManager-wait-online.service
13.630s zramswap.service
11.286s fwupd.service
8.153s rsyslog.service
6.858s gpm.service
6.835s e2scrub_reap.service
6.449s ModemManager.service
6.439s networking.service
5.881s packagekit.service
3.696s systemd-udevd.service
3.322s apparmor.service
3.277s exim4.service
2.910s alsa-restore.service
1.611s systemd-tmpfiles-setup.service
1.540s bluetooth.service
1.448s systemd-journal-flush.service
1.353s keyboard-setup.service
1.322s atd.service
1.239s systemd-modules-load.service
1.216s binfmt-support.service
1.060s [email protected]
994ms systemd-journald.service
983ms upower.service
937ms systemd-sysusers.service
914ms ifupdown-wait-online.service
904ms pppd-dns.service
710ms systemd-udev-trigger.service
666ms dev-hugepages.mount
650ms dev-mqueue.mount
649ms sys-kernel-debug.mount
647ms sys-kernel-tracing.mount
607ms console-setup.service
590ms systemd-tmpfiles-setup-dev.service
570ms systemd-timesyncd.service
508ms systemd-random-seed.service
461ms systemd-backlight@backlight:intel_backlight.service
433ms [email protected]
422ms systemd-remount-fs.service
365ms systemd-sysctl.service
279ms kmod-static-nodes.service
246ms proc-sys-fs-binfmt_misc.mount
202ms systemd-rfkill.service
130ms systemd-user-sessions.service
104ms systemd-update-utmp.service
25ms [email protected]
14ms systemd-update-utmp-runlevel.service
6ms sys-fs-fuse-connections.mount
4ms ifupdown-pre.service

# cat systemd-analyze-critical-chain_5.7.0-rc7-6-amd64-clang.txt
The time when unit became active or started is printed after the "@" character.
The time the unit took to start is printed after the "+" character.

graphical.target @45.123s
└─multi-user.target @45.122s
└─exim4.service @41.842s +3.277s
└─network-online.target @41.840s
└─NetworkManager-wait-online.service @27.259s +14.580s
└─NetworkManager.service @9.522s +17.730s
└─dbus.service @9.514s
└─basic.target @9.328s
└─sockets.target @9.327s
└─dbus.socket @9.327s
└─sysinit.target @9.202s

└─systemd-backlight@backlight:intel_backlight.service @21.927s +461ms
└─system-systemd\x2dbacklight.slice @21.925s
└─system.slice @2.529s
└─-.slice @2.529s

Will you send a separate patch on this?

Feel free to add:
Reported-by: Sedat Dilek <[email protected]>
Tested-by: Sedat Dilek <[email protected]>

My kernel-config and dmesg-output are attached.

Thanks Jens!

- Sedat -


Attachments:
dmesg-T-5.7.0-rc7-6-amd64-clang.txt (69.52 kB)
config-5.7.0-rc7-6-amd64-clang (224.17 kB)
Download all attachments

2020-05-31 07:15:01

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Sun, May 31, 2020 at 9:04 AM Sedat Dilek <[email protected]> wrote:
>
> On Sun, May 31, 2020 at 3:57 AM Jens Axboe <[email protected]> wrote:
> >
> > On 5/30/20 12:57 PM, Sedat Dilek wrote:
> > > Here are the numbers with your patchset:
> > >
> > > # cat systemd-analyze-time_5.7.0-rc7-4-amd64-clang_2nd-try.txt
> > > Startup finished in 7.229s (kernel) + 1min 18.304s (userspace) = 1min 25.534s
> > > graphical.target reached after 1min 18.286s in userspace
> >
> > Can you see if this makes a difference?
> >
> > diff --git a/include/linux/blk_types.h b/include/linux/blk_types.h
> > index c296463c15eb..ccb895f911b1 100644
> > --- a/include/linux/blk_types.h
> > +++ b/include/linux/blk_types.h
> > @@ -374,8 +374,7 @@ enum req_flag_bits {
> > #define REQ_INTEGRITY (1ULL << __REQ_INTEGRITY)
> > #define REQ_FUA (1ULL << __REQ_FUA)
> > #define REQ_PREFLUSH (1ULL << __REQ_PREFLUSH)
> > -#define REQ_RAHEAD \
> > - ((1ULL << __REQ_RAHEAD) | (1ULL << __REQ_NOWAIT))
> > +#define REQ_RAHEAD (1ULL << __REQ_RAHEAD)
> > #define REQ_BACKGROUND (1ULL << __REQ_BACKGROUND)
> > #define REQ_NOWAIT (1ULL << __REQ_NOWAIT)
> > #define REQ_CGROUP_PUNT (1ULL << __REQ_CGROUP_PUNT)
> >
>
> Looks good!
>
> With your patch I now get...
>
> # cat systemd-analyze-time_5.7.0-rc7-6-amd64-clang.txt
> Startup finished in 6.199s (kernel) + 45.143s (userspace) = 51.343s
> graphical.target reached after 45.123s in userspace
>
> # cat systemd-analyze-blame_5.7.0-rc7-6-amd64-clang.txt
> 25.029s udisks2.service
> 24.582s accounts-daemon.service
> 19.257s dev-sdc2.device
> 18.016s polkit.service
> 17.857s avahi-daemon.service
> 17.730s NetworkManager.service
> 17.433s rtkit-daemon.service
> 16.558s switcheroo-control.service
> 16.547s wpa_supplicant.service
> 16.538s systemd-logind.service
> 16.078s smartmontools.service
> 14.982s fwupd-refresh.service
> 14.580s NetworkManager-wait-online.service
> 13.630s zramswap.service
> 11.286s fwupd.service
> 8.153s rsyslog.service
> 6.858s gpm.service
> 6.835s e2scrub_reap.service
> 6.449s ModemManager.service
> 6.439s networking.service
> 5.881s packagekit.service
> 3.696s systemd-udevd.service
> 3.322s apparmor.service
> 3.277s exim4.service
> 2.910s alsa-restore.service
> 1.611s systemd-tmpfiles-setup.service
> 1.540s bluetooth.service
> 1.448s systemd-journal-flush.service
> 1.353s keyboard-setup.service
> 1.322s atd.service
> 1.239s systemd-modules-load.service
> 1.216s binfmt-support.service
> 1.060s [email protected]
> 994ms systemd-journald.service
> 983ms upower.service
> 937ms systemd-sysusers.service
> 914ms ifupdown-wait-online.service
> 904ms pppd-dns.service
> 710ms systemd-udev-trigger.service
> 666ms dev-hugepages.mount
> 650ms dev-mqueue.mount
> 649ms sys-kernel-debug.mount
> 647ms sys-kernel-tracing.mount
> 607ms console-setup.service
> 590ms systemd-tmpfiles-setup-dev.service
> 570ms systemd-timesyncd.service
> 508ms systemd-random-seed.service
> 461ms systemd-backlight@backlight:intel_backlight.service
> 433ms [email protected]
> 422ms systemd-remount-fs.service
> 365ms systemd-sysctl.service
> 279ms kmod-static-nodes.service
> 246ms proc-sys-fs-binfmt_misc.mount
> 202ms systemd-rfkill.service
> 130ms systemd-user-sessions.service
> 104ms systemd-update-utmp.service
> 25ms [email protected]
> 14ms systemd-update-utmp-runlevel.service
> 6ms sys-fs-fuse-connections.mount
> 4ms ifupdown-pre.service
>
> # cat systemd-analyze-critical-chain_5.7.0-rc7-6-amd64-clang.txt
> The time when unit became active or started is printed after the "@" character.
> The time the unit took to start is printed after the "+" character.
>
> graphical.target @45.123s
> └─multi-user.target @45.122s
> └─exim4.service @41.842s +3.277s
> └─network-online.target @41.840s
> └─NetworkManager-wait-online.service @27.259s +14.580s
> └─NetworkManager.service @9.522s +17.730s
> └─dbus.service @9.514s
> └─basic.target @9.328s
> └─sockets.target @9.327s
> └─dbus.socket @9.327s
> └─sysinit.target @9.202s
>
> └─systemd-backlight@backlight:intel_backlight.service @21.927s +461ms
> └─system-systemd\x2dbacklight.slice @21.925s
> └─system.slice @2.529s
> └─-.slice @2.529s
>
> Will you send a separate patch on this?
>
> Feel free to add:
> Reported-by: Sedat Dilek <[email protected]>
> Tested-by: Sedat Dilek <[email protected]>
>
> My kernel-config and dmesg-output are attached.
>
> Thanks Jens!
>

2nd-try (reboot into system with activated WLAN before startup)...

# systemd-analyze --no-pager time
Startup finished in 6.044s (kernel) + 41.370s (userspace) = 47.414s
graphical.target reached after 40.431s in userspace

# systemd-analyze --no-pager blame | head -20
24.674s udisks2.service
24.348s accounts-daemon.service
19.504s dev-sdc2.device
18.677s polkit.service
18.283s avahi-daemon.service
18.231s NetworkManager.service
17.918s rtkit-daemon.service
16.948s switcheroo-control.service
16.932s wpa_supplicant.service
16.928s systemd-logind.service
15.335s smartmontools.service
14.063s zramswap.service
11.507s NetworkManager-wait-online.service
11.279s rsyslog.service
7.490s gpm.service
6.556s packagekit.service
6.506s networking.service
6.446s e2scrub_reap.service
5.404s ModemManager.service
3.329s systemd-udevd.service

# systemd-analyze critical-chain
The time when unit became active or started is printed after the "@" character.
The time the unit took to start is printed after the "+" character.

graphical.target @40.431s
└─multi-user.target @40.431s
└─exim4.service @38.726s +1.701s
└─network-online.target @38.723s
└─NetworkManager-wait-online.service @27.215s +11.507s
└─NetworkManager.service @8.978s +18.231s
└─dbus.service @8.971s
└─basic.target @8.862s
└─sockets.target @8.861s
└─dbus.socket @8.861s
└─sysinit.target @8.723s

└─systemd-backlight@backlight:intel_backlight.service @20.555s +812ms
└─system-systemd\x2dbacklight.slice @19.665s
└─system.slice @2.473s
└─-.slice @2.473s

Nice!

- Sedat -

2020-06-01 13:37:53

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi Jens,

with Linux v5.7 final I switched to linux-block.git/for-next and reverted...

"block: read-ahead submission should imply no-wait as well"

...and see no boot-slowdowns.

Regards,
- Sedat -

2020-06-01 14:07:41

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/1/20 7:35 AM, Sedat Dilek wrote:
> Hi Jens,
>
> with Linux v5.7 final I switched to linux-block.git/for-next and reverted...
>
> "block: read-ahead submission should imply no-wait as well"
>
> ...and see no boot-slowdowns.

Can you try with these patches applied instead? Or pull my async-readahead
branch from the same location.

--
Jens Axboe


Attachments:
0006-Revert-block-read-ahead-submission-should-imply-no-w.patch (1.02 kB)
0005-fs-make-mpage_readpages-take-a-struct-kiocb-argument.patch (10.75 kB)
0004-iomap-set-REQ_NOWAIT-on-bio-if-IOCB_NOWAIT-is-set-in.patch (3.79 kB)
0003-mm-make-generic_file_buffered_read-use-iocb-read-ahe.patch (1.20 kB)
0002-mm-provide-read-ahead-helpers-that-take-a-struct-kio.patch (4.08 kB)
0001-fs-make-aops-readpages-take-kiocb-argument.patch (28.36 kB)
Download all attachments

2020-06-01 14:17:22

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/1/20 8:13 AM, Sedat Dilek wrote:
> On Mon, Jun 1, 2020 at 4:04 PM Jens Axboe <[email protected]> wrote:
>>
>> On 6/1/20 7:35 AM, Sedat Dilek wrote:
>>> Hi Jens,
>>>
>>> with Linux v5.7 final I switched to linux-block.git/for-next and reverted...
>>>
>>> "block: read-ahead submission should imply no-wait as well"
>>>
>>> ...and see no boot-slowdowns.
>>
>> Can you try with these patches applied instead? Or pull my async-readahead
>> branch from the same location.
>>
>
> Yes, I can do that.
> I pulled from linux-block.git#async-readahead and will report later.
>
> Any specific testing desired by you?

Just do your boot timing test and see if it works, thanks.

--
Jens Axboe

2020-06-01 14:18:03

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Mon, Jun 1, 2020 at 4:04 PM Jens Axboe <[email protected]> wrote:
>
> On 6/1/20 7:35 AM, Sedat Dilek wrote:
> > Hi Jens,
> >
> > with Linux v5.7 final I switched to linux-block.git/for-next and reverted...
> >
> > "block: read-ahead submission should imply no-wait as well"
> >
> > ...and see no boot-slowdowns.
>
> Can you try with these patches applied instead? Or pull my async-readahead
> branch from the same location.
>

Yes, I can do that.
I pulled from linux-block.git#async-readahead and will report later.

Any specific testing desired by you?

- Sedat -

2020-06-01 14:20:53

by Matthew Wilcox

[permalink] [raw]
Subject: Re: [PATCH 03/12] mm: abstract out wake_page_match() from wake_page_function()

On Tue, May 26, 2020 at 01:51:14PM -0600, Jens Axboe wrote:
> +++ b/include/linux/pagemap.h
> @@ -456,6 +456,43 @@ static inline pgoff_t linear_page_index(struct vm_area_struct *vma,
> return pgoff;
> }
>
> +/* This has the same layout as wait_bit_key - see fs/cachefiles/rdwr.c */
> +struct wait_page_key {
> + struct page *page;
> + int bit_nr;
> + int page_match;
> +};

I know you only moved the struct, and the comment along with it, but now
that this struct is in pagemap.h, I think cachefiles should be updated
to use wait_page_key instead of wait_bit_key. Dave?

2020-06-01 14:29:04

by Matthew Wilcox

[permalink] [raw]
Subject: Re: [PATCH 04/12] mm: add support for async page locking

On Tue, May 26, 2020 at 01:51:15PM -0600, Jens Axboe wrote:
> +static int __wait_on_page_locked_async(struct page *page,
> + struct wait_page_queue *wait, bool set)
> +{
> + struct wait_queue_head *q = page_waitqueue(page);
> + int ret = 0;
> +
> + wait->page = page;
> + wait->bit_nr = PG_locked;
> +
> + spin_lock_irq(&q->lock);
> + if (set)
> + ret = !trylock_page(page);
> + else
> + ret = PageLocked(page);
> + if (ret) {
> + __add_wait_queue_entry_tail(q, &wait->wait);
> + SetPageWaiters(page);
> + if (set)
> + ret = !trylock_page(page);
> + else
> + ret = PageLocked(page);

Between the callers and this function, we actually look at PG_lock three
times; once in the caller, then after taking the spinlock, then after
adding ourselves to the waitqueue. I understand the first and third, but
is it really worth doing the second test? It feels unlikely to succeed
and only saves us setting PageWaiters.

2020-06-01 14:37:24

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/1/20 8:14 AM, Jens Axboe wrote:
> On 6/1/20 8:13 AM, Sedat Dilek wrote:
>> On Mon, Jun 1, 2020 at 4:04 PM Jens Axboe <[email protected]> wrote:
>>>
>>> On 6/1/20 7:35 AM, Sedat Dilek wrote:
>>>> Hi Jens,
>>>>
>>>> with Linux v5.7 final I switched to linux-block.git/for-next and reverted...
>>>>
>>>> "block: read-ahead submission should imply no-wait as well"
>>>>
>>>> ...and see no boot-slowdowns.
>>>
>>> Can you try with these patches applied instead? Or pull my async-readahead
>>> branch from the same location.
>>>
>>
>> Yes, I can do that.
>> I pulled from linux-block.git#async-readahead and will report later.
>>
>> Any specific testing desired by you?
>
> Just do your boot timing test and see if it works, thanks.

Actually, can you just re-test with the current async-buffered.6 branch?
I think the major surgery should wait for 5.9, we can do this a bit
easier without having to touch everything around us.

--
Jens Axboe

2020-06-01 14:48:48

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Mon, Jun 1, 2020 at 4:35 PM Jens Axboe <[email protected]> wrote:
>
> On 6/1/20 8:14 AM, Jens Axboe wrote:
> > On 6/1/20 8:13 AM, Sedat Dilek wrote:
> >> On Mon, Jun 1, 2020 at 4:04 PM Jens Axboe <[email protected]> wrote:
> >>>
> >>> On 6/1/20 7:35 AM, Sedat Dilek wrote:
> >>>> Hi Jens,
> >>>>
> >>>> with Linux v5.7 final I switched to linux-block.git/for-next and reverted...
> >>>>
> >>>> "block: read-ahead submission should imply no-wait as well"
> >>>>
> >>>> ...and see no boot-slowdowns.
> >>>
> >>> Can you try with these patches applied instead? Or pull my async-readahead
> >>> branch from the same location.
> >>>
> >>
> >> Yes, I can do that.
> >> I pulled from linux-block.git#async-readahead and will report later.
> >>
> >> Any specific testing desired by you?
> >
> > Just do your boot timing test and see if it works, thanks.
>
> Actually, can you just re-test with the current async-buffered.6 branch?
> I think the major surgery should wait for 5.9, we can do this a bit
> easier without having to touch everything around us.
>

With linux-block.git#async-readahead:

mycompiler -Wp,-MD,kernel/.sys.o.d -nostdinc -isystem
/home/dileks/src/llvm-toolchain/install/lib/clang/10.0.1rc1/include
-I./arch/x86/include -I./arch/x86/include/generated -I./include
-I./arch/x86/include/uapi -I./arch/x86/include/generated/uapi
-I./include/uapi -I./include/generated/uapi -include
./include/linux/kconfig.h -include ./include/linux/compiler_types.h
-D__KERNEL__ -Qunused-arguments -Wall -Wundef
-Werror=strict-prototypes -Wno-trigraphs -fno-strict-aliasing
-fno-common -fshort-wchar -fno-PIE
-Werror=implicit-function-declaration -Werror=implicit-int
-Wno-format-security -std=gnu89 -no-integrated-as
-Werror=unknown-warning-option -mno-sse -mno-mmx -mno-sse2 -mno-3dnow
-mno-avx -m64 -mno-80387 -mstack-alignment=8 -mtune=generic
-mno-red-zone -mcmodel=kernel -Wno-sign-compare
-fno-asynchronous-unwind-tables -mretpoline-external-thunk
-fno-delete-null-pointer-checks -Wno-address-of-packed-member -O2
-Wframe-larger-than=2048 -fstack-protector-strong
-Wno-format-invalid-specifier -Wno-gnu -mno-global-merge
-Wno-unused-const-variable -g -gz=zlib -pg -mfentry -DCC_USING_FENTRY
-Wdeclaration-after-statement -Wvla -Wno-pointer-sign
-Wno-array-bounds -fno-strict-overflow -fno-merge-all-constants
-fno-stack-check -Werror=date-time -Werror=incompatible-pointer-types
-fmacro-prefix-map=./= -fcf-protection=none -Wno-initializer-overrides
-Wno-format -Wno-sign-compare -Wno-format-zero-length
-Wno-tautological-constant-out-of-range-compare
-DKBUILD_MODFILE='"kernel/sys"' -DKBUILD_BASENAME='"sys"'
-DKBUILD_MODNAME='"sys"' -c -o kernel/sys.o kernel/sys.c
fs/9p/vfs_addr.c:112:4: error: use of undeclared identifier 'filp'
filp->private_data);
^
1 error generated.
make[5]: *** [scripts/Makefile.build:267: fs/9p/vfs_addr.o] Error 1
make[4]: *** [scripts/Makefile.build:488: fs/9p] Error 2
make[3]: *** [Makefile:1735: fs] Error 2
make[3]: *** Waiting for unfinished jobs....

I guess block.git#async-buffered.6 needs the same revert of "block:
read-ahead submission should imply no-wait as well".

- Sedat -

2020-06-01 14:51:12

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/1/20 8:43 AM, Sedat Dilek wrote:
> On Mon, Jun 1, 2020 at 4:35 PM Jens Axboe <[email protected]> wrote:
>>
>> On 6/1/20 8:14 AM, Jens Axboe wrote:
>>> On 6/1/20 8:13 AM, Sedat Dilek wrote:
>>>> On Mon, Jun 1, 2020 at 4:04 PM Jens Axboe <[email protected]> wrote:
>>>>>
>>>>> On 6/1/20 7:35 AM, Sedat Dilek wrote:
>>>>>> Hi Jens,
>>>>>>
>>>>>> with Linux v5.7 final I switched to linux-block.git/for-next and reverted...
>>>>>>
>>>>>> "block: read-ahead submission should imply no-wait as well"
>>>>>>
>>>>>> ...and see no boot-slowdowns.
>>>>>
>>>>> Can you try with these patches applied instead? Or pull my async-readahead
>>>>> branch from the same location.
>>>>>
>>>>
>>>> Yes, I can do that.
>>>> I pulled from linux-block.git#async-readahead and will report later.
>>>>
>>>> Any specific testing desired by you?
>>>
>>> Just do your boot timing test and see if it works, thanks.
>>
>> Actually, can you just re-test with the current async-buffered.6 branch?
>> I think the major surgery should wait for 5.9, we can do this a bit
>> easier without having to touch everything around us.
>>
>
> With linux-block.git#async-readahead:
>
> mycompiler -Wp,-MD,kernel/.sys.o.d -nostdinc -isystem
> /home/dileks/src/llvm-toolchain/install/lib/clang/10.0.1rc1/include
> -I./arch/x86/include -I./arch/x86/include/generated -I./include
> -I./arch/x86/include/uapi -I./arch/x86/include/generated/uapi
> -I./include/uapi -I./include/generated/uapi -include
> ./include/linux/kconfig.h -include ./include/linux/compiler_types.h
> -D__KERNEL__ -Qunused-arguments -Wall -Wundef
> -Werror=strict-prototypes -Wno-trigraphs -fno-strict-aliasing
> -fno-common -fshort-wchar -fno-PIE
> -Werror=implicit-function-declaration -Werror=implicit-int
> -Wno-format-security -std=gnu89 -no-integrated-as
> -Werror=unknown-warning-option -mno-sse -mno-mmx -mno-sse2 -mno-3dnow
> -mno-avx -m64 -mno-80387 -mstack-alignment=8 -mtune=generic
> -mno-red-zone -mcmodel=kernel -Wno-sign-compare
> -fno-asynchronous-unwind-tables -mretpoline-external-thunk
> -fno-delete-null-pointer-checks -Wno-address-of-packed-member -O2
> -Wframe-larger-than=2048 -fstack-protector-strong
> -Wno-format-invalid-specifier -Wno-gnu -mno-global-merge
> -Wno-unused-const-variable -g -gz=zlib -pg -mfentry -DCC_USING_FENTRY
> -Wdeclaration-after-statement -Wvla -Wno-pointer-sign
> -Wno-array-bounds -fno-strict-overflow -fno-merge-all-constants
> -fno-stack-check -Werror=date-time -Werror=incompatible-pointer-types
> -fmacro-prefix-map=./= -fcf-protection=none -Wno-initializer-overrides
> -Wno-format -Wno-sign-compare -Wno-format-zero-length
> -Wno-tautological-constant-out-of-range-compare
> -DKBUILD_MODFILE='"kernel/sys"' -DKBUILD_BASENAME='"sys"'
> -DKBUILD_MODNAME='"sys"' -c -o kernel/sys.o kernel/sys.c
> fs/9p/vfs_addr.c:112:4: error: use of undeclared identifier 'filp'
> filp->private_data);
> ^
> 1 error generated.
> make[5]: *** [scripts/Makefile.build:267: fs/9p/vfs_addr.o] Error 1
> make[4]: *** [scripts/Makefile.build:488: fs/9p] Error 2
> make[3]: *** [Makefile:1735: fs] Error 2
> make[3]: *** Waiting for unfinished jobs....
>
> I guess block.git#async-buffered.6 needs the same revert of "block:
> read-ahead submission should imply no-wait as well".

Sorry, forgot to push out the updated version. But as per previous
email, I think that major work should wait. Just try the updated
async-buffered.6 branch instead.

--
Jens Axboe

2020-06-01 14:56:26

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Mon, Jun 1, 2020 at 4:46 PM Jens Axboe <[email protected]> wrote:
>
> On 6/1/20 8:43 AM, Sedat Dilek wrote:
> > On Mon, Jun 1, 2020 at 4:35 PM Jens Axboe <[email protected]> wrote:
> >>
> >> On 6/1/20 8:14 AM, Jens Axboe wrote:
> >>> On 6/1/20 8:13 AM, Sedat Dilek wrote:
> >>>> On Mon, Jun 1, 2020 at 4:04 PM Jens Axboe <[email protected]> wrote:
> >>>>>
> >>>>> On 6/1/20 7:35 AM, Sedat Dilek wrote:
> >>>>>> Hi Jens,
> >>>>>>
> >>>>>> with Linux v5.7 final I switched to linux-block.git/for-next and reverted...
> >>>>>>
> >>>>>> "block: read-ahead submission should imply no-wait as well"
> >>>>>>
> >>>>>> ...and see no boot-slowdowns.
> >>>>>
> >>>>> Can you try with these patches applied instead? Or pull my async-readahead
> >>>>> branch from the same location.
> >>>>>
> >>>>
> >>>> Yes, I can do that.
> >>>> I pulled from linux-block.git#async-readahead and will report later.
> >>>>
> >>>> Any specific testing desired by you?
> >>>
> >>> Just do your boot timing test and see if it works, thanks.
> >>
> >> Actually, can you just re-test with the current async-buffered.6 branch?
> >> I think the major surgery should wait for 5.9, we can do this a bit
> >> easier without having to touch everything around us.
> >>
> >
> > With linux-block.git#async-readahead:
> >
> > mycompiler -Wp,-MD,kernel/.sys.o.d -nostdinc -isystem
> > /home/dileks/src/llvm-toolchain/install/lib/clang/10.0.1rc1/include
> > -I./arch/x86/include -I./arch/x86/include/generated -I./include
> > -I./arch/x86/include/uapi -I./arch/x86/include/generated/uapi
> > -I./include/uapi -I./include/generated/uapi -include
> > ./include/linux/kconfig.h -include ./include/linux/compiler_types.h
> > -D__KERNEL__ -Qunused-arguments -Wall -Wundef
> > -Werror=strict-prototypes -Wno-trigraphs -fno-strict-aliasing
> > -fno-common -fshort-wchar -fno-PIE
> > -Werror=implicit-function-declaration -Werror=implicit-int
> > -Wno-format-security -std=gnu89 -no-integrated-as
> > -Werror=unknown-warning-option -mno-sse -mno-mmx -mno-sse2 -mno-3dnow
> > -mno-avx -m64 -mno-80387 -mstack-alignment=8 -mtune=generic
> > -mno-red-zone -mcmodel=kernel -Wno-sign-compare
> > -fno-asynchronous-unwind-tables -mretpoline-external-thunk
> > -fno-delete-null-pointer-checks -Wno-address-of-packed-member -O2
> > -Wframe-larger-than=2048 -fstack-protector-strong
> > -Wno-format-invalid-specifier -Wno-gnu -mno-global-merge
> > -Wno-unused-const-variable -g -gz=zlib -pg -mfentry -DCC_USING_FENTRY
> > -Wdeclaration-after-statement -Wvla -Wno-pointer-sign
> > -Wno-array-bounds -fno-strict-overflow -fno-merge-all-constants
> > -fno-stack-check -Werror=date-time -Werror=incompatible-pointer-types
> > -fmacro-prefix-map=./= -fcf-protection=none -Wno-initializer-overrides
> > -Wno-format -Wno-sign-compare -Wno-format-zero-length
> > -Wno-tautological-constant-out-of-range-compare
> > -DKBUILD_MODFILE='"kernel/sys"' -DKBUILD_BASENAME='"sys"'
> > -DKBUILD_MODNAME='"sys"' -c -o kernel/sys.o kernel/sys.c
> > fs/9p/vfs_addr.c:112:4: error: use of undeclared identifier 'filp'
> > filp->private_data);
> > ^
> > 1 error generated.
> > make[5]: *** [scripts/Makefile.build:267: fs/9p/vfs_addr.o] Error 1
> > make[4]: *** [scripts/Makefile.build:488: fs/9p] Error 2
> > make[3]: *** [Makefile:1735: fs] Error 2
> > make[3]: *** Waiting for unfinished jobs....
> >
> > I guess block.git#async-buffered.6 needs the same revert of "block:
> > read-ahead submission should imply no-wait as well".
>
> Sorry, forgot to push out the updated version. But as per previous
> email, I think that major work should wait. Just try the updated
> async-buffered.6 branch instead.
>

Hehe, I was looking at git.k.o URLs.

I just pulled from
<https://git.kernel.dk/cgit/linux-block/log/?h=async-buffered.6>.

- Sedat -

2020-06-01 17:20:17

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCH 04/12] mm: add support for async page locking

On 6/1/20 8:26 AM, Matthew Wilcox wrote:
> On Tue, May 26, 2020 at 01:51:15PM -0600, Jens Axboe wrote:
>> +static int __wait_on_page_locked_async(struct page *page,
>> + struct wait_page_queue *wait, bool set)
>> +{
>> + struct wait_queue_head *q = page_waitqueue(page);
>> + int ret = 0;
>> +
>> + wait->page = page;
>> + wait->bit_nr = PG_locked;
>> +
>> + spin_lock_irq(&q->lock);
>> + if (set)
>> + ret = !trylock_page(page);
>> + else
>> + ret = PageLocked(page);
>> + if (ret) {
>> + __add_wait_queue_entry_tail(q, &wait->wait);
>> + SetPageWaiters(page);
>> + if (set)
>> + ret = !trylock_page(page);
>> + else
>> + ret = PageLocked(page);
>
> Between the callers and this function, we actually look at PG_lock three
> times; once in the caller, then after taking the spinlock, then after
> adding ourselves to the waitqueue. I understand the first and third, but
> is it really worth doing the second test? It feels unlikely to succeed
> and only saves us setting PageWaiters.

That's probably true, and we can skip the 2nd one. I'll make the change.

--
Jens Axboe

2020-06-01 20:21:11

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Mon, Jun 1, 2020 at 4:46 PM Jens Axboe <[email protected]> wrote:
>
> On 6/1/20 8:43 AM, Sedat Dilek wrote:
> > On Mon, Jun 1, 2020 at 4:35 PM Jens Axboe <[email protected]> wrote:
> >>
> >> On 6/1/20 8:14 AM, Jens Axboe wrote:
> >>> On 6/1/20 8:13 AM, Sedat Dilek wrote:
> >>>> On Mon, Jun 1, 2020 at 4:04 PM Jens Axboe <[email protected]> wrote:
> >>>>>
> >>>>> On 6/1/20 7:35 AM, Sedat Dilek wrote:
> >>>>>> Hi Jens,
> >>>>>>
> >>>>>> with Linux v5.7 final I switched to linux-block.git/for-next and reverted...
> >>>>>>
> >>>>>> "block: read-ahead submission should imply no-wait as well"
> >>>>>>
> >>>>>> ...and see no boot-slowdowns.
> >>>>>
> >>>>> Can you try with these patches applied instead? Or pull my async-readahead
> >>>>> branch from the same location.
> >>>>>
> >>>>
> >>>> Yes, I can do that.
> >>>> I pulled from linux-block.git#async-readahead and will report later.
> >>>>
> >>>> Any specific testing desired by you?
> >>>
> >>> Just do your boot timing test and see if it works, thanks.
> >>
> >> Actually, can you just re-test with the current async-buffered.6 branch?
> >> I think the major surgery should wait for 5.9, we can do this a bit
> >> easier without having to touch everything around us.
> >>
> >
> > With linux-block.git#async-readahead:
> >
> > mycompiler -Wp,-MD,kernel/.sys.o.d -nostdinc -isystem
> > /home/dileks/src/llvm-toolchain/install/lib/clang/10.0.1rc1/include
> > -I./arch/x86/include -I./arch/x86/include/generated -I./include
> > -I./arch/x86/include/uapi -I./arch/x86/include/generated/uapi
> > -I./include/uapi -I./include/generated/uapi -include
> > ./include/linux/kconfig.h -include ./include/linux/compiler_types.h
> > -D__KERNEL__ -Qunused-arguments -Wall -Wundef
> > -Werror=strict-prototypes -Wno-trigraphs -fno-strict-aliasing
> > -fno-common -fshort-wchar -fno-PIE
> > -Werror=implicit-function-declaration -Werror=implicit-int
> > -Wno-format-security -std=gnu89 -no-integrated-as
> > -Werror=unknown-warning-option -mno-sse -mno-mmx -mno-sse2 -mno-3dnow
> > -mno-avx -m64 -mno-80387 -mstack-alignment=8 -mtune=generic
> > -mno-red-zone -mcmodel=kernel -Wno-sign-compare
> > -fno-asynchronous-unwind-tables -mretpoline-external-thunk
> > -fno-delete-null-pointer-checks -Wno-address-of-packed-member -O2
> > -Wframe-larger-than=2048 -fstack-protector-strong
> > -Wno-format-invalid-specifier -Wno-gnu -mno-global-merge
> > -Wno-unused-const-variable -g -gz=zlib -pg -mfentry -DCC_USING_FENTRY
> > -Wdeclaration-after-statement -Wvla -Wno-pointer-sign
> > -Wno-array-bounds -fno-strict-overflow -fno-merge-all-constants
> > -fno-stack-check -Werror=date-time -Werror=incompatible-pointer-types
> > -fmacro-prefix-map=./= -fcf-protection=none -Wno-initializer-overrides
> > -Wno-format -Wno-sign-compare -Wno-format-zero-length
> > -Wno-tautological-constant-out-of-range-compare
> > -DKBUILD_MODFILE='"kernel/sys"' -DKBUILD_BASENAME='"sys"'
> > -DKBUILD_MODNAME='"sys"' -c -o kernel/sys.o kernel/sys.c
> > fs/9p/vfs_addr.c:112:4: error: use of undeclared identifier 'filp'
> > filp->private_data);
> > ^
> > 1 error generated.
> > make[5]: *** [scripts/Makefile.build:267: fs/9p/vfs_addr.o] Error 1
> > make[4]: *** [scripts/Makefile.build:488: fs/9p] Error 2
> > make[3]: *** [Makefile:1735: fs] Error 2
> > make[3]: *** Waiting for unfinished jobs....
> >
> > I guess block.git#async-buffered.6 needs the same revert of "block:
> > read-ahead submission should imply no-wait as well".
>
> Sorry, forgot to push out the updated version. But as per previous
> email, I think that major work should wait. Just try the updated
> async-buffered.6 branch instead.
>
> --
> Jens Axboe
>

Looks good.

$ cat systemd-analyze_time_5.7.0-2-amd64-clang.txt
Startup finished in 6.054s (kernel) + 42.626s (userspace) = 48.680s
graphical.target reached after 42.605s in userspace

$ cat systemd-analyze_blame_5.7.0-2-amd64-clang.txt
25.777s udisks2.service
24.527s accounts-daemon.service
18.228s polkit.service
17.593s avahi-daemon.service
17.554s NetworkManager.service
17.526s rtkit-daemon.service
17.417s dev-sdc2.device
17.200s switcheroo-control.service
17.186s wpa_supplicant.service
17.174s systemd-logind.service
15.714s smartmontools.service
14.255s zramswap.service
13.495s NetworkManager-wait-online.service
11.092s rsyslog.service
8.900s networking.service
7.817s gpm.service
6.571s ModemManager.service
6.403s packagekit.service
5.701s e2scrub_reap.service
3.714s alsa-restore.service
3.509s systemd-udevd.service
3.286s apparmor.service
3.028s atd.service
2.361s exim4.service
1.532s systemd-tmpfiles-setup.service
1.213s systemd-journal-flush.service
1.163s keyboard-setup.service
1.011s bluetooth.service
1.008s binfmt-support.service
995ms systemd-modules-load.service
987ms pppd-dns.service
919ms systemd-journald.service
875ms upower.service
856ms systemd-sysusers.service
750ms systemd-udev-trigger.service
728ms dev-hugepages.mount
725ms dev-mqueue.mount
724ms sys-kernel-debug.mount
723ms sys-kernel-tracing.mount
648ms [email protected]
640ms ifupdown-wait-online.service
609ms systemd-timesyncd.service
579ms systemd-tmpfiles-setup-dev.service
521ms systemd-backlight@backlight:intel_backlight.service
477ms systemd-remount-fs.service
454ms systemd-sysctl.service
441ms systemd-random-seed.service
389ms console-setup.service
306ms systemd-rfkill.service
266ms kmod-static-nodes.service
244ms [email protected]
150ms proc-sys-fs-binfmt_misc.mount
131ms systemd-update-utmp.service
98ms systemd-user-sessions.service
26ms [email protected]
15ms systemd-update-utmp-runlevel.service
6ms sys-fs-fuse-connections.mount
3ms ifupdown-pre.service

$ cat systemd-analyze_critical-chain_5.7.0-2-amd64-clang.txt
The time when unit became active or started is printed after the "@" character.
The time the unit took to start is printed after the "+" character.

graphical.target @42.605s
└─multi-user.target @42.604s
└─exim4.service @40.240s +2.361s
└─network-online.target @40.237s
└─NetworkManager-wait-online.service @26.740s +13.495s
└─NetworkManager.service @9.177s +17.554s
└─dbus.service @9.170s
└─basic.target @9.081s
└─sockets.target @9.080s
└─dbus.socket @9.080s
└─sysinit.target @8.814s

└─systemd-backlight@backlight:intel_backlight.service @23.030s +521ms
└─system-systemd\x2dbacklight.slice @22.251s
└─system.slice @2.429s
└─-.slice @2.429s

Hope this helps you.

- Sedat -

2020-06-04 01:35:38

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

On 2020-06-03 19:04:17 -0600, Jens Axboe wrote:
> > The workload that triggers the bug within a few seconds is postgres
> > doing a parallel sequential scan of a large table (and aggregating the
> > data, but that shouldn't matter). In the triggering case that boils down
> > to 9 processes sequentially reading a number of 1GB files (we chunk
> > tables internally into smaller files). Each process will read a 512kB
> > chunk of the file on its own, and then claim the next 512kB from a
> > shared memory location. Most of the IO will be READV requests, reading
> > 16 * 8kB into postgres' buffer pool (which may or may not be neighboring
> > 8kB pages).
>
> I'll try and reproduce this, any chance you have a test case that can
> be run so I don't have to write one from scratch? The more detailed
> instructions the better.

It shouldn't be too hard to write you a detailed script for reproducing
the issue. But it'd not be an all that minimal reproducer, unless it
also triggers on smaller scale (it's a 130GB database that triggers the
problem reliably, and small tables don't seem to do so reliably).

I'll try to write that up after I set up kvm / repro there.

One thing I forgot in the earlier email: I ran the benchmark using 'perf
stat -a -e ...'. I'm fairly, but not absolutely, certain that it also
triggered without that. I don't think it's related, but I thought I
better mention it.


> I have a known issue with request starvation, wonder if that could be it.
> I'm going to rebase the branch on top of the aops->readahead() changes
> shortly, and fix that issue. Hopefully that's what's plaguing your run
> here, but if not, I'll hunt that one down.

FWIW, I had iostat -xm /dev/nvme1n1 1 running during this. Shortly
before the crash I see:

Device r/s rMB/s rrqm/s %rrqm r_await rareq-sz w/s wMB/s wrqm/s %wrqm w_await wareq-sz d/s dMB/s drqm/s %drqm d_await dareq-sz f/s f_await aqu-sz %util
nvme1n1 6221.00 956.09 3428.00 35.53 0.24 157.38 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 1.48 99.00

Device r/s rMB/s rrqm/s %rrqm r_await rareq-sz w/s wMB/s wrqm/s %wrqm w_await wareq-sz d/s dMB/s drqm/s %drqm d_await dareq-sz f/s f_await aqu-sz %util
nvme1n1 6456.00 978.83 3439.00 34.75 0.21 155.25 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 0.00 1.38 98.70

It's maybe also worth noting that in this workload the results are
*worse* than when using 5.7-rc7 io_uring. So perhaps request starvation
isn't the worst guess...

Greetings,

Andres Freund

2020-06-04 01:54:17

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

I was trying to benchmark the benefits of this for the io_uring using
postgres I am working on. The initial results where quite promising
(reducing cpu usage significantly, to lower than non-uring sync io). But
unfortunately trying another workload triggered both panics and before
that seemingly returned wrong data.

I first saw that problem with b360d424ce02, which was
linux-block/async-buffered.6 at the time. After hitting the issue, I
updated to the current linux-block/async-buffered.6, but the problem
persists.

The workload that triggers the bug within a few seconds is postgres
doing a parallel sequential scan of a large table (and aggregating the
data, but that shouldn't matter). In the triggering case that boils down
to 9 processes sequentially reading a number of 1GB files (we chunk
tables internally into smaller files). Each process will read a 512kB
chunk of the file on its own, and then claim the next 512kB from a
shared memory location. Most of the IO will be READV requests, reading
16 * 8kB into postgres' buffer pool (which may or may not be neighboring
8kB pages).

The io submissions in this case will all be done to the same io_uring,
targeting MAP_HUGETLB | MAP_SHARED | MAP_ANONYMOUS memory. The data is
on xfs.

The benchmark starts with dropping caches, which helpfully is visible in
dmesg... That takes a few seconds, so the actual buffered io_uring load
starts soon after.

[ 125.526092] tee (7775): drop_caches: 3
[ 146.264327] kill_fasync: bad magic number in fasync_struct!

After this, but before the oops, I see some userspace memory or
non-io-uring pipe reads being corrupted.


[ 146.264327] kill_fasync: bad magic number in fasync_struct!
[ 146.285175] kill_fasync: bad magic number in fasync_struct!
[ 146.290793] kill_fasync: bad magic number in fasync_struct!
[ 146.285175] kill_fasync: bad magic number in fasync_struct!
[ 146.290793] kill_fasync: bad magic number in fasync_struct!
[ 157.071979] BUG: kernel NULL pointer dereference, address: 0000000000000020
[ 157.078945] #PF: supervisor read access in kernel mode
[ 157.084082] #PF: error_code(0x0000) - not-present page
[ 157.089225] PGD 0 P4D 0
[ 157.091763] Oops: 0000 [#1] SMP NOPTI
[ 157.095429] CPU: 3 PID: 7756 Comm: postgres Not tainted 5.7.0-rc7-andres-00133-gc8707bf69395 #41
[ 157.104208] Hardware name: Supermicro SYS-7049A-T/X11DAi-N, BIOS 3.2 11/13/2019
[ 157.111518] RIP: 0010:xfs_log_commit_cil+0x356/0x7f0
[ 157.116478] Code: 00 00 48 89 7e 08 49 89 76 30 48 89 11 48 89 4a 08 8b 4c 24 2c 48 89 85 c0 00 00 00 48 89 85 c8 00 00 00 49 8b 46 20 45 31 c9 <8b> 70 20 85 f6 0f 84 d9 02 00 00 45 31 c0 85 c9 7e 4f 41 8b 46 2c
[ 157.135226] RSP: 0018:ffffc90021fefb00 EFLAGS: 00010246
[ 157.140452] RAX: 0000000000000000 RBX: ffff8897a9ea80b0 RCX: 0000000000000458
[ 157.147582] RDX: ffff8897a9ea80c0 RSI: ffff88afc39a1200 RDI: ffff88afb1ae2c18
[ 157.154715] RBP: ffff8897a9ea8000 R08: ffffc90021fefb30 R09: 0000000000000000
[ 157.161848] R10: 0000000000000084 R11: 0000000000000012 R12: ffff8897721bd400
[ 157.168983] R13: ffff88afb1ae2c00 R14: ffff88afb19860c0 R15: ffff8897a9ea80a0
[ 157.176115] FS: 00007ffbbe9a1980(0000) GS:ffff8897e0cc0000(0000) knlGS:0000000000000000
[ 157.184199] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
[ 157.189946] CR2: 0000000000000020 CR3: 000000179c0ac005 CR4: 00000000007606e0
[ 157.197081] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
[ 157.204212] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
[ 157.211343] PKRU: 55555554
[ 157.214048] Call Trace:
[ 157.216498] __xfs_trans_commit+0xa1/0x340
[ 157.220596] xfs_create+0x4cc/0x5e0
[ 157.224087] xfs_generic_create+0x241/0x310
[ 157.228274] path_openat+0xdb7/0x1020
[ 157.231940] do_filp_open+0x91/0x100
[ 157.235518] ? __sys_recvfrom+0xe4/0x170
[ 157.239444] ? _cond_resched+0x19/0x30
[ 157.243196] do_sys_openat2+0x215/0x2d0
[ 157.247037] do_sys_open+0x44/0x80
[ 157.250443] do_syscall_64+0x48/0x130
[ 157.254108] entry_SYSCALL_64_after_hwframe+0x44/0xa9
[ 157.259161] RIP: 0033:0x7ffbc0ded307
[ 157.262738] Code: 25 00 00 41 00 3d 00 00 41 00 74 47 64 8b 04 25 18 00 00 00 85 c0 75 6b 44 89 e2 48 89 ee bf 9c ff ff ff b8 01 01 00 00 0f 05 <48> 3d 00 f0 ff ff 0f 87 95 00 00 00 48 8b 4c 24 28 64 48 33 0c 25
[ 157.281485] RSP: 002b:00007fff520a4940 EFLAGS: 00000246 ORIG_RAX: 0000000000000101
[ 157.289052] RAX: ffffffffffffffda RBX: 00005561dc02eca0 RCX: 00007ffbc0ded307
[ 157.296185] RDX: 0000000000000241 RSI: 00005561dc013470 RDI: 00000000ffffff9c
[ 157.303317] RBP: 00005561dc013470 R08: 0000000000000004 R09: 0000000000000001
[ 157.310447] R10: 00000000000001b6 R11: 0000000000000246 R12: 0000000000000241
[ 157.317575] R13: 00005561dc02eca0 R14: 0000000000000001 R15: 0000000000000000
[ 157.324705] Modules linked in: isst_if_common squashfs nls_iso8859_1 nls_cp437 snd_hda_codec_realtek vfat snd_hda_codec_generic fat iwlmvm ledtrig_audio snd_hda_codec_hdmi x86_pkg_temp_thermal intel_powerclamp snd_hda_intel snd_intel_dspcfg iwlwifi efi_pstore btusb snd_hda_codec btrtl btbcm efivars snd_hwdep btintel snd_hda_core iTCO_wdt iTCO_vendor_support mei_me mei loop coretemp hid_logitech_hidpp hid_logitech_dj hid_lenovo amdgpu gpu_sched i40e ixgbe ast drm_vram_helper drm_ttm_helper ttm mdio xhci_pci xhci_hcd
[ 157.370249] CR2: 0000000000000020

There's a lot more of these later, some interspersed (possibly related
to grabbing the output via serial->bmc->network->ipmitool). I've
attached the whole dmesg and .config.

What would be helpful for debugging? Should I try the v5 branch instead?

I'll try setting up a VM + passing through NVME to be able to test this
without fear... In one instance I did see some minor corruption on
another device & fs (ext4 on dm-crypt on nvme). It's all backed up,
but...

Greetings,

Andres Freund


Attachments:
(No filename) (5.82 kB)
dmesg (232.51 kB)
.config (179.69 kB)
Download all attachments

2020-06-04 02:01:15

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/3/20 6:59 PM, Andres Freund wrote:
> Hi,
>
> I was trying to benchmark the benefits of this for the io_uring using
> postgres I am working on. The initial results where quite promising
> (reducing cpu usage significantly, to lower than non-uring sync io). But
> unfortunately trying another workload triggered both panics and before
> that seemingly returned wrong data.
>
> I first saw that problem with b360d424ce02, which was
> linux-block/async-buffered.6 at the time. After hitting the issue, I
> updated to the current linux-block/async-buffered.6, but the problem
> persists.
>
> The workload that triggers the bug within a few seconds is postgres
> doing a parallel sequential scan of a large table (and aggregating the
> data, but that shouldn't matter). In the triggering case that boils down
> to 9 processes sequentially reading a number of 1GB files (we chunk
> tables internally into smaller files). Each process will read a 512kB
> chunk of the file on its own, and then claim the next 512kB from a
> shared memory location. Most of the IO will be READV requests, reading
> 16 * 8kB into postgres' buffer pool (which may or may not be neighboring
> 8kB pages).

I'll try and reproduce this, any chance you have a test case that can
be run so I don't have to write one from scratch? The more detailed
instructions the better.

> The io submissions in this case will all be done to the same io_uring,
> targeting MAP_HUGETLB | MAP_SHARED | MAP_ANONYMOUS memory. The data is
> on xfs.
>
> The benchmark starts with dropping caches, which helpfully is visible in
> dmesg... That takes a few seconds, so the actual buffered io_uring load
> starts soon after.
>
> [ 125.526092] tee (7775): drop_caches: 3
> [ 146.264327] kill_fasync: bad magic number in fasync_struct!
>
> After this, but before the oops, I see some userspace memory or
> non-io-uring pipe reads being corrupted.
>
>
> [ 146.264327] kill_fasync: bad magic number in fasync_struct!
> [ 146.285175] kill_fasync: bad magic number in fasync_struct!
> [ 146.290793] kill_fasync: bad magic number in fasync_struct!
> [ 146.285175] kill_fasync: bad magic number in fasync_struct!
> [ 146.290793] kill_fasync: bad magic number in fasync_struct!
> [ 157.071979] BUG: kernel NULL pointer dereference, address: 0000000000000020
> [ 157.078945] #PF: supervisor read access in kernel mode
> [ 157.084082] #PF: error_code(0x0000) - not-present page
> [ 157.089225] PGD 0 P4D 0
> [ 157.091763] Oops: 0000 [#1] SMP NOPTI
> [ 157.095429] CPU: 3 PID: 7756 Comm: postgres Not tainted 5.7.0-rc7-andres-00133-gc8707bf69395 #41
> [ 157.104208] Hardware name: Supermicro SYS-7049A-T/X11DAi-N, BIOS 3.2 11/13/2019
> [ 157.111518] RIP: 0010:xfs_log_commit_cil+0x356/0x7f0
> [ 157.116478] Code: 00 00 48 89 7e 08 49 89 76 30 48 89 11 48 89 4a 08 8b 4c 24 2c 48 89 85 c0 00 00 00 48 89 85 c8 00 00 00 49 8b 46 20 45 31 c9 <8b> 70 20 85 f6 0f 84 d9 02 00 00 45 31 c0 85 c9 7e 4f 41 8b 46 2c
> [ 157.135226] RSP: 0018:ffffc90021fefb00 EFLAGS: 00010246
> [ 157.140452] RAX: 0000000000000000 RBX: ffff8897a9ea80b0 RCX: 0000000000000458
> [ 157.147582] RDX: ffff8897a9ea80c0 RSI: ffff88afc39a1200 RDI: ffff88afb1ae2c18
> [ 157.154715] RBP: ffff8897a9ea8000 R08: ffffc90021fefb30 R09: 0000000000000000
> [ 157.161848] R10: 0000000000000084 R11: 0000000000000012 R12: ffff8897721bd400
> [ 157.168983] R13: ffff88afb1ae2c00 R14: ffff88afb19860c0 R15: ffff8897a9ea80a0
> [ 157.176115] FS: 00007ffbbe9a1980(0000) GS:ffff8897e0cc0000(0000) knlGS:0000000000000000
> [ 157.184199] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> [ 157.189946] CR2: 0000000000000020 CR3: 000000179c0ac005 CR4: 00000000007606e0
> [ 157.197081] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> [ 157.204212] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> [ 157.211343] PKRU: 55555554
> [ 157.214048] Call Trace:
> [ 157.216498] __xfs_trans_commit+0xa1/0x340
> [ 157.220596] xfs_create+0x4cc/0x5e0
> [ 157.224087] xfs_generic_create+0x241/0x310
> [ 157.228274] path_openat+0xdb7/0x1020
> [ 157.231940] do_filp_open+0x91/0x100
> [ 157.235518] ? __sys_recvfrom+0xe4/0x170
> [ 157.239444] ? _cond_resched+0x19/0x30
> [ 157.243196] do_sys_openat2+0x215/0x2d0
> [ 157.247037] do_sys_open+0x44/0x80
> [ 157.250443] do_syscall_64+0x48/0x130
> [ 157.254108] entry_SYSCALL_64_after_hwframe+0x44/0xa9
> [ 157.259161] RIP: 0033:0x7ffbc0ded307
> [ 157.262738] Code: 25 00 00 41 00 3d 00 00 41 00 74 47 64 8b 04 25 18 00 00 00 85 c0 75 6b 44 89 e2 48 89 ee bf 9c ff ff ff b8 01 01 00 00 0f 05 <48> 3d 00 f0 ff ff 0f 87 95 00 00 00 48 8b 4c 24 28 64 48 33 0c 25
> [ 157.281485] RSP: 002b:00007fff520a4940 EFLAGS: 00000246 ORIG_RAX: 0000000000000101
> [ 157.289052] RAX: ffffffffffffffda RBX: 00005561dc02eca0 RCX: 00007ffbc0ded307
> [ 157.296185] RDX: 0000000000000241 RSI: 00005561dc013470 RDI: 00000000ffffff9c
> [ 157.303317] RBP: 00005561dc013470 R08: 0000000000000004 R09: 0000000000000001
> [ 157.310447] R10: 00000000000001b6 R11: 0000000000000246 R12: 0000000000000241
> [ 157.317575] R13: 00005561dc02eca0 R14: 0000000000000001 R15: 0000000000000000
> [ 157.324705] Modules linked in: isst_if_common squashfs nls_iso8859_1 nls_cp437 snd_hda_codec_realtek vfat snd_hda_codec_generic fat iwlmvm ledtrig_audio snd_hda_codec_hdmi x86_pkg_temp_thermal intel_powerclamp snd_hda_intel snd_intel_dspcfg iwlwifi efi_pstore btusb snd_hda_codec btrtl btbcm efivars snd_hwdep btintel snd_hda_core iTCO_wdt iTCO_vendor_support mei_me mei loop coretemp hid_logitech_hidpp hid_logitech_dj hid_lenovo amdgpu gpu_sched i40e ixgbe ast drm_vram_helper drm_ttm_helper ttm mdio xhci_pci xhci_hcd
> [ 157.370249] CR2: 0000000000000020
>
> There's a lot more of these later, some interspersed (possibly related
> to grabbing the output via serial->bmc->network->ipmitool). I've
> attached the whole dmesg and .config.
>
> What would be helpful for debugging? Should I try the v5 branch instead?
>
> I'll try setting up a VM + passing through NVME to be able to test this
> without fear... In one instance I did see some minor corruption on
> another device & fs (ext4 on dm-crypt on nvme). It's all backed up,
> but...

I have a known issue with request starvation, wonder if that could be it.
I'm going to rebase the branch on top of the aops->readahead() changes
shortly, and fix that issue. Hopefully that's what's plaguing your run
here, but if not, I'll hunt that one down.

Thanks for testing!

--
Jens Axboe

2020-06-05 14:45:00

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/3/20 7:04 PM, Jens Axboe wrote:
> On 6/3/20 6:59 PM, Andres Freund wrote:
>> Hi,
>>
>> I was trying to benchmark the benefits of this for the io_uring using
>> postgres I am working on. The initial results where quite promising
>> (reducing cpu usage significantly, to lower than non-uring sync io). But
>> unfortunately trying another workload triggered both panics and before
>> that seemingly returned wrong data.
>>
>> I first saw that problem with b360d424ce02, which was
>> linux-block/async-buffered.6 at the time. After hitting the issue, I
>> updated to the current linux-block/async-buffered.6, but the problem
>> persists.
>>
>> The workload that triggers the bug within a few seconds is postgres
>> doing a parallel sequential scan of a large table (and aggregating the
>> data, but that shouldn't matter). In the triggering case that boils down
>> to 9 processes sequentially reading a number of 1GB files (we chunk
>> tables internally into smaller files). Each process will read a 512kB
>> chunk of the file on its own, and then claim the next 512kB from a
>> shared memory location. Most of the IO will be READV requests, reading
>> 16 * 8kB into postgres' buffer pool (which may or may not be neighboring
>> 8kB pages).
>
> I'll try and reproduce this, any chance you have a test case that can
> be run so I don't have to write one from scratch? The more detailed
> instructions the better.

Can you try with async-buffered.7? I've rebased it on a new mechanism,
and doing something like what you describe above I haven't been able
to trigger anything bad. I'd try your test case specifically, so do let
know if it's something I can run.

--
Jens Axboe

2020-06-05 20:00:42

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

On 2020-06-03 18:30:45 -0700, Andres Freund wrote:
> > I'll try and reproduce this, any chance you have a test case that can
> > be run so I don't have to write one from scratch? The more detailed
> > instructions the better.
>
> It shouldn't be too hard to write you a detailed script for reproducing
> the issue. But it'd not be an all that minimal reproducer, unless it
> also triggers on smaller scale (it's a 130GB database that triggers the
> problem reliably, and small tables don't seem to do so reliably).
>
> I'll try to write that up after I set up kvm / repro there.

Sorry, took a bit longer. Ran into some issues with my aio branch when
running on 5.6. Not yet sure if that's my fault, or io_uring's.


Here's a recipe that repros reliably for me. For me it triggers reliably
at this size (-s 1000 ends up around ~16GB).


# Build aio branch of postgres
sudo mkdir /srv/src
sudo chown andres. /srv/src
cd /srv/src
git clone -b aio https://github.com/anarazel/postgres.git pg
cd pg
./configure --enable-depend --with-liburing --prefix=/home/andres/bin/
make -j48 -s install

# check that target directory is in PATH
type psql
psql is /home/andres/bin/bin/psql

in one shell:
# create database directory, start postgres
echo 1200 | sudo tee /proc/sys/vm/nr_hugepages
initdb /srv/pguringcrash
postgres -D /srv/pguringcrash/ -c bgwriter_delay=10ms -c bgwriter_lru_maxpages=1000 -c wal_level=minimal -c max_wal_senders=0 -c wal_buffers=128MB -c max_wal_size=100GB -c shared_buffers=2GB -c max_parallel_workers_per_gather=8 -c huge_pages=on

in the other shell:
# fill with test data (skipping index creation etc)
pgbench -i -s 1000 postgres -IdtGv -q

# and trigger a parallel sequential scan, using 8 additional workers
echo 3 | sudo tee /proc/sys/vm/drop_caches && psql -c 'SET max_parallel_workers_per_gather=8' -c 'SELECT SUM(abalance) FROM pgbench_accounts ;'


For me that faily reliably triggers the issue within a few seconds.

[ 323.312854] BUG: unable to handle page fault for address: 00007f0f1428c000
[ 323.319879] #PF: supervisor read access in kernel mode
[ 323.321391] #PF: error_code(0x0001) - permissions violation
[ 323.323019] PGD 102e760067 P4D 102e760067 PUD 1037878067 PMD 8000000fc7e000e7
[ 323.325065] Oops: 0001 [#1] SMP NOPTI
[ 323.326248] CPU: 1 PID: 3145 Comm: postgres Not tainted 5.7.0-rc7-andres-00133-gc8707bf69395 #43
[ 323.328653] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
[ 323.330365] RIP: 0010:__wake_up_common+0x53/0x140
[ 323.331276] Code: 41 f6 01 04 0f 85 a3 00 00 00 48 8b 43 08 48 8d 78 e8 48 8d 43 08 48 89 c1 48 89 04 24 48 8d 47 18 48 39 c1 0f 84 b8 00 00 00 <48> 8b 47 18 89 54 24 14 31 ed 4c 8d 60 e8 8b 1f f6 c3 04 75 52 48
[ 323.334419] RSP: 0018:ffffc90000dcbdc8 EFLAGS: 00010086
[ 323.335409] RAX: 00007f0f1428c000 RBX: ffff889038b189f8 RCX: ffff889038b18a00
[ 323.336723] RDX: 0000000000000001 RSI: 0000000000000003 RDI: 00007f0f1428bfe8
[ 323.338035] RBP: 0000000000000246 R08: 0000000000000001 R09: ffffc90000dcbe18
[ 323.339331] R10: ffff889036a5e750 R11: 0000000000000000 R12: 0000000000000001
[ 323.340312] R13: 0000000000000003 R14: 0000000000000000 R15: 0000000000000001
[ 323.341204] FS: 00007f0f6f271740(0000) GS:ffff88903f840000(0000) knlGS:0000000000000000
[ 323.342217] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
[ 323.342950] CR2: 00007f0f1428c000 CR3: 0000001037ed0005 CR4: 0000000000760ee0
[ 323.343836] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
[ 323.344724] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
[ 323.345613] PKRU: 55555554
[ 323.346017] Call Trace:
[ 323.346435] __wake_up_common_lock+0x7c/0xc0
[ 323.347015] ep_free+0x30/0xd0
[ 323.347468] ep_eventpoll_release+0x16/0x20
[ 323.348040] __fput+0xda/0x240
[ 323.348498] task_work_run+0x62/0x90
[ 323.349015] exit_to_usermode_loop+0xbd/0xe0
[ 323.349573] do_syscall_64+0xf2/0x130
[ 323.349987] entry_SYSCALL_64_after_hwframe+0x44/0xa9
[ 323.350523] RIP: 0033:0x7f0f6f5a5f13
[ 323.350906] Code: 00 00 f7 d8 64 89 02 48 c7 c0 ff ff ff ff eb bb 0f 1f 80 00 00 00 00 64 8b 04 25 18 00 00 00 85 c0 75 14 b8 03 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 45 c3 0f 1f 40 00 48 83 ec 18 89 7c 24 0c e8
[ 323.352501] RSP: 002b:00007ffdf3eab5a8 EFLAGS: 00000246 ORIG_RAX: 0000000000000003
[ 323.353277] RAX: 0000000000000000 RBX: 0000000000000000 RCX: 00007f0f6f5a5f13
[ 323.353947] RDX: 0000000000000001 RSI: 000055ef9f5cb360 RDI: 000000000000002a
[ 323.354587] RBP: 000055ef9f5cb2e8 R08: 00000000000280e8 R09: 0000000000000143
[ 323.355227] R10: 000000000000000a R11: 0000000000000246 R12: 000055ef9f5cb2e8
[ 323.355872] R13: 0000000000000008 R14: 00000000ffffffff R15: 00007f0f6d3407a4
[ 323.356510] Modules linked in: 9pnet_virtio isst_if_common iTCO_wdt 9pnet xhci_pci iTCO_vendor_support xhci_hcd
[ 323.357477] CR2: 00007f0f1428c000
[ 323.357866] ---[ end trace 081898b3c6b5ab1b ]---


I'll try your new branch next.

Greetings,

Andres Freund

2020-06-05 20:23:55

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/5/20 2:20 PM, Andres Freund wrote:
> Hi,
>
> On 2020-06-05 08:42:28 -0600, Jens Axboe wrote:
>> Can you try with async-buffered.7? I've rebased it on a new mechanism,
>> and doing something like what you describe above I haven't been able
>> to trigger anything bad. I'd try your test case specifically, so do let
>> know if it's something I can run.
>
> I tried my test on async-buffered.7?, and I get hangs very quickly after
> starting. Unfortunately, I don't seem to get an OOPSs, not sure yet why.
>
> Let me know if my test triggers for you.
>
> I'll go and try to figure out why I don't see an oops...

I'll try the reproducer! Thanks for testing.

--
Jens Axboe

2020-06-05 20:24:39

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

On 2020-06-05 08:42:28 -0600, Jens Axboe wrote:
> Can you try with async-buffered.7? I've rebased it on a new mechanism,
> and doing something like what you describe above I haven't been able
> to trigger anything bad. I'd try your test case specifically, so do let
> know if it's something I can run.

I tried my test on async-buffered.7?, and I get hangs very quickly after
starting. Unfortunately, I don't seem to get an OOPSs, not sure yet why.

Let me know if my test triggers for you.

I'll go and try to figure out why I don't see an oops...

Regards,

Andres

2020-06-05 20:38:30

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

On 2020-06-05 13:20:28 -0700, Andres Freund wrote:
> I'll go and try to figure out why I don't see an oops...

Err, that probably was a typo on my end in the serial console
config. After fixing that, I did get the below.

If helpful I can try with debugging enabled or such.

[ 67.910265] tee (2577): drop_caches: 3
[ 68.059674] BUG: unable to handle page fault for address: 00007f0b16a3c03c
[ 68.062021] #PF: supervisor read access in kernel mode
[ 68.063742] #PF: error_code(0x0000) - not-present page
[ 68.065517] PGD 102e044067 P4D 102e044067 PUD 102bf7a067 PMD 0
[ 68.067519] Oops: 0000 [#1] SMP NOPTI
[ 68.068800] CPU: 2 PID: 2554 Comm: postgres Not tainted 5.7.0-andres-10123-g87823242260e #44
[ 68.071505] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
[ 68.074139] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
[ 68.075389] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
[ 68.079125] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
[ 68.080260] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
[ 68.084115] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
[ 68.085374] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
[ 68.086409] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
[ 68.087447] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
[ 68.088697] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
[ 68.089903] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
[ 68.090776] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
[ 68.091834] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
[ 68.092902] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
[ 68.093967] PKRU: 55555554
[ 68.094427] Call Trace:
[ 68.094845] ? __schedule+0x2ac/0x720
[ 68.095350] ? schedule+0x55/0xc0
[ 68.095797] ? ep_read_events_proc+0xd0/0xd0
[ 68.096354] ep_scan_ready_list.constprop.0+0x16c/0x190
[ 68.097016] ep_poll+0x2a3/0x440
[ 68.097449] ? wait_woken+0x70/0x70
[ 68.097904] do_epoll_wait+0xb0/0xd0
[ 68.098375] __x64_sys_epoll_wait+0x1a/0x20
[ 68.098913] do_syscall_64+0x48/0x130
[ 68.099393] entry_SYSCALL_64_after_hwframe+0x44/0xa9
[ 68.100030] RIP: 0033:0x7f0b97845606
[ 68.100498] Code: 48 83 c8 ff c3 66 2e 0f 1f 84 00 00 00 00 00 0f 1f 44 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 11 b8 e8 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 5a c3 90 48 83 ec 28 89 54 24 18 48 89 74 24
[ 68.102718] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
[ 68.103644] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
[ 68.104533] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
[ 68.105418] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
[ 68.106296] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
[ 68.107187] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
[ 68.108104] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
[ 68.109505] CR2: 00007f0b16a3c03c
[ 68.109962] ---[ end trace 0ca39a5ed99162ce ]---
[ 68.110547] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
[ 68.111214] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
[ 68.113435] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
[ 68.114111] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
[ 68.115016] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
[ 68.115902] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
[ 68.116810] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
[ 68.117663] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
[ 68.118520] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
[ 68.119482] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
[ 68.120181] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
[ 68.121043] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
[ 68.121904] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
[ 68.122790] PKRU: 55555554
[ 68.123401] general protection fault, probably for non-canonical address 0xfeeda989fef06266: 0000 [#2] SMP NOPTI
[ 68.125052] CPU: 2 PID: 2554 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
[ 68.126260] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
[ 68.127274] RIP: 0010:__pv_queued_spin_lock_slowpath+0x1a1/0x2b0
[ 68.128032] Code: c4 c1 ea 12 41 be 01 00 00 00 8d 42 ff 41 83 e4 03 4c 8d 6b 14 49 c1 e4 05 48 98 49 81 c4 00 c7 02 00 4c 03 24 c5 e0 e6 fd 82 <49> 89 1c 24 b8 00 80 00 00 eb 15 84 c0 75 0a 41 0f b6 54 24 14 84
[ 68.130221] RSP: 0018:ffffc90000befce8 EFLAGS: 00010086
[ 68.130867] RAX: 0000000000003ffe RBX: ffff88903f8ac700 RCX: 0000000000000001
[ 68.131752] RDX: 0000000000003fff RSI: 0000000000000000 RDI: 0000000000000000
[ 68.132637] RBP: ffff889037617924 R08: 0000000000000000 R09: ffffc90000befdf8
[ 68.133513] R10: ffff8890334d2bf0 R11: 0000000000000018 R12: feeda989fef06266
[ 68.134399] R13: ffff88903f8ac714 R14: 0000000000000001 R15: 00000000000c0000
[ 68.135323] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
[ 68.136307] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
[ 68.137004] CR2: 00007f0b16a3c03c CR3: 000000000360a005 CR4: 0000000000760ee0
[ 68.137866] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
[ 68.138748] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
[ 68.139606] PKRU: 55555554
[ 68.139973] Call Trace:
[ 68.140313] queued_read_lock_slowpath+0x6c/0x70
[ 68.140915] _raw_read_lock_irqsave+0x26/0x30
[ 68.141485] ep_poll_callback+0x3e/0x2b0
[ 68.142007] ? set_next_entity+0xab/0x1f0
[ 68.142541] __wake_up_common+0x7a/0x140
[ 68.143077] __wake_up_common_lock+0x7c/0xc0
[ 68.143651] pipe_release+0x5b/0xd0
[ 68.144150] __fput+0xda/0x240
[ 68.144574] task_work_run+0x62/0x90
[ 68.145046] do_exit+0x35c/0xa70
[ 68.145505] ? do_epoll_wait+0xb0/0xd0
[ 68.146000] rewind_stack_do_exit+0x17/0x20
[ 68.146538] RIP: 0033:0x7f0b97845606
[ 68.146988] Code: Bad RIP value.
[ 68.147405] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
[ 68.148314] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
[ 68.149157] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
[ 68.150021] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
[ 68.150831] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
[ 68.151640] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
[ 68.152459] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
[ 68.153707] ---[ end trace 0ca39a5ed99162cf ]---
[ 68.154282] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
[ 68.154884] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
[ 68.156976] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
[ 68.157614] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
[ 68.158436] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
[ 68.159269] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
[ 68.160092] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
[ 68.160920] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
[ 68.161746] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
[ 68.162701] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
[ 68.163386] CR2: 00007f0b978455dc CR3: 000000000360a005 CR4: 0000000000760ee0
[ 68.164226] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
[ 68.165079] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
[ 68.165931] PKRU: 55555554
[ 68.166298] Fixing recursive fault but reboot is needed!
[ 128.173729] rcu: INFO: rcu_sched detected stalls on CPUs/tasks:
[ 128.179267] rcu: 11-...0: (1 GPs behind) idle=c56/1/0x4000000000000000 softirq=6897/6898 fqs=5976
[ 128.182232] rcu: 19-...0: (1 GPs behind) idle=492/1/0x4000000000000000 softirq=1023/1023 fqs=5976
[ 128.185217] (detected by 10, t=18003 jiffies, g=15789, q=631)
[ 128.186863] Sending NMI from CPU 10 to CPUs 11:
[ 128.188902] NMI backtrace for cpu 11
[ 128.188903] CPU: 11 PID: 2546 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
[ 128.188904] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
[ 128.188904] RIP: 0010:queued_read_lock_slowpath+0x5b/0x70
[ 128.188906] Code: 03 00 02 00 00 8b 03 84 c0 74 08 f3 90 8b 13 84 d2 75 f8 48 89 ef e8 74 e6 ff ff 66 90 5b 5d c3 8b 07 84 c0 74 08 f3 90 8b 03 <84> c0 75 f8 5b 5d c3 89 c6 48 89 ef e8 e4 e8 ff ff 66 90 eb bf 0f
[ 128.188907] RSP: 0018:ffffc90000348c08 EFLAGS: 00000086
[ 128.188908] RAX: 0000000037617cc0 RBX: ffff889037617920 RCX: 00000000000000c3
[ 128.188909] RDX: 0000000000000001 RSI: 0000000000000001 RDI: ffff889037617920
[ 128.188910] RBP: ffff889037bb1a80 R08: 00000000000000c3 R09: ffffc90000348cd8
[ 128.188910] R10: 0100000000000000 R11: 00000000de2ee17e R12: 0000000000000046
[ 128.188911] R13: ffff889037617920 R14: 0000000000000001 R15: 00000000000000c3
[ 128.188912] FS: 00007f0b97743740(0000) GS:ffff88903fac0000(0000) knlGS:0000000000000000
[ 128.188912] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
[ 128.188913] CR2: 00007f0b16afc00e CR3: 000000102e5c6002 CR4: 0000000000760ee0
[ 128.188914] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
[ 128.188914] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
[ 128.188915] PKRU: 55555554
[ 128.188915] Call Trace:
[ 128.188916] <IRQ>
[ 128.188916] _raw_read_lock_irqsave+0x26/0x30
[ 128.188917] ep_poll_callback+0x3e/0x2b0
[ 128.188917] ? cpumask_next_and+0x19/0x20
[ 128.188918] ? update_sd_lb_stats.constprop.0+0xfe/0x810
[ 128.188918] __wake_up_common+0x7a/0x140
[ 128.188919] __wake_up_common_lock+0x7c/0xc0
[ 128.188919] sock_def_readable+0x37/0x60
[ 128.188920] __udp_enqueue_schedule_skb+0x168/0x260
[ 128.188920] udpv6_queue_rcv_one_skb+0x284/0x3c0
[ 128.188921] udp6_unicast_rcv_skb.isra.0+0x44/0xa0
[ 128.188921] ip6_protocol_deliver_rcu+0x235/0x4b0
[ 128.188922] ip6_input_finish+0x11/0x20
[ 128.188922] ip6_input+0xa2/0xb0
[ 128.188923] ? ip6_protocol_deliver_rcu+0x4b0/0x4b0
[ 128.188923] ipv6_rcv+0xc0/0xd0
[ 128.188924] ? ip6_rcv_finish_core.isra.0+0xd0/0xd0
[ 128.188924] __netif_receive_skb_one_core+0x63/0xa0
[ 128.188925] process_backlog+0x98/0x140
[ 128.188925] net_rx_action+0x13a/0x370
[ 128.188926] __do_softirq+0xe0/0x2ca
[ 128.188926] do_softirq_own_stack+0x2a/0x40
[ 128.188926] </IRQ>
[ 128.188927] do_softirq.part.0+0x2b/0x30
[ 128.188927] __local_bh_enable_ip+0x4b/0x50
[ 128.188928] ip6_finish_output2+0x264/0x5b0
[ 128.188928] ip6_output+0x73/0x120
[ 128.188929] ? __ip6_finish_output+0x110/0x110
[ 128.188929] ip6_send_skb+0x1e/0x60
[ 128.188930] udp_v6_send_skb.isra.0+0x197/0x460
[ 128.188930] udpv6_sendmsg+0xb4f/0xdb0
[ 128.188931] ? ip_reply_glue_bits+0x40/0x40
[ 128.188931] ? update_load_avg+0x78/0x630
[ 128.188932] ? update_curr+0x73/0x1d0
[ 128.188932] ? __sys_sendto+0x108/0x190
[ 128.188933] __sys_sendto+0x108/0x190
[ 128.188933] ? __fput+0x1a5/0x240
[ 128.188934] ? _cond_resched+0x19/0x30
[ 128.188934] ? task_work_run+0x67/0x90
[ 128.188935] __x64_sys_sendto+0x25/0x30
[ 128.188935] do_syscall_64+0x48/0x130
[ 128.188936] entry_SYSCALL_64_after_hwframe+0x44/0xa9
[ 128.188936] RIP: 0033:0x7f0b97a7826c
[ 128.188945] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
[ 128.188946] RSP: 002b:00007ffe80ffcea8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
[ 128.188947] RAX: ffffffffffffffda RBX: 0000000000000058 RCX: 00007f0b97a7826c
[ 128.188948] RDX: 0000000000000058 RSI: 000055fb765264c0 RDI: 0000000000000009
[ 128.188949] RBP: 000055fb765264c0 R08: 0000000000000000 R09: 0000000000000000
[ 128.188949] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffcfc0
[ 128.188950] R13: 000000000002c600 R14: 0000000000000000 R15: 000055fb7629b751
[ 128.188957] Sending NMI from CPU 10 to CPUs 19:
[ 128.239939] NMI backtrace for cpu 19
[ 128.239940] CPU: 19 PID: 2587 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
[ 128.239940] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
[ 128.239940] RIP: 0010:kvm_wait+0x30/0x50
[ 128.239941] Code: 8b 05 ac cb f4 7e a9 00 00 f0 00 75 1f 9c 58 fa 0f b6 17 40 38 d6 75 12 f6 c4 02 75 10 e9 07 00 00 00 0f 00 2d 6f ce 54 01 f4 <50> 9d c3 e9 07 00 00 00 0f 00 2d 5f ce 54 01 fb f4 eb ed 66 66 2e
[ 128.239942] RSP: 0018:ffffc90000cc78e8 EFLAGS: 00000046
[ 128.239942] RAX: 0000000000000046 RBX: ffff88903fcec700 RCX: 0000000000000008
[ 128.239943] RDX: 0000000000000003 RSI: 0000000000000003 RDI: ffff889039049d80
[ 128.239943] RBP: ffff889039049d80 R08: ffff88907ffe9f80 R09: 00000000000000f8
[ 128.239944] R10: 0000000000000000 R11: 0000000000000001 R12: 0000000000000000
[ 128.239944] R13: 0000000000000001 R14: 0000000000000100 R15: 0000000000500000
[ 128.239944] FS: 00007f0b97743740(0000) GS:ffff88903fcc0000(0000) knlGS:0000000000000000
[ 128.239945] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
[ 128.239945] CR2: 000055fb76a96030 CR3: 0000001035882004 CR4: 0000000000760ee0
[ 128.239945] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
[ 128.239946] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
[ 128.239946] PKRU: 55555554
[ 128.239946] Call Trace:
[ 128.239946] __pv_queued_spin_lock_slowpath+0x26e/0x2b0
[ 128.239947] _raw_spin_lock_irqsave+0x25/0x30
[ 128.239947] __wake_up_common_lock+0x62/0xc0
[ 128.239947] sock_def_write_space+0x49/0x90
[ 128.239948] sock_wfree+0x68/0xb0
[ 128.239948] loopback_xmit+0x49/0xe0
[ 128.239948] dev_hard_start_xmit+0x8d/0x1e0
[ 128.239948] __dev_queue_xmit+0x721/0x8e0
[ 128.239949] ip6_finish_output2+0x250/0x5b0
[ 128.239949] ip6_output+0x73/0x120
[ 128.239949] ? __ip6_finish_output+0x110/0x110
[ 128.239950] ip6_send_skb+0x1e/0x60
[ 128.239950] udp_v6_send_skb.isra.0+0x197/0x460
[ 128.239950] udpv6_sendmsg+0xb4f/0xdb0
[ 128.239950] ? release_pages+0x28f/0x2f0
[ 128.239950] ? ip_reply_glue_bits+0x40/0x40
[ 128.239951] ? _cond_resched+0x19/0x30
[ 128.239951] ? unmap_page_range+0x678/0xa60
[ 128.239951] ? __sys_sendto+0x108/0x190
[ 128.239951] __sys_sendto+0x108/0x190
[ 128.239952] ? __fput+0x1a5/0x240
[ 128.239952] ? _cond_resched+0x19/0x30
[ 128.239952] ? task_work_run+0x67/0x90
[ 128.239952] __x64_sys_sendto+0x25/0x30
[ 128.239953] do_syscall_64+0x48/0x130
[ 128.239953] entry_SYSCALL_64_after_hwframe+0x44/0xa9
[ 128.239953] RIP: 0033:0x7f0b97a7826c
[ 128.239954] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
[ 128.239954] RSP: 002b:00007ffe80ffd7b8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
[ 128.239955] RAX: ffffffffffffffda RBX: 00000000000003a8 RCX: 00007f0b97a7826c
[ 128.239955] RDX: 00000000000003a8 RSI: 00007ffe80ffd800 RDI: 0000000000000009
[ 128.239955] RBP: 00007ffe80ffd800 R08: 0000000000000000 R09: 0000000000000000
[ 128.239956] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffd800
[ 128.239956] R13: 00007ffe80ffd800 R14: 000000000000000e R15: 000055fb76b37e58

Greetings,

Andres Freund

2020-06-05 20:58:21

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/5/20 2:36 PM, Andres Freund wrote:
> Hi,
>
> On 2020-06-05 13:20:28 -0700, Andres Freund wrote:
>> I'll go and try to figure out why I don't see an oops...
>
> Err, that probably was a typo on my end in the serial console
> config. After fixing that, I did get the below.
>
> If helpful I can try with debugging enabled or such.
>
> [ 67.910265] tee (2577): drop_caches: 3
> [ 68.059674] BUG: unable to handle page fault for address: 00007f0b16a3c03c
> [ 68.062021] #PF: supervisor read access in kernel mode
> [ 68.063742] #PF: error_code(0x0000) - not-present page
> [ 68.065517] PGD 102e044067 P4D 102e044067 PUD 102bf7a067 PMD 0
> [ 68.067519] Oops: 0000 [#1] SMP NOPTI
> [ 68.068800] CPU: 2 PID: 2554 Comm: postgres Not tainted 5.7.0-andres-10123-g87823242260e #44
> [ 68.071505] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
> [ 68.074139] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
> [ 68.075389] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
> [ 68.079125] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
> [ 68.080260] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
> [ 68.084115] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
> [ 68.085374] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
> [ 68.086409] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
> [ 68.087447] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
> [ 68.088697] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
> [ 68.089903] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> [ 68.090776] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
> [ 68.091834] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> [ 68.092902] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> [ 68.093967] PKRU: 55555554
> [ 68.094427] Call Trace:
> [ 68.094845] ? __schedule+0x2ac/0x720
> [ 68.095350] ? schedule+0x55/0xc0
> [ 68.095797] ? ep_read_events_proc+0xd0/0xd0
> [ 68.096354] ep_scan_ready_list.constprop.0+0x16c/0x190
> [ 68.097016] ep_poll+0x2a3/0x440
> [ 68.097449] ? wait_woken+0x70/0x70
> [ 68.097904] do_epoll_wait+0xb0/0xd0
> [ 68.098375] __x64_sys_epoll_wait+0x1a/0x20
> [ 68.098913] do_syscall_64+0x48/0x130
> [ 68.099393] entry_SYSCALL_64_after_hwframe+0x44/0xa9
> [ 68.100030] RIP: 0033:0x7f0b97845606
> [ 68.100498] Code: 48 83 c8 ff c3 66 2e 0f 1f 84 00 00 00 00 00 0f 1f 44 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 11 b8 e8 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 5a c3 90 48 83 ec 28 89 54 24 18 48 89 74 24
> [ 68.102718] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
> [ 68.103644] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
> [ 68.104533] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
> [ 68.105418] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
> [ 68.106296] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
> [ 68.107187] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
> [ 68.108104] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
> [ 68.109505] CR2: 00007f0b16a3c03c
> [ 68.109962] ---[ end trace 0ca39a5ed99162ce ]---
> [ 68.110547] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
> [ 68.111214] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
> [ 68.113435] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
> [ 68.114111] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
> [ 68.115016] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
> [ 68.115902] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
> [ 68.116810] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
> [ 68.117663] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
> [ 68.118520] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
> [ 68.119482] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> [ 68.120181] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
> [ 68.121043] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> [ 68.121904] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> [ 68.122790] PKRU: 55555554
> [ 68.123401] general protection fault, probably for non-canonical address 0xfeeda989fef06266: 0000 [#2] SMP NOPTI
> [ 68.125052] CPU: 2 PID: 2554 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
> [ 68.126260] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
> [ 68.127274] RIP: 0010:__pv_queued_spin_lock_slowpath+0x1a1/0x2b0
> [ 68.128032] Code: c4 c1 ea 12 41 be 01 00 00 00 8d 42 ff 41 83 e4 03 4c 8d 6b 14 49 c1 e4 05 48 98 49 81 c4 00 c7 02 00 4c 03 24 c5 e0 e6 fd 82 <49> 89 1c 24 b8 00 80 00 00 eb 15 84 c0 75 0a 41 0f b6 54 24 14 84
> [ 68.130221] RSP: 0018:ffffc90000befce8 EFLAGS: 00010086
> [ 68.130867] RAX: 0000000000003ffe RBX: ffff88903f8ac700 RCX: 0000000000000001
> [ 68.131752] RDX: 0000000000003fff RSI: 0000000000000000 RDI: 0000000000000000
> [ 68.132637] RBP: ffff889037617924 R08: 0000000000000000 R09: ffffc90000befdf8
> [ 68.133513] R10: ffff8890334d2bf0 R11: 0000000000000018 R12: feeda989fef06266
> [ 68.134399] R13: ffff88903f8ac714 R14: 0000000000000001 R15: 00000000000c0000
> [ 68.135323] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
> [ 68.136307] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> [ 68.137004] CR2: 00007f0b16a3c03c CR3: 000000000360a005 CR4: 0000000000760ee0
> [ 68.137866] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> [ 68.138748] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> [ 68.139606] PKRU: 55555554
> [ 68.139973] Call Trace:
> [ 68.140313] queued_read_lock_slowpath+0x6c/0x70
> [ 68.140915] _raw_read_lock_irqsave+0x26/0x30
> [ 68.141485] ep_poll_callback+0x3e/0x2b0
> [ 68.142007] ? set_next_entity+0xab/0x1f0
> [ 68.142541] __wake_up_common+0x7a/0x140
> [ 68.143077] __wake_up_common_lock+0x7c/0xc0
> [ 68.143651] pipe_release+0x5b/0xd0
> [ 68.144150] __fput+0xda/0x240
> [ 68.144574] task_work_run+0x62/0x90
> [ 68.145046] do_exit+0x35c/0xa70
> [ 68.145505] ? do_epoll_wait+0xb0/0xd0
> [ 68.146000] rewind_stack_do_exit+0x17/0x20
> [ 68.146538] RIP: 0033:0x7f0b97845606
> [ 68.146988] Code: Bad RIP value.
> [ 68.147405] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
> [ 68.148314] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
> [ 68.149157] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
> [ 68.150021] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
> [ 68.150831] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
> [ 68.151640] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
> [ 68.152459] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
> [ 68.153707] ---[ end trace 0ca39a5ed99162cf ]---
> [ 68.154282] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
> [ 68.154884] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
> [ 68.156976] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
> [ 68.157614] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
> [ 68.158436] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
> [ 68.159269] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
> [ 68.160092] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
> [ 68.160920] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
> [ 68.161746] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
> [ 68.162701] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> [ 68.163386] CR2: 00007f0b978455dc CR3: 000000000360a005 CR4: 0000000000760ee0
> [ 68.164226] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> [ 68.165079] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> [ 68.165931] PKRU: 55555554
> [ 68.166298] Fixing recursive fault but reboot is needed!
> [ 128.173729] rcu: INFO: rcu_sched detected stalls on CPUs/tasks:
> [ 128.179267] rcu: 11-...0: (1 GPs behind) idle=c56/1/0x4000000000000000 softirq=6897/6898 fqs=5976
> [ 128.182232] rcu: 19-...0: (1 GPs behind) idle=492/1/0x4000000000000000 softirq=1023/1023 fqs=5976
> [ 128.185217] (detected by 10, t=18003 jiffies, g=15789, q=631)
> [ 128.186863] Sending NMI from CPU 10 to CPUs 11:
> [ 128.188902] NMI backtrace for cpu 11
> [ 128.188903] CPU: 11 PID: 2546 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
> [ 128.188904] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
> [ 128.188904] RIP: 0010:queued_read_lock_slowpath+0x5b/0x70
> [ 128.188906] Code: 03 00 02 00 00 8b 03 84 c0 74 08 f3 90 8b 13 84 d2 75 f8 48 89 ef e8 74 e6 ff ff 66 90 5b 5d c3 8b 07 84 c0 74 08 f3 90 8b 03 <84> c0 75 f8 5b 5d c3 89 c6 48 89 ef e8 e4 e8 ff ff 66 90 eb bf 0f
> [ 128.188907] RSP: 0018:ffffc90000348c08 EFLAGS: 00000086
> [ 128.188908] RAX: 0000000037617cc0 RBX: ffff889037617920 RCX: 00000000000000c3
> [ 128.188909] RDX: 0000000000000001 RSI: 0000000000000001 RDI: ffff889037617920
> [ 128.188910] RBP: ffff889037bb1a80 R08: 00000000000000c3 R09: ffffc90000348cd8
> [ 128.188910] R10: 0100000000000000 R11: 00000000de2ee17e R12: 0000000000000046
> [ 128.188911] R13: ffff889037617920 R14: 0000000000000001 R15: 00000000000000c3
> [ 128.188912] FS: 00007f0b97743740(0000) GS:ffff88903fac0000(0000) knlGS:0000000000000000
> [ 128.188912] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> [ 128.188913] CR2: 00007f0b16afc00e CR3: 000000102e5c6002 CR4: 0000000000760ee0
> [ 128.188914] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> [ 128.188914] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> [ 128.188915] PKRU: 55555554
> [ 128.188915] Call Trace:
> [ 128.188916] <IRQ>
> [ 128.188916] _raw_read_lock_irqsave+0x26/0x30
> [ 128.188917] ep_poll_callback+0x3e/0x2b0
> [ 128.188917] ? cpumask_next_and+0x19/0x20
> [ 128.188918] ? update_sd_lb_stats.constprop.0+0xfe/0x810
> [ 128.188918] __wake_up_common+0x7a/0x140
> [ 128.188919] __wake_up_common_lock+0x7c/0xc0
> [ 128.188919] sock_def_readable+0x37/0x60
> [ 128.188920] __udp_enqueue_schedule_skb+0x168/0x260
> [ 128.188920] udpv6_queue_rcv_one_skb+0x284/0x3c0
> [ 128.188921] udp6_unicast_rcv_skb.isra.0+0x44/0xa0
> [ 128.188921] ip6_protocol_deliver_rcu+0x235/0x4b0
> [ 128.188922] ip6_input_finish+0x11/0x20
> [ 128.188922] ip6_input+0xa2/0xb0
> [ 128.188923] ? ip6_protocol_deliver_rcu+0x4b0/0x4b0
> [ 128.188923] ipv6_rcv+0xc0/0xd0
> [ 128.188924] ? ip6_rcv_finish_core.isra.0+0xd0/0xd0
> [ 128.188924] __netif_receive_skb_one_core+0x63/0xa0
> [ 128.188925] process_backlog+0x98/0x140
> [ 128.188925] net_rx_action+0x13a/0x370
> [ 128.188926] __do_softirq+0xe0/0x2ca
> [ 128.188926] do_softirq_own_stack+0x2a/0x40
> [ 128.188926] </IRQ>
> [ 128.188927] do_softirq.part.0+0x2b/0x30
> [ 128.188927] __local_bh_enable_ip+0x4b/0x50
> [ 128.188928] ip6_finish_output2+0x264/0x5b0
> [ 128.188928] ip6_output+0x73/0x120
> [ 128.188929] ? __ip6_finish_output+0x110/0x110
> [ 128.188929] ip6_send_skb+0x1e/0x60
> [ 128.188930] udp_v6_send_skb.isra.0+0x197/0x460
> [ 128.188930] udpv6_sendmsg+0xb4f/0xdb0
> [ 128.188931] ? ip_reply_glue_bits+0x40/0x40
> [ 128.188931] ? update_load_avg+0x78/0x630
> [ 128.188932] ? update_curr+0x73/0x1d0
> [ 128.188932] ? __sys_sendto+0x108/0x190
> [ 128.188933] __sys_sendto+0x108/0x190
> [ 128.188933] ? __fput+0x1a5/0x240
> [ 128.188934] ? _cond_resched+0x19/0x30
> [ 128.188934] ? task_work_run+0x67/0x90
> [ 128.188935] __x64_sys_sendto+0x25/0x30
> [ 128.188935] do_syscall_64+0x48/0x130
> [ 128.188936] entry_SYSCALL_64_after_hwframe+0x44/0xa9
> [ 128.188936] RIP: 0033:0x7f0b97a7826c
> [ 128.188945] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
> [ 128.188946] RSP: 002b:00007ffe80ffcea8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
> [ 128.188947] RAX: ffffffffffffffda RBX: 0000000000000058 RCX: 00007f0b97a7826c
> [ 128.188948] RDX: 0000000000000058 RSI: 000055fb765264c0 RDI: 0000000000000009
> [ 128.188949] RBP: 000055fb765264c0 R08: 0000000000000000 R09: 0000000000000000
> [ 128.188949] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffcfc0
> [ 128.188950] R13: 000000000002c600 R14: 0000000000000000 R15: 000055fb7629b751
> [ 128.188957] Sending NMI from CPU 10 to CPUs 19:
> [ 128.239939] NMI backtrace for cpu 19
> [ 128.239940] CPU: 19 PID: 2587 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
> [ 128.239940] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
> [ 128.239940] RIP: 0010:kvm_wait+0x30/0x50
> [ 128.239941] Code: 8b 05 ac cb f4 7e a9 00 00 f0 00 75 1f 9c 58 fa 0f b6 17 40 38 d6 75 12 f6 c4 02 75 10 e9 07 00 00 00 0f 00 2d 6f ce 54 01 f4 <50> 9d c3 e9 07 00 00 00 0f 00 2d 5f ce 54 01 fb f4 eb ed 66 66 2e
> [ 128.239942] RSP: 0018:ffffc90000cc78e8 EFLAGS: 00000046
> [ 128.239942] RAX: 0000000000000046 RBX: ffff88903fcec700 RCX: 0000000000000008
> [ 128.239943] RDX: 0000000000000003 RSI: 0000000000000003 RDI: ffff889039049d80
> [ 128.239943] RBP: ffff889039049d80 R08: ffff88907ffe9f80 R09: 00000000000000f8
> [ 128.239944] R10: 0000000000000000 R11: 0000000000000001 R12: 0000000000000000
> [ 128.239944] R13: 0000000000000001 R14: 0000000000000100 R15: 0000000000500000
> [ 128.239944] FS: 00007f0b97743740(0000) GS:ffff88903fcc0000(0000) knlGS:0000000000000000
> [ 128.239945] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> [ 128.239945] CR2: 000055fb76a96030 CR3: 0000001035882004 CR4: 0000000000760ee0
> [ 128.239945] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> [ 128.239946] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> [ 128.239946] PKRU: 55555554
> [ 128.239946] Call Trace:
> [ 128.239946] __pv_queued_spin_lock_slowpath+0x26e/0x2b0
> [ 128.239947] _raw_spin_lock_irqsave+0x25/0x30
> [ 128.239947] __wake_up_common_lock+0x62/0xc0
> [ 128.239947] sock_def_write_space+0x49/0x90
> [ 128.239948] sock_wfree+0x68/0xb0
> [ 128.239948] loopback_xmit+0x49/0xe0
> [ 128.239948] dev_hard_start_xmit+0x8d/0x1e0
> [ 128.239948] __dev_queue_xmit+0x721/0x8e0
> [ 128.239949] ip6_finish_output2+0x250/0x5b0
> [ 128.239949] ip6_output+0x73/0x120
> [ 128.239949] ? __ip6_finish_output+0x110/0x110
> [ 128.239950] ip6_send_skb+0x1e/0x60
> [ 128.239950] udp_v6_send_skb.isra.0+0x197/0x460
> [ 128.239950] udpv6_sendmsg+0xb4f/0xdb0
> [ 128.239950] ? release_pages+0x28f/0x2f0
> [ 128.239950] ? ip_reply_glue_bits+0x40/0x40
> [ 128.239951] ? _cond_resched+0x19/0x30
> [ 128.239951] ? unmap_page_range+0x678/0xa60
> [ 128.239951] ? __sys_sendto+0x108/0x190
> [ 128.239951] __sys_sendto+0x108/0x190
> [ 128.239952] ? __fput+0x1a5/0x240
> [ 128.239952] ? _cond_resched+0x19/0x30
> [ 128.239952] ? task_work_run+0x67/0x90
> [ 128.239952] __x64_sys_sendto+0x25/0x30
> [ 128.239953] do_syscall_64+0x48/0x130
> [ 128.239953] entry_SYSCALL_64_after_hwframe+0x44/0xa9
> [ 128.239953] RIP: 0033:0x7f0b97a7826c
> [ 128.239954] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
> [ 128.239954] RSP: 002b:00007ffe80ffd7b8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
> [ 128.239955] RAX: ffffffffffffffda RBX: 00000000000003a8 RCX: 00007f0b97a7826c
> [ 128.239955] RDX: 00000000000003a8 RSI: 00007ffe80ffd800 RDI: 0000000000000009
> [ 128.239955] RBP: 00007ffe80ffd800 R08: 0000000000000000 R09: 0000000000000000
> [ 128.239956] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffd800
> [ 128.239956] R13: 00007ffe80ffd800 R14: 000000000000000e R15: 000055fb76b37e58

I can reproduce this, and I see what it is. I'll send out a patch soonish.

--
Jens Axboe

2020-06-05 21:16:15

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/5/20 2:53 PM, Jens Axboe wrote:
> On 6/5/20 2:36 PM, Andres Freund wrote:
>> Hi,
>>
>> On 2020-06-05 13:20:28 -0700, Andres Freund wrote:
>>> I'll go and try to figure out why I don't see an oops...
>>
>> Err, that probably was a typo on my end in the serial console
>> config. After fixing that, I did get the below.
>>
>> If helpful I can try with debugging enabled or such.
>>
>> [ 67.910265] tee (2577): drop_caches: 3
>> [ 68.059674] BUG: unable to handle page fault for address: 00007f0b16a3c03c
>> [ 68.062021] #PF: supervisor read access in kernel mode
>> [ 68.063742] #PF: error_code(0x0000) - not-present page
>> [ 68.065517] PGD 102e044067 P4D 102e044067 PUD 102bf7a067 PMD 0
>> [ 68.067519] Oops: 0000 [#1] SMP NOPTI
>> [ 68.068800] CPU: 2 PID: 2554 Comm: postgres Not tainted 5.7.0-andres-10123-g87823242260e #44
>> [ 68.071505] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>> [ 68.074139] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>> [ 68.075389] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>> [ 68.079125] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>> [ 68.080260] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>> [ 68.084115] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>> [ 68.085374] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>> [ 68.086409] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>> [ 68.087447] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>> [ 68.088697] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>> [ 68.089903] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>> [ 68.090776] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
>> [ 68.091834] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>> [ 68.092902] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>> [ 68.093967] PKRU: 55555554
>> [ 68.094427] Call Trace:
>> [ 68.094845] ? __schedule+0x2ac/0x720
>> [ 68.095350] ? schedule+0x55/0xc0
>> [ 68.095797] ? ep_read_events_proc+0xd0/0xd0
>> [ 68.096354] ep_scan_ready_list.constprop.0+0x16c/0x190
>> [ 68.097016] ep_poll+0x2a3/0x440
>> [ 68.097449] ? wait_woken+0x70/0x70
>> [ 68.097904] do_epoll_wait+0xb0/0xd0
>> [ 68.098375] __x64_sys_epoll_wait+0x1a/0x20
>> [ 68.098913] do_syscall_64+0x48/0x130
>> [ 68.099393] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>> [ 68.100030] RIP: 0033:0x7f0b97845606
>> [ 68.100498] Code: 48 83 c8 ff c3 66 2e 0f 1f 84 00 00 00 00 00 0f 1f 44 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 11 b8 e8 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 5a c3 90 48 83 ec 28 89 54 24 18 48 89 74 24
>> [ 68.102718] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
>> [ 68.103644] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
>> [ 68.104533] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
>> [ 68.105418] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
>> [ 68.106296] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
>> [ 68.107187] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
>> [ 68.108104] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
>> [ 68.109505] CR2: 00007f0b16a3c03c
>> [ 68.109962] ---[ end trace 0ca39a5ed99162ce ]---
>> [ 68.110547] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>> [ 68.111214] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>> [ 68.113435] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>> [ 68.114111] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>> [ 68.115016] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>> [ 68.115902] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>> [ 68.116810] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>> [ 68.117663] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>> [ 68.118520] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>> [ 68.119482] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>> [ 68.120181] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
>> [ 68.121043] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>> [ 68.121904] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>> [ 68.122790] PKRU: 55555554
>> [ 68.123401] general protection fault, probably for non-canonical address 0xfeeda989fef06266: 0000 [#2] SMP NOPTI
>> [ 68.125052] CPU: 2 PID: 2554 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>> [ 68.126260] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>> [ 68.127274] RIP: 0010:__pv_queued_spin_lock_slowpath+0x1a1/0x2b0
>> [ 68.128032] Code: c4 c1 ea 12 41 be 01 00 00 00 8d 42 ff 41 83 e4 03 4c 8d 6b 14 49 c1 e4 05 48 98 49 81 c4 00 c7 02 00 4c 03 24 c5 e0 e6 fd 82 <49> 89 1c 24 b8 00 80 00 00 eb 15 84 c0 75 0a 41 0f b6 54 24 14 84
>> [ 68.130221] RSP: 0018:ffffc90000befce8 EFLAGS: 00010086
>> [ 68.130867] RAX: 0000000000003ffe RBX: ffff88903f8ac700 RCX: 0000000000000001
>> [ 68.131752] RDX: 0000000000003fff RSI: 0000000000000000 RDI: 0000000000000000
>> [ 68.132637] RBP: ffff889037617924 R08: 0000000000000000 R09: ffffc90000befdf8
>> [ 68.133513] R10: ffff8890334d2bf0 R11: 0000000000000018 R12: feeda989fef06266
>> [ 68.134399] R13: ffff88903f8ac714 R14: 0000000000000001 R15: 00000000000c0000
>> [ 68.135323] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>> [ 68.136307] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>> [ 68.137004] CR2: 00007f0b16a3c03c CR3: 000000000360a005 CR4: 0000000000760ee0
>> [ 68.137866] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>> [ 68.138748] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>> [ 68.139606] PKRU: 55555554
>> [ 68.139973] Call Trace:
>> [ 68.140313] queued_read_lock_slowpath+0x6c/0x70
>> [ 68.140915] _raw_read_lock_irqsave+0x26/0x30
>> [ 68.141485] ep_poll_callback+0x3e/0x2b0
>> [ 68.142007] ? set_next_entity+0xab/0x1f0
>> [ 68.142541] __wake_up_common+0x7a/0x140
>> [ 68.143077] __wake_up_common_lock+0x7c/0xc0
>> [ 68.143651] pipe_release+0x5b/0xd0
>> [ 68.144150] __fput+0xda/0x240
>> [ 68.144574] task_work_run+0x62/0x90
>> [ 68.145046] do_exit+0x35c/0xa70
>> [ 68.145505] ? do_epoll_wait+0xb0/0xd0
>> [ 68.146000] rewind_stack_do_exit+0x17/0x20
>> [ 68.146538] RIP: 0033:0x7f0b97845606
>> [ 68.146988] Code: Bad RIP value.
>> [ 68.147405] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
>> [ 68.148314] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
>> [ 68.149157] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
>> [ 68.150021] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
>> [ 68.150831] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
>> [ 68.151640] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
>> [ 68.152459] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
>> [ 68.153707] ---[ end trace 0ca39a5ed99162cf ]---
>> [ 68.154282] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>> [ 68.154884] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>> [ 68.156976] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>> [ 68.157614] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>> [ 68.158436] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>> [ 68.159269] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>> [ 68.160092] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>> [ 68.160920] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>> [ 68.161746] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>> [ 68.162701] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>> [ 68.163386] CR2: 00007f0b978455dc CR3: 000000000360a005 CR4: 0000000000760ee0
>> [ 68.164226] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>> [ 68.165079] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>> [ 68.165931] PKRU: 55555554
>> [ 68.166298] Fixing recursive fault but reboot is needed!
>> [ 128.173729] rcu: INFO: rcu_sched detected stalls on CPUs/tasks:
>> [ 128.179267] rcu: 11-...0: (1 GPs behind) idle=c56/1/0x4000000000000000 softirq=6897/6898 fqs=5976
>> [ 128.182232] rcu: 19-...0: (1 GPs behind) idle=492/1/0x4000000000000000 softirq=1023/1023 fqs=5976
>> [ 128.185217] (detected by 10, t=18003 jiffies, g=15789, q=631)
>> [ 128.186863] Sending NMI from CPU 10 to CPUs 11:
>> [ 128.188902] NMI backtrace for cpu 11
>> [ 128.188903] CPU: 11 PID: 2546 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>> [ 128.188904] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>> [ 128.188904] RIP: 0010:queued_read_lock_slowpath+0x5b/0x70
>> [ 128.188906] Code: 03 00 02 00 00 8b 03 84 c0 74 08 f3 90 8b 13 84 d2 75 f8 48 89 ef e8 74 e6 ff ff 66 90 5b 5d c3 8b 07 84 c0 74 08 f3 90 8b 03 <84> c0 75 f8 5b 5d c3 89 c6 48 89 ef e8 e4 e8 ff ff 66 90 eb bf 0f
>> [ 128.188907] RSP: 0018:ffffc90000348c08 EFLAGS: 00000086
>> [ 128.188908] RAX: 0000000037617cc0 RBX: ffff889037617920 RCX: 00000000000000c3
>> [ 128.188909] RDX: 0000000000000001 RSI: 0000000000000001 RDI: ffff889037617920
>> [ 128.188910] RBP: ffff889037bb1a80 R08: 00000000000000c3 R09: ffffc90000348cd8
>> [ 128.188910] R10: 0100000000000000 R11: 00000000de2ee17e R12: 0000000000000046
>> [ 128.188911] R13: ffff889037617920 R14: 0000000000000001 R15: 00000000000000c3
>> [ 128.188912] FS: 00007f0b97743740(0000) GS:ffff88903fac0000(0000) knlGS:0000000000000000
>> [ 128.188912] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>> [ 128.188913] CR2: 00007f0b16afc00e CR3: 000000102e5c6002 CR4: 0000000000760ee0
>> [ 128.188914] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>> [ 128.188914] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>> [ 128.188915] PKRU: 55555554
>> [ 128.188915] Call Trace:
>> [ 128.188916] <IRQ>
>> [ 128.188916] _raw_read_lock_irqsave+0x26/0x30
>> [ 128.188917] ep_poll_callback+0x3e/0x2b0
>> [ 128.188917] ? cpumask_next_and+0x19/0x20
>> [ 128.188918] ? update_sd_lb_stats.constprop.0+0xfe/0x810
>> [ 128.188918] __wake_up_common+0x7a/0x140
>> [ 128.188919] __wake_up_common_lock+0x7c/0xc0
>> [ 128.188919] sock_def_readable+0x37/0x60
>> [ 128.188920] __udp_enqueue_schedule_skb+0x168/0x260
>> [ 128.188920] udpv6_queue_rcv_one_skb+0x284/0x3c0
>> [ 128.188921] udp6_unicast_rcv_skb.isra.0+0x44/0xa0
>> [ 128.188921] ip6_protocol_deliver_rcu+0x235/0x4b0
>> [ 128.188922] ip6_input_finish+0x11/0x20
>> [ 128.188922] ip6_input+0xa2/0xb0
>> [ 128.188923] ? ip6_protocol_deliver_rcu+0x4b0/0x4b0
>> [ 128.188923] ipv6_rcv+0xc0/0xd0
>> [ 128.188924] ? ip6_rcv_finish_core.isra.0+0xd0/0xd0
>> [ 128.188924] __netif_receive_skb_one_core+0x63/0xa0
>> [ 128.188925] process_backlog+0x98/0x140
>> [ 128.188925] net_rx_action+0x13a/0x370
>> [ 128.188926] __do_softirq+0xe0/0x2ca
>> [ 128.188926] do_softirq_own_stack+0x2a/0x40
>> [ 128.188926] </IRQ>
>> [ 128.188927] do_softirq.part.0+0x2b/0x30
>> [ 128.188927] __local_bh_enable_ip+0x4b/0x50
>> [ 128.188928] ip6_finish_output2+0x264/0x5b0
>> [ 128.188928] ip6_output+0x73/0x120
>> [ 128.188929] ? __ip6_finish_output+0x110/0x110
>> [ 128.188929] ip6_send_skb+0x1e/0x60
>> [ 128.188930] udp_v6_send_skb.isra.0+0x197/0x460
>> [ 128.188930] udpv6_sendmsg+0xb4f/0xdb0
>> [ 128.188931] ? ip_reply_glue_bits+0x40/0x40
>> [ 128.188931] ? update_load_avg+0x78/0x630
>> [ 128.188932] ? update_curr+0x73/0x1d0
>> [ 128.188932] ? __sys_sendto+0x108/0x190
>> [ 128.188933] __sys_sendto+0x108/0x190
>> [ 128.188933] ? __fput+0x1a5/0x240
>> [ 128.188934] ? _cond_resched+0x19/0x30
>> [ 128.188934] ? task_work_run+0x67/0x90
>> [ 128.188935] __x64_sys_sendto+0x25/0x30
>> [ 128.188935] do_syscall_64+0x48/0x130
>> [ 128.188936] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>> [ 128.188936] RIP: 0033:0x7f0b97a7826c
>> [ 128.188945] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
>> [ 128.188946] RSP: 002b:00007ffe80ffcea8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
>> [ 128.188947] RAX: ffffffffffffffda RBX: 0000000000000058 RCX: 00007f0b97a7826c
>> [ 128.188948] RDX: 0000000000000058 RSI: 000055fb765264c0 RDI: 0000000000000009
>> [ 128.188949] RBP: 000055fb765264c0 R08: 0000000000000000 R09: 0000000000000000
>> [ 128.188949] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffcfc0
>> [ 128.188950] R13: 000000000002c600 R14: 0000000000000000 R15: 000055fb7629b751
>> [ 128.188957] Sending NMI from CPU 10 to CPUs 19:
>> [ 128.239939] NMI backtrace for cpu 19
>> [ 128.239940] CPU: 19 PID: 2587 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>> [ 128.239940] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>> [ 128.239940] RIP: 0010:kvm_wait+0x30/0x50
>> [ 128.239941] Code: 8b 05 ac cb f4 7e a9 00 00 f0 00 75 1f 9c 58 fa 0f b6 17 40 38 d6 75 12 f6 c4 02 75 10 e9 07 00 00 00 0f 00 2d 6f ce 54 01 f4 <50> 9d c3 e9 07 00 00 00 0f 00 2d 5f ce 54 01 fb f4 eb ed 66 66 2e
>> [ 128.239942] RSP: 0018:ffffc90000cc78e8 EFLAGS: 00000046
>> [ 128.239942] RAX: 0000000000000046 RBX: ffff88903fcec700 RCX: 0000000000000008
>> [ 128.239943] RDX: 0000000000000003 RSI: 0000000000000003 RDI: ffff889039049d80
>> [ 128.239943] RBP: ffff889039049d80 R08: ffff88907ffe9f80 R09: 00000000000000f8
>> [ 128.239944] R10: 0000000000000000 R11: 0000000000000001 R12: 0000000000000000
>> [ 128.239944] R13: 0000000000000001 R14: 0000000000000100 R15: 0000000000500000
>> [ 128.239944] FS: 00007f0b97743740(0000) GS:ffff88903fcc0000(0000) knlGS:0000000000000000
>> [ 128.239945] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>> [ 128.239945] CR2: 000055fb76a96030 CR3: 0000001035882004 CR4: 0000000000760ee0
>> [ 128.239945] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>> [ 128.239946] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>> [ 128.239946] PKRU: 55555554
>> [ 128.239946] Call Trace:
>> [ 128.239946] __pv_queued_spin_lock_slowpath+0x26e/0x2b0
>> [ 128.239947] _raw_spin_lock_irqsave+0x25/0x30
>> [ 128.239947] __wake_up_common_lock+0x62/0xc0
>> [ 128.239947] sock_def_write_space+0x49/0x90
>> [ 128.239948] sock_wfree+0x68/0xb0
>> [ 128.239948] loopback_xmit+0x49/0xe0
>> [ 128.239948] dev_hard_start_xmit+0x8d/0x1e0
>> [ 128.239948] __dev_queue_xmit+0x721/0x8e0
>> [ 128.239949] ip6_finish_output2+0x250/0x5b0
>> [ 128.239949] ip6_output+0x73/0x120
>> [ 128.239949] ? __ip6_finish_output+0x110/0x110
>> [ 128.239950] ip6_send_skb+0x1e/0x60
>> [ 128.239950] udp_v6_send_skb.isra.0+0x197/0x460
>> [ 128.239950] udpv6_sendmsg+0xb4f/0xdb0
>> [ 128.239950] ? release_pages+0x28f/0x2f0
>> [ 128.239950] ? ip_reply_glue_bits+0x40/0x40
>> [ 128.239951] ? _cond_resched+0x19/0x30
>> [ 128.239951] ? unmap_page_range+0x678/0xa60
>> [ 128.239951] ? __sys_sendto+0x108/0x190
>> [ 128.239951] __sys_sendto+0x108/0x190
>> [ 128.239952] ? __fput+0x1a5/0x240
>> [ 128.239952] ? _cond_resched+0x19/0x30
>> [ 128.239952] ? task_work_run+0x67/0x90
>> [ 128.239952] __x64_sys_sendto+0x25/0x30
>> [ 128.239953] do_syscall_64+0x48/0x130
>> [ 128.239953] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>> [ 128.239953] RIP: 0033:0x7f0b97a7826c
>> [ 128.239954] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
>> [ 128.239954] RSP: 002b:00007ffe80ffd7b8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
>> [ 128.239955] RAX: ffffffffffffffda RBX: 00000000000003a8 RCX: 00007f0b97a7826c
>> [ 128.239955] RDX: 00000000000003a8 RSI: 00007ffe80ffd800 RDI: 0000000000000009
>> [ 128.239955] RBP: 00007ffe80ffd800 R08: 0000000000000000 R09: 0000000000000000
>> [ 128.239956] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffd800
>> [ 128.239956] R13: 00007ffe80ffd800 R14: 000000000000000e R15: 000055fb76b37e58
>
> I can reproduce this, and I see what it is. I'll send out a patch soonish.

Thinko, can you try with this on top?


diff --git a/fs/io_uring.c b/fs/io_uring.c
index d22830a423f1..ca96ece3ac18 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -2787,9 +2787,11 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
/* if we can retry, do so with the callbacks armed */
if (io_rw_should_retry(req)) {
ret2 = io_iter_do_read(req, &iter);
- if (ret2 != -EAGAIN) {
+ if (ret2 == -EIOCBQUEUED) {
+ return 0;
+ } else if (ret2 != -EAGAIN) {
kiocb_done(kiocb, ret2);
- goto out_free;
+ return 0;
}
}
kiocb->ki_flags &= ~IOCB_WAITQ;

--
Jens Axboe

2020-06-05 21:27:10

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/5/20 3:13 PM, Jens Axboe wrote:
> On 6/5/20 2:53 PM, Jens Axboe wrote:
>> On 6/5/20 2:36 PM, Andres Freund wrote:
>>> Hi,
>>>
>>> On 2020-06-05 13:20:28 -0700, Andres Freund wrote:
>>>> I'll go and try to figure out why I don't see an oops...
>>>
>>> Err, that probably was a typo on my end in the serial console
>>> config. After fixing that, I did get the below.
>>>
>>> If helpful I can try with debugging enabled or such.
>>>
>>> [ 67.910265] tee (2577): drop_caches: 3
>>> [ 68.059674] BUG: unable to handle page fault for address: 00007f0b16a3c03c
>>> [ 68.062021] #PF: supervisor read access in kernel mode
>>> [ 68.063742] #PF: error_code(0x0000) - not-present page
>>> [ 68.065517] PGD 102e044067 P4D 102e044067 PUD 102bf7a067 PMD 0
>>> [ 68.067519] Oops: 0000 [#1] SMP NOPTI
>>> [ 68.068800] CPU: 2 PID: 2554 Comm: postgres Not tainted 5.7.0-andres-10123-g87823242260e #44
>>> [ 68.071505] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>>> [ 68.074139] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>>> [ 68.075389] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>>> [ 68.079125] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>>> [ 68.080260] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>>> [ 68.084115] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>>> [ 68.085374] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>>> [ 68.086409] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>>> [ 68.087447] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>>> [ 68.088697] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>>> [ 68.089903] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>> [ 68.090776] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
>>> [ 68.091834] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>> [ 68.092902] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>> [ 68.093967] PKRU: 55555554
>>> [ 68.094427] Call Trace:
>>> [ 68.094845] ? __schedule+0x2ac/0x720
>>> [ 68.095350] ? schedule+0x55/0xc0
>>> [ 68.095797] ? ep_read_events_proc+0xd0/0xd0
>>> [ 68.096354] ep_scan_ready_list.constprop.0+0x16c/0x190
>>> [ 68.097016] ep_poll+0x2a3/0x440
>>> [ 68.097449] ? wait_woken+0x70/0x70
>>> [ 68.097904] do_epoll_wait+0xb0/0xd0
>>> [ 68.098375] __x64_sys_epoll_wait+0x1a/0x20
>>> [ 68.098913] do_syscall_64+0x48/0x130
>>> [ 68.099393] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>>> [ 68.100030] RIP: 0033:0x7f0b97845606
>>> [ 68.100498] Code: 48 83 c8 ff c3 66 2e 0f 1f 84 00 00 00 00 00 0f 1f 44 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 11 b8 e8 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 5a c3 90 48 83 ec 28 89 54 24 18 48 89 74 24
>>> [ 68.102718] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
>>> [ 68.103644] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
>>> [ 68.104533] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
>>> [ 68.105418] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
>>> [ 68.106296] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
>>> [ 68.107187] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
>>> [ 68.108104] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
>>> [ 68.109505] CR2: 00007f0b16a3c03c
>>> [ 68.109962] ---[ end trace 0ca39a5ed99162ce ]---
>>> [ 68.110547] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>>> [ 68.111214] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>>> [ 68.113435] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>>> [ 68.114111] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>>> [ 68.115016] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>>> [ 68.115902] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>>> [ 68.116810] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>>> [ 68.117663] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>>> [ 68.118520] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>>> [ 68.119482] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>> [ 68.120181] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
>>> [ 68.121043] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>> [ 68.121904] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>> [ 68.122790] PKRU: 55555554
>>> [ 68.123401] general protection fault, probably for non-canonical address 0xfeeda989fef06266: 0000 [#2] SMP NOPTI
>>> [ 68.125052] CPU: 2 PID: 2554 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>>> [ 68.126260] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>>> [ 68.127274] RIP: 0010:__pv_queued_spin_lock_slowpath+0x1a1/0x2b0
>>> [ 68.128032] Code: c4 c1 ea 12 41 be 01 00 00 00 8d 42 ff 41 83 e4 03 4c 8d 6b 14 49 c1 e4 05 48 98 49 81 c4 00 c7 02 00 4c 03 24 c5 e0 e6 fd 82 <49> 89 1c 24 b8 00 80 00 00 eb 15 84 c0 75 0a 41 0f b6 54 24 14 84
>>> [ 68.130221] RSP: 0018:ffffc90000befce8 EFLAGS: 00010086
>>> [ 68.130867] RAX: 0000000000003ffe RBX: ffff88903f8ac700 RCX: 0000000000000001
>>> [ 68.131752] RDX: 0000000000003fff RSI: 0000000000000000 RDI: 0000000000000000
>>> [ 68.132637] RBP: ffff889037617924 R08: 0000000000000000 R09: ffffc90000befdf8
>>> [ 68.133513] R10: ffff8890334d2bf0 R11: 0000000000000018 R12: feeda989fef06266
>>> [ 68.134399] R13: ffff88903f8ac714 R14: 0000000000000001 R15: 00000000000c0000
>>> [ 68.135323] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>>> [ 68.136307] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>> [ 68.137004] CR2: 00007f0b16a3c03c CR3: 000000000360a005 CR4: 0000000000760ee0
>>> [ 68.137866] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>> [ 68.138748] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>> [ 68.139606] PKRU: 55555554
>>> [ 68.139973] Call Trace:
>>> [ 68.140313] queued_read_lock_slowpath+0x6c/0x70
>>> [ 68.140915] _raw_read_lock_irqsave+0x26/0x30
>>> [ 68.141485] ep_poll_callback+0x3e/0x2b0
>>> [ 68.142007] ? set_next_entity+0xab/0x1f0
>>> [ 68.142541] __wake_up_common+0x7a/0x140
>>> [ 68.143077] __wake_up_common_lock+0x7c/0xc0
>>> [ 68.143651] pipe_release+0x5b/0xd0
>>> [ 68.144150] __fput+0xda/0x240
>>> [ 68.144574] task_work_run+0x62/0x90
>>> [ 68.145046] do_exit+0x35c/0xa70
>>> [ 68.145505] ? do_epoll_wait+0xb0/0xd0
>>> [ 68.146000] rewind_stack_do_exit+0x17/0x20
>>> [ 68.146538] RIP: 0033:0x7f0b97845606
>>> [ 68.146988] Code: Bad RIP value.
>>> [ 68.147405] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
>>> [ 68.148314] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
>>> [ 68.149157] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
>>> [ 68.150021] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
>>> [ 68.150831] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
>>> [ 68.151640] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
>>> [ 68.152459] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
>>> [ 68.153707] ---[ end trace 0ca39a5ed99162cf ]---
>>> [ 68.154282] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>>> [ 68.154884] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>>> [ 68.156976] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>>> [ 68.157614] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>>> [ 68.158436] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>>> [ 68.159269] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>>> [ 68.160092] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>>> [ 68.160920] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>>> [ 68.161746] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>>> [ 68.162701] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>> [ 68.163386] CR2: 00007f0b978455dc CR3: 000000000360a005 CR4: 0000000000760ee0
>>> [ 68.164226] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>> [ 68.165079] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>> [ 68.165931] PKRU: 55555554
>>> [ 68.166298] Fixing recursive fault but reboot is needed!
>>> [ 128.173729] rcu: INFO: rcu_sched detected stalls on CPUs/tasks:
>>> [ 128.179267] rcu: 11-...0: (1 GPs behind) idle=c56/1/0x4000000000000000 softirq=6897/6898 fqs=5976
>>> [ 128.182232] rcu: 19-...0: (1 GPs behind) idle=492/1/0x4000000000000000 softirq=1023/1023 fqs=5976
>>> [ 128.185217] (detected by 10, t=18003 jiffies, g=15789, q=631)
>>> [ 128.186863] Sending NMI from CPU 10 to CPUs 11:
>>> [ 128.188902] NMI backtrace for cpu 11
>>> [ 128.188903] CPU: 11 PID: 2546 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>>> [ 128.188904] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>>> [ 128.188904] RIP: 0010:queued_read_lock_slowpath+0x5b/0x70
>>> [ 128.188906] Code: 03 00 02 00 00 8b 03 84 c0 74 08 f3 90 8b 13 84 d2 75 f8 48 89 ef e8 74 e6 ff ff 66 90 5b 5d c3 8b 07 84 c0 74 08 f3 90 8b 03 <84> c0 75 f8 5b 5d c3 89 c6 48 89 ef e8 e4 e8 ff ff 66 90 eb bf 0f
>>> [ 128.188907] RSP: 0018:ffffc90000348c08 EFLAGS: 00000086
>>> [ 128.188908] RAX: 0000000037617cc0 RBX: ffff889037617920 RCX: 00000000000000c3
>>> [ 128.188909] RDX: 0000000000000001 RSI: 0000000000000001 RDI: ffff889037617920
>>> [ 128.188910] RBP: ffff889037bb1a80 R08: 00000000000000c3 R09: ffffc90000348cd8
>>> [ 128.188910] R10: 0100000000000000 R11: 00000000de2ee17e R12: 0000000000000046
>>> [ 128.188911] R13: ffff889037617920 R14: 0000000000000001 R15: 00000000000000c3
>>> [ 128.188912] FS: 00007f0b97743740(0000) GS:ffff88903fac0000(0000) knlGS:0000000000000000
>>> [ 128.188912] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>> [ 128.188913] CR2: 00007f0b16afc00e CR3: 000000102e5c6002 CR4: 0000000000760ee0
>>> [ 128.188914] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>> [ 128.188914] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>> [ 128.188915] PKRU: 55555554
>>> [ 128.188915] Call Trace:
>>> [ 128.188916] <IRQ>
>>> [ 128.188916] _raw_read_lock_irqsave+0x26/0x30
>>> [ 128.188917] ep_poll_callback+0x3e/0x2b0
>>> [ 128.188917] ? cpumask_next_and+0x19/0x20
>>> [ 128.188918] ? update_sd_lb_stats.constprop.0+0xfe/0x810
>>> [ 128.188918] __wake_up_common+0x7a/0x140
>>> [ 128.188919] __wake_up_common_lock+0x7c/0xc0
>>> [ 128.188919] sock_def_readable+0x37/0x60
>>> [ 128.188920] __udp_enqueue_schedule_skb+0x168/0x260
>>> [ 128.188920] udpv6_queue_rcv_one_skb+0x284/0x3c0
>>> [ 128.188921] udp6_unicast_rcv_skb.isra.0+0x44/0xa0
>>> [ 128.188921] ip6_protocol_deliver_rcu+0x235/0x4b0
>>> [ 128.188922] ip6_input_finish+0x11/0x20
>>> [ 128.188922] ip6_input+0xa2/0xb0
>>> [ 128.188923] ? ip6_protocol_deliver_rcu+0x4b0/0x4b0
>>> [ 128.188923] ipv6_rcv+0xc0/0xd0
>>> [ 128.188924] ? ip6_rcv_finish_core.isra.0+0xd0/0xd0
>>> [ 128.188924] __netif_receive_skb_one_core+0x63/0xa0
>>> [ 128.188925] process_backlog+0x98/0x140
>>> [ 128.188925] net_rx_action+0x13a/0x370
>>> [ 128.188926] __do_softirq+0xe0/0x2ca
>>> [ 128.188926] do_softirq_own_stack+0x2a/0x40
>>> [ 128.188926] </IRQ>
>>> [ 128.188927] do_softirq.part.0+0x2b/0x30
>>> [ 128.188927] __local_bh_enable_ip+0x4b/0x50
>>> [ 128.188928] ip6_finish_output2+0x264/0x5b0
>>> [ 128.188928] ip6_output+0x73/0x120
>>> [ 128.188929] ? __ip6_finish_output+0x110/0x110
>>> [ 128.188929] ip6_send_skb+0x1e/0x60
>>> [ 128.188930] udp_v6_send_skb.isra.0+0x197/0x460
>>> [ 128.188930] udpv6_sendmsg+0xb4f/0xdb0
>>> [ 128.188931] ? ip_reply_glue_bits+0x40/0x40
>>> [ 128.188931] ? update_load_avg+0x78/0x630
>>> [ 128.188932] ? update_curr+0x73/0x1d0
>>> [ 128.188932] ? __sys_sendto+0x108/0x190
>>> [ 128.188933] __sys_sendto+0x108/0x190
>>> [ 128.188933] ? __fput+0x1a5/0x240
>>> [ 128.188934] ? _cond_resched+0x19/0x30
>>> [ 128.188934] ? task_work_run+0x67/0x90
>>> [ 128.188935] __x64_sys_sendto+0x25/0x30
>>> [ 128.188935] do_syscall_64+0x48/0x130
>>> [ 128.188936] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>>> [ 128.188936] RIP: 0033:0x7f0b97a7826c
>>> [ 128.188945] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
>>> [ 128.188946] RSP: 002b:00007ffe80ffcea8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
>>> [ 128.188947] RAX: ffffffffffffffda RBX: 0000000000000058 RCX: 00007f0b97a7826c
>>> [ 128.188948] RDX: 0000000000000058 RSI: 000055fb765264c0 RDI: 0000000000000009
>>> [ 128.188949] RBP: 000055fb765264c0 R08: 0000000000000000 R09: 0000000000000000
>>> [ 128.188949] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffcfc0
>>> [ 128.188950] R13: 000000000002c600 R14: 0000000000000000 R15: 000055fb7629b751
>>> [ 128.188957] Sending NMI from CPU 10 to CPUs 19:
>>> [ 128.239939] NMI backtrace for cpu 19
>>> [ 128.239940] CPU: 19 PID: 2587 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>>> [ 128.239940] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>>> [ 128.239940] RIP: 0010:kvm_wait+0x30/0x50
>>> [ 128.239941] Code: 8b 05 ac cb f4 7e a9 00 00 f0 00 75 1f 9c 58 fa 0f b6 17 40 38 d6 75 12 f6 c4 02 75 10 e9 07 00 00 00 0f 00 2d 6f ce 54 01 f4 <50> 9d c3 e9 07 00 00 00 0f 00 2d 5f ce 54 01 fb f4 eb ed 66 66 2e
>>> [ 128.239942] RSP: 0018:ffffc90000cc78e8 EFLAGS: 00000046
>>> [ 128.239942] RAX: 0000000000000046 RBX: ffff88903fcec700 RCX: 0000000000000008
>>> [ 128.239943] RDX: 0000000000000003 RSI: 0000000000000003 RDI: ffff889039049d80
>>> [ 128.239943] RBP: ffff889039049d80 R08: ffff88907ffe9f80 R09: 00000000000000f8
>>> [ 128.239944] R10: 0000000000000000 R11: 0000000000000001 R12: 0000000000000000
>>> [ 128.239944] R13: 0000000000000001 R14: 0000000000000100 R15: 0000000000500000
>>> [ 128.239944] FS: 00007f0b97743740(0000) GS:ffff88903fcc0000(0000) knlGS:0000000000000000
>>> [ 128.239945] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>> [ 128.239945] CR2: 000055fb76a96030 CR3: 0000001035882004 CR4: 0000000000760ee0
>>> [ 128.239945] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>> [ 128.239946] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>> [ 128.239946] PKRU: 55555554
>>> [ 128.239946] Call Trace:
>>> [ 128.239946] __pv_queued_spin_lock_slowpath+0x26e/0x2b0
>>> [ 128.239947] _raw_spin_lock_irqsave+0x25/0x30
>>> [ 128.239947] __wake_up_common_lock+0x62/0xc0
>>> [ 128.239947] sock_def_write_space+0x49/0x90
>>> [ 128.239948] sock_wfree+0x68/0xb0
>>> [ 128.239948] loopback_xmit+0x49/0xe0
>>> [ 128.239948] dev_hard_start_xmit+0x8d/0x1e0
>>> [ 128.239948] __dev_queue_xmit+0x721/0x8e0
>>> [ 128.239949] ip6_finish_output2+0x250/0x5b0
>>> [ 128.239949] ip6_output+0x73/0x120
>>> [ 128.239949] ? __ip6_finish_output+0x110/0x110
>>> [ 128.239950] ip6_send_skb+0x1e/0x60
>>> [ 128.239950] udp_v6_send_skb.isra.0+0x197/0x460
>>> [ 128.239950] udpv6_sendmsg+0xb4f/0xdb0
>>> [ 128.239950] ? release_pages+0x28f/0x2f0
>>> [ 128.239950] ? ip_reply_glue_bits+0x40/0x40
>>> [ 128.239951] ? _cond_resched+0x19/0x30
>>> [ 128.239951] ? unmap_page_range+0x678/0xa60
>>> [ 128.239951] ? __sys_sendto+0x108/0x190
>>> [ 128.239951] __sys_sendto+0x108/0x190
>>> [ 128.239952] ? __fput+0x1a5/0x240
>>> [ 128.239952] ? _cond_resched+0x19/0x30
>>> [ 128.239952] ? task_work_run+0x67/0x90
>>> [ 128.239952] __x64_sys_sendto+0x25/0x30
>>> [ 128.239953] do_syscall_64+0x48/0x130
>>> [ 128.239953] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>>> [ 128.239953] RIP: 0033:0x7f0b97a7826c
>>> [ 128.239954] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
>>> [ 128.239954] RSP: 002b:00007ffe80ffd7b8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
>>> [ 128.239955] RAX: ffffffffffffffda RBX: 00000000000003a8 RCX: 00007f0b97a7826c
>>> [ 128.239955] RDX: 00000000000003a8 RSI: 00007ffe80ffd800 RDI: 0000000000000009
>>> [ 128.239955] RBP: 00007ffe80ffd800 R08: 0000000000000000 R09: 0000000000000000
>>> [ 128.239956] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffd800
>>> [ 128.239956] R13: 00007ffe80ffd800 R14: 000000000000000e R15: 000055fb76b37e58
>>
>> I can reproduce this, and I see what it is. I'll send out a patch soonish.
>
> Thinko, can you try with this on top?

Sorry that was incomplete, please use this one!

diff --git a/fs/io_uring.c b/fs/io_uring.c
index d22830a423f1..0075107dd09a 100644
--- a/fs/io_uring.c
+++ b/fs/io_uring.c
@@ -2779,7 +2779,7 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
ret = io_setup_async_rw(req, io_size, iovec,
inline_vecs, &iter);
if (ret)
- goto out_free;
+ goto out;
/* any defer here is final, must blocking retry */
if (!(req->flags & REQ_F_NOWAIT) &&
!file_can_poll(req->file))
@@ -2787,18 +2787,18 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
/* if we can retry, do so with the callbacks armed */
if (io_rw_should_retry(req)) {
ret2 = io_iter_do_read(req, &iter);
- if (ret2 != -EAGAIN) {
+ if (ret2 == -EIOCBQUEUED) {
+ goto out;
+ } else if (ret2 != -EAGAIN) {
kiocb_done(kiocb, ret2);
- goto out_free;
+ goto out;
}
}
kiocb->ki_flags &= ~IOCB_WAITQ;
return -EAGAIN;
}
}
-out_free:
- kfree(iovec);
- req->flags &= ~REQ_F_NEED_CLEANUP;
+out:
return ret;
}

--
Jens Axboe

2020-06-05 22:32:58

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

On 2020-06-05 15:21:34 -0600, Jens Axboe wrote:
> >> I can reproduce this, and I see what it is. I'll send out a patch soonish.
> >
> > Thinko, can you try with this on top?
>
> Sorry that was incomplete, please use this one!

That seems to fix it! Yay.


Bulk buffered reads somehow don't quite seem to be performing that well
though, looking into it. Could be on the pg side too.

Greetings,

Andres Freund

2020-06-05 22:38:42

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

On 2020-06-05 15:30:44 -0700, Andres Freund wrote:
> On 2020-06-05 15:21:34 -0600, Jens Axboe wrote:
> > >> I can reproduce this, and I see what it is. I'll send out a patch soonish.
> > >
> > > Thinko, can you try with this on top?
> >
> > Sorry that was incomplete, please use this one!
>
> That seems to fix it! Yay.
>
>
> Bulk buffered reads somehow don't quite seem to be performing that well
> though, looking into it. Could be on the pg side too.

While looking into that, I played with setting
/sys/<dev>/queue/read_ahead_kb to 0 and noticed that seems to result in
all/most IO done in workers. Is that to be expected?

Greetings,

Andres Freund

2020-06-05 22:51:54

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/5/20 4:36 PM, Andres Freund wrote:
> Hi,
>
> On 2020-06-05 15:30:44 -0700, Andres Freund wrote:
>> On 2020-06-05 15:21:34 -0600, Jens Axboe wrote:
>>>>> I can reproduce this, and I see what it is. I'll send out a patch soonish.
>>>>
>>>> Thinko, can you try with this on top?
>>>
>>> Sorry that was incomplete, please use this one!
>>
>> That seems to fix it! Yay.
>>
>>
>> Bulk buffered reads somehow don't quite seem to be performing that well
>> though, looking into it. Could be on the pg side too.
>
> While looking into that, I played with setting
> /sys/<dev>/queue/read_ahead_kb to 0 and noticed that seems to result in
> all/most IO done in workers. Is that to be expected?

Yes that's expected, if we have to fallback to ->readpage(), then it'll
go to a worker. read-ahead is what drives the async nature of it, as we
issue the range (plus more, depending on RA window) as read-ahead for
the normal read, then wait for it.

--
Jens Axboe

2020-06-05 22:56:43

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

On 2020-06-05 16:49:24 -0600, Jens Axboe wrote:
> Yes that's expected, if we have to fallback to ->readpage(), then it'll
> go to a worker. read-ahead is what drives the async nature of it, as we
> issue the range (plus more, depending on RA window) as read-ahead for
> the normal read, then wait for it.

But I assume async would still work for files with POSIX_FADV_RANDOM
set, or not? Assuming the system wide setting isn't zero, of course.

Greetings,

Andres Freund

2020-06-05 22:58:58

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/5/20 4:54 PM, Andres Freund wrote:
> Hi,
>
> On 2020-06-05 16:49:24 -0600, Jens Axboe wrote:
>> Yes that's expected, if we have to fallback to ->readpage(), then it'll
>> go to a worker. read-ahead is what drives the async nature of it, as we
>> issue the range (plus more, depending on RA window) as read-ahead for
>> the normal read, then wait for it.
>
> But I assume async would still work for files with POSIX_FADV_RANDOM
> set, or not? Assuming the system wide setting isn't zero, of course.

Yes it'll work if FADV_RANDOM is set. But just not if read-ahead is
totally disabled. I guess we could make that work too, though not sure
that it's super important.

--
Jens Axboe

2020-06-05 23:06:47

by Andres Freund

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

Hi,

On 2020-06-05 16:56:44 -0600, Jens Axboe wrote:
> On 6/5/20 4:54 PM, Andres Freund wrote:
> > On 2020-06-05 16:49:24 -0600, Jens Axboe wrote:
> >> Yes that's expected, if we have to fallback to ->readpage(), then it'll
> >> go to a worker. read-ahead is what drives the async nature of it, as we
> >> issue the range (plus more, depending on RA window) as read-ahead for
> >> the normal read, then wait for it.
> >
> > But I assume async would still work for files with POSIX_FADV_RANDOM
> > set, or not? Assuming the system wide setting isn't zero, of course.
>
> Yes it'll work if FADV_RANDOM is set.

Cool.


> But just not if read-ahead is totally disabled. I guess we could make
> that work too, though not sure that it's super important.

It's not from my end. I was just trying to check if the reduced
performance I saw was related to interactions between PG prefetching and
kernel level prefetching. And changing the /sys entry seemed easier than
making postgres set POSIX_FADV_RANDOM...

Greetings,

Andres Freund

2020-06-06 00:38:12

by Sedat Dilek

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On Fri, Jun 5, 2020 at 11:24 PM Jens Axboe <[email protected]> wrote:
>
> On 6/5/20 3:13 PM, Jens Axboe wrote:
> > On 6/5/20 2:53 PM, Jens Axboe wrote:
> >> On 6/5/20 2:36 PM, Andres Freund wrote:
> >>> Hi,
> >>>
> >>> On 2020-06-05 13:20:28 -0700, Andres Freund wrote:
> >>>> I'll go and try to figure out why I don't see an oops...
> >>>
> >>> Err, that probably was a typo on my end in the serial console
> >>> config. After fixing that, I did get the below.
> >>>
> >>> If helpful I can try with debugging enabled or such.
> >>>
> >>> [ 67.910265] tee (2577): drop_caches: 3
> >>> [ 68.059674] BUG: unable to handle page fault for address: 00007f0b16a3c03c
> >>> [ 68.062021] #PF: supervisor read access in kernel mode
> >>> [ 68.063742] #PF: error_code(0x0000) - not-present page
> >>> [ 68.065517] PGD 102e044067 P4D 102e044067 PUD 102bf7a067 PMD 0
> >>> [ 68.067519] Oops: 0000 [#1] SMP NOPTI
> >>> [ 68.068800] CPU: 2 PID: 2554 Comm: postgres Not tainted 5.7.0-andres-10123-g87823242260e #44
> >>> [ 68.071505] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
> >>> [ 68.074139] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
> >>> [ 68.075389] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
> >>> [ 68.079125] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
> >>> [ 68.080260] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
> >>> [ 68.084115] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
> >>> [ 68.085374] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
> >>> [ 68.086409] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
> >>> [ 68.087447] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
> >>> [ 68.088697] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
> >>> [ 68.089903] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> >>> [ 68.090776] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
> >>> [ 68.091834] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> >>> [ 68.092902] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> >>> [ 68.093967] PKRU: 55555554
> >>> [ 68.094427] Call Trace:
> >>> [ 68.094845] ? __schedule+0x2ac/0x720
> >>> [ 68.095350] ? schedule+0x55/0xc0
> >>> [ 68.095797] ? ep_read_events_proc+0xd0/0xd0
> >>> [ 68.096354] ep_scan_ready_list.constprop.0+0x16c/0x190
> >>> [ 68.097016] ep_poll+0x2a3/0x440
> >>> [ 68.097449] ? wait_woken+0x70/0x70
> >>> [ 68.097904] do_epoll_wait+0xb0/0xd0
> >>> [ 68.098375] __x64_sys_epoll_wait+0x1a/0x20
> >>> [ 68.098913] do_syscall_64+0x48/0x130
> >>> [ 68.099393] entry_SYSCALL_64_after_hwframe+0x44/0xa9
> >>> [ 68.100030] RIP: 0033:0x7f0b97845606
> >>> [ 68.100498] Code: 48 83 c8 ff c3 66 2e 0f 1f 84 00 00 00 00 00 0f 1f 44 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 11 b8 e8 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 5a c3 90 48 83 ec 28 89 54 24 18 48 89 74 24
> >>> [ 68.102718] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
> >>> [ 68.103644] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
> >>> [ 68.104533] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
> >>> [ 68.105418] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
> >>> [ 68.106296] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
> >>> [ 68.107187] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
> >>> [ 68.108104] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
> >>> [ 68.109505] CR2: 00007f0b16a3c03c
> >>> [ 68.109962] ---[ end trace 0ca39a5ed99162ce ]---
> >>> [ 68.110547] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
> >>> [ 68.111214] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
> >>> [ 68.113435] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
> >>> [ 68.114111] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
> >>> [ 68.115016] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
> >>> [ 68.115902] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
> >>> [ 68.116810] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
> >>> [ 68.117663] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
> >>> [ 68.118520] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
> >>> [ 68.119482] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> >>> [ 68.120181] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
> >>> [ 68.121043] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> >>> [ 68.121904] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> >>> [ 68.122790] PKRU: 55555554
> >>> [ 68.123401] general protection fault, probably for non-canonical address 0xfeeda989fef06266: 0000 [#2] SMP NOPTI
> >>> [ 68.125052] CPU: 2 PID: 2554 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
> >>> [ 68.126260] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
> >>> [ 68.127274] RIP: 0010:__pv_queued_spin_lock_slowpath+0x1a1/0x2b0
> >>> [ 68.128032] Code: c4 c1 ea 12 41 be 01 00 00 00 8d 42 ff 41 83 e4 03 4c 8d 6b 14 49 c1 e4 05 48 98 49 81 c4 00 c7 02 00 4c 03 24 c5 e0 e6 fd 82 <49> 89 1c 24 b8 00 80 00 00 eb 15 84 c0 75 0a 41 0f b6 54 24 14 84
> >>> [ 68.130221] RSP: 0018:ffffc90000befce8 EFLAGS: 00010086
> >>> [ 68.130867] RAX: 0000000000003ffe RBX: ffff88903f8ac700 RCX: 0000000000000001
> >>> [ 68.131752] RDX: 0000000000003fff RSI: 0000000000000000 RDI: 0000000000000000
> >>> [ 68.132637] RBP: ffff889037617924 R08: 0000000000000000 R09: ffffc90000befdf8
> >>> [ 68.133513] R10: ffff8890334d2bf0 R11: 0000000000000018 R12: feeda989fef06266
> >>> [ 68.134399] R13: ffff88903f8ac714 R14: 0000000000000001 R15: 00000000000c0000
> >>> [ 68.135323] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
> >>> [ 68.136307] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> >>> [ 68.137004] CR2: 00007f0b16a3c03c CR3: 000000000360a005 CR4: 0000000000760ee0
> >>> [ 68.137866] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> >>> [ 68.138748] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> >>> [ 68.139606] PKRU: 55555554
> >>> [ 68.139973] Call Trace:
> >>> [ 68.140313] queued_read_lock_slowpath+0x6c/0x70
> >>> [ 68.140915] _raw_read_lock_irqsave+0x26/0x30
> >>> [ 68.141485] ep_poll_callback+0x3e/0x2b0
> >>> [ 68.142007] ? set_next_entity+0xab/0x1f0
> >>> [ 68.142541] __wake_up_common+0x7a/0x140
> >>> [ 68.143077] __wake_up_common_lock+0x7c/0xc0
> >>> [ 68.143651] pipe_release+0x5b/0xd0
> >>> [ 68.144150] __fput+0xda/0x240
> >>> [ 68.144574] task_work_run+0x62/0x90
> >>> [ 68.145046] do_exit+0x35c/0xa70
> >>> [ 68.145505] ? do_epoll_wait+0xb0/0xd0
> >>> [ 68.146000] rewind_stack_do_exit+0x17/0x20
> >>> [ 68.146538] RIP: 0033:0x7f0b97845606
> >>> [ 68.146988] Code: Bad RIP value.
> >>> [ 68.147405] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
> >>> [ 68.148314] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
> >>> [ 68.149157] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
> >>> [ 68.150021] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
> >>> [ 68.150831] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
> >>> [ 68.151640] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
> >>> [ 68.152459] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
> >>> [ 68.153707] ---[ end trace 0ca39a5ed99162cf ]---
> >>> [ 68.154282] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
> >>> [ 68.154884] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
> >>> [ 68.156976] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
> >>> [ 68.157614] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
> >>> [ 68.158436] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
> >>> [ 68.159269] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
> >>> [ 68.160092] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
> >>> [ 68.160920] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
> >>> [ 68.161746] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
> >>> [ 68.162701] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> >>> [ 68.163386] CR2: 00007f0b978455dc CR3: 000000000360a005 CR4: 0000000000760ee0
> >>> [ 68.164226] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> >>> [ 68.165079] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> >>> [ 68.165931] PKRU: 55555554
> >>> [ 68.166298] Fixing recursive fault but reboot is needed!
> >>> [ 128.173729] rcu: INFO: rcu_sched detected stalls on CPUs/tasks:
> >>> [ 128.179267] rcu: 11-...0: (1 GPs behind) idle=c56/1/0x4000000000000000 softirq=6897/6898 fqs=5976
> >>> [ 128.182232] rcu: 19-...0: (1 GPs behind) idle=492/1/0x4000000000000000 softirq=1023/1023 fqs=5976
> >>> [ 128.185217] (detected by 10, t=18003 jiffies, g=15789, q=631)
> >>> [ 128.186863] Sending NMI from CPU 10 to CPUs 11:
> >>> [ 128.188902] NMI backtrace for cpu 11
> >>> [ 128.188903] CPU: 11 PID: 2546 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
> >>> [ 128.188904] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
> >>> [ 128.188904] RIP: 0010:queued_read_lock_slowpath+0x5b/0x70
> >>> [ 128.188906] Code: 03 00 02 00 00 8b 03 84 c0 74 08 f3 90 8b 13 84 d2 75 f8 48 89 ef e8 74 e6 ff ff 66 90 5b 5d c3 8b 07 84 c0 74 08 f3 90 8b 03 <84> c0 75 f8 5b 5d c3 89 c6 48 89 ef e8 e4 e8 ff ff 66 90 eb bf 0f
> >>> [ 128.188907] RSP: 0018:ffffc90000348c08 EFLAGS: 00000086
> >>> [ 128.188908] RAX: 0000000037617cc0 RBX: ffff889037617920 RCX: 00000000000000c3
> >>> [ 128.188909] RDX: 0000000000000001 RSI: 0000000000000001 RDI: ffff889037617920
> >>> [ 128.188910] RBP: ffff889037bb1a80 R08: 00000000000000c3 R09: ffffc90000348cd8
> >>> [ 128.188910] R10: 0100000000000000 R11: 00000000de2ee17e R12: 0000000000000046
> >>> [ 128.188911] R13: ffff889037617920 R14: 0000000000000001 R15: 00000000000000c3
> >>> [ 128.188912] FS: 00007f0b97743740(0000) GS:ffff88903fac0000(0000) knlGS:0000000000000000
> >>> [ 128.188912] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> >>> [ 128.188913] CR2: 00007f0b16afc00e CR3: 000000102e5c6002 CR4: 0000000000760ee0
> >>> [ 128.188914] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> >>> [ 128.188914] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> >>> [ 128.188915] PKRU: 55555554
> >>> [ 128.188915] Call Trace:
> >>> [ 128.188916] <IRQ>
> >>> [ 128.188916] _raw_read_lock_irqsave+0x26/0x30
> >>> [ 128.188917] ep_poll_callback+0x3e/0x2b0
> >>> [ 128.188917] ? cpumask_next_and+0x19/0x20
> >>> [ 128.188918] ? update_sd_lb_stats.constprop.0+0xfe/0x810
> >>> [ 128.188918] __wake_up_common+0x7a/0x140
> >>> [ 128.188919] __wake_up_common_lock+0x7c/0xc0
> >>> [ 128.188919] sock_def_readable+0x37/0x60
> >>> [ 128.188920] __udp_enqueue_schedule_skb+0x168/0x260
> >>> [ 128.188920] udpv6_queue_rcv_one_skb+0x284/0x3c0
> >>> [ 128.188921] udp6_unicast_rcv_skb.isra.0+0x44/0xa0
> >>> [ 128.188921] ip6_protocol_deliver_rcu+0x235/0x4b0
> >>> [ 128.188922] ip6_input_finish+0x11/0x20
> >>> [ 128.188922] ip6_input+0xa2/0xb0
> >>> [ 128.188923] ? ip6_protocol_deliver_rcu+0x4b0/0x4b0
> >>> [ 128.188923] ipv6_rcv+0xc0/0xd0
> >>> [ 128.188924] ? ip6_rcv_finish_core.isra.0+0xd0/0xd0
> >>> [ 128.188924] __netif_receive_skb_one_core+0x63/0xa0
> >>> [ 128.188925] process_backlog+0x98/0x140
> >>> [ 128.188925] net_rx_action+0x13a/0x370
> >>> [ 128.188926] __do_softirq+0xe0/0x2ca
> >>> [ 128.188926] do_softirq_own_stack+0x2a/0x40
> >>> [ 128.188926] </IRQ>
> >>> [ 128.188927] do_softirq.part.0+0x2b/0x30
> >>> [ 128.188927] __local_bh_enable_ip+0x4b/0x50
> >>> [ 128.188928] ip6_finish_output2+0x264/0x5b0
> >>> [ 128.188928] ip6_output+0x73/0x120
> >>> [ 128.188929] ? __ip6_finish_output+0x110/0x110
> >>> [ 128.188929] ip6_send_skb+0x1e/0x60
> >>> [ 128.188930] udp_v6_send_skb.isra.0+0x197/0x460
> >>> [ 128.188930] udpv6_sendmsg+0xb4f/0xdb0
> >>> [ 128.188931] ? ip_reply_glue_bits+0x40/0x40
> >>> [ 128.188931] ? update_load_avg+0x78/0x630
> >>> [ 128.188932] ? update_curr+0x73/0x1d0
> >>> [ 128.188932] ? __sys_sendto+0x108/0x190
> >>> [ 128.188933] __sys_sendto+0x108/0x190
> >>> [ 128.188933] ? __fput+0x1a5/0x240
> >>> [ 128.188934] ? _cond_resched+0x19/0x30
> >>> [ 128.188934] ? task_work_run+0x67/0x90
> >>> [ 128.188935] __x64_sys_sendto+0x25/0x30
> >>> [ 128.188935] do_syscall_64+0x48/0x130
> >>> [ 128.188936] entry_SYSCALL_64_after_hwframe+0x44/0xa9
> >>> [ 128.188936] RIP: 0033:0x7f0b97a7826c
> >>> [ 128.188945] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
> >>> [ 128.188946] RSP: 002b:00007ffe80ffcea8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
> >>> [ 128.188947] RAX: ffffffffffffffda RBX: 0000000000000058 RCX: 00007f0b97a7826c
> >>> [ 128.188948] RDX: 0000000000000058 RSI: 000055fb765264c0 RDI: 0000000000000009
> >>> [ 128.188949] RBP: 000055fb765264c0 R08: 0000000000000000 R09: 0000000000000000
> >>> [ 128.188949] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffcfc0
> >>> [ 128.188950] R13: 000000000002c600 R14: 0000000000000000 R15: 000055fb7629b751
> >>> [ 128.188957] Sending NMI from CPU 10 to CPUs 19:
> >>> [ 128.239939] NMI backtrace for cpu 19
> >>> [ 128.239940] CPU: 19 PID: 2587 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
> >>> [ 128.239940] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
> >>> [ 128.239940] RIP: 0010:kvm_wait+0x30/0x50
> >>> [ 128.239941] Code: 8b 05 ac cb f4 7e a9 00 00 f0 00 75 1f 9c 58 fa 0f b6 17 40 38 d6 75 12 f6 c4 02 75 10 e9 07 00 00 00 0f 00 2d 6f ce 54 01 f4 <50> 9d c3 e9 07 00 00 00 0f 00 2d 5f ce 54 01 fb f4 eb ed 66 66 2e
> >>> [ 128.239942] RSP: 0018:ffffc90000cc78e8 EFLAGS: 00000046
> >>> [ 128.239942] RAX: 0000000000000046 RBX: ffff88903fcec700 RCX: 0000000000000008
> >>> [ 128.239943] RDX: 0000000000000003 RSI: 0000000000000003 RDI: ffff889039049d80
> >>> [ 128.239943] RBP: ffff889039049d80 R08: ffff88907ffe9f80 R09: 00000000000000f8
> >>> [ 128.239944] R10: 0000000000000000 R11: 0000000000000001 R12: 0000000000000000
> >>> [ 128.239944] R13: 0000000000000001 R14: 0000000000000100 R15: 0000000000500000
> >>> [ 128.239944] FS: 00007f0b97743740(0000) GS:ffff88903fcc0000(0000) knlGS:0000000000000000
> >>> [ 128.239945] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
> >>> [ 128.239945] CR2: 000055fb76a96030 CR3: 0000001035882004 CR4: 0000000000760ee0
> >>> [ 128.239945] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
> >>> [ 128.239946] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
> >>> [ 128.239946] PKRU: 55555554
> >>> [ 128.239946] Call Trace:
> >>> [ 128.239946] __pv_queued_spin_lock_slowpath+0x26e/0x2b0
> >>> [ 128.239947] _raw_spin_lock_irqsave+0x25/0x30
> >>> [ 128.239947] __wake_up_common_lock+0x62/0xc0
> >>> [ 128.239947] sock_def_write_space+0x49/0x90
> >>> [ 128.239948] sock_wfree+0x68/0xb0
> >>> [ 128.239948] loopback_xmit+0x49/0xe0
> >>> [ 128.239948] dev_hard_start_xmit+0x8d/0x1e0
> >>> [ 128.239948] __dev_queue_xmit+0x721/0x8e0
> >>> [ 128.239949] ip6_finish_output2+0x250/0x5b0
> >>> [ 128.239949] ip6_output+0x73/0x120
> >>> [ 128.239949] ? __ip6_finish_output+0x110/0x110
> >>> [ 128.239950] ip6_send_skb+0x1e/0x60
> >>> [ 128.239950] udp_v6_send_skb.isra.0+0x197/0x460
> >>> [ 128.239950] udpv6_sendmsg+0xb4f/0xdb0
> >>> [ 128.239950] ? release_pages+0x28f/0x2f0
> >>> [ 128.239950] ? ip_reply_glue_bits+0x40/0x40
> >>> [ 128.239951] ? _cond_resched+0x19/0x30
> >>> [ 128.239951] ? unmap_page_range+0x678/0xa60
> >>> [ 128.239951] ? __sys_sendto+0x108/0x190
> >>> [ 128.239951] __sys_sendto+0x108/0x190
> >>> [ 128.239952] ? __fput+0x1a5/0x240
> >>> [ 128.239952] ? _cond_resched+0x19/0x30
> >>> [ 128.239952] ? task_work_run+0x67/0x90
> >>> [ 128.239952] __x64_sys_sendto+0x25/0x30
> >>> [ 128.239953] do_syscall_64+0x48/0x130
> >>> [ 128.239953] entry_SYSCALL_64_after_hwframe+0x44/0xa9
> >>> [ 128.239953] RIP: 0033:0x7f0b97a7826c
> >>> [ 128.239954] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
> >>> [ 128.239954] RSP: 002b:00007ffe80ffd7b8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
> >>> [ 128.239955] RAX: ffffffffffffffda RBX: 00000000000003a8 RCX: 00007f0b97a7826c
> >>> [ 128.239955] RDX: 00000000000003a8 RSI: 00007ffe80ffd800 RDI: 0000000000000009
> >>> [ 128.239955] RBP: 00007ffe80ffd800 R08: 0000000000000000 R09: 0000000000000000
> >>> [ 128.239956] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffd800
> >>> [ 128.239956] R13: 00007ffe80ffd800 R14: 000000000000000e R15: 000055fb76b37e58
> >>
> >> I can reproduce this, and I see what it is. I'll send out a patch soonish.
> >
> > Thinko, can you try with this on top?
>
> Sorry that was incomplete, please use this one!
>
> diff --git a/fs/io_uring.c b/fs/io_uring.c
> index d22830a423f1..0075107dd09a 100644
> --- a/fs/io_uring.c
> +++ b/fs/io_uring.c
> @@ -2779,7 +2779,7 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
> ret = io_setup_async_rw(req, io_size, iovec,
> inline_vecs, &iter);
> if (ret)
> - goto out_free;
> + goto out;
> /* any defer here is final, must blocking retry */
> if (!(req->flags & REQ_F_NOWAIT) &&
> !file_can_poll(req->file))
> @@ -2787,18 +2787,18 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
> /* if we can retry, do so with the callbacks armed */
> if (io_rw_should_retry(req)) {
> ret2 = io_iter_do_read(req, &iter);
> - if (ret2 != -EAGAIN) {
> + if (ret2 == -EIOCBQUEUED) {
> + goto out;
> + } else if (ret2 != -EAGAIN) {
> kiocb_done(kiocb, ret2);
> - goto out_free;
> + goto out;
> }
> }
> kiocb->ki_flags &= ~IOCB_WAITQ;
> return -EAGAIN;
> }
> }
> -out_free:
> - kfree(iovec);
> - req->flags &= ~REQ_F_NEED_CLEANUP;
> +out:
> return ret;
> }
>
> --
> Jens Axboe
>

Hi Jens,

I have tested with today's linux-block.git#async-buffered.7 and my
usual testcases:

[ FIO ]

root# echo 1 > /sys/kernel/debug/tracing/events/io_uring/io_uring_create/enable

root# cat /sys/kernel/debug/tracing/events/io_uring/io_uring_create/enable
1

root# cat buf-test-dileks-min
[buf-test-dileks-min]
filename=/mnt/backup/thinkpad-t530/home/sdi/Downloads/iso-images/iso_debian-stretch/firmware-stretch-live-amd64/debian-live-9.2.0-amd64-kde+nonfree.iso
buffered=1
ioengine=io_uring

root# fio ./buf-test-dileks-min
buf-test-dileks-min: (g=0): rw=read, bs=(R) 4096B-4096B, (W)
4096B-4096B, (T) 4096B-4096B, ioengine=io_uring, iodepth=1
fio-3.20
Starting 1 process
Jobs: 1 (f=1): [R(1)][100.0%][r=87.1MiB/s][r=22.3k IOPS][eta 00m:00s]
buf-test-dileks-min: (groupid=0, jobs=1): err= 0: pid=4076: Sat Jun 6
02:22:20 2020
read: IOPS=20.8k, BW=81.2MiB/s (85.1MB/s)(2811MiB/34626msec)
slat (usec): min=2, max=395, avg= 8.95, stdev=18.43
clat (nsec): min=130, max=159571k, avg=36797.18, stdev=520010.15
lat (usec): min=2, max=159779, avg=46.04, stdev=528.95
clat percentiles (nsec):
| 1.00th=[ 225], 5.00th=[ 302], 10.00th=[ 330],
| 20.00th=[ 358], 30.00th=[ 378], 40.00th=[ 390],
| 50.00th=[ 406], 60.00th=[ 418], 70.00th=[ 430],
| 80.00th=[ 446], 90.00th=[ 478], 95.00th=[ 732],
| 99.00th=[ 1531904], 99.50th=[ 1695744], 99.90th=[ 3555328],
| 99.95th=[ 3686400], 99.99th=[16711680]
bw ( KiB/s): min=34235, max=94554, per=100.00%, avg=83141.75,
stdev=11118.80, samples=69
iops : min= 8558, max=23638, avg=20785.14, stdev=2779.73, samples=69
lat (nsec) : 250=1.63%, 500=89.57%, 750=3.94%, 1000=0.85%
lat (usec) : 2=1.55%, 4=0.21%, 10=0.03%, 20=0.01%, 50=0.02%
lat (usec) : 100=0.01%, 250=0.50%, 500=0.11%, 750=0.02%, 1000=0.01%
lat (msec) : 2=1.18%, 4=0.34%, 10=0.01%, 20=0.01%, 50=0.01%
lat (msec) : 100=0.01%, 250=0.01%
cpu : usr=9.80%, sys=17.79%, ctx=15869, majf=0, minf=15
IO depths : 1=100.0%, 2=0.0%, 4=0.0%, 8=0.0%, 16=0.0%, 32=0.0%, >=64=0.0%
submit : 0=0.0%, 4=100.0%, 8=0.0%, 16=0.0%, 32=0.0%, 64=0.0%, >=64=0.0%
complete : 0=0.0%, 4=100.0%, 8=0.0%, 16=0.0%, 32=0.0%, 64=0.0%, >=64=0.0%
issued rwts: total=719568,0,0,0 short=0,0,0,0 dropped=0,0,0,0
latency : target=0, window=0, percentile=100.00%, depth=1

Run status group 0 (all jobs):
READ: bw=81.2MiB/s (85.1MB/s), 81.2MiB/s-81.2MiB/s
(85.1MB/s-85.1MB/s), io=2811MiB (2947MB), run=34626-34626msec

Disk stats (read/write):
sdc: ios=15784/53, merge=0/89, ticks=57005/536, in_queue=58011, util=99.75%

root# fio --showcmd ./buf-test-dileks-min
fio --name=buf-test-dileks-min --buffered=1 --ioengine=io_uring
--filename=/mnt/backup/thinkpad-t530/home/sdi/Downloads/iso-images/iso_debian-stretch/firmware-stretch-live-amd64/debian-live-9.2.0-amd64-kde+nonfree.iso

root# tail /sys/kernel/debug/tracing/trace
# entries-in-buffer/entries-written: 1/1 #P:4
#
# _-----=> irqs-off
# / _----=> need-resched
# | / _---=> hardirq/softirq
# || / _--=> preempt-depth
# ||| / delay
# TASK-PID CPU# |||| TIMESTAMP FUNCTION
# | | | |||| | |
fio-4076 [001] .... 641.669846: io_uring_create: ring
00000000a88caba9, fd 5 sq size 1, cq size 2, flags 0


[ SYSTEMD-ANALYZE ]

user$ sudo systemd-analyze --no-pager time
Startup finished in 6.228s (kernel) + 41.559s (userspace) = 47.787s
graphical.target reached after 41.539s in userspace

user$ sudo systemd-analyze --no-pager blame
19.800s accounts-daemon.service
19.001s udisks2.service
13.282s NetworkManager-wait-online.service
9.956s polkit.service
9.590s avahi-daemon.service
9.479s NetworkManager.service
9.436s rtkit-daemon.service
9.367s systemd-logind.service
9.272s switcheroo-control.service
9.257s wpa_supplicant.service
9.215s dev-sdc2.device
9.079s powertop.service
8.875s smartmontools.service
8.108s ModemManager.service
7.070s zramswap.service
5.443s networking.service
5.181s rsyslog.service
4.870s gpm.service
3.846s e2scrub_reap.service
3.718s systemd-fsck@dev-disk-by\x2duuid-cfd78a9b\x2dfadd\x2d4870\x2d88c2\x2d5277dae4e59f.service
3.550s apparmor.service
1.807s systemd-journal-flush.service
1.570s systemd-tmpfiles-setup.service
1.269s keyboard-setup.service
1.259s atd.service
1.207s systemd-udevd.service
1.055s bluetooth.service
1.052s binfmt-support.service
935ms exim4.service
901ms systemd-journald.service
850ms systemd-modules-load.service
843ms systemd-sysusers.service
814ms upower.service
807ms dev-hugepages.mount
803ms pppd-dns.service
790ms dev-mqueue.mount
771ms sys-kernel-debug.mount
770ms sys-kernel-tracing.mount
697ms systemd-timesyncd.service
677ms [email protected]
630ms ifupdown-wait-online.service
604ms systemd-rfkill.service
561ms systemd-tmpfiles-setup-dev.service
521ms systemd-remount-fs.service
476ms systemd-sysctl.service
397ms systemd-random-seed.service
394ms systemd-backlight@backlight:intel_backlight.service
376ms systemd-tmpfiles-clean.service
289ms kmod-static-nodes.service
269ms [email protected]
194ms console-setup.service
182ms systemd-udev-trigger.service
134ms systemd-update-utmp.service
125ms proc-sys-fs-binfmt_misc.mount
69ms systemd-user-sessions.service
52ms alsa-restore.service
21ms [email protected]
17ms mnt-sandisk.mount
13ms systemd-update-utmp-runlevel.service
6ms sys-fs-fuse-connections.mount
4ms ifupdown-pre.service

user$ sudo systemd-analyze --no-pager critical-chain
The time when unit became active or started is printed after the "@" character.
The time the unit took to start is printed after the "+" character.

graphical.target @41.539s
└─multi-user.target @41.538s
└─exim4.service @40.601s +935ms
└─network-online.target @40.599s
└─NetworkManager-wait-online.service @27.314s +13.282s
└─NetworkManager.service @17.831s +9.479s
└─dbus.service @17.824s
└─basic.target @17.675s
└─sockets.target @17.672s
└─dbus.socket @17.670s
└─sysinit.target @17.584s
└─systemd-udev-trigger.service @26.747s +182ms
└─systemd-udevd-kernel.socket @3.027s
└─system.slice @2.440s
└─-.slice @2.440s

Hope this helps you.

If you want some further informations and/or testing, please let me
know and give clear instructions.

Thanks.

Regards,
- Sedat -

2020-06-06 16:07:25

by Jens Axboe

[permalink] [raw]
Subject: Re: [PATCHSET v5 0/12] Add support for async buffered reads

On 6/5/20 6:33 PM, Sedat Dilek wrote:
> On Fri, Jun 5, 2020 at 11:24 PM Jens Axboe <[email protected]> wrote:
>>
>> On 6/5/20 3:13 PM, Jens Axboe wrote:
>>> On 6/5/20 2:53 PM, Jens Axboe wrote:
>>>> On 6/5/20 2:36 PM, Andres Freund wrote:
>>>>> Hi,
>>>>>
>>>>> On 2020-06-05 13:20:28 -0700, Andres Freund wrote:
>>>>>> I'll go and try to figure out why I don't see an oops...
>>>>>
>>>>> Err, that probably was a typo on my end in the serial console
>>>>> config. After fixing that, I did get the below.
>>>>>
>>>>> If helpful I can try with debugging enabled or such.
>>>>>
>>>>> [ 67.910265] tee (2577): drop_caches: 3
>>>>> [ 68.059674] BUG: unable to handle page fault for address: 00007f0b16a3c03c
>>>>> [ 68.062021] #PF: supervisor read access in kernel mode
>>>>> [ 68.063742] #PF: error_code(0x0000) - not-present page
>>>>> [ 68.065517] PGD 102e044067 P4D 102e044067 PUD 102bf7a067 PMD 0
>>>>> [ 68.067519] Oops: 0000 [#1] SMP NOPTI
>>>>> [ 68.068800] CPU: 2 PID: 2554 Comm: postgres Not tainted 5.7.0-andres-10123-g87823242260e #44
>>>>> [ 68.071505] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>>>>> [ 68.074139] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>>>>> [ 68.075389] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>>>>> [ 68.079125] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>>>>> [ 68.080260] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>>>>> [ 68.084115] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>>>>> [ 68.085374] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>>>>> [ 68.086409] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>>>>> [ 68.087447] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>>>>> [ 68.088697] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>>>>> [ 68.089903] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>>>> [ 68.090776] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
>>>>> [ 68.091834] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>>>> [ 68.092902] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>>>> [ 68.093967] PKRU: 55555554
>>>>> [ 68.094427] Call Trace:
>>>>> [ 68.094845] ? __schedule+0x2ac/0x720
>>>>> [ 68.095350] ? schedule+0x55/0xc0
>>>>> [ 68.095797] ? ep_read_events_proc+0xd0/0xd0
>>>>> [ 68.096354] ep_scan_ready_list.constprop.0+0x16c/0x190
>>>>> [ 68.097016] ep_poll+0x2a3/0x440
>>>>> [ 68.097449] ? wait_woken+0x70/0x70
>>>>> [ 68.097904] do_epoll_wait+0xb0/0xd0
>>>>> [ 68.098375] __x64_sys_epoll_wait+0x1a/0x20
>>>>> [ 68.098913] do_syscall_64+0x48/0x130
>>>>> [ 68.099393] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>>>>> [ 68.100030] RIP: 0033:0x7f0b97845606
>>>>> [ 68.100498] Code: 48 83 c8 ff c3 66 2e 0f 1f 84 00 00 00 00 00 0f 1f 44 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 11 b8 e8 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 5a c3 90 48 83 ec 28 89 54 24 18 48 89 74 24
>>>>> [ 68.102718] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
>>>>> [ 68.103644] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
>>>>> [ 68.104533] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
>>>>> [ 68.105418] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
>>>>> [ 68.106296] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
>>>>> [ 68.107187] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
>>>>> [ 68.108104] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
>>>>> [ 68.109505] CR2: 00007f0b16a3c03c
>>>>> [ 68.109962] ---[ end trace 0ca39a5ed99162ce ]---
>>>>> [ 68.110547] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>>>>> [ 68.111214] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>>>>> [ 68.113435] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>>>>> [ 68.114111] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>>>>> [ 68.115016] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>>>>> [ 68.115902] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>>>>> [ 68.116810] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>>>>> [ 68.117663] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>>>>> [ 68.118520] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>>>>> [ 68.119482] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>>>> [ 68.120181] CR2: 00007f0b16a3c03c CR3: 000000102cf7c004 CR4: 0000000000760ee0
>>>>> [ 68.121043] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>>>> [ 68.121904] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>>>> [ 68.122790] PKRU: 55555554
>>>>> [ 68.123401] general protection fault, probably for non-canonical address 0xfeeda989fef06266: 0000 [#2] SMP NOPTI
>>>>> [ 68.125052] CPU: 2 PID: 2554 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>>>>> [ 68.126260] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>>>>> [ 68.127274] RIP: 0010:__pv_queued_spin_lock_slowpath+0x1a1/0x2b0
>>>>> [ 68.128032] Code: c4 c1 ea 12 41 be 01 00 00 00 8d 42 ff 41 83 e4 03 4c 8d 6b 14 49 c1 e4 05 48 98 49 81 c4 00 c7 02 00 4c 03 24 c5 e0 e6 fd 82 <49> 89 1c 24 b8 00 80 00 00 eb 15 84 c0 75 0a 41 0f b6 54 24 14 84
>>>>> [ 68.130221] RSP: 0018:ffffc90000befce8 EFLAGS: 00010086
>>>>> [ 68.130867] RAX: 0000000000003ffe RBX: ffff88903f8ac700 RCX: 0000000000000001
>>>>> [ 68.131752] RDX: 0000000000003fff RSI: 0000000000000000 RDI: 0000000000000000
>>>>> [ 68.132637] RBP: ffff889037617924 R08: 0000000000000000 R09: ffffc90000befdf8
>>>>> [ 68.133513] R10: ffff8890334d2bf0 R11: 0000000000000018 R12: feeda989fef06266
>>>>> [ 68.134399] R13: ffff88903f8ac714 R14: 0000000000000001 R15: 00000000000c0000
>>>>> [ 68.135323] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>>>>> [ 68.136307] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>>>> [ 68.137004] CR2: 00007f0b16a3c03c CR3: 000000000360a005 CR4: 0000000000760ee0
>>>>> [ 68.137866] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>>>> [ 68.138748] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>>>> [ 68.139606] PKRU: 55555554
>>>>> [ 68.139973] Call Trace:
>>>>> [ 68.140313] queued_read_lock_slowpath+0x6c/0x70
>>>>> [ 68.140915] _raw_read_lock_irqsave+0x26/0x30
>>>>> [ 68.141485] ep_poll_callback+0x3e/0x2b0
>>>>> [ 68.142007] ? set_next_entity+0xab/0x1f0
>>>>> [ 68.142541] __wake_up_common+0x7a/0x140
>>>>> [ 68.143077] __wake_up_common_lock+0x7c/0xc0
>>>>> [ 68.143651] pipe_release+0x5b/0xd0
>>>>> [ 68.144150] __fput+0xda/0x240
>>>>> [ 68.144574] task_work_run+0x62/0x90
>>>>> [ 68.145046] do_exit+0x35c/0xa70
>>>>> [ 68.145505] ? do_epoll_wait+0xb0/0xd0
>>>>> [ 68.146000] rewind_stack_do_exit+0x17/0x20
>>>>> [ 68.146538] RIP: 0033:0x7f0b97845606
>>>>> [ 68.146988] Code: Bad RIP value.
>>>>> [ 68.147405] RSP: 002b:00007ffe80ffdba8 EFLAGS: 00000246 ORIG_RAX: 00000000000000e8
>>>>> [ 68.148314] RAX: ffffffffffffffda RBX: 000055fb76a9a998 RCX: 00007f0b97845606
>>>>> [ 68.149157] RDX: 0000000000000001 RSI: 000055fb76a9aa10 RDI: 0000000000000005
>>>>> [ 68.150021] RBP: 0000000005000007 R08: 0000000005000007 R09: 0000000000000003
>>>>> [ 68.150831] R10: 00000000ffffffff R11: 0000000000000246 R12: 000055fb76a9a998
>>>>> [ 68.151640] R13: 0000000000000001 R14: 0000000000000009 R15: 000055fb76a9a998
>>>>> [ 68.152459] Modules linked in: 9pnet_virtio isst_if_common xhci_pci 9pnet iTCO_wdt intel_pmc_bxt xhci_hcd iTCO_vendor_support
>>>>> [ 68.153707] ---[ end trace 0ca39a5ed99162cf ]---
>>>>> [ 68.154282] RIP: 0010:__mutex_lock.isra.0+0x76/0x4e0
>>>>> [ 68.154884] Code: b7 01 00 00 48 39 c6 0f 84 a4 03 00 00 65 48 8b 04 25 80 7d 01 00 48 8b 00 a8 08 75 18 49 8b 06 48 83 e0 f8 0f 84 2f 02 00 00 <8b> 50 3c 85 d2 0f 85 12 02 00 00 65 48 8b 04 25 80 7d 01 00 48 8b
>>>>> [ 68.156976] RSP: 0018:ffffc90000befd50 EFLAGS: 00010206
>>>>> [ 68.157614] RAX: 00007f0b16a3c000 RBX: ffff889037617910 RCX: 0000000000000000
>>>>> [ 68.158436] RDX: 00007f0b16a3c000 RSI: ffff88902a4aca80 RDI: ffff8890376178c0
>>>>> [ 68.159269] RBP: ffffc90000befdf0 R08: 0000000000000001 R09: 0000000000000000
>>>>> [ 68.160092] R10: ffff889037617920 R11: ffff88903f8abaf0 R12: ffffc90000befe08
>>>>> [ 68.160920] R13: 0000000000000000 R14: ffff8890376178c0 R15: 0000000000000000
>>>>> [ 68.161746] FS: 00007f0b97743740(0000) GS:ffff88903f880000(0000) knlGS:0000000000000000
>>>>> [ 68.162701] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>>>> [ 68.163386] CR2: 00007f0b978455dc CR3: 000000000360a005 CR4: 0000000000760ee0
>>>>> [ 68.164226] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>>>> [ 68.165079] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>>>> [ 68.165931] PKRU: 55555554
>>>>> [ 68.166298] Fixing recursive fault but reboot is needed!
>>>>> [ 128.173729] rcu: INFO: rcu_sched detected stalls on CPUs/tasks:
>>>>> [ 128.179267] rcu: 11-...0: (1 GPs behind) idle=c56/1/0x4000000000000000 softirq=6897/6898 fqs=5976
>>>>> [ 128.182232] rcu: 19-...0: (1 GPs behind) idle=492/1/0x4000000000000000 softirq=1023/1023 fqs=5976
>>>>> [ 128.185217] (detected by 10, t=18003 jiffies, g=15789, q=631)
>>>>> [ 128.186863] Sending NMI from CPU 10 to CPUs 11:
>>>>> [ 128.188902] NMI backtrace for cpu 11
>>>>> [ 128.188903] CPU: 11 PID: 2546 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>>>>> [ 128.188904] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>>>>> [ 128.188904] RIP: 0010:queued_read_lock_slowpath+0x5b/0x70
>>>>> [ 128.188906] Code: 03 00 02 00 00 8b 03 84 c0 74 08 f3 90 8b 13 84 d2 75 f8 48 89 ef e8 74 e6 ff ff 66 90 5b 5d c3 8b 07 84 c0 74 08 f3 90 8b 03 <84> c0 75 f8 5b 5d c3 89 c6 48 89 ef e8 e4 e8 ff ff 66 90 eb bf 0f
>>>>> [ 128.188907] RSP: 0018:ffffc90000348c08 EFLAGS: 00000086
>>>>> [ 128.188908] RAX: 0000000037617cc0 RBX: ffff889037617920 RCX: 00000000000000c3
>>>>> [ 128.188909] RDX: 0000000000000001 RSI: 0000000000000001 RDI: ffff889037617920
>>>>> [ 128.188910] RBP: ffff889037bb1a80 R08: 00000000000000c3 R09: ffffc90000348cd8
>>>>> [ 128.188910] R10: 0100000000000000 R11: 00000000de2ee17e R12: 0000000000000046
>>>>> [ 128.188911] R13: ffff889037617920 R14: 0000000000000001 R15: 00000000000000c3
>>>>> [ 128.188912] FS: 00007f0b97743740(0000) GS:ffff88903fac0000(0000) knlGS:0000000000000000
>>>>> [ 128.188912] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>>>> [ 128.188913] CR2: 00007f0b16afc00e CR3: 000000102e5c6002 CR4: 0000000000760ee0
>>>>> [ 128.188914] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>>>> [ 128.188914] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>>>> [ 128.188915] PKRU: 55555554
>>>>> [ 128.188915] Call Trace:
>>>>> [ 128.188916] <IRQ>
>>>>> [ 128.188916] _raw_read_lock_irqsave+0x26/0x30
>>>>> [ 128.188917] ep_poll_callback+0x3e/0x2b0
>>>>> [ 128.188917] ? cpumask_next_and+0x19/0x20
>>>>> [ 128.188918] ? update_sd_lb_stats.constprop.0+0xfe/0x810
>>>>> [ 128.188918] __wake_up_common+0x7a/0x140
>>>>> [ 128.188919] __wake_up_common_lock+0x7c/0xc0
>>>>> [ 128.188919] sock_def_readable+0x37/0x60
>>>>> [ 128.188920] __udp_enqueue_schedule_skb+0x168/0x260
>>>>> [ 128.188920] udpv6_queue_rcv_one_skb+0x284/0x3c0
>>>>> [ 128.188921] udp6_unicast_rcv_skb.isra.0+0x44/0xa0
>>>>> [ 128.188921] ip6_protocol_deliver_rcu+0x235/0x4b0
>>>>> [ 128.188922] ip6_input_finish+0x11/0x20
>>>>> [ 128.188922] ip6_input+0xa2/0xb0
>>>>> [ 128.188923] ? ip6_protocol_deliver_rcu+0x4b0/0x4b0
>>>>> [ 128.188923] ipv6_rcv+0xc0/0xd0
>>>>> [ 128.188924] ? ip6_rcv_finish_core.isra.0+0xd0/0xd0
>>>>> [ 128.188924] __netif_receive_skb_one_core+0x63/0xa0
>>>>> [ 128.188925] process_backlog+0x98/0x140
>>>>> [ 128.188925] net_rx_action+0x13a/0x370
>>>>> [ 128.188926] __do_softirq+0xe0/0x2ca
>>>>> [ 128.188926] do_softirq_own_stack+0x2a/0x40
>>>>> [ 128.188926] </IRQ>
>>>>> [ 128.188927] do_softirq.part.0+0x2b/0x30
>>>>> [ 128.188927] __local_bh_enable_ip+0x4b/0x50
>>>>> [ 128.188928] ip6_finish_output2+0x264/0x5b0
>>>>> [ 128.188928] ip6_output+0x73/0x120
>>>>> [ 128.188929] ? __ip6_finish_output+0x110/0x110
>>>>> [ 128.188929] ip6_send_skb+0x1e/0x60
>>>>> [ 128.188930] udp_v6_send_skb.isra.0+0x197/0x460
>>>>> [ 128.188930] udpv6_sendmsg+0xb4f/0xdb0
>>>>> [ 128.188931] ? ip_reply_glue_bits+0x40/0x40
>>>>> [ 128.188931] ? update_load_avg+0x78/0x630
>>>>> [ 128.188932] ? update_curr+0x73/0x1d0
>>>>> [ 128.188932] ? __sys_sendto+0x108/0x190
>>>>> [ 128.188933] __sys_sendto+0x108/0x190
>>>>> [ 128.188933] ? __fput+0x1a5/0x240
>>>>> [ 128.188934] ? _cond_resched+0x19/0x30
>>>>> [ 128.188934] ? task_work_run+0x67/0x90
>>>>> [ 128.188935] __x64_sys_sendto+0x25/0x30
>>>>> [ 128.188935] do_syscall_64+0x48/0x130
>>>>> [ 128.188936] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>>>>> [ 128.188936] RIP: 0033:0x7f0b97a7826c
>>>>> [ 128.188945] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
>>>>> [ 128.188946] RSP: 002b:00007ffe80ffcea8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
>>>>> [ 128.188947] RAX: ffffffffffffffda RBX: 0000000000000058 RCX: 00007f0b97a7826c
>>>>> [ 128.188948] RDX: 0000000000000058 RSI: 000055fb765264c0 RDI: 0000000000000009
>>>>> [ 128.188949] RBP: 000055fb765264c0 R08: 0000000000000000 R09: 0000000000000000
>>>>> [ 128.188949] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffcfc0
>>>>> [ 128.188950] R13: 000000000002c600 R14: 0000000000000000 R15: 000055fb7629b751
>>>>> [ 128.188957] Sending NMI from CPU 10 to CPUs 19:
>>>>> [ 128.239939] NMI backtrace for cpu 19
>>>>> [ 128.239940] CPU: 19 PID: 2587 Comm: postgres Tainted: G D 5.7.0-andres-10123-g87823242260e #44
>>>>> [ 128.239940] Hardware name: QEMU Standard PC (Q35 + ICH9, 2009), BIOS 1.13.0-1 04/01/2014
>>>>> [ 128.239940] RIP: 0010:kvm_wait+0x30/0x50
>>>>> [ 128.239941] Code: 8b 05 ac cb f4 7e a9 00 00 f0 00 75 1f 9c 58 fa 0f b6 17 40 38 d6 75 12 f6 c4 02 75 10 e9 07 00 00 00 0f 00 2d 6f ce 54 01 f4 <50> 9d c3 e9 07 00 00 00 0f 00 2d 5f ce 54 01 fb f4 eb ed 66 66 2e
>>>>> [ 128.239942] RSP: 0018:ffffc90000cc78e8 EFLAGS: 00000046
>>>>> [ 128.239942] RAX: 0000000000000046 RBX: ffff88903fcec700 RCX: 0000000000000008
>>>>> [ 128.239943] RDX: 0000000000000003 RSI: 0000000000000003 RDI: ffff889039049d80
>>>>> [ 128.239943] RBP: ffff889039049d80 R08: ffff88907ffe9f80 R09: 00000000000000f8
>>>>> [ 128.239944] R10: 0000000000000000 R11: 0000000000000001 R12: 0000000000000000
>>>>> [ 128.239944] R13: 0000000000000001 R14: 0000000000000100 R15: 0000000000500000
>>>>> [ 128.239944] FS: 00007f0b97743740(0000) GS:ffff88903fcc0000(0000) knlGS:0000000000000000
>>>>> [ 128.239945] CS: 0010 DS: 0000 ES: 0000 CR0: 0000000080050033
>>>>> [ 128.239945] CR2: 000055fb76a96030 CR3: 0000001035882004 CR4: 0000000000760ee0
>>>>> [ 128.239945] DR0: 0000000000000000 DR1: 0000000000000000 DR2: 0000000000000000
>>>>> [ 128.239946] DR3: 0000000000000000 DR6: 00000000fffe0ff0 DR7: 0000000000000400
>>>>> [ 128.239946] PKRU: 55555554
>>>>> [ 128.239946] Call Trace:
>>>>> [ 128.239946] __pv_queued_spin_lock_slowpath+0x26e/0x2b0
>>>>> [ 128.239947] _raw_spin_lock_irqsave+0x25/0x30
>>>>> [ 128.239947] __wake_up_common_lock+0x62/0xc0
>>>>> [ 128.239947] sock_def_write_space+0x49/0x90
>>>>> [ 128.239948] sock_wfree+0x68/0xb0
>>>>> [ 128.239948] loopback_xmit+0x49/0xe0
>>>>> [ 128.239948] dev_hard_start_xmit+0x8d/0x1e0
>>>>> [ 128.239948] __dev_queue_xmit+0x721/0x8e0
>>>>> [ 128.239949] ip6_finish_output2+0x250/0x5b0
>>>>> [ 128.239949] ip6_output+0x73/0x120
>>>>> [ 128.239949] ? __ip6_finish_output+0x110/0x110
>>>>> [ 128.239950] ip6_send_skb+0x1e/0x60
>>>>> [ 128.239950] udp_v6_send_skb.isra.0+0x197/0x460
>>>>> [ 128.239950] udpv6_sendmsg+0xb4f/0xdb0
>>>>> [ 128.239950] ? release_pages+0x28f/0x2f0
>>>>> [ 128.239950] ? ip_reply_glue_bits+0x40/0x40
>>>>> [ 128.239951] ? _cond_resched+0x19/0x30
>>>>> [ 128.239951] ? unmap_page_range+0x678/0xa60
>>>>> [ 128.239951] ? __sys_sendto+0x108/0x190
>>>>> [ 128.239951] __sys_sendto+0x108/0x190
>>>>> [ 128.239952] ? __fput+0x1a5/0x240
>>>>> [ 128.239952] ? _cond_resched+0x19/0x30
>>>>> [ 128.239952] ? task_work_run+0x67/0x90
>>>>> [ 128.239952] __x64_sys_sendto+0x25/0x30
>>>>> [ 128.239953] do_syscall_64+0x48/0x130
>>>>> [ 128.239953] entry_SYSCALL_64_after_hwframe+0x44/0xa9
>>>>> [ 128.239953] RIP: 0033:0x7f0b97a7826c
>>>>> [ 128.239954] Code: c0 ff ff ff ff eb b9 0f 1f 80 00 00 00 00 41 89 ca 64 8b 04 25 18 00 00 00 85 c0 75 19 45 31 c9 45 31 c0 b8 2c 00 00 00 0f 05 <48> 3d 00 f0 ff ff 77 64 c3 0f 1f 00 55 48 83 ec 20 48 89 54 24 10
>>>>> [ 128.239954] RSP: 002b:00007ffe80ffd7b8 EFLAGS: 00000246 ORIG_RAX: 000000000000002c
>>>>> [ 128.239955] RAX: ffffffffffffffda RBX: 00000000000003a8 RCX: 00007f0b97a7826c
>>>>> [ 128.239955] RDX: 00000000000003a8 RSI: 00007ffe80ffd800 RDI: 0000000000000009
>>>>> [ 128.239955] RBP: 00007ffe80ffd800 R08: 0000000000000000 R09: 0000000000000000
>>>>> [ 128.239956] R10: 0000000000000000 R11: 0000000000000246 R12: 00007ffe80ffd800
>>>>> [ 128.239956] R13: 00007ffe80ffd800 R14: 000000000000000e R15: 000055fb76b37e58
>>>>
>>>> I can reproduce this, and I see what it is. I'll send out a patch soonish.
>>>
>>> Thinko, can you try with this on top?
>>
>> Sorry that was incomplete, please use this one!
>>
>> diff --git a/fs/io_uring.c b/fs/io_uring.c
>> index d22830a423f1..0075107dd09a 100644
>> --- a/fs/io_uring.c
>> +++ b/fs/io_uring.c
>> @@ -2779,7 +2779,7 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
>> ret = io_setup_async_rw(req, io_size, iovec,
>> inline_vecs, &iter);
>> if (ret)
>> - goto out_free;
>> + goto out;
>> /* any defer here is final, must blocking retry */
>> if (!(req->flags & REQ_F_NOWAIT) &&
>> !file_can_poll(req->file))
>> @@ -2787,18 +2787,18 @@ static int io_read(struct io_kiocb *req, bool force_nonblock)
>> /* if we can retry, do so with the callbacks armed */
>> if (io_rw_should_retry(req)) {
>> ret2 = io_iter_do_read(req, &iter);
>> - if (ret2 != -EAGAIN) {
>> + if (ret2 == -EIOCBQUEUED) {
>> + goto out;
>> + } else if (ret2 != -EAGAIN) {
>> kiocb_done(kiocb, ret2);
>> - goto out_free;
>> + goto out;
>> }
>> }
>> kiocb->ki_flags &= ~IOCB_WAITQ;
>> return -EAGAIN;
>> }
>> }
>> -out_free:
>> - kfree(iovec);
>> - req->flags &= ~REQ_F_NEED_CLEANUP;
>> +out:
>> return ret;
>> }
>>
>> --
>> Jens Axboe
>>
>
> Hi Jens,
>
> I have tested with today's linux-block.git#async-buffered.7 and my
> usual testcases:
>
> [ FIO ]
>
> root# echo 1 > /sys/kernel/debug/tracing/events/io_uring/io_uring_create/enable
>
> root# cat /sys/kernel/debug/tracing/events/io_uring/io_uring_create/enable
> 1
>
> root# cat buf-test-dileks-min
> [buf-test-dileks-min]
> filename=/mnt/backup/thinkpad-t530/home/sdi/Downloads/iso-images/iso_debian-stretch/firmware-stretch-live-amd64/debian-live-9.2.0-amd64-kde+nonfree.iso
> buffered=1
> ioengine=io_uring
>
> root# fio ./buf-test-dileks-min
> buf-test-dileks-min: (g=0): rw=read, bs=(R) 4096B-4096B, (W)
> 4096B-4096B, (T) 4096B-4096B, ioengine=io_uring, iodepth=1
> fio-3.20
> Starting 1 process
> Jobs: 1 (f=1): [R(1)][100.0%][r=87.1MiB/s][r=22.3k IOPS][eta 00m:00s]
> buf-test-dileks-min: (groupid=0, jobs=1): err= 0: pid=4076: Sat Jun 6
> 02:22:20 2020
> read: IOPS=20.8k, BW=81.2MiB/s (85.1MB/s)(2811MiB/34626msec)
> slat (usec): min=2, max=395, avg= 8.95, stdev=18.43
> clat (nsec): min=130, max=159571k, avg=36797.18, stdev=520010.15
> lat (usec): min=2, max=159779, avg=46.04, stdev=528.95
> clat percentiles (nsec):
> | 1.00th=[ 225], 5.00th=[ 302], 10.00th=[ 330],
> | 20.00th=[ 358], 30.00th=[ 378], 40.00th=[ 390],
> | 50.00th=[ 406], 60.00th=[ 418], 70.00th=[ 430],
> | 80.00th=[ 446], 90.00th=[ 478], 95.00th=[ 732],
> | 99.00th=[ 1531904], 99.50th=[ 1695744], 99.90th=[ 3555328],
> | 99.95th=[ 3686400], 99.99th=[16711680]
> bw ( KiB/s): min=34235, max=94554, per=100.00%, avg=83141.75,
> stdev=11118.80, samples=69
> iops : min= 8558, max=23638, avg=20785.14, stdev=2779.73, samples=69
> lat (nsec) : 250=1.63%, 500=89.57%, 750=3.94%, 1000=0.85%
> lat (usec) : 2=1.55%, 4=0.21%, 10=0.03%, 20=0.01%, 50=0.02%
> lat (usec) : 100=0.01%, 250=0.50%, 500=0.11%, 750=0.02%, 1000=0.01%
> lat (msec) : 2=1.18%, 4=0.34%, 10=0.01%, 20=0.01%, 50=0.01%
> lat (msec) : 100=0.01%, 250=0.01%
> cpu : usr=9.80%, sys=17.79%, ctx=15869, majf=0, minf=15
> IO depths : 1=100.0%, 2=0.0%, 4=0.0%, 8=0.0%, 16=0.0%, 32=0.0%, >=64=0.0%
> submit : 0=0.0%, 4=100.0%, 8=0.0%, 16=0.0%, 32=0.0%, 64=0.0%, >=64=0.0%
> complete : 0=0.0%, 4=100.0%, 8=0.0%, 16=0.0%, 32=0.0%, 64=0.0%, >=64=0.0%
> issued rwts: total=719568,0,0,0 short=0,0,0,0 dropped=0,0,0,0
> latency : target=0, window=0, percentile=100.00%, depth=1
>
> Run status group 0 (all jobs):
> READ: bw=81.2MiB/s (85.1MB/s), 81.2MiB/s-81.2MiB/s
> (85.1MB/s-85.1MB/s), io=2811MiB (2947MB), run=34626-34626msec
>
> Disk stats (read/write):
> sdc: ios=15784/53, merge=0/89, ticks=57005/536, in_queue=58011, util=99.75%
>
> root# fio --showcmd ./buf-test-dileks-min
> fio --name=buf-test-dileks-min --buffered=1 --ioengine=io_uring
> --filename=/mnt/backup/thinkpad-t530/home/sdi/Downloads/iso-images/iso_debian-stretch/firmware-stretch-live-amd64/debian-live-9.2.0-amd64-kde+nonfree.iso
>
> root# tail /sys/kernel/debug/tracing/trace
> # entries-in-buffer/entries-written: 1/1 #P:4
> #
> # _-----=> irqs-off
> # / _----=> need-resched
> # | / _---=> hardirq/softirq
> # || / _--=> preempt-depth
> # ||| / delay
> # TASK-PID CPU# |||| TIMESTAMP FUNCTION
> # | | | |||| | |
> fio-4076 [001] .... 641.669846: io_uring_create: ring
> 00000000a88caba9, fd 5 sq size 1, cq size 2, flags 0
>
>
> [ SYSTEMD-ANALYZE ]
>
> user$ sudo systemd-analyze --no-pager time
> Startup finished in 6.228s (kernel) + 41.559s (userspace) = 47.787s
> graphical.target reached after 41.539s in userspace
>
> user$ sudo systemd-analyze --no-pager blame
> 19.800s accounts-daemon.service
> 19.001s udisks2.service
> 13.282s NetworkManager-wait-online.service
> 9.956s polkit.service
> 9.590s avahi-daemon.service
> 9.479s NetworkManager.service
> 9.436s rtkit-daemon.service
> 9.367s systemd-logind.service
> 9.272s switcheroo-control.service
> 9.257s wpa_supplicant.service
> 9.215s dev-sdc2.device
> 9.079s powertop.service
> 8.875s smartmontools.service
> 8.108s ModemManager.service
> 7.070s zramswap.service
> 5.443s networking.service
> 5.181s rsyslog.service
> 4.870s gpm.service
> 3.846s e2scrub_reap.service
> 3.718s systemd-fsck@dev-disk-by\x2duuid-cfd78a9b\x2dfadd\x2d4870\x2d88c2\x2d5277dae4e59f.service
> 3.550s apparmor.service
> 1.807s systemd-journal-flush.service
> 1.570s systemd-tmpfiles-setup.service
> 1.269s keyboard-setup.service
> 1.259s atd.service
> 1.207s systemd-udevd.service
> 1.055s bluetooth.service
> 1.052s binfmt-support.service
> 935ms exim4.service
> 901ms systemd-journald.service
> 850ms systemd-modules-load.service
> 843ms systemd-sysusers.service
> 814ms upower.service
> 807ms dev-hugepages.mount
> 803ms pppd-dns.service
> 790ms dev-mqueue.mount
> 771ms sys-kernel-debug.mount
> 770ms sys-kernel-tracing.mount
> 697ms systemd-timesyncd.service
> 677ms [email protected]
> 630ms ifupdown-wait-online.service
> 604ms systemd-rfkill.service
> 561ms systemd-tmpfiles-setup-dev.service
> 521ms systemd-remount-fs.service
> 476ms systemd-sysctl.service
> 397ms systemd-random-seed.service
> 394ms systemd-backlight@backlight:intel_backlight.service
> 376ms systemd-tmpfiles-clean.service
> 289ms kmod-static-nodes.service
> 269ms [email protected]
> 194ms console-setup.service
> 182ms systemd-udev-trigger.service
> 134ms systemd-update-utmp.service
> 125ms proc-sys-fs-binfmt_misc.mount
> 69ms systemd-user-sessions.service
> 52ms alsa-restore.service
> 21ms [email protected]
> 17ms mnt-sandisk.mount
> 13ms systemd-update-utmp-runlevel.service
> 6ms sys-fs-fuse-connections.mount
> 4ms ifupdown-pre.service
>
> user$ sudo systemd-analyze --no-pager critical-chain
> The time when unit became active or started is printed after the "@" character.
> The time the unit took to start is printed after the "+" character.
>
> graphical.target @41.539s
> └─multi-user.target @41.538s
> └─exim4.service @40.601s +935ms
> └─network-online.target @40.599s
> └─NetworkManager-wait-online.service @27.314s +13.282s
> └─NetworkManager.service @17.831s +9.479s
> └─dbus.service @17.824s
> └─basic.target @17.675s
> └─sockets.target @17.672s
> └─dbus.socket @17.670s
> └─sysinit.target @17.584s
> └─systemd-udev-trigger.service @26.747s +182ms
> └─systemd-udevd-kernel.socket @3.027s
> └─system.slice @2.440s
> └─-.slice @2.440s
>
> Hope this helps you.
>
> If you want some further informations and/or testing, please let me
> know and give clear instructions.

Thanks for testing, appreciate it!

--
Jens Axboe