qemu-devel.nongnu.org archive mirror
 help / color / mirror / Atom feed
* [PATCH RFC 0/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
@ 2025-07-16 18:38 Brian Song
  2025-07-16 18:38 ` [PATCH RFC 1/1] " Brian Song
  2025-07-20 16:13 ` [PATCH RFC 0/1] " Stefan Hajnoczi
  0 siblings, 2 replies; 19+ messages in thread
From: Brian Song @ 2025-07-16 18:38 UTC (permalink / raw)
  To: qemu-block
  Cc: qemu-devel, armbru, bschubert, fam, hibriansong, hreitz, kwolf,
	stefanha

This RFC patch represents an initial implementation of the FUSE-over- 
io_uring Exports idea proposed for Google Summer of Code (2025) under 
the QEMU community:
https://wiki.qemu.org/Google_Summer_of_Code_2025#FUSE-over-io_uring_exports

The implementation approach is primarily borrowed from how libfuse 
interacts with the kernel.

FUSE-over-io_uring (https://docs.kernel.org/next/filesystems/fuse-io- 
uring.html) has been officially merged into the Linux kernel. The idea 
is to replace the traditional /dev/fuse based communication with a more 
efficient io_uring-based approach. In this model, userspace registers 
io_uring SQEs via the FUSE_IO_URING_CMD_REGISTER opcode, and then waits 
for the kernel to forward FUSE requests as CQEs. These are processed by 
the FUSE exports implementation in userspace and then committed back to 
the kernel using FUSE_IO_URING_CMD_COMMIT_AND_FETCH.

To enable this feature in qemu-export-daemon, simply add the uring=on 
option to the export configuration.

As this patch is still in the RFC stage, it currently supports **only 
single thread**. Due to protocol requirements in FUSE-over-io_uring,
the number of FUSE threads must match the number of CPUs. Therefore,
this initial version only works on single-core VMs (i.e., QEMU started
with -smp 1) or single core machine.

Brian Song (1):
  block/export: FUSE-over-io_uring Support for QEMU FUSE Exports

 block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
 docs/tools/qemu-storage-daemon.rst   |  10 +-
 qapi/block-export.json               |   6 +-
 storage-daemon/qemu-storage-daemon.c |   1 +
 util/fdmon-io_uring.c                |   5 +-
 5 files changed, 420 insertions(+), 25 deletions(-)

-- 
2.50.1



^ permalink raw reply	[flat|nested] 19+ messages in thread

* [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-16 18:38 [PATCH RFC 0/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports Brian Song
@ 2025-07-16 18:38 ` Brian Song
  2025-07-17  6:03   ` Markus Armbruster
                     ` (3 more replies)
  2025-07-20 16:13 ` [PATCH RFC 0/1] " Stefan Hajnoczi
  1 sibling, 4 replies; 19+ messages in thread
From: Brian Song @ 2025-07-16 18:38 UTC (permalink / raw)
  To: qemu-block
  Cc: qemu-devel, armbru, bschubert, fam, hibriansong, hreitz, kwolf,
	stefanha

This work provides an initial implementation of fuse-over-io_uring
support for QEMU export. According to the fuse-over-io_uring protocol
specification, the userspace side must create the same number of queues
as the number of CPUs (nr_cpu), just like the kernel. Currently, each
queue contains only a single SQE entry, which is used to validate the
correctness of the fuse-over-io_uring functionality.

All FUSE read and write operations interact with the kernel via io
vectors embedded in the SQE entry during submission and CQE fetching.
The req_header and op_payload members of each entry are included as
parts of the io vector: req_header carries the FUSE operation header,
and op_payload carries the data payload, such as file attributes in a
getattr reply, file content in a read reply, or file content being
written to the FUSE client in a write operation.

At present, multi-threading support is still incomplete. In addition,
handling connection termination and managing the "drained" state of a
FUSE block export in QEMU remain as pending work.

Suggested-by: Kevin Wolf <kwolf@redhat.com>
Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
Signed-off-by: Brian Song <hibriansong@gmail.com>

---
 block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
 docs/tools/qemu-storage-daemon.rst   |  10 +-
 qapi/block-export.json               |   6 +-
 storage-daemon/qemu-storage-daemon.c |   1 +
 util/fdmon-io_uring.c                |   5 +-
 5 files changed, 420 insertions(+), 25 deletions(-)

diff --git a/block/export/fuse.c b/block/export/fuse.c
index c0ad4696ce..637d36186a 100644
--- a/block/export/fuse.c
+++ b/block/export/fuse.c
@@ -48,6 +48,11 @@
 #include <linux/fs.h>
 #endif
 
+#define FUSE_DEFAULT_MAX_PAGES_PER_REQ 32
+
+/* room needed in buffer to accommodate header */
+#define FUSE_BUFFER_HEADER_SIZE 0x1000
+
 /* Prevent overly long bounce buffer allocations */
 #define FUSE_MAX_READ_BYTES (MIN(BDRV_REQUEST_MAX_BYTES, 1 * 1024 * 1024))
 /*
@@ -64,6 +69,26 @@
 
 typedef struct FuseExport FuseExport;
 
+struct FuseQueue;
+
+typedef struct FuseRingEnt {
+    /* back pointer */
+    struct FuseQueue *q;
+
+    /* commit id of a fuse request */
+    uint64_t req_commit_id;
+
+    /* fuse request header and payload */
+    struct fuse_uring_req_header *req_header;
+    void *op_payload;
+    size_t req_payload_sz;
+
+    /* The vector passed to the kernel */
+    struct iovec iov[2];
+
+    CqeHandler fuse_cqe_handler;
+} FuseRingEnt;
+
 /*
  * One FUSE "queue", representing one FUSE FD from which requests are fetched
  * and processed.  Each queue is tied to an AioContext.
@@ -73,6 +98,7 @@ typedef struct FuseQueue {
 
     AioContext *ctx;
     int fuse_fd;
+    int qid;
 
     /*
      * The request buffer must be able to hold a full write, and/or at least
@@ -109,6 +135,17 @@ typedef struct FuseQueue {
      * Free this buffer with qemu_vfree().
      */
     void *spillover_buf;
+
+#ifdef CONFIG_LINUX_IO_URING
+    FuseRingEnt ent;
+
+    /*
+     * TODO
+     * Support multi-threaded FUSE over io_uring by using eventfd and allocating
+     * an extra SQE for each thread to be notified when the connection
+     * shuts down.
+     */
+#endif
 } FuseQueue;
 
 /*
@@ -148,6 +185,7 @@ struct FuseExport {
     bool growable;
     /* Whether allow_other was used as a mount option or not */
     bool allow_other;
+    bool is_uring;
 
     mode_t st_mode;
     uid_t st_uid;
@@ -257,6 +295,126 @@ static const BlockDevOps fuse_export_blk_dev_ops = {
     .drained_poll  = fuse_export_drained_poll,
 };
 
+#ifdef CONFIG_LINUX_IO_URING
+static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent);
+
+static void coroutine_fn co_fuse_uring_queue_handle_cqes(void *opaque)
+{
+    CqeHandler *cqe_handler = opaque;
+    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
+    FuseExport *exp = ent->q->exp;
+
+    fuse_uring_co_process_request(ent);
+
+    fuse_dec_in_flight(exp);
+}
+
+static void fuse_uring_cqe_handler(CqeHandler *cqe_handler)
+{
+    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
+    FuseQueue *q = ent->q;
+    Coroutine *co;
+    FuseExport *exp = ent->q->exp;
+
+    int err = cqe_handler->cqe.res;
+    if (err != 0) {
+        /* TODO end_conn support */
+
+        /* -ENOTCONN is ok on umount  */
+        if (err != -EINTR && err != -EOPNOTSUPP &&
+            err != -EAGAIN && err != -ENOTCONN) {
+            fuse_export_halt(exp);
+        }
+    } else {
+        co = qemu_coroutine_create(co_fuse_uring_queue_handle_cqes,
+                            cqe_handler);
+        /* Decremented by co_fuse_uring_queue_handle_cqes() */
+        fuse_inc_in_flight(q->exp);
+        qemu_coroutine_enter(co);
+    }
+}
+
+static void fuse_uring_sqe_set_req_data(struct fuse_uring_cmd_req *req,
+                    const unsigned int qid,
+                    const unsigned int commit_id)
+{
+    req->qid = qid;
+    req->commit_id = commit_id;
+    req->flags = 0;
+}
+
+static void fuse_uring_sqe_prepare(struct io_uring_sqe *sqe, FuseRingEnt *ent,
+               __u32 cmd_op)
+{
+    sqe->opcode = IORING_OP_URING_CMD;
+
+    sqe->fd = ent->q->fuse_fd;
+    sqe->rw_flags = 0;
+    sqe->ioprio = 0;
+    sqe->off = 0;
+
+    sqe->cmd_op = cmd_op;
+    sqe->__pad1 = 0;
+}
+
+static void fuse_uring_prep_sqe_register(struct io_uring_sqe *sqe, void *opaque)
+{
+    FuseQueue *q = opaque;
+    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
+
+    fuse_uring_sqe_prepare(sqe, &q->ent, FUSE_IO_URING_CMD_REGISTER);
+
+    sqe->addr = (uint64_t)(q->ent.iov);
+    sqe->len = 2;
+
+    fuse_uring_sqe_set_req_data(req, q->qid, 0);
+}
+
+static void fuse_uring_start(FuseExport *exp, struct fuse_init_out *out)
+{
+    /*
+     * Since we didn't enable the FUSE_MAX_PAGES feature, the value of
+     * fc->max_pages should be FUSE_DEFAULT_MAX_PAGES_PER_REQ, which is set by
+     * the kernel by default. Also, max_write should not exceed
+     * FUSE_DEFAULT_MAX_PAGES_PER_REQ * PAGE_SIZE.
+     */
+    size_t bufsize = out->max_write + FUSE_BUFFER_HEADER_SIZE;
+
+    if (!(out->flags & FUSE_MAX_PAGES)) {
+        /*
+         * bufsize = MIN(FUSE_DEFAULT_MAX_PAGES_PER_REQ *
+         *       qemu_real_host_page_size() + FUSE_BUFFER_HEADER_SIZE, bufsize);
+         */
+        bufsize = FUSE_DEFAULT_MAX_PAGES_PER_REQ * qemu_real_host_page_size()
+                         + FUSE_BUFFER_HEADER_SIZE;
+    }
+
+    for (int i = 0; i < exp->num_queues; i++) {
+        FuseQueue *q = &exp->queues[i];
+
+        q->ent.q = q;
+
+        q->ent.req_header = g_malloc0(sizeof(struct fuse_uring_req_header));
+        q->ent.req_payload_sz = bufsize - FUSE_BUFFER_HEADER_SIZE;
+        q->ent.op_payload = g_malloc0(q->ent.req_payload_sz);
+
+        q->ent.iov[0] = (struct iovec) {
+            q->ent.req_header,
+            sizeof(struct fuse_uring_req_header)
+        };
+        q->ent.iov[1] = (struct iovec) {
+            q->ent.op_payload,
+            q->ent.req_payload_sz
+        };
+
+        exp->queues[i].ent.fuse_cqe_handler.cb = fuse_uring_cqe_handler;
+
+        aio_add_sqe(fuse_uring_prep_sqe_register, &(exp->queues[i]),
+            &(exp->queues[i].ent.fuse_cqe_handler));
+    }
+}
+#endif
+
 static int fuse_export_create(BlockExport *blk_exp,
                               BlockExportOptions *blk_exp_args,
                               AioContext *const *multithread,
@@ -280,6 +438,7 @@ static int fuse_export_create(BlockExport *blk_exp,
 
         for (size_t i = 0; i < mt_count; i++) {
             exp->queues[i] = (FuseQueue) {
+                .qid = i,
                 .exp = exp,
                 .ctx = multithread[i],
                 .fuse_fd = -1,
@@ -293,6 +452,7 @@ static int fuse_export_create(BlockExport *blk_exp,
         exp->num_queues = 1;
         exp->queues = g_new(FuseQueue, 1);
         exp->queues[0] = (FuseQueue) {
+            .qid = 0,
             .exp = exp,
             .ctx = exp->common.ctx,
             .fuse_fd = -1,
@@ -312,6 +472,8 @@ static int fuse_export_create(BlockExport *blk_exp,
         }
     }
 
+    exp->is_uring = args->uring ? true : false;
+
     blk_set_dev_ops(exp->common.blk, &fuse_export_blk_dev_ops, exp);
 
     /*
@@ -597,6 +759,22 @@ static void read_from_fuse_fd(void *opaque)
     qemu_coroutine_enter(co);
 }
 
+#ifdef CONFIG_LINUX_IO_URING
+static void fuse_export_delete_uring(FuseExport *exp)
+{
+    exp->is_uring = false;
+
+    /*
+     * TODO
+     * end_conn handling
+     */
+    for (size_t qid = 0; qid < exp->num_queues; qid++) {
+        g_free(exp->queues[qid].ent.req_header);
+        g_free(exp->queues[qid].ent.op_payload);
+    }
+}
+#endif
+
 static void fuse_export_shutdown(BlockExport *blk_exp)
 {
     FuseExport *exp = container_of(blk_exp, FuseExport, common);
@@ -618,6 +796,11 @@ static void fuse_export_delete(BlockExport *blk_exp)
 {
     FuseExport *exp = container_of(blk_exp, FuseExport, common);
 
+#ifdef CONFIG_LINUX_IO_URING
+    if (exp->is_uring)
+        fuse_export_delete_uring(exp);
+#endif
+
     for (int i = 0; i < exp->num_queues; i++) {
         FuseQueue *q = &exp->queues[i];
 
@@ -687,15 +870,22 @@ static ssize_t coroutine_fn
 fuse_co_init(FuseExport *exp, struct fuse_init_out *out,
              uint32_t max_readahead, uint32_t flags)
 {
-    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO;
+    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO
+                                     | FUSE_INIT_EXT;
+    uint64_t outargflags = flags;
+
+#ifdef CONFIG_LINUX_IO_URING
+    if (exp->is_uring)
+        outargflags |= FUSE_OVER_IO_URING;
+#endif
 
     *out = (struct fuse_init_out) {
         .major = FUSE_KERNEL_VERSION,
         .minor = FUSE_KERNEL_MINOR_VERSION,
         .max_readahead = max_readahead,
         .max_write = FUSE_MAX_WRITE_BYTES,
-        .flags = flags & supported_flags,
-        .flags2 = 0,
+        .flags = outargflags & supported_flags,
+        .flags2 = outargflags >> 32,
 
         /* libfuse maximum: 2^16 - 1 */
         .max_background = UINT16_MAX,
@@ -943,6 +1133,9 @@ fuse_co_read(FuseExport *exp, void **bufptr, uint64_t offset, uint32_t size)
  * Data in @in_place_buf is assumed to be overwritten after yielding, so will
  * be copied to a bounce buffer beforehand.  @spillover_buf in contrast is
  * assumed to be exclusively owned and will be used as-is.
+ * In FUSE-over-io_uring mode, the actual op_payload content is stored in
+ * @spillover_buf. To ensure this buffer is used for writing, @in_place_buf
+ * is explicitly set to NULL.
  * Return the number of bytes written to *out on success, and -errno on error.
  */
 static ssize_t coroutine_fn
@@ -950,8 +1143,8 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
               uint64_t offset, uint32_t size,
               const void *in_place_buf, const void *spillover_buf)
 {
-    size_t in_place_size;
-    void *copied;
+    size_t in_place_size = 0;
+    void *copied = NULL;
     int64_t blk_len;
     int ret;
     struct iovec iov[2];
@@ -966,10 +1159,12 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
         return -EACCES;
     }
 
-    /* Must copy to bounce buffer before potentially yielding */
-    in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
-    copied = blk_blockalign(exp->common.blk, in_place_size);
-    memcpy(copied, in_place_buf, in_place_size);
+    if (in_place_buf) {
+        /* Must copy to bounce buffer before potentially yielding */
+        in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
+        copied = blk_blockalign(exp->common.blk, in_place_size);
+        memcpy(copied, in_place_buf, in_place_size);
+    }
 
     /**
      * Clients will expect short writes at EOF, so we have to limit
@@ -993,26 +1188,37 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
         }
     }
 
-    iov[0] = (struct iovec) {
-        .iov_base = copied,
-        .iov_len = in_place_size,
-    };
-    if (size > FUSE_IN_PLACE_WRITE_BYTES) {
-        assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
-        iov[1] = (struct iovec) {
-            .iov_base = (void *)spillover_buf,
-            .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
+    if (in_place_buf) {
+        iov[0] = (struct iovec) {
+            .iov_base = copied,
+            .iov_len = in_place_size,
         };
-        qemu_iovec_init_external(&qiov, iov, 2);
+        if (size > FUSE_IN_PLACE_WRITE_BYTES) {
+            assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
+            iov[1] = (struct iovec) {
+                .iov_base = (void *)spillover_buf,
+                .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
+            };
+            qemu_iovec_init_external(&qiov, iov, 2);
+        } else {
+            qemu_iovec_init_external(&qiov, iov, 1);
+        }
     } else {
+        /* fuse over io_uring */
+        iov[0] = (struct iovec) {
+            .iov_base = (void *)spillover_buf,
+            .iov_len = size,
+        };
         qemu_iovec_init_external(&qiov, iov, 1);
     }
+
     ret = blk_co_pwritev(exp->common.blk, offset, size, &qiov, 0);
     if (ret < 0) {
         goto fail_free_buffer;
     }
 
-    qemu_vfree(copied);
+    if (in_place_buf)
+        qemu_vfree(copied);
 
     *out = (struct fuse_write_out) {
         .size = size,
@@ -1020,7 +1226,9 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
     return sizeof(*out);
 
 fail_free_buffer:
-    qemu_vfree(copied);
+    if (in_place_buf) {
+        qemu_vfree(copied);
+    }
     return ret;
 }
 
@@ -1409,6 +1617,12 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
         const struct fuse_init_in *in = FUSE_IN_OP_STRUCT(init, q);
         ret = fuse_co_init(exp, FUSE_OUT_OP_STRUCT(init, out_buf),
                            in->max_readahead, in->flags);
+#ifdef CONFIG_LINUX_IO_URING
+        /* Set up fuse over io_uring after replying to the first FUSE_INIT */
+        if (exp->is_uring) {
+            fuse_uring_start(exp, FUSE_OUT_OP_STRUCT(init, out_buf));
+        }
+#endif
         break;
     }
 
@@ -1515,6 +1729,173 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
     qemu_vfree(spillover_buf);
 }
 
+#ifdef CONFIG_LINUX_IO_URING
+static void fuse_uring_prep_sqe_commit(struct io_uring_sqe *sqe, void *opaque)
+{
+    FuseRingEnt *ent = opaque;
+    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
+
+    fuse_uring_sqe_prepare(sqe, ent, FUSE_IO_URING_CMD_COMMIT_AND_FETCH);
+    fuse_uring_sqe_set_req_data(req, ent->q->qid,
+                                     ent->req_commit_id);
+}
+
+static void
+fuse_uring_write_response(FuseRingEnt *ent, uint32_t req_id, ssize_t ret,
+                          const void *out_op_hdr, const void *buf)
+{
+    struct fuse_uring_req_header *rrh = ent->req_header;
+    struct fuse_out_header *out_header = (struct fuse_out_header *)&rrh->in_out;
+    struct fuse_uring_ent_in_out *ent_in_out =
+        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
+
+    if (buf) {
+        memcpy(ent->op_payload, buf, ret);
+    } else if (ret > 0) {
+        if (ret > ent->req_payload_sz) {
+            warn_report("data size %zu exceeds payload buffer size %zu",
+                        ret, ent->req_payload_sz);
+                        ret = -EINVAL;
+        } else {
+            memcpy(ent->op_payload, out_op_hdr, ret);
+        }
+    }
+
+    out_header->error  = ret < 0 ? ret : 0;
+    out_header->unique = req_id;
+    /* out_header->len = ret > 0 ? ret : 0; */
+    ent_in_out->payload_sz = ret > 0 ? ret : 0;
+
+    aio_add_sqe(fuse_uring_prep_sqe_commit, ent,
+                    &ent->fuse_cqe_handler);
+}
+
+static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent)
+{
+    FuseQueue *q = ent->q;
+    FuseExport *exp = q->exp;
+    struct fuse_uring_req_header *rrh = ent->req_header;
+    struct fuse_uring_ent_in_out *ent_in_out =
+        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
+
+    char out_op_hdr[MAX_CONST(sizeof(struct fuse_init_out),
+                 MAX_CONST(sizeof(struct fuse_open_out),
+                 MAX_CONST(sizeof(struct fuse_attr_out),
+                 MAX_CONST(sizeof(struct fuse_write_out),
+                           sizeof(struct fuse_lseek_out)))))];
+
+    void *out_data_buffer = NULL;
+
+    uint32_t opcode;
+    uint64_t req_id;
+
+    struct fuse_in_header *in_hdr = (struct fuse_in_header *)&rrh->in_out;
+    opcode = in_hdr->opcode;
+    req_id = in_hdr->unique;
+
+    ent->req_commit_id = ent_in_out->commit_id;
+
+    if (unlikely(ent->req_commit_id == 0)) {
+        /*
+         * If this happens kernel will not find the response - it will
+         * be stuck forever - better to abort immediately.
+         */
+        error_report("If this happens kernel will not find the response"
+        " - it will be stuck forever - better to abort immediately.");
+        fuse_export_halt(exp);
+        fuse_dec_in_flight(exp);
+        return;
+    }
+
+    ssize_t ret;
+
+    switch (opcode) {
+    case FUSE_OPEN:
+        ret = fuse_co_open(exp, (struct fuse_open_out *)out_op_hdr);
+        break;
+
+    case FUSE_RELEASE:
+        ret = 0;
+        break;
+
+    case FUSE_LOOKUP:
+        ret = -ENOENT; /* There is no node but the root node */
+        break;
+
+    case FUSE_GETATTR:
+        ret = fuse_co_getattr(exp, (struct fuse_attr_out *)out_op_hdr);
+        break;
+
+    case FUSE_SETATTR: {
+        const struct fuse_setattr_in *in =
+                        (const struct fuse_setattr_in *)&rrh->op_in;
+        ret = fuse_co_setattr(exp, (struct fuse_attr_out *)out_op_hdr,
+                              in->valid, in->size, in->mode, in->uid, in->gid);
+        break;
+    }
+
+    case FUSE_READ: {
+        const struct fuse_read_in *in =
+                        (const struct fuse_read_in *)&rrh->op_in;
+        ret = fuse_co_read(exp, &out_data_buffer, in->offset, in->size);
+        break;
+    }
+
+    case FUSE_WRITE: {
+        const struct fuse_write_in *in =
+                        (const struct fuse_write_in *)&rrh->op_in;
+
+        assert(in->size == ent_in_out->payload_sz);
+
+        /*
+         * poll_fuse_fd() has checked that in_hdr->len matches the number of
+         * bytes read, which cannot exceed the max_write value we set
+         * (FUSE_MAX_WRITE_BYTES).  So we know that FUSE_MAX_WRITE_BYTES >=
+         * in_hdr->len >= in->size + X, so this assertion must hold.
+         */
+        assert(in->size <= FUSE_MAX_WRITE_BYTES);
+
+        ret = fuse_co_write(exp, (struct fuse_write_out *)out_op_hdr,
+                            in->offset, in->size, NULL, ent->op_payload);
+        break;
+    }
+
+    case FUSE_FALLOCATE: {
+        const struct fuse_fallocate_in *in =
+                        (const struct fuse_fallocate_in *)&rrh->op_in;
+        ret = fuse_co_fallocate(exp, in->offset, in->length, in->mode);
+        break;
+    }
+
+    case FUSE_FSYNC:
+        ret = fuse_co_fsync(exp);
+        break;
+
+    case FUSE_FLUSH:
+        ret = fuse_co_flush(exp);
+        break;
+
+#ifdef CONFIG_FUSE_LSEEK
+    case FUSE_LSEEK: {
+        const struct fuse_lseek_in *in =
+                        (const struct fuse_lseek_in *)&rrh->op_in;
+        ret = fuse_co_lseek(exp, (struct fuse_lseek_out *)out_op_hdr,
+                            in->offset, in->whence);
+        break;
+    }
+#endif
+
+    default:
+        ret = -ENOSYS;
+    }
+
+    fuse_uring_write_response(ent, req_id, ret, out_op_hdr, out_data_buffer);
+
+    if (out_data_buffer)
+        qemu_vfree(out_data_buffer);
+}
+#endif
+
 const BlockExportDriver blk_exp_fuse = {
     .type               = BLOCK_EXPORT_TYPE_FUSE,
     .instance_size      = sizeof(FuseExport),
diff --git a/docs/tools/qemu-storage-daemon.rst b/docs/tools/qemu-storage-daemon.rst
index 35ab2d7807..4ec0648e95 100644
--- a/docs/tools/qemu-storage-daemon.rst
+++ b/docs/tools/qemu-storage-daemon.rst
@@ -78,7 +78,7 @@ Standard options:
 .. option:: --export [type=]nbd,id=<id>,node-name=<node-name>[,name=<export-name>][,writable=on|off][,bitmap=<name>]
   --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=unix,addr.path=<socket-path>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
   --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=fd,addr.str=<fd>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
-  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto]
+  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto][,uring=on|off]
   --export [type=]vduse-blk,id=<id>,node-name=<node-name>,name=<vduse-name>[,writable=on|off][,num-queues=<num-queues>][,queue-size=<queue-size>][,logical-block-size=<block-size>][,serial=<serial-number>]
 
   is a block export definition. ``node-name`` is the block node that should be
@@ -111,7 +111,13 @@ Standard options:
   that enabling this option as a non-root user requires enabling the
   user_allow_other option in the global fuse.conf configuration file.  Setting
   ``allow-other`` to auto (the default) will try enabling this option, and on
-  error fall back to disabling it.
+  error fall back to disabling it. Once ``uring`` is enabled
+  (off by default), the initialization of FUSE-over-io_uring-related settings
+  will be performed in the FUSE_INIT request handler. This setup bypasses
+  the traditional /dev/fuse communication mechanism and instead uses io_uring
+  for handling FUSE operations.
+
+
 
   The ``vduse-blk`` export type takes a ``name`` (must be unique across the host)
   to create the VDUSE device.
diff --git a/qapi/block-export.json b/qapi/block-export.json
index 9ae703ad01..7d14f3f1ba 100644
--- a/qapi/block-export.json
+++ b/qapi/block-export.json
@@ -184,12 +184,16 @@
 #     mount the export with allow_other, and if that fails, try again
 #     without.  (since 6.1; default: auto)
 #
+# @uring: If we enable uring option, it will enable FUSE over io_uring
+#         feature for QEMU FUSE export.  (default: false)
+#
 # Since: 6.0
 ##
 { 'struct': 'BlockExportOptionsFuse',
   'data': { 'mountpoint': 'str',
             '*growable': 'bool',
-            '*allow-other': 'FuseExportAllowOther' },
+            '*allow-other': 'FuseExportAllowOther',
+            '*uring': 'bool' },
   'if': 'CONFIG_FUSE' }
 
 ##
diff --git a/storage-daemon/qemu-storage-daemon.c b/storage-daemon/qemu-storage-daemon.c
index eb72561358..803538db29 100644
--- a/storage-daemon/qemu-storage-daemon.c
+++ b/storage-daemon/qemu-storage-daemon.c
@@ -107,6 +107,7 @@ static void help(void)
 #ifdef CONFIG_FUSE
 "  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>\n"
 "           [,growable=on|off][,writable=on|off][,allow-other=on|off|auto]\n"
+"           [,fuse-over-uring=on|off]"
 "                         export the specified block node over FUSE\n"
 "\n"
 #endif /* CONFIG_FUSE */
diff --git a/util/fdmon-io_uring.c b/util/fdmon-io_uring.c
index d2433d1d99..68d3fe8e01 100644
--- a/util/fdmon-io_uring.c
+++ b/util/fdmon-io_uring.c
@@ -452,10 +452,13 @@ static const FDMonOps fdmon_io_uring_ops = {
 void fdmon_io_uring_setup(AioContext *ctx, Error **errp)
 {
     int ret;
+    int flags;
 
     ctx->io_uring_fd_tag = NULL;
+    flags = IORING_SETUP_SQE128;
 
-    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES, &ctx->fdmon_io_uring, 0);
+    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES,
+                            &ctx->fdmon_io_uring, flags);
     if (ret != 0) {
         error_setg_errno(errp, -ret, "Failed to initialize io_uring");
         return;
-- 
2.50.1



^ permalink raw reply related	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-16 18:38 ` [PATCH RFC 1/1] " Brian Song
@ 2025-07-17  6:03   ` Markus Armbruster
  2025-07-22 12:00     ` Brian Song
  2025-07-21  0:53   ` Stefan Hajnoczi
                     ` (2 subsequent siblings)
  3 siblings, 1 reply; 19+ messages in thread
From: Markus Armbruster @ 2025-07-17  6:03 UTC (permalink / raw)
  To: Brian Song
  Cc: qemu-block, qemu-devel, bschubert, fam, hreitz, kwolf, stefanha

Brian Song <hibriansong@gmail.com> writes:

> This work provides an initial implementation of fuse-over-io_uring
> support for QEMU export. According to the fuse-over-io_uring protocol
> specification, the userspace side must create the same number of queues
> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
> queue contains only a single SQE entry, which is used to validate the
> correctness of the fuse-over-io_uring functionality.
>
> All FUSE read and write operations interact with the kernel via io
> vectors embedded in the SQE entry during submission and CQE fetching.
> The req_header and op_payload members of each entry are included as
> parts of the io vector: req_header carries the FUSE operation header,
> and op_payload carries the data payload, such as file attributes in a
> getattr reply, file content in a read reply, or file content being
> written to the FUSE client in a write operation.
>
> At present, multi-threading support is still incomplete. In addition,
> handling connection termination and managing the "drained" state of a
> FUSE block export in QEMU remain as pending work.
>
> Suggested-by: Kevin Wolf <kwolf@redhat.com>
> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
> Signed-off-by: Brian Song <hibriansong@gmail.com>

[...]

> diff --git a/docs/tools/qemu-storage-daemon.rst b/docs/tools/qemu-storage-daemon.rst
> index 35ab2d7807..4ec0648e95 100644
> --- a/docs/tools/qemu-storage-daemon.rst
> +++ b/docs/tools/qemu-storage-daemon.rst
> @@ -78,7 +78,7 @@ Standard options:
>  .. option:: --export [type=]nbd,id=<id>,node-name=<node-name>[,name=<export-name>][,writable=on|off][,bitmap=<name>]
>    --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=unix,addr.path=<socket-path>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>    --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=fd,addr.str=<fd>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
> -  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto]
> +  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto][,uring=on|off]
>    --export [type=]vduse-blk,id=<id>,node-name=<node-name>,name=<vduse-name>[,writable=on|off][,num-queues=<num-queues>][,queue-size=<queue-size>][,logical-block-size=<block-size>][,serial=<serial-number>]
>  
>    is a block export definition. ``node-name`` is the block node that should be
> @@ -111,7 +111,13 @@ Standard options:
>    that enabling this option as a non-root user requires enabling the
>    user_allow_other option in the global fuse.conf configuration file.  Setting
>    ``allow-other`` to auto (the default) will try enabling this option, and on
> -  error fall back to disabling it.
> +  error fall back to disabling it. Once ``uring`` is enabled
> +  (off by default), the initialization of FUSE-over-io_uring-related settings
> +  will be performed in the FUSE_INIT request handler. This setup bypasses
> +  the traditional /dev/fuse communication mechanism and instead uses io_uring
> +  for handling FUSE operations.
> +
> +

Drop the additional blank lines, please.

This is user-facing documentation.  Do users care about "the FUSE_INIT
request handler"?

>  
>    The ``vduse-blk`` export type takes a ``name`` (must be unique across the host)
>    to create the VDUSE device.
> diff --git a/qapi/block-export.json b/qapi/block-export.json
> index 9ae703ad01..7d14f3f1ba 100644
> --- a/qapi/block-export.json
> +++ b/qapi/block-export.json
> @@ -184,12 +184,16 @@
>  #     mount the export with allow_other, and if that fails, try again
>  #     without.  (since 6.1; default: auto)
>  #
> +# @uring: If we enable uring option, it will enable FUSE over io_uring
> +#         feature for QEMU FUSE export.  (default: false)
> +#

Missing (since 10.2).

Please format just like everywhere else:

   # @uring: If we enable uring option, it will enable FUSE over
   #     io_uring feature for QEMU FUSE export.  (default: false)


Kernel documentation calls the thing "FUSE-over-io-uring":
https://docs.kernel.org/filesystems/fuse-io-uring.html

The text feels awkward.  Here's my attempt:

   # @uring: Use FUSE-over-io-uring.  (since 10.2; default: false)

>  # Since: 6.0
>  ##
>  { 'struct': 'BlockExportOptionsFuse',
>    'data': { 'mountpoint': 'str',
>              '*growable': 'bool',
> -            '*allow-other': 'FuseExportAllowOther' },
> +            '*allow-other': 'FuseExportAllowOther',
> +            '*uring': 'bool' },
>    'if': 'CONFIG_FUSE' }
>  
>  ##

[...]



^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 0/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-16 18:38 [PATCH RFC 0/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports Brian Song
  2025-07-16 18:38 ` [PATCH RFC 1/1] " Brian Song
@ 2025-07-20 16:13 ` Stefan Hajnoczi
  2025-07-22 12:00   ` Brian Song
  1 sibling, 1 reply; 19+ messages in thread
From: Stefan Hajnoczi @ 2025-07-20 16:13 UTC (permalink / raw)
  To: Brian Song; +Cc: qemu-block, qemu-devel, armbru, bschubert, fam, hreitz, kwolf

[-- Attachment #1: Type: text/plain, Size: 2026 bytes --]

On Wed, Jul 16, 2025 at 02:38:23PM -0400, Brian Song wrote:
> This RFC patch represents an initial implementation of the FUSE-over- 
> io_uring Exports idea proposed for Google Summer of Code (2025) under 
> the QEMU community:
> https://wiki.qemu.org/Google_Summer_of_Code_2025#FUSE-over-io_uring_exports
> 
> The implementation approach is primarily borrowed from how libfuse 
> interacts with the kernel.
> 
> FUSE-over-io_uring (https://docs.kernel.org/next/filesystems/fuse-io- 
> uring.html) has been officially merged into the Linux kernel. The idea 
> is to replace the traditional /dev/fuse based communication with a more 
> efficient io_uring-based approach. In this model, userspace registers 
> io_uring SQEs via the FUSE_IO_URING_CMD_REGISTER opcode, and then waits 
> for the kernel to forward FUSE requests as CQEs. These are processed by 
> the FUSE exports implementation in userspace and then committed back to 
> the kernel using FUSE_IO_URING_CMD_COMMIT_AND_FETCH.
> 
> To enable this feature in qemu-export-daemon, simply add the uring=on 
> option to the export configuration.

For consistency with the --blockdev file,aio=io_uring option, please
name it "io_uring=on" instead of "uring=on".

> 
> As this patch is still in the RFC stage, it currently supports **only 
> single thread**. Due to protocol requirements in FUSE-over-io_uring,
> the number of FUSE threads must match the number of CPUs. Therefore,
> this initial version only works on single-core VMs (i.e., QEMU started
> with -smp 1) or single core machine.
> 
> Brian Song (1):
>   block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
> 
>  block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>  docs/tools/qemu-storage-daemon.rst   |  10 +-
>  qapi/block-export.json               |   6 +-
>  storage-daemon/qemu-storage-daemon.c |   1 +
>  util/fdmon-io_uring.c                |   5 +-
>  5 files changed, 420 insertions(+), 25 deletions(-)
> 
> -- 
> 2.50.1
> 

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 488 bytes --]

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-16 18:38 ` [PATCH RFC 1/1] " Brian Song
  2025-07-17  6:03   ` Markus Armbruster
@ 2025-07-21  0:53   ` Stefan Hajnoczi
  2025-07-22 12:00     ` Brian Song
  2025-07-22 14:06     ` Bernd Schubert
  2025-07-21 13:51   ` Bernd Schubert
  2025-07-22 13:32   ` Kevin Wolf
  3 siblings, 2 replies; 19+ messages in thread
From: Stefan Hajnoczi @ 2025-07-21  0:53 UTC (permalink / raw)
  To: Brian Song; +Cc: qemu-block, qemu-devel, armbru, bschubert, fam, hreitz, kwolf

[-- Attachment #1: Type: text/plain, Size: 27553 bytes --]

On Wed, Jul 16, 2025 at 02:38:24PM -0400, Brian Song wrote:
> This work provides an initial implementation of fuse-over-io_uring
> support for QEMU export. According to the fuse-over-io_uring protocol
> specification, the userspace side must create the same number of queues
> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
> queue contains only a single SQE entry, which is used to validate the
> correctness of the fuse-over-io_uring functionality.
> 
> All FUSE read and write operations interact with the kernel via io
> vectors embedded in the SQE entry during submission and CQE fetching.
> The req_header and op_payload members of each entry are included as
> parts of the io vector: req_header carries the FUSE operation header,
> and op_payload carries the data payload, such as file attributes in a
> getattr reply, file content in a read reply, or file content being
> written to the FUSE client in a write operation.
> 
> At present, multi-threading support is still incomplete. In addition,
> handling connection termination and managing the "drained" state of a
> FUSE block export in QEMU remain as pending work.
> 
> Suggested-by: Kevin Wolf <kwolf@redhat.com>
> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
> Signed-off-by: Brian Song <hibriansong@gmail.com>
> 
> ---
>  block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>  docs/tools/qemu-storage-daemon.rst   |  10 +-
>  qapi/block-export.json               |   6 +-
>  storage-daemon/qemu-storage-daemon.c |   1 +
>  util/fdmon-io_uring.c                |   5 +-
>  5 files changed, 420 insertions(+), 25 deletions(-)

Here is feedback from a first pass over this patch.

> 
> diff --git a/block/export/fuse.c b/block/export/fuse.c
> index c0ad4696ce..637d36186a 100644
> --- a/block/export/fuse.c
> +++ b/block/export/fuse.c
> @@ -48,6 +48,11 @@
>  #include <linux/fs.h>
>  #endif
>  
> +#define FUSE_DEFAULT_MAX_PAGES_PER_REQ 32
> +
> +/* room needed in buffer to accommodate header */
> +#define FUSE_BUFFER_HEADER_SIZE 0x1000
> +
>  /* Prevent overly long bounce buffer allocations */
>  #define FUSE_MAX_READ_BYTES (MIN(BDRV_REQUEST_MAX_BYTES, 1 * 1024 * 1024))
>  /*
> @@ -64,6 +69,26 @@
>  
>  typedef struct FuseExport FuseExport;
>  
> +struct FuseQueue;

Use "typedef struct FuseQueue FuseQueue;" here...

> +
> +typedef struct FuseRingEnt {
> +    /* back pointer */
> +    struct FuseQueue *q;

...and then this can be "FuseQueue *q;" so that QEMU coding style is
followed.

> +
> +    /* commit id of a fuse request */
> +    uint64_t req_commit_id;
> +
> +    /* fuse request header and payload */
> +    struct fuse_uring_req_header *req_header;
> +    void *op_payload;
> +    size_t req_payload_sz;
> +
> +    /* The vector passed to the kernel */
> +    struct iovec iov[2];
> +
> +    CqeHandler fuse_cqe_handler;
> +} FuseRingEnt;
> +
>  /*
>   * One FUSE "queue", representing one FUSE FD from which requests are fetched
>   * and processed.  Each queue is tied to an AioContext.
> @@ -73,6 +98,7 @@ typedef struct FuseQueue {
>  
>      AioContext *ctx;
>      int fuse_fd;
> +    int qid;

Could this go inside #ifdef CONFIG_LINUX_IO_URING? It seems to be
specific to FUSE-over-io_uring.

>  
>      /*
>       * The request buffer must be able to hold a full write, and/or at least
> @@ -109,6 +135,17 @@ typedef struct FuseQueue {
>       * Free this buffer with qemu_vfree().
>       */
>      void *spillover_buf;
> +
> +#ifdef CONFIG_LINUX_IO_URING
> +    FuseRingEnt ent;
> +
> +    /*
> +     * TODO
> +     * Support multi-threaded FUSE over io_uring by using eventfd and allocating
> +     * an extra SQE for each thread to be notified when the connection
> +     * shuts down.
> +     */

eventfd and the extra SQE won't be necessary because
aio_bh_schedule_oneshot() can be used to cause threads to execute a
function.

(I think this comment effectively says that connection shutdown still
needs to be implemented. The implementation details don't matter at this
point.)

> +#endif
>  } FuseQueue;
>  
>  /*
> @@ -148,6 +185,7 @@ struct FuseExport {
>      bool growable;
>      /* Whether allow_other was used as a mount option or not */
>      bool allow_other;
> +    bool is_uring;
>  
>      mode_t st_mode;
>      uid_t st_uid;
> @@ -257,6 +295,126 @@ static const BlockDevOps fuse_export_blk_dev_ops = {
>      .drained_poll  = fuse_export_drained_poll,
>  };
>  
> +#ifdef CONFIG_LINUX_IO_URING
> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent);
> +
> +static void coroutine_fn co_fuse_uring_queue_handle_cqes(void *opaque)
> +{
> +    CqeHandler *cqe_handler = opaque;
> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);

Passing ent in opaque instead of cqe_handler would simplify this.

> +    FuseExport *exp = ent->q->exp;
> +
> +    fuse_uring_co_process_request(ent);
> +
> +    fuse_dec_in_flight(exp);
> +}
> +
> +static void fuse_uring_cqe_handler(CqeHandler *cqe_handler)
> +{
> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
> +    FuseQueue *q = ent->q;
> +    Coroutine *co;
> +    FuseExport *exp = ent->q->exp;
> +
> +    int err = cqe_handler->cqe.res;
> +    if (err != 0) {
> +        /* TODO end_conn support */
> +
> +        /* -ENOTCONN is ok on umount  */
> +        if (err != -EINTR && err != -EOPNOTSUPP &&
> +            err != -EAGAIN && err != -ENOTCONN) {
> +            fuse_export_halt(exp);
> +        }
> +    } else {
> +        co = qemu_coroutine_create(co_fuse_uring_queue_handle_cqes,
> +                            cqe_handler);
> +        /* Decremented by co_fuse_uring_queue_handle_cqes() */
> +        fuse_inc_in_flight(q->exp);

Can this be moved inside co_fuse_uring_queue_handle_cqes() to avoid
calling inc/dec from different functions? That would make the code
easier to understand and more robust against future bugs.

> +        qemu_coroutine_enter(co);
> +    }
> +}
> +
> +static void fuse_uring_sqe_set_req_data(struct fuse_uring_cmd_req *req,
> +                    const unsigned int qid,
> +                    const unsigned int commit_id)
> +{
> +    req->qid = qid;
> +    req->commit_id = commit_id;
> +    req->flags = 0;
> +}
> +
> +static void fuse_uring_sqe_prepare(struct io_uring_sqe *sqe, FuseRingEnt *ent,
> +               __u32 cmd_op)
> +{
> +    sqe->opcode = IORING_OP_URING_CMD;
> +
> +    sqe->fd = ent->q->fuse_fd;
> +    sqe->rw_flags = 0;
> +    sqe->ioprio = 0;
> +    sqe->off = 0;
> +
> +    sqe->cmd_op = cmd_op;
> +    sqe->__pad1 = 0;
> +}
> +
> +static void fuse_uring_prep_sqe_register(struct io_uring_sqe *sqe, void *opaque)
> +{
> +    FuseQueue *q = opaque;
> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
> +
> +    fuse_uring_sqe_prepare(sqe, &q->ent, FUSE_IO_URING_CMD_REGISTER);
> +
> +    sqe->addr = (uint64_t)(q->ent.iov);
> +    sqe->len = 2;
> +
> +    fuse_uring_sqe_set_req_data(req, q->qid, 0);
> +}
> +
> +static void fuse_uring_start(FuseExport *exp, struct fuse_init_out *out)
> +{
> +    /*
> +     * Since we didn't enable the FUSE_MAX_PAGES feature, the value of
> +     * fc->max_pages should be FUSE_DEFAULT_MAX_PAGES_PER_REQ, which is set by
> +     * the kernel by default. Also, max_write should not exceed
> +     * FUSE_DEFAULT_MAX_PAGES_PER_REQ * PAGE_SIZE.
> +     */
> +    size_t bufsize = out->max_write + FUSE_BUFFER_HEADER_SIZE;
> +
> +    if (!(out->flags & FUSE_MAX_PAGES)) {
> +        /*
> +         * bufsize = MIN(FUSE_DEFAULT_MAX_PAGES_PER_REQ *
> +         *       qemu_real_host_page_size() + FUSE_BUFFER_HEADER_SIZE, bufsize);
> +         */
> +        bufsize = FUSE_DEFAULT_MAX_PAGES_PER_REQ * qemu_real_host_page_size()
> +                         + FUSE_BUFFER_HEADER_SIZE;
> +    }
> +
> +    for (int i = 0; i < exp->num_queues; i++) {
> +        FuseQueue *q = &exp->queues[i];
> +
> +        q->ent.q = q;
> +
> +        q->ent.req_header = g_malloc0(sizeof(struct fuse_uring_req_header));

It's probably easier to embed the header as a FuseRingEnt field instead
of heap allocating it.

> +        q->ent.req_payload_sz = bufsize - FUSE_BUFFER_HEADER_SIZE;
> +        q->ent.op_payload = g_malloc0(q->ent.req_payload_sz);


> +
> +        q->ent.iov[0] = (struct iovec) {
> +            q->ent.req_header,
> +            sizeof(struct fuse_uring_req_header)
> +        };
> +        q->ent.iov[1] = (struct iovec) {
> +            q->ent.op_payload,
> +            q->ent.req_payload_sz
> +        };
> +
> +        exp->queues[i].ent.fuse_cqe_handler.cb = fuse_uring_cqe_handler;
> +
> +        aio_add_sqe(fuse_uring_prep_sqe_register, &(exp->queues[i]),
> +            &(exp->queues[i].ent.fuse_cqe_handler));
> +    }
> +}
> +#endif
> +
>  static int fuse_export_create(BlockExport *blk_exp,
>                                BlockExportOptions *blk_exp_args,
>                                AioContext *const *multithread,
> @@ -280,6 +438,7 @@ static int fuse_export_create(BlockExport *blk_exp,
>  
>          for (size_t i = 0; i < mt_count; i++) {
>              exp->queues[i] = (FuseQueue) {
> +                .qid = i,
>                  .exp = exp,
>                  .ctx = multithread[i],
>                  .fuse_fd = -1,
> @@ -293,6 +452,7 @@ static int fuse_export_create(BlockExport *blk_exp,
>          exp->num_queues = 1;
>          exp->queues = g_new(FuseQueue, 1);
>          exp->queues[0] = (FuseQueue) {
> +            .qid = 0,
>              .exp = exp,
>              .ctx = exp->common.ctx,
>              .fuse_fd = -1,
> @@ -312,6 +472,8 @@ static int fuse_export_create(BlockExport *blk_exp,
>          }
>      }
>  
> +    exp->is_uring = args->uring ? true : false;
> +
>      blk_set_dev_ops(exp->common.blk, &fuse_export_blk_dev_ops, exp);
>  
>      /*
> @@ -597,6 +759,22 @@ static void read_from_fuse_fd(void *opaque)
>      qemu_coroutine_enter(co);
>  }
>  
> +#ifdef CONFIG_LINUX_IO_URING
> +static void fuse_export_delete_uring(FuseExport *exp)
> +{
> +    exp->is_uring = false;
> +
> +    /*
> +     * TODO
> +     * end_conn handling
> +     */
> +    for (size_t qid = 0; qid < exp->num_queues; qid++) {
> +        g_free(exp->queues[qid].ent.req_header);
> +        g_free(exp->queues[qid].ent.op_payload);
> +    }
> +}
> +#endif
> +
>  static void fuse_export_shutdown(BlockExport *blk_exp)
>  {
>      FuseExport *exp = container_of(blk_exp, FuseExport, common);
> @@ -618,6 +796,11 @@ static void fuse_export_delete(BlockExport *blk_exp)
>  {
>      FuseExport *exp = container_of(blk_exp, FuseExport, common);
>  
> +#ifdef CONFIG_LINUX_IO_URING
> +    if (exp->is_uring)
> +        fuse_export_delete_uring(exp);
> +#endif
> +
>      for (int i = 0; i < exp->num_queues; i++) {
>          FuseQueue *q = &exp->queues[i];
>  
> @@ -687,15 +870,22 @@ static ssize_t coroutine_fn
>  fuse_co_init(FuseExport *exp, struct fuse_init_out *out,
>               uint32_t max_readahead, uint32_t flags)
>  {
> -    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO;
> +    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO
> +                                     | FUSE_INIT_EXT;
> +    uint64_t outargflags = flags;
> +
> +#ifdef CONFIG_LINUX_IO_URING
> +    if (exp->is_uring)
> +        outargflags |= FUSE_OVER_IO_URING;
> +#endif
>  
>      *out = (struct fuse_init_out) {
>          .major = FUSE_KERNEL_VERSION,
>          .minor = FUSE_KERNEL_MINOR_VERSION,
>          .max_readahead = max_readahead,
>          .max_write = FUSE_MAX_WRITE_BYTES,
> -        .flags = flags & supported_flags,
> -        .flags2 = 0,
> +        .flags = outargflags & supported_flags,
> +        .flags2 = outargflags >> 32,
>  
>          /* libfuse maximum: 2^16 - 1 */
>          .max_background = UINT16_MAX,
> @@ -943,6 +1133,9 @@ fuse_co_read(FuseExport *exp, void **bufptr, uint64_t offset, uint32_t size)
>   * Data in @in_place_buf is assumed to be overwritten after yielding, so will
>   * be copied to a bounce buffer beforehand.  @spillover_buf in contrast is
>   * assumed to be exclusively owned and will be used as-is.
> + * In FUSE-over-io_uring mode, the actual op_payload content is stored in
> + * @spillover_buf. To ensure this buffer is used for writing, @in_place_buf
> + * is explicitly set to NULL.
>   * Return the number of bytes written to *out on success, and -errno on error.
>   */
>  static ssize_t coroutine_fn
> @@ -950,8 +1143,8 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>                uint64_t offset, uint32_t size,
>                const void *in_place_buf, const void *spillover_buf)
>  {
> -    size_t in_place_size;
> -    void *copied;
> +    size_t in_place_size = 0;
> +    void *copied = NULL;
>      int64_t blk_len;
>      int ret;
>      struct iovec iov[2];
> @@ -966,10 +1159,12 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>          return -EACCES;
>      }
>  
> -    /* Must copy to bounce buffer before potentially yielding */
> -    in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
> -    copied = blk_blockalign(exp->common.blk, in_place_size);
> -    memcpy(copied, in_place_buf, in_place_size);
> +    if (in_place_buf) {
> +        /* Must copy to bounce buffer before potentially yielding */
> +        in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
> +        copied = blk_blockalign(exp->common.blk, in_place_size);
> +        memcpy(copied, in_place_buf, in_place_size);
> +    }
>  
>      /**
>       * Clients will expect short writes at EOF, so we have to limit
> @@ -993,26 +1188,37 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>          }
>      }
>  
> -    iov[0] = (struct iovec) {
> -        .iov_base = copied,
> -        .iov_len = in_place_size,
> -    };
> -    if (size > FUSE_IN_PLACE_WRITE_BYTES) {
> -        assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
> -        iov[1] = (struct iovec) {
> -            .iov_base = (void *)spillover_buf,
> -            .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
> +    if (in_place_buf) {
> +        iov[0] = (struct iovec) {
> +            .iov_base = copied,
> +            .iov_len = in_place_size,
>          };
> -        qemu_iovec_init_external(&qiov, iov, 2);
> +        if (size > FUSE_IN_PLACE_WRITE_BYTES) {
> +            assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
> +            iov[1] = (struct iovec) {
> +                .iov_base = (void *)spillover_buf,
> +                .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
> +            };
> +            qemu_iovec_init_external(&qiov, iov, 2);
> +        } else {
> +            qemu_iovec_init_external(&qiov, iov, 1);
> +        }
>      } else {
> +        /* fuse over io_uring */
> +        iov[0] = (struct iovec) {
> +            .iov_base = (void *)spillover_buf,
> +            .iov_len = size,
> +        };
>          qemu_iovec_init_external(&qiov, iov, 1);
>      }
> +
>      ret = blk_co_pwritev(exp->common.blk, offset, size, &qiov, 0);
>      if (ret < 0) {
>          goto fail_free_buffer;
>      }
>  
> -    qemu_vfree(copied);
> +    if (in_place_buf)
> +        qemu_vfree(copied);
>  
>      *out = (struct fuse_write_out) {
>          .size = size,
> @@ -1020,7 +1226,9 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>      return sizeof(*out);
>  
>  fail_free_buffer:
> -    qemu_vfree(copied);
> +    if (in_place_buf) {
> +        qemu_vfree(copied);
> +    }
>      return ret;
>  }
>  
> @@ -1409,6 +1617,12 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
>          const struct fuse_init_in *in = FUSE_IN_OP_STRUCT(init, q);
>          ret = fuse_co_init(exp, FUSE_OUT_OP_STRUCT(init, out_buf),
>                             in->max_readahead, in->flags);
> +#ifdef CONFIG_LINUX_IO_URING
> +        /* Set up fuse over io_uring after replying to the first FUSE_INIT */
> +        if (exp->is_uring) {
> +            fuse_uring_start(exp, FUSE_OUT_OP_STRUCT(init, out_buf));
> +        }
> +#endif
>          break;
>      }
>  
> @@ -1515,6 +1729,173 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
>      qemu_vfree(spillover_buf);
>  }
>  
> +#ifdef CONFIG_LINUX_IO_URING
> +static void fuse_uring_prep_sqe_commit(struct io_uring_sqe *sqe, void *opaque)
> +{
> +    FuseRingEnt *ent = opaque;
> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
> +
> +    fuse_uring_sqe_prepare(sqe, ent, FUSE_IO_URING_CMD_COMMIT_AND_FETCH);
> +    fuse_uring_sqe_set_req_data(req, ent->q->qid,
> +                                     ent->req_commit_id);
> +}
> +
> +static void
> +fuse_uring_write_response(FuseRingEnt *ent, uint32_t req_id, ssize_t ret,
> +                          const void *out_op_hdr, const void *buf)
> +{
> +    struct fuse_uring_req_header *rrh = ent->req_header;
> +    struct fuse_out_header *out_header = (struct fuse_out_header *)&rrh->in_out;
> +    struct fuse_uring_ent_in_out *ent_in_out =
> +        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
> +
> +    if (buf) {
> +        memcpy(ent->op_payload, buf, ret);
> +    } else if (ret > 0) {
> +        if (ret > ent->req_payload_sz) {
> +            warn_report("data size %zu exceeds payload buffer size %zu",
> +                        ret, ent->req_payload_sz);
> +                        ret = -EINVAL;
> +        } else {
> +            memcpy(ent->op_payload, out_op_hdr, ret);
> +        }
> +    }
> +
> +    out_header->error  = ret < 0 ? ret : 0;
> +    out_header->unique = req_id;
> +    /* out_header->len = ret > 0 ? ret : 0; */
> +    ent_in_out->payload_sz = ret > 0 ? ret : 0;
> +
> +    aio_add_sqe(fuse_uring_prep_sqe_commit, ent,
> +                    &ent->fuse_cqe_handler);
> +}
> +
> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent)
> +{
> +    FuseQueue *q = ent->q;
> +    FuseExport *exp = q->exp;
> +    struct fuse_uring_req_header *rrh = ent->req_header;
> +    struct fuse_uring_ent_in_out *ent_in_out =
> +        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
> +
> +    char out_op_hdr[MAX_CONST(sizeof(struct fuse_init_out),
> +                 MAX_CONST(sizeof(struct fuse_open_out),
> +                 MAX_CONST(sizeof(struct fuse_attr_out),
> +                 MAX_CONST(sizeof(struct fuse_write_out),
> +                           sizeof(struct fuse_lseek_out)))))];

A union would be cleaner because the casts below could be avoided and
MAX_CONST() would be unnecessary:

  union {
      struct fuse_init_out init;
      struct fuse_open out open;
      struct fuse_attr_out attr;
      struct fuse_write_out write;
      struct fuse_lseek_out lseek;
  } out_op_hdr;

> +
> +    void *out_data_buffer = NULL;
> +
> +    uint32_t opcode;
> +    uint64_t req_id;
> +
> +    struct fuse_in_header *in_hdr = (struct fuse_in_header *)&rrh->in_out;
> +    opcode = in_hdr->opcode;
> +    req_id = in_hdr->unique;
> +
> +    ent->req_commit_id = ent_in_out->commit_id;
> +
> +    if (unlikely(ent->req_commit_id == 0)) {
> +        /*
> +         * If this happens kernel will not find the response - it will
> +         * be stuck forever - better to abort immediately.
> +         */
> +        error_report("If this happens kernel will not find the response"
> +        " - it will be stuck forever - better to abort immediately.");
> +        fuse_export_halt(exp);
> +        fuse_dec_in_flight(exp);
> +        return;
> +    }
> +
> +    ssize_t ret;
> +
> +    switch (opcode) {
> +    case FUSE_OPEN:
> +        ret = fuse_co_open(exp, (struct fuse_open_out *)out_op_hdr);
> +        break;
> +
> +    case FUSE_RELEASE:
> +        ret = 0;
> +        break;
> +
> +    case FUSE_LOOKUP:
> +        ret = -ENOENT; /* There is no node but the root node */
> +        break;
> +
> +    case FUSE_GETATTR:
> +        ret = fuse_co_getattr(exp, (struct fuse_attr_out *)out_op_hdr);
> +        break;
> +
> +    case FUSE_SETATTR: {
> +        const struct fuse_setattr_in *in =
> +                        (const struct fuse_setattr_in *)&rrh->op_in;
> +        ret = fuse_co_setattr(exp, (struct fuse_attr_out *)out_op_hdr,
> +                              in->valid, in->size, in->mode, in->uid, in->gid);
> +        break;
> +    }
> +
> +    case FUSE_READ: {
> +        const struct fuse_read_in *in =
> +                        (const struct fuse_read_in *)&rrh->op_in;
> +        ret = fuse_co_read(exp, &out_data_buffer, in->offset, in->size);
> +        break;
> +    }
> +
> +    case FUSE_WRITE: {
> +        const struct fuse_write_in *in =
> +                        (const struct fuse_write_in *)&rrh->op_in;
> +
> +        assert(in->size == ent_in_out->payload_sz);
> +
> +        /*
> +         * poll_fuse_fd() has checked that in_hdr->len matches the number of
> +         * bytes read, which cannot exceed the max_write value we set
> +         * (FUSE_MAX_WRITE_BYTES).  So we know that FUSE_MAX_WRITE_BYTES >=
> +         * in_hdr->len >= in->size + X, so this assertion must hold.
> +         */
> +        assert(in->size <= FUSE_MAX_WRITE_BYTES);
> +
> +        ret = fuse_co_write(exp, (struct fuse_write_out *)out_op_hdr,
> +                            in->offset, in->size, NULL, ent->op_payload);
> +        break;
> +    }
> +
> +    case FUSE_FALLOCATE: {
> +        const struct fuse_fallocate_in *in =
> +                        (const struct fuse_fallocate_in *)&rrh->op_in;
> +        ret = fuse_co_fallocate(exp, in->offset, in->length, in->mode);
> +        break;
> +    }
> +
> +    case FUSE_FSYNC:
> +        ret = fuse_co_fsync(exp);
> +        break;
> +
> +    case FUSE_FLUSH:
> +        ret = fuse_co_flush(exp);
> +        break;
> +
> +#ifdef CONFIG_FUSE_LSEEK
> +    case FUSE_LSEEK: {
> +        const struct fuse_lseek_in *in =
> +                        (const struct fuse_lseek_in *)&rrh->op_in;
> +        ret = fuse_co_lseek(exp, (struct fuse_lseek_out *)out_op_hdr,
> +                            in->offset, in->whence);
> +        break;
> +    }
> +#endif
> +
> +    default:
> +        ret = -ENOSYS;
> +    }

It would be nice to reuse the non-io_uring code rather than duplicating
the switch statement that covers each FUSE opcode. Is the memory layout
so different that the code cannot be shared?

> +
> +    fuse_uring_write_response(ent, req_id, ret, out_op_hdr, out_data_buffer);
> +
> +    if (out_data_buffer)
> +        qemu_vfree(out_data_buffer);
> +}
> +#endif
> +
>  const BlockExportDriver blk_exp_fuse = {
>      .type               = BLOCK_EXPORT_TYPE_FUSE,
>      .instance_size      = sizeof(FuseExport),
> diff --git a/docs/tools/qemu-storage-daemon.rst b/docs/tools/qemu-storage-daemon.rst
> index 35ab2d7807..4ec0648e95 100644
> --- a/docs/tools/qemu-storage-daemon.rst
> +++ b/docs/tools/qemu-storage-daemon.rst
> @@ -78,7 +78,7 @@ Standard options:
>  .. option:: --export [type=]nbd,id=<id>,node-name=<node-name>[,name=<export-name>][,writable=on|off][,bitmap=<name>]
>    --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=unix,addr.path=<socket-path>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>    --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=fd,addr.str=<fd>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
> -  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto]
> +  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto][,uring=on|off]
>    --export [type=]vduse-blk,id=<id>,node-name=<node-name>,name=<vduse-name>[,writable=on|off][,num-queues=<num-queues>][,queue-size=<queue-size>][,logical-block-size=<block-size>][,serial=<serial-number>]
>  
>    is a block export definition. ``node-name`` is the block node that should be
> @@ -111,7 +111,13 @@ Standard options:
>    that enabling this option as a non-root user requires enabling the
>    user_allow_other option in the global fuse.conf configuration file.  Setting
>    ``allow-other`` to auto (the default) will try enabling this option, and on
> -  error fall back to disabling it.
> +  error fall back to disabling it. Once ``uring`` is enabled
> +  (off by default), the initialization of FUSE-over-io_uring-related settings
> +  will be performed in the FUSE_INIT request handler. This setup bypasses
> +  the traditional /dev/fuse communication mechanism and instead uses io_uring
> +  for handling FUSE operations.
> +
> +
>  
>    The ``vduse-blk`` export type takes a ``name`` (must be unique across the host)
>    to create the VDUSE device.
> diff --git a/qapi/block-export.json b/qapi/block-export.json
> index 9ae703ad01..7d14f3f1ba 100644
> --- a/qapi/block-export.json
> +++ b/qapi/block-export.json
> @@ -184,12 +184,16 @@
>  #     mount the export with allow_other, and if that fails, try again
>  #     without.  (since 6.1; default: auto)
>  #
> +# @uring: If we enable uring option, it will enable FUSE over io_uring
> +#         feature for QEMU FUSE export.  (default: false)
> +#
>  # Since: 6.0
>  ##
>  { 'struct': 'BlockExportOptionsFuse',
>    'data': { 'mountpoint': 'str',
>              '*growable': 'bool',
> -            '*allow-other': 'FuseExportAllowOther' },
> +            '*allow-other': 'FuseExportAllowOther',
> +            '*uring': 'bool' },
>    'if': 'CONFIG_FUSE' }
>  
>  ##
> diff --git a/storage-daemon/qemu-storage-daemon.c b/storage-daemon/qemu-storage-daemon.c
> index eb72561358..803538db29 100644
> --- a/storage-daemon/qemu-storage-daemon.c
> +++ b/storage-daemon/qemu-storage-daemon.c
> @@ -107,6 +107,7 @@ static void help(void)
>  #ifdef CONFIG_FUSE
>  "  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>\n"
>  "           [,growable=on|off][,writable=on|off][,allow-other=on|off|auto]\n"
> +"           [,fuse-over-uring=on|off]"
>  "                         export the specified block node over FUSE\n"
>  "\n"
>  #endif /* CONFIG_FUSE */
> diff --git a/util/fdmon-io_uring.c b/util/fdmon-io_uring.c
> index d2433d1d99..68d3fe8e01 100644
> --- a/util/fdmon-io_uring.c
> +++ b/util/fdmon-io_uring.c
> @@ -452,10 +452,13 @@ static const FDMonOps fdmon_io_uring_ops = {
>  void fdmon_io_uring_setup(AioContext *ctx, Error **errp)
>  {
>      int ret;
> +    int flags;
>  
>      ctx->io_uring_fd_tag = NULL;
> +    flags = IORING_SETUP_SQE128;
>  
> -    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES, &ctx->fdmon_io_uring, 0);
> +    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES,
> +                            &ctx->fdmon_io_uring, flags);
>      if (ret != 0) {
>          error_setg_errno(errp, -ret, "Failed to initialize io_uring");
>          return;
> -- 
> 2.50.1
> 

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 488 bytes --]

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-16 18:38 ` [PATCH RFC 1/1] " Brian Song
  2025-07-17  6:03   ` Markus Armbruster
  2025-07-21  0:53   ` Stefan Hajnoczi
@ 2025-07-21 13:51   ` Bernd Schubert
  2025-07-21 18:26     ` Stefan Hajnoczi
                       ` (2 more replies)
  2025-07-22 13:32   ` Kevin Wolf
  3 siblings, 3 replies; 19+ messages in thread
From: Bernd Schubert @ 2025-07-21 13:51 UTC (permalink / raw)
  To: Brian Song, qemu-block@nongnu.org
  Cc: qemu-devel@nongnu.org, armbru@redhat.com, fam@euphon.net,
	hreitz@redhat.com, kwolf@redhat.com, stefanha@redhat.com

Hi Brian,

nice work! Just some minor comments from my side below.

On 7/16/25 20:38, Brian Song wrote:
> [You don't often get email from hibriansong@gmail.com. Learn why this is important at https://aka.ms/LearnAboutSenderIdentification ]
> 
> This work provides an initial implementation of fuse-over-io_uring
> support for QEMU export. According to the fuse-over-io_uring protocol
> specification, the userspace side must create the same number of queues
> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
> queue contains only a single SQE entry, which is used to validate the
> correctness of the fuse-over-io_uring functionality.
> 
> All FUSE read and write operations interact with the kernel via io
> vectors embedded in the SQE entry during submission and CQE fetching.
> The req_header and op_payload members of each entry are included as
> parts of the io vector: req_header carries the FUSE operation header,
> and op_payload carries the data payload, such as file attributes in a
> getattr reply, file content in a read reply, or file content being
> written to the FUSE client in a write operation.
> 
> At present, multi-threading support is still incomplete. In addition,
> handling connection termination and managing the "drained" state of a
> FUSE block export in QEMU remain as pending work.
> 
> Suggested-by: Kevin Wolf <kwolf@redhat.com>
> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
> Signed-off-by: Brian Song <hibriansong@gmail.com>
> 
> ---
>  block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>  docs/tools/qemu-storage-daemon.rst   |  10 +-
>  qapi/block-export.json               |   6 +-
>  storage-daemon/qemu-storage-daemon.c |   1 +
>  util/fdmon-io_uring.c                |   5 +-
>  5 files changed, 420 insertions(+), 25 deletions(-)
> 
> diff --git a/block/export/fuse.c b/block/export/fuse.c
> index c0ad4696ce..637d36186a 100644
> --- a/block/export/fuse.c
> +++ b/block/export/fuse.c
> @@ -48,6 +48,11 @@
>  #include <linux/fs.h>
>  #endif
> 
> +#define FUSE_DEFAULT_MAX_PAGES_PER_REQ 32
> +
> +/* room needed in buffer to accommodate header */
> +#define FUSE_BUFFER_HEADER_SIZE 0x1000
> +
>  /* Prevent overly long bounce buffer allocations */
>  #define FUSE_MAX_READ_BYTES (MIN(BDRV_REQUEST_MAX_BYTES, 1 * 1024 * 1024))
>  /*
> @@ -64,6 +69,26 @@
> 
>  typedef struct FuseExport FuseExport;
> 
> +struct FuseQueue;
> +
> +typedef struct FuseRingEnt {
> +    /* back pointer */
> +    struct FuseQueue *q;
> +
> +    /* commit id of a fuse request */
> +    uint64_t req_commit_id;
> +
> +    /* fuse request header and payload */
> +    struct fuse_uring_req_header *req_header;
> +    void *op_payload;
> +    size_t req_payload_sz;

Note that this will be needed only later, when we have multiple request sizes per queue. I had added it to libfuse only pro-actively to avoid the need to change too much code later on.

> +
> +    /* The vector passed to the kernel */
> +    struct iovec iov[2];
> +
> +    CqeHandler fuse_cqe_handler;
> +} FuseRingEnt;
> +
>  /*
>   * One FUSE "queue", representing one FUSE FD from which requests are fetched
>   * and processed.  Each queue is tied to an AioContext.
> @@ -73,6 +98,7 @@ typedef struct FuseQueue {
> 
>      AioContext *ctx;
>      int fuse_fd;
> +    int qid;
> 
>      /*
>       * The request buffer must be able to hold a full write, and/or at least
> @@ -109,6 +135,17 @@ typedef struct FuseQueue {
>       * Free this buffer with qemu_vfree().
>       */
>      void *spillover_buf;
> +
> +#ifdef CONFIG_LINUX_IO_URING
> +    FuseRingEnt ent;
> +
> +    /*
> +     * TODO
> +     * Support multi-threaded FUSE over io_uring by using eventfd and allocating
> +     * an extra SQE for each thread to be notified when the connection
> +     * shuts down.
> +     */
> +#endif
>  } FuseQueue;
> 
>  /*
> @@ -148,6 +185,7 @@ struct FuseExport {
>      bool growable;
>      /* Whether allow_other was used as a mount option or not */
>      bool allow_other;
> +    bool is_uring;
> 
>      mode_t st_mode;
>      uid_t st_uid;
> @@ -257,6 +295,126 @@ static const BlockDevOps fuse_export_blk_dev_ops = {
>      .drained_poll  = fuse_export_drained_poll,
>  };
> 
> +#ifdef CONFIG_LINUX_IO_URING
> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent);
> +
> +static void coroutine_fn co_fuse_uring_queue_handle_cqes(void *opaque)
> +{
> +    CqeHandler *cqe_handler = opaque;
> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
> +    FuseExport *exp = ent->q->exp;
> +
> +    fuse_uring_co_process_request(ent);
> +
> +    fuse_dec_in_flight(exp);
> +}
> +
> +static void fuse_uring_cqe_handler(CqeHandler *cqe_handler)
> +{
> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
> +    FuseQueue *q = ent->q;
> +    Coroutine *co;
> +    FuseExport *exp = ent->q->exp;
> +
> +    int err = cqe_handler->cqe.res;
> +    if (err != 0) {
> +        /* TODO end_conn support */
> +
> +        /* -ENOTCONN is ok on umount  */
> +        if (err != -EINTR && err != -EOPNOTSUPP &&
> +            err != -EAGAIN && err != -ENOTCONN) {
> +            fuse_export_halt(exp);
> +        }
> +    } else {
> +        co = qemu_coroutine_create(co_fuse_uring_queue_handle_cqes,
> +                            cqe_handler);
> +        /* Decremented by co_fuse_uring_queue_handle_cqes() */
> +        fuse_inc_in_flight(q->exp);
> +        qemu_coroutine_enter(co);
> +    }
> +}
> +
> +static void fuse_uring_sqe_set_req_data(struct fuse_uring_cmd_req *req,
> +                    const unsigned int qid,
> +                    const unsigned int commit_id)
> +{
> +    req->qid = qid;
> +    req->commit_id = commit_id;
> +    req->flags = 0;
> +}
> +
> +static void fuse_uring_sqe_prepare(struct io_uring_sqe *sqe, FuseRingEnt *ent,
> +               __u32 cmd_op)
> +{
> +    sqe->opcode = IORING_OP_URING_CMD;
> +
> +    sqe->fd = ent->q->fuse_fd;
> +    sqe->rw_flags = 0;
> +    sqe->ioprio = 0;
> +    sqe->off = 0;
> +
> +    sqe->cmd_op = cmd_op;
> +    sqe->__pad1 = 0;
> +}
> +
> +static void fuse_uring_prep_sqe_register(struct io_uring_sqe *sqe, void *opaque)
> +{
> +    FuseQueue *q = opaque;
> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
> +
> +    fuse_uring_sqe_prepare(sqe, &q->ent, FUSE_IO_URING_CMD_REGISTER);
> +
> +    sqe->addr = (uint64_t)(q->ent.iov);
> +    sqe->len = 2;
> +
> +    fuse_uring_sqe_set_req_data(req, q->qid, 0);
> +}
> +
> +static void fuse_uring_start(FuseExport *exp, struct fuse_init_out *out)
> +{
> +    /*
> +     * Since we didn't enable the FUSE_MAX_PAGES feature, the value of
> +     * fc->max_pages should be FUSE_DEFAULT_MAX_PAGES_PER_REQ, which is set by
> +     * the kernel by default. Also, max_write should not exceed
> +     * FUSE_DEFAULT_MAX_PAGES_PER_REQ * PAGE_SIZE.

From kernel point of view these things always cause headache, because now it
is impossible to change defaults. Can't it set FUSE_MAX_PAGES and then define
the actual limit?

> +     */
> +    size_t bufsize = out->max_write + FUSE_BUFFER_HEADER_SIZE;
> +
> +    if (!(out->flags & FUSE_MAX_PAGES)) {
> +        /*
> +         * bufsize = MIN(FUSE_DEFAULT_MAX_PAGES_PER_REQ *
> +         *       qemu_real_host_page_size() + FUSE_BUFFER_HEADER_SIZE, bufsize);
> +         */
> +        bufsize = FUSE_DEFAULT_MAX_PAGES_PER_REQ * qemu_real_host_page_size()
> +                         + FUSE_BUFFER_HEADER_SIZE;
> +    }
> +
> +    for (int i = 0; i < exp->num_queues; i++) {
> +        FuseQueue *q = &exp->queues[i];
> +
> +        q->ent.q = q;
> +
> +        q->ent.req_header = g_malloc0(sizeof(struct fuse_uring_req_header));
> +        q->ent.req_payload_sz = bufsize - FUSE_BUFFER_HEADER_SIZE;
> +        q->ent.op_payload = g_malloc0(q->ent.req_payload_sz);
> +
> +        q->ent.iov[0] = (struct iovec) {
> +            q->ent.req_header,
> +            sizeof(struct fuse_uring_req_header)
> +        };
> +        q->ent.iov[1] = (struct iovec) {
> +            q->ent.op_payload,
> +            q->ent.req_payload_sz
> +        };
> +
> +        exp->queues[i].ent.fuse_cqe_handler.cb = fuse_uring_cqe_handler;
> +
> +        aio_add_sqe(fuse_uring_prep_sqe_register, &(exp->queues[i]),
> +            &(exp->queues[i].ent.fuse_cqe_handler));
> +    }
> +}
> +#endif
> +
>  static int fuse_export_create(BlockExport *blk_exp,
>                                BlockExportOptions *blk_exp_args,
>                                AioContext *const *multithread,
> @@ -280,6 +438,7 @@ static int fuse_export_create(BlockExport *blk_exp,
> 
>          for (size_t i = 0; i < mt_count; i++) {
>              exp->queues[i] = (FuseQueue) {
> +                .qid = i,
>                  .exp = exp,
>                  .ctx = multithread[i],
>                  .fuse_fd = -1,
> @@ -293,6 +452,7 @@ static int fuse_export_create(BlockExport *blk_exp,
>          exp->num_queues = 1;
>          exp->queues = g_new(FuseQueue, 1);
>          exp->queues[0] = (FuseQueue) {
> +            .qid = 0,
>              .exp = exp,
>              .ctx = exp->common.ctx,
>              .fuse_fd = -1,
> @@ -312,6 +472,8 @@ static int fuse_export_create(BlockExport *blk_exp,
>          }
>      }
> 
> +    exp->is_uring = args->uring ? true : false;
> +
>      blk_set_dev_ops(exp->common.blk, &fuse_export_blk_dev_ops, exp);
> 
>      /*
> @@ -597,6 +759,22 @@ static void read_from_fuse_fd(void *opaque)
>      qemu_coroutine_enter(co);
>  }
> 
> +#ifdef CONFIG_LINUX_IO_URING
> +static void fuse_export_delete_uring(FuseExport *exp)
> +{
> +    exp->is_uring = false;
> +
> +    /*
> +     * TODO
> +     * end_conn handling
> +     */
> +    for (size_t qid = 0; qid < exp->num_queues; qid++) {
> +        g_free(exp->queues[qid].ent.req_header);
> +        g_free(exp->queues[qid].ent.op_payload);
> +    }
> +}
> +#endif
> +
>  static void fuse_export_shutdown(BlockExport *blk_exp)
>  {
>      FuseExport *exp = container_of(blk_exp, FuseExport, common);
> @@ -618,6 +796,11 @@ static void fuse_export_delete(BlockExport *blk_exp)
>  {
>      FuseExport *exp = container_of(blk_exp, FuseExport, common);
> 
> +#ifdef CONFIG_LINUX_IO_URING
> +    if (exp->is_uring)
> +        fuse_export_delete_uring(exp);
> +#endif
> +
>      for (int i = 0; i < exp->num_queues; i++) {
>          FuseQueue *q = &exp->queues[i];
> 
> @@ -687,15 +870,22 @@ static ssize_t coroutine_fn
>  fuse_co_init(FuseExport *exp, struct fuse_init_out *out,
>               uint32_t max_readahead, uint32_t flags)
>  {
> -    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO;
> +    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO
> +                                     | FUSE_INIT_EXT;
> +    uint64_t outargflags = flags;
> +
> +#ifdef CONFIG_LINUX_IO_URING
> +    if (exp->is_uring)
> +        outargflags |= FUSE_OVER_IO_URING;
> +#endif
> 
>      *out = (struct fuse_init_out) {
>          .major = FUSE_KERNEL_VERSION,
>          .minor = FUSE_KERNEL_MINOR_VERSION,
>          .max_readahead = max_readahead,
>          .max_write = FUSE_MAX_WRITE_BYTES,
> -        .flags = flags & supported_flags,
> -        .flags2 = 0,
> +        .flags = outargflags & supported_flags,
> +        .flags2 = outargflags >> 32,
> 
>          /* libfuse maximum: 2^16 - 1 */
>          .max_background = UINT16_MAX,
> @@ -943,6 +1133,9 @@ fuse_co_read(FuseExport *exp, void **bufptr, uint64_t offset, uint32_t size)
>   * Data in @in_place_buf is assumed to be overwritten after yielding, so will
>   * be copied to a bounce buffer beforehand.  @spillover_buf in contrast is
>   * assumed to be exclusively owned and will be used as-is.
> + * In FUSE-over-io_uring mode, the actual op_payload content is stored in
> + * @spillover_buf. To ensure this buffer is used for writing, @in_place_buf
> + * is explicitly set to NULL.
>   * Return the number of bytes written to *out on success, and -errno on error.
>   */
>  static ssize_t coroutine_fn
> @@ -950,8 +1143,8 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>                uint64_t offset, uint32_t size,
>                const void *in_place_buf, const void *spillover_buf)
>  {
> -    size_t in_place_size;
> -    void *copied;
> +    size_t in_place_size = 0;
> +    void *copied = NULL;
>      int64_t blk_len;
>      int ret;
>      struct iovec iov[2];
> @@ -966,10 +1159,12 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>          return -EACCES;
>      }
> 
> -    /* Must copy to bounce buffer before potentially yielding */
> -    in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
> -    copied = blk_blockalign(exp->common.blk, in_place_size);
> -    memcpy(copied, in_place_buf, in_place_size);
> +    if (in_place_buf) {
> +        /* Must copy to bounce buffer before potentially yielding */
> +        in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
> +        copied = blk_blockalign(exp->common.blk, in_place_size);
> +        memcpy(copied, in_place_buf, in_place_size);
> +    }
> 
>      /**
>       * Clients will expect short writes at EOF, so we have to limit
> @@ -993,26 +1188,37 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>          }
>      }
> 
> -    iov[0] = (struct iovec) {
> -        .iov_base = copied,
> -        .iov_len = in_place_size,
> -    };
> -    if (size > FUSE_IN_PLACE_WRITE_BYTES) {
> -        assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
> -        iov[1] = (struct iovec) {
> -            .iov_base = (void *)spillover_buf,
> -            .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
> +    if (in_place_buf) {
> +        iov[0] = (struct iovec) {
> +            .iov_base = copied,
> +            .iov_len = in_place_size,
>          };
> -        qemu_iovec_init_external(&qiov, iov, 2);
> +        if (size > FUSE_IN_PLACE_WRITE_BYTES) {
> +            assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
> +            iov[1] = (struct iovec) {
> +                .iov_base = (void *)spillover_buf,
> +                .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
> +            };
> +            qemu_iovec_init_external(&qiov, iov, 2);
> +        } else {
> +            qemu_iovec_init_external(&qiov, iov, 1);
> +        }
>      } else {
> +        /* fuse over io_uring */
> +        iov[0] = (struct iovec) {
> +            .iov_base = (void *)spillover_buf,
> +            .iov_len = size,
> +        };
>          qemu_iovec_init_external(&qiov, iov, 1);
>      }
> +
>      ret = blk_co_pwritev(exp->common.blk, offset, size, &qiov, 0);
>      if (ret < 0) {
>          goto fail_free_buffer;
>      }
> 
> -    qemu_vfree(copied);
> +    if (in_place_buf)
> +        qemu_vfree(copied);
> 
>      *out = (struct fuse_write_out) {
>          .size = size,
> @@ -1020,7 +1226,9 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>      return sizeof(*out);
> 
>  fail_free_buffer:
> -    qemu_vfree(copied);
> +    if (in_place_buf) {
> +        qemu_vfree(copied);
> +    }
>      return ret;
>  }
> 
> @@ -1409,6 +1617,12 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
>          const struct fuse_init_in *in = FUSE_IN_OP_STRUCT(init, q);
>          ret = fuse_co_init(exp, FUSE_OUT_OP_STRUCT(init, out_buf),
>                             in->max_readahead, in->flags);
> +#ifdef CONFIG_LINUX_IO_URING
> +        /* Set up fuse over io_uring after replying to the first FUSE_INIT */
> +        if (exp->is_uring) {
> +            fuse_uring_start(exp, FUSE_OUT_OP_STRUCT(init, out_buf));
> +        }
> +#endif
>          break;
>      }
> 
> @@ -1515,6 +1729,173 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
>      qemu_vfree(spillover_buf);
>  }
> 
> +#ifdef CONFIG_LINUX_IO_URING
> +static void fuse_uring_prep_sqe_commit(struct io_uring_sqe *sqe, void *opaque)
> +{
> +    FuseRingEnt *ent = opaque;
> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
> +
> +    fuse_uring_sqe_prepare(sqe, ent, FUSE_IO_URING_CMD_COMMIT_AND_FETCH);
> +    fuse_uring_sqe_set_req_data(req, ent->q->qid,
> +                                     ent->req_commit_id);
> +}
> +
> +static void
> +fuse_uring_write_response(FuseRingEnt *ent, uint32_t req_id, ssize_t ret,
> +                          const void *out_op_hdr, const void *buf)

Isn't it more a "send_response"? With /dev/fuse there was a write() 
operation, but with io-uring it just fills an SQE?

> +{
> +    struct fuse_uring_req_header *rrh = ent->req_header;
> +    struct fuse_out_header *out_header = (struct fuse_out_header *)&rrh->in_out;
> +    struct fuse_uring_ent_in_out *ent_in_out =
> +        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
> +
> +    if (buf) {
> +        memcpy(ent->op_payload, buf, ret);
> +    } else if (ret > 0) {
> +        if (ret > ent->req_payload_sz) {
> +            warn_report("data size %zu exceeds payload buffer size %zu",
> +                        ret, ent->req_payload_sz);
> +                        ret = -EINVAL;
> +        } else {
> +            memcpy(ent->op_payload, out_op_hdr, ret);
> +        }
> +    }

I would try to optimize that away. Like

if (buf && buf != ent->op_payload) {
	/* copy needed */
}

> +
> +    out_header->error  = ret < 0 ? ret : 0;
> +    out_header->unique = req_id;
> +    /* out_header->len = ret > 0 ? ret : 0; */
> +    ent_in_out->payload_sz = ret > 0 ? ret : 0;
> +
> +    aio_add_sqe(fuse_uring_prep_sqe_commit, ent,
> +                    &ent->fuse_cqe_handler);
> +}
> +
> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent)
> +{
> +    FuseQueue *q = ent->q;
> +    FuseExport *exp = q->exp;
> +    struct fuse_uring_req_header *rrh = ent->req_header;
> +    struct fuse_uring_ent_in_out *ent_in_out =
> +        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
> +
> +    char out_op_hdr[MAX_CONST(sizeof(struct fuse_init_out),
> +                 MAX_CONST(sizeof(struct fuse_open_out),
> +                 MAX_CONST(sizeof(struct fuse_attr_out),
> +                 MAX_CONST(sizeof(struct fuse_write_out),
> +                           sizeof(struct fuse_lseek_out)))))];
> +
> +    void *out_data_buffer = NULL;
> +
> +    uint32_t opcode;
> +    uint64_t req_id;
> +
> +    struct fuse_in_header *in_hdr = (struct fuse_in_header *)&rrh->in_out;
> +    opcode = in_hdr->opcode;
> +    req_id = in_hdr->unique;
> +
> +    ent->req_commit_id = ent_in_out->commit_id;
> +
> +    if (unlikely(ent->req_commit_id == 0)) {
> +        /*
> +         * If this happens kernel will not find the response - it will
> +         * be stuck forever - better to abort immediately.
> +         */
> +        error_report("If this happens kernel will not find the response"
> +        " - it will be stuck forever - better to abort immediately.");

Dunno about qemu style, but isn't it better to keep the string in a single
line to make it greppable?

> +        fuse_export_halt(exp);
> +        fuse_dec_in_flight(exp);
> +        return;
> +    }
> +
> +    ssize_t ret;
> +
> +    switch (opcode) {
> +    case FUSE_OPEN:
> +        ret = fuse_co_open(exp, (struct fuse_open_out *)out_op_hdr);
> +        break;
> +
> +    case FUSE_RELEASE:
> +        ret = 0;
> +        break;
> +
> +    case FUSE_LOOKUP:
> +        ret = -ENOENT; /* There is no node but the root node */
> +        break;
> +
> +    case FUSE_GETATTR:
> +        ret = fuse_co_getattr(exp, (struct fuse_attr_out *)out_op_hdr);
> +        break;
> +
> +    case FUSE_SETATTR: {
> +        const struct fuse_setattr_in *in =
> +                        (const struct fuse_setattr_in *)&rrh->op_in;
> +        ret = fuse_co_setattr(exp, (struct fuse_attr_out *)out_op_hdr,
> +                              in->valid, in->size, in->mode, in->uid, in->gid);
> +        break;
> +    }
> +
> +    case FUSE_READ: {
> +        const struct fuse_read_in *in =
> +                        (const struct fuse_read_in *)&rrh->op_in;
> +        ret = fuse_co_read(exp, &out_data_buffer, in->offset, in->size);
> +        break;
> +    }

There is room for optimization here - this could just pass the payload
buffer and let it directly copy the data into it? 

> +
> +    case FUSE_WRITE: {
> +        const struct fuse_write_in *in =
> +                        (const struct fuse_write_in *)&rrh->op_in;
> +
> +        assert(in->size == ent_in_out->payload_sz);
> +
> +        /*
> +         * poll_fuse_fd() has checked that in_hdr->len matches the number of
> +         * bytes read, which cannot exceed the max_write value we set
> +         * (FUSE_MAX_WRITE_BYTES).  So we know that FUSE_MAX_WRITE_BYTES >=
> +         * in_hdr->len >= in->size + X, so this assertion must hold.
> +         */
> +        assert(in->size <= FUSE_MAX_WRITE_BYTES);
> +
> +        ret = fuse_co_write(exp, (struct fuse_write_out *)out_op_hdr,
> +                            in->offset, in->size, NULL, ent->op_payload);
> +        break;
> +    }
> +
> +    case FUSE_FALLOCATE: {
> +        const struct fuse_fallocate_in *in =
> +                        (const struct fuse_fallocate_in *)&rrh->op_in;
> +        ret = fuse_co_fallocate(exp, in->offset, in->length, in->mode);
> +        break;
> +    }
> +
> +    case FUSE_FSYNC:
> +        ret = fuse_co_fsync(exp);
> +        break;
> +
> +    case FUSE_FLUSH:
> +        ret = fuse_co_flush(exp);
> +        break;
> +
> +#ifdef CONFIG_FUSE_LSEEK
> +    case FUSE_LSEEK: {
> +        const struct fuse_lseek_in *in =
> +                        (const struct fuse_lseek_in *)&rrh->op_in;
> +        ret = fuse_co_lseek(exp, (struct fuse_lseek_out *)out_op_hdr,
> +                            in->offset, in->whence);
> +        break;
> +    }
> +#endif
> +
> +    default:
> +        ret = -ENOSYS;
> +    }
> +
> +    fuse_uring_write_response(ent, req_id, ret, out_op_hdr, out_data_buffer);
> +
> +    if (out_data_buffer)
> +        qemu_vfree(out_data_buffer);
> +}
> +#endif
> +
>  const BlockExportDriver blk_exp_fuse = {
>      .type               = BLOCK_EXPORT_TYPE_FUSE,
>      .instance_size      = sizeof(FuseExport),
> diff --git a/docs/tools/qemu-storage-daemon.rst b/docs/tools/qemu-storage-daemon.rst
> index 35ab2d7807..4ec0648e95 100644
> --- a/docs/tools/qemu-storage-daemon.rst
> +++ b/docs/tools/qemu-storage-daemon.rst
> @@ -78,7 +78,7 @@ Standard options:
>  .. option:: --export [type=]nbd,id=<id>,node-name=<node-name>[,name=<export-name>][,writable=on|off][,bitmap=<name>]
>    --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=unix,addr.path=<socket-path>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>    --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=fd,addr.str=<fd>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
> -  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto]
> +  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto][,uring=on|off]
>    --export [type=]vduse-blk,id=<id>,node-name=<node-name>,name=<vduse-name>[,writable=on|off][,num-queues=<num-queues>][,queue-size=<queue-size>][,logical-block-size=<block-size>][,serial=<serial-number>]
> 
>    is a block export definition. ``node-name`` is the block node that should be
> @@ -111,7 +111,13 @@ Standard options:
>    that enabling this option as a non-root user requires enabling the
>    user_allow_other option in the global fuse.conf configuration file.  Setting
>    ``allow-other`` to auto (the default) will try enabling this option, and on
> -  error fall back to disabling it.
> +  error fall back to disabling it. Once ``uring`` is enabled
> +  (off by default), the initialization of FUSE-over-io_uring-related settings
> +  will be performed in the FUSE_INIT request handler. This setup bypasses
> +  the traditional /dev/fuse communication mechanism and instead uses io_uring
> +  for handling FUSE operations.
> +
> +
> 
>    The ``vduse-blk`` export type takes a ``name`` (must be unique across the host)
>    to create the VDUSE device.
> diff --git a/qapi/block-export.json b/qapi/block-export.json
> index 9ae703ad01..7d14f3f1ba 100644
> --- a/qapi/block-export.json
> +++ b/qapi/block-export.json
> @@ -184,12 +184,16 @@
>  #     mount the export with allow_other, and if that fails, try again
>  #     without.  (since 6.1; default: auto)
>  #
> +# @uring: If we enable uring option, it will enable FUSE over io_uring
> +#         feature for QEMU FUSE export.  (default: false)
> +#
>  # Since: 6.0
>  ##
>  { 'struct': 'BlockExportOptionsFuse',
>    'data': { 'mountpoint': 'str',
>              '*growable': 'bool',
> -            '*allow-other': 'FuseExportAllowOther' },
> +            '*allow-other': 'FuseExportAllowOther',
> +            '*uring': 'bool' },
>    'if': 'CONFIG_FUSE' }
> 
>  ##
> diff --git a/storage-daemon/qemu-storage-daemon.c b/storage-daemon/qemu-storage-daemon.c
> index eb72561358..803538db29 100644
> --- a/storage-daemon/qemu-storage-daemon.c
> +++ b/storage-daemon/qemu-storage-daemon.c
> @@ -107,6 +107,7 @@ static void help(void)
>  #ifdef CONFIG_FUSE
>  "  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>\n"
>  "           [,growable=on|off][,writable=on|off][,allow-other=on|off|auto]\n"
> +"           [,fuse-over-uring=on|off]"
>  "                         export the specified block node over FUSE\n"
>  "\n"
>  #endif /* CONFIG_FUSE */
> diff --git a/util/fdmon-io_uring.c b/util/fdmon-io_uring.c
> index d2433d1d99..68d3fe8e01 100644
> --- a/util/fdmon-io_uring.c
> +++ b/util/fdmon-io_uring.c
> @@ -452,10 +452,13 @@ static const FDMonOps fdmon_io_uring_ops = {
>  void fdmon_io_uring_setup(AioContext *ctx, Error **errp)
>  {
>      int ret;
> +    int flags;
> 
>      ctx->io_uring_fd_tag = NULL;
> +    flags = IORING_SETUP_SQE128;
> 
> -    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES, &ctx->fdmon_io_uring, 0);
> +    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES,
> +                            &ctx->fdmon_io_uring, flags);
>      if (ret != 0) {
>          error_setg_errno(errp, -ret, "Failed to initialize io_uring");
>          return;

I need test these flags again, might be worthful for qemu as well

	/* These flags should help to increase performance, but actually
	 * make it a bit slower - reason should get investigated.
	 */
	if (0) {
		/* Has the main slow down effect */
		params.flags |= IORING_SETUP_SINGLE_ISSUER;

		// params.flags |= IORING_SETUP_DEFER_TASKRUN;
		params.flags |= IORING_SETUP_TASKRUN_FLAG;

		/* Second main effect to make it slower */
		params.flags |= IORING_SETUP_COOP_TASKRUN;
	}




Thanks,
Bernd


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-21 13:51   ` Bernd Schubert
@ 2025-07-21 18:26     ` Stefan Hajnoczi
  2025-07-22 12:00     ` Brian Song
  2025-07-24 20:36     ` Stefan Hajnoczi
  2 siblings, 0 replies; 19+ messages in thread
From: Stefan Hajnoczi @ 2025-07-21 18:26 UTC (permalink / raw)
  To: Bernd Schubert
  Cc: Brian Song, qemu-block@nongnu.org, qemu-devel@nongnu.org,
	armbru@redhat.com, fam@euphon.net, hreitz@redhat.com,
	kwolf@redhat.com

[-- Attachment #1: Type: text/plain, Size: 1059 bytes --]

On Mon, Jul 21, 2025 at 01:51:44PM +0000, Bernd Schubert wrote:
> I need test these flags again, might be worthful for qemu as well
> 
> 	/* These flags should help to increase performance, but actually
> 	 * make it a bit slower - reason should get investigated.
> 	 */
> 	if (0) {
> 		/* Has the main slow down effect */
> 		params.flags |= IORING_SETUP_SINGLE_ISSUER;
> 
> 		// params.flags |= IORING_SETUP_DEFER_TASKRUN;
> 		params.flags |= IORING_SETUP_TASKRUN_FLAG;
> 
> 		/* Second main effect to make it slower */
> 		params.flags |= IORING_SETUP_COOP_TASKRUN;
> 	}

Yes, they are interesting. Thanks for mentioning them!

An earlier attempt at enabling IORING_SETUP_SINGLE_ISSUER in QEMU (still
needs benchmarking):
https://lore.kernel.org/qemu-devel/174293621917.22751.11381319865102029969-0%40git.sr.ht/T/#t

Let's consider these flags as a separate patch series. I have a
benchmarking setup that can be used for measurement, so I'll take this
as a TODO item for myself to work on over the coming days.

Thanks,
Stefan

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 488 bytes --]

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-17  6:03   ` Markus Armbruster
@ 2025-07-22 12:00     ` Brian Song
  0 siblings, 0 replies; 19+ messages in thread
From: Brian Song @ 2025-07-22 12:00 UTC (permalink / raw)
  To: Markus Armbruster
  Cc: qemu-block, qemu-devel, bschubert, fam, hreitz, kwolf, stefanha

On 7/17/25 2:03 AM, Markus Armbruster wrote:
> Brian Song <hibriansong@gmail.com> writes:
>
>> This work provides an initial implementation of fuse-over-io_uring
>> support for QEMU export. According to the fuse-over-io_uring protocol
>> specification, the userspace side must create the same number of queues
>> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
>> queue contains only a single SQE entry, which is used to validate the
>> correctness of the fuse-over-io_uring functionality.
>>
>> All FUSE read and write operations interact with the kernel via io
>> vectors embedded in the SQE entry during submission and CQE fetching.
>> The req_header and op_payload members of each entry are included as
>> parts of the io vector: req_header carries the FUSE operation header,
>> and op_payload carries the data payload, such as file attributes in a
>> getattr reply, file content in a read reply, or file content being
>> written to the FUSE client in a write operation.
>>
>> At present, multi-threading support is still incomplete. In addition,
>> handling connection termination and managing the "drained" state of a
>> FUSE block export in QEMU remain as pending work.
>>
>> Suggested-by: Kevin Wolf <kwolf@redhat.com>
>> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
>> Signed-off-by: Brian Song <hibriansong@gmail.com>
>
> [...]
>
>> diff --git a/docs/tools/qemu-storage-daemon.rst b/docs/tools/qemu-storage-daemon.rst
>> index 35ab2d7807..4ec0648e95 100644
>> --- a/docs/tools/qemu-storage-daemon.rst
>> +++ b/docs/tools/qemu-storage-daemon.rst
>> @@ -78,7 +78,7 @@ Standard options:
>>   .. option:: --export [type=]nbd,id=<id>,node-name=<node-name>[,name=<export-name>][,writable=on|off][,bitmap=<name>]
>>     --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=unix,addr.path=<socket-path>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>>     --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=fd,addr.str=<fd>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>> -  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto]
>> +  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto][,uring=on|off]
>>     --export [type=]vduse-blk,id=<id>,node-name=<node-name>,name=<vduse-name>[,writable=on|off][,num-queues=<num-queues>][,queue-size=<queue-size>][,logical-block-size=<block-size>][,serial=<serial-number>]
>>
>>     is a block export definition. ``node-name`` is the block node that should be
>> @@ -111,7 +111,13 @@ Standard options:
>>     that enabling this option as a non-root user requires enabling the
>>     user_allow_other option in the global fuse.conf configuration file.  Setting
>>     ``allow-other`` to auto (the default) will try enabling this option, and on
>> -  error fall back to disabling it.
>> +  error fall back to disabling it. Once ``uring`` is enabled
>> +  (off by default), the initialization of FUSE-over-io_uring-related settings
>> +  will be performed in the FUSE_INIT request handler. This setup bypasses
>> +  the traditional /dev/fuse communication mechanism and instead uses io_uring
>> +  for handling FUSE operations.
>> +
>> +
>
> Drop the additional blank lines, please.
>
> This is user-facing documentation.  Do users care about "the FUSE_INIT
> request handler"?
>
>>
>>     The ``vduse-blk`` export type takes a ``name`` (must be unique across the host)
>>     to create the VDUSE device.
>> diff --git a/qapi/block-export.json b/qapi/block-export.json
>> index 9ae703ad01..7d14f3f1ba 100644
>> --- a/qapi/block-export.json
>> +++ b/qapi/block-export.json
>> @@ -184,12 +184,16 @@
>>   #     mount the export with allow_other, and if that fails, try again
>>   #     without.  (since 6.1; default: auto)
>>   #
>> +# @uring: If we enable uring option, it will enable FUSE over io_uring
>> +#         feature for QEMU FUSE export.  (default: false)
>> +#
>
> Missing (since 10.2).
>
> Please format just like everywhere else:
>
>     # @uring: If we enable uring option, it will enable FUSE over
>     #     io_uring feature for QEMU FUSE export.  (default: false)
>
>
> Kernel documentation calls the thing "FUSE-over-io-uring":
> https://docs.kernel.org/filesystems/fuse-io-uring.html
>
> The text feels awkward.  Here's my attempt:
>
>     # @uring: Use FUSE-over-io-uring.  (since 10.2; default: false)
>

Thanks for pointing them out! I've fixed them. :)


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-21  0:53   ` Stefan Hajnoczi
@ 2025-07-22 12:00     ` Brian Song
  2025-07-22 15:17       ` Kevin Wolf
  2025-07-22 14:06     ` Bernd Schubert
  1 sibling, 1 reply; 19+ messages in thread
From: Brian Song @ 2025-07-22 12:00 UTC (permalink / raw)
  To: Stefan Hajnoczi
  Cc: qemu-block, qemu-devel, armbru, bschubert, fam, hreitz, kwolf

On 7/20/25 8:53 PM, Stefan Hajnoczi wrote:
> On Wed, Jul 16, 2025 at 02:38:24PM -0400, Brian Song wrote:
>> This work provides an initial implementation of fuse-over-io_uring
>> support for QEMU export. According to the fuse-over-io_uring protocol
>> specification, the userspace side must create the same number of queues
>> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
>> queue contains only a single SQE entry, which is used to validate the
>> correctness of the fuse-over-io_uring functionality.
>>
>> All FUSE read and write operations interact with the kernel via io
>> vectors embedded in the SQE entry during submission and CQE fetching.
>> The req_header and op_payload members of each entry are included as
>> parts of the io vector: req_header carries the FUSE operation header,
>> and op_payload carries the data payload, such as file attributes in a
>> getattr reply, file content in a read reply, or file content being
>> written to the FUSE client in a write operation.
>>
>> At present, multi-threading support is still incomplete. In addition,
>> handling connection termination and managing the "drained" state of a
>> FUSE block export in QEMU remain as pending work.
>>
>> Suggested-by: Kevin Wolf <kwolf@redhat.com>
>> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
>> Signed-off-by: Brian Song <hibriansong@gmail.com>
>>
>> ---
>>   block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>>   docs/tools/qemu-storage-daemon.rst   |  10 +-
>>   qapi/block-export.json               |   6 +-
>>   storage-daemon/qemu-storage-daemon.c |   1 +
>>   util/fdmon-io_uring.c                |   5 +-
>>   5 files changed, 420 insertions(+), 25 deletions(-)
>
> Here is feedback from a first pass over this patch.
>
>>
>> diff --git a/block/export/fuse.c b/block/export/fuse.c
>> index c0ad4696ce..637d36186a 100644
>> --- a/block/export/fuse.c
>> +++ b/block/export/fuse.c
>> @@ -48,6 +48,11 @@
>>   #include <linux/fs.h>
>>   #endif
>>
>> +#define FUSE_DEFAULT_MAX_PAGES_PER_REQ 32
>> +
>> +/* room needed in buffer to accommodate header */
>> +#define FUSE_BUFFER_HEADER_SIZE 0x1000
>> +
>>   /* Prevent overly long bounce buffer allocations */
>>   #define FUSE_MAX_READ_BYTES (MIN(BDRV_REQUEST_MAX_BYTES, 1 * 1024 * 1024))
>>   /*
>> @@ -64,6 +69,26 @@
>>
>>   typedef struct FuseExport FuseExport;
>>
>> +struct FuseQueue;
>
> Use "typedef struct FuseQueue FuseQueue;" here...
>
>> +
>> +typedef struct FuseRingEnt {
>> +    /* back pointer */
>> +    struct FuseQueue *q;
>
> ...and then this can be "FuseQueue *q;" so that QEMU coding style is
> followed.

Done.

>> +
>> +    /* commit id of a fuse request */
>> +    uint64_t req_commit_id;
>> +
>> +    /* fuse request header and payload */
>> +    struct fuse_uring_req_header *req_header;
>> +    void *op_payload;
>> +    size_t req_payload_sz;
>> +
>> +    /* The vector passed to the kernel */
>> +    struct iovec iov[2];
>> +
>> +    CqeHandler fuse_cqe_handler;
>> +} FuseRingEnt;
>> +
>>   /*
>>    * One FUSE "queue", representing one FUSE FD from which requests are fetched
>>    * and processed.  Each queue is tied to an AioContext.
>> @@ -73,6 +98,7 @@ typedef struct FuseQueue {
>>
>>       AioContext *ctx;
>>       int fuse_fd;
>> +    int qid;
>
> Could this go inside #ifdef CONFIG_LINUX_IO_URING? It seems to be
> specific to FUSE-over-io_uring.

Done.

>>
>>       /*
>>        * The request buffer must be able to hold a full write, and/or at least
>> @@ -109,6 +135,17 @@ typedef struct FuseQueue {
>>        * Free this buffer with qemu_vfree().
>>        */
>>       void *spillover_buf;
>> +
>> +#ifdef CONFIG_LINUX_IO_URING
>> +    FuseRingEnt ent;
>> +
>> +    /*
>> +     * TODO
>> +     * Support multi-threaded FUSE over io_uring by using eventfd and allocating
>> +     * an extra SQE for each thread to be notified when the connection
>> +     * shuts down.
>> +     */
>
> eventfd and the extra SQE won't be necessary because
> aio_bh_schedule_oneshot() can be used to cause threads to execute a
> function.
>
> (I think this comment effectively says that connection shutdown still
> needs to be implemented. The implementation details don't matter at this
> point.)
>
>> +#endif
>>   } FuseQueue;
>>
>>   /*
>> @@ -148,6 +185,7 @@ struct FuseExport {
>>       bool growable;
>>       /* Whether allow_other was used as a mount option or not */
>>       bool allow_other;
>> +    bool is_uring;
>>
>>       mode_t st_mode;
>>       uid_t st_uid;
>> @@ -257,6 +295,126 @@ static const BlockDevOps fuse_export_blk_dev_ops = {
>>       .drained_poll  = fuse_export_drained_poll,
>>   };
>>
>> +#ifdef CONFIG_LINUX_IO_URING
>> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent);
>> +
>> +static void coroutine_fn co_fuse_uring_queue_handle_cqes(void *opaque)
>> +{
>> +    CqeHandler *cqe_handler = opaque;
>> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
>
> Passing ent in opaque instead of cqe_handler would simplify this.
>

Done.

>> +    FuseExport *exp = ent->q->exp;
>> +
>> +    fuse_uring_co_process_request(ent);
>> +
>> +    fuse_dec_in_flight(exp);
>> +}
>> +
>> +static void fuse_uring_cqe_handler(CqeHandler *cqe_handler)
>> +{
>> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
>> +    FuseQueue *q = ent->q;
>> +    Coroutine *co;
>> +    FuseExport *exp = ent->q->exp;
>> +
>> +    int err = cqe_handler->cqe.res;
>> +    if (err != 0) {
>> +        /* TODO end_conn support */
>> +
>> +        /* -ENOTCONN is ok on umount  */
>> +        if (err != -EINTR && err != -EOPNOTSUPP &&
>> +            err != -EAGAIN && err != -ENOTCONN) {
>> +            fuse_export_halt(exp);
>> +        }
>> +    } else {
>> +        co = qemu_coroutine_create(co_fuse_uring_queue_handle_cqes,
>> +                            cqe_handler);
>> +        /* Decremented by co_fuse_uring_queue_handle_cqes() */
>> +        fuse_inc_in_flight(q->exp);
>
> Can this be moved inside co_fuse_uring_queue_handle_cqes() to avoid
> calling inc/dec from different functions? That would make the code
> easier to understand and more robust against future bugs.
>

Yes, I kept some of the logic the same as in the traditional approach
for handling requests


>> +        qemu_coroutine_enter(co);
>> +    }
>> +}
>> +
>> +static void fuse_uring_sqe_set_req_data(struct fuse_uring_cmd_req *req,
>> +                    const unsigned int qid,
>> +                    const unsigned int commit_id)
>> +{
>> +    req->qid = qid;
>> +    req->commit_id = commit_id;
>> +    req->flags = 0;
>> +}
>> +
>> +static void fuse_uring_sqe_prepare(struct io_uring_sqe *sqe, FuseRingEnt *ent,
>> +               __u32 cmd_op)
>> +{
>> +    sqe->opcode = IORING_OP_URING_CMD;
>> +
>> +    sqe->fd = ent->q->fuse_fd;
>> +    sqe->rw_flags = 0;
>> +    sqe->ioprio = 0;
>> +    sqe->off = 0;
>> +
>> +    sqe->cmd_op = cmd_op;
>> +    sqe->__pad1 = 0;
>> +}
>> +
>> +static void fuse_uring_prep_sqe_register(struct io_uring_sqe *sqe, void *opaque)
>> +{
>> +    FuseQueue *q = opaque;
>> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
>> +
>> +    fuse_uring_sqe_prepare(sqe, &q->ent, FUSE_IO_URING_CMD_REGISTER);
>> +
>> +    sqe->addr = (uint64_t)(q->ent.iov);
>> +    sqe->len = 2;
>> +
>> +    fuse_uring_sqe_set_req_data(req, q->qid, 0);
>> +}
>> +
>> +static void fuse_uring_start(FuseExport *exp, struct fuse_init_out *out)
>> +{
>> +    /*
>> +     * Since we didn't enable the FUSE_MAX_PAGES feature, the value of
>> +     * fc->max_pages should be FUSE_DEFAULT_MAX_PAGES_PER_REQ, which is set by
>> +     * the kernel by default. Also, max_write should not exceed
>> +     * FUSE_DEFAULT_MAX_PAGES_PER_REQ * PAGE_SIZE.
>> +     */
>> +    size_t bufsize = out->max_write + FUSE_BUFFER_HEADER_SIZE;
>> +
>> +    if (!(out->flags & FUSE_MAX_PAGES)) {
>> +        /*
>> +         * bufsize = MIN(FUSE_DEFAULT_MAX_PAGES_PER_REQ *
>> +         *       qemu_real_host_page_size() + FUSE_BUFFER_HEADER_SIZE, bufsize);
>> +         */
>> +        bufsize = FUSE_DEFAULT_MAX_PAGES_PER_REQ * qemu_real_host_page_size()
>> +                         + FUSE_BUFFER_HEADER_SIZE;
>> +    }
>> +
>> +    for (int i = 0; i < exp->num_queues; i++) {
>> +        FuseQueue *q = &exp->queues[i];
>> +
>> +        q->ent.q = q;
>> +
>> +        q->ent.req_header = g_malloc0(sizeof(struct fuse_uring_req_header));
>
> It's probably easier to embed the header as a FuseRingEnt field instead
> of heap allocating it.

Done.

>
>> +        q->ent.req_payload_sz = bufsize - FUSE_BUFFER_HEADER_SIZE;
>> +        q->ent.op_payload = g_malloc0(q->ent.req_payload_sz);
>
>
>> +
>> +        q->ent.iov[0] = (struct iovec) {
>> +            q->ent.req_header,
>> +            sizeof(struct fuse_uring_req_header)
>> +        };
>> +        q->ent.iov[1] = (struct iovec) {
>> +            q->ent.op_payload,
>> +            q->ent.req_payload_sz
>> +        };
>> +
>> +        exp->queues[i].ent.fuse_cqe_handler.cb = fuse_uring_cqe_handler;
>> +
>> +        aio_add_sqe(fuse_uring_prep_sqe_register, &(exp->queues[i]),
>> +            &(exp->queues[i].ent.fuse_cqe_handler));
>> +    }
>> +}
>> +#endif
>> +
>>   static int fuse_export_create(BlockExport *blk_exp,
>>                                 BlockExportOptions *blk_exp_args,
>>                                 AioContext *const *multithread,
>> @@ -280,6 +438,7 @@ static int fuse_export_create(BlockExport *blk_exp,
>>
>>           for (size_t i = 0; i < mt_count; i++) {
>>               exp->queues[i] = (FuseQueue) {
>> +                .qid = i,
>>                   .exp = exp,
>>                   .ctx = multithread[i],
>>                   .fuse_fd = -1,
>> @@ -293,6 +452,7 @@ static int fuse_export_create(BlockExport *blk_exp,
>>           exp->num_queues = 1;
>>           exp->queues = g_new(FuseQueue, 1);
>>           exp->queues[0] = (FuseQueue) {
>> +            .qid = 0,
>>               .exp = exp,
>>               .ctx = exp->common.ctx,
>>               .fuse_fd = -1,
>> @@ -312,6 +472,8 @@ static int fuse_export_create(BlockExport *blk_exp,
>>           }
>>       }
>>
>> +    exp->is_uring = args->uring ? true : false;
>> +
>>       blk_set_dev_ops(exp->common.blk, &fuse_export_blk_dev_ops, exp);
>>
>>       /*
>> @@ -597,6 +759,22 @@ static void read_from_fuse_fd(void *opaque)
>>       qemu_coroutine_enter(co);
>>   }
>>
>> +#ifdef CONFIG_LINUX_IO_URING
>> +static void fuse_export_delete_uring(FuseExport *exp)
>> +{
>> +    exp->is_uring = false;
>> +
>> +    /*
>> +     * TODO
>> +     * end_conn handling
>> +     */
>> +    for (size_t qid = 0; qid < exp->num_queues; qid++) {
>> +        g_free(exp->queues[qid].ent.req_header);
>> +        g_free(exp->queues[qid].ent.op_payload);
>> +    }
>> +}
>> +#endif
>> +
>>   static void fuse_export_shutdown(BlockExport *blk_exp)
>>   {
>>       FuseExport *exp = container_of(blk_exp, FuseExport, common);
>> @@ -618,6 +796,11 @@ static void fuse_export_delete(BlockExport *blk_exp)
>>   {
>>       FuseExport *exp = container_of(blk_exp, FuseExport, common);
>>
>> +#ifdef CONFIG_LINUX_IO_URING
>> +    if (exp->is_uring)
>> +        fuse_export_delete_uring(exp);
>> +#endif
>> +
>>       for (int i = 0; i < exp->num_queues; i++) {
>>           FuseQueue *q = &exp->queues[i];
>>
>> @@ -687,15 +870,22 @@ static ssize_t coroutine_fn
>>   fuse_co_init(FuseExport *exp, struct fuse_init_out *out,
>>                uint32_t max_readahead, uint32_t flags)
>>   {
>> -    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO;
>> +    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO
>> +                                     | FUSE_INIT_EXT;
>> +    uint64_t outargflags = flags;
>> +
>> +#ifdef CONFIG_LINUX_IO_URING
>> +    if (exp->is_uring)
>> +        outargflags |= FUSE_OVER_IO_URING;
>> +#endif
>>
>>       *out = (struct fuse_init_out) {
>>           .major = FUSE_KERNEL_VERSION,
>>           .minor = FUSE_KERNEL_MINOR_VERSION,
>>           .max_readahead = max_readahead,
>>           .max_write = FUSE_MAX_WRITE_BYTES,
>> -        .flags = flags & supported_flags,
>> -        .flags2 = 0,
>> +        .flags = outargflags & supported_flags,
>> +        .flags2 = outargflags >> 32,
>>
>>           /* libfuse maximum: 2^16 - 1 */
>>           .max_background = UINT16_MAX,
>> @@ -943,6 +1133,9 @@ fuse_co_read(FuseExport *exp, void **bufptr, uint64_t offset, uint32_t size)
>>    * Data in @in_place_buf is assumed to be overwritten after yielding, so will
>>    * be copied to a bounce buffer beforehand.  @spillover_buf in contrast is
>>    * assumed to be exclusively owned and will be used as-is.
>> + * In FUSE-over-io_uring mode, the actual op_payload content is stored in
>> + * @spillover_buf. To ensure this buffer is used for writing, @in_place_buf
>> + * is explicitly set to NULL.
>>    * Return the number of bytes written to *out on success, and -errno on error.
>>    */
>>   static ssize_t coroutine_fn
>> @@ -950,8 +1143,8 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>>                 uint64_t offset, uint32_t size,
>>                 const void *in_place_buf, const void *spillover_buf)
>>   {
>> -    size_t in_place_size;
>> -    void *copied;
>> +    size_t in_place_size = 0;
>> +    void *copied = NULL;
>>       int64_t blk_len;
>>       int ret;
>>       struct iovec iov[2];
>> @@ -966,10 +1159,12 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>>           return -EACCES;
>>       }
>>
>> -    /* Must copy to bounce buffer before potentially yielding */
>> -    in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
>> -    copied = blk_blockalign(exp->common.blk, in_place_size);
>> -    memcpy(copied, in_place_buf, in_place_size);
>> +    if (in_place_buf) {
>> +        /* Must copy to bounce buffer before potentially yielding */
>> +        in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
>> +        copied = blk_blockalign(exp->common.blk, in_place_size);
>> +        memcpy(copied, in_place_buf, in_place_size);
>> +    }
>>
>>       /**
>>        * Clients will expect short writes at EOF, so we have to limit
>> @@ -993,26 +1188,37 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>>           }
>>       }
>>
>> -    iov[0] = (struct iovec) {
>> -        .iov_base = copied,
>> -        .iov_len = in_place_size,
>> -    };
>> -    if (size > FUSE_IN_PLACE_WRITE_BYTES) {
>> -        assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
>> -        iov[1] = (struct iovec) {
>> -            .iov_base = (void *)spillover_buf,
>> -            .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
>> +    if (in_place_buf) {
>> +        iov[0] = (struct iovec) {
>> +            .iov_base = copied,
>> +            .iov_len = in_place_size,
>>           };
>> -        qemu_iovec_init_external(&qiov, iov, 2);
>> +        if (size > FUSE_IN_PLACE_WRITE_BYTES) {
>> +            assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
>> +            iov[1] = (struct iovec) {
>> +                .iov_base = (void *)spillover_buf,
>> +                .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
>> +            };
>> +            qemu_iovec_init_external(&qiov, iov, 2);
>> +        } else {
>> +            qemu_iovec_init_external(&qiov, iov, 1);
>> +        }
>>       } else {
>> +        /* fuse over io_uring */
>> +        iov[0] = (struct iovec) {
>> +            .iov_base = (void *)spillover_buf,
>> +            .iov_len = size,
>> +        };
>>           qemu_iovec_init_external(&qiov, iov, 1);
>>       }
>> +
>>       ret = blk_co_pwritev(exp->common.blk, offset, size, &qiov, 0);
>>       if (ret < 0) {
>>           goto fail_free_buffer;
>>       }
>>
>> -    qemu_vfree(copied);
>> +    if (in_place_buf)
>> +        qemu_vfree(copied);
>>
>>       *out = (struct fuse_write_out) {
>>           .size = size,
>> @@ -1020,7 +1226,9 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>>       return sizeof(*out);
>>
>>   fail_free_buffer:
>> -    qemu_vfree(copied);
>> +    if (in_place_buf) {
>> +        qemu_vfree(copied);
>> +    }
>>       return ret;
>>   }
>>
>> @@ -1409,6 +1617,12 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
>>           const struct fuse_init_in *in = FUSE_IN_OP_STRUCT(init, q);
>>           ret = fuse_co_init(exp, FUSE_OUT_OP_STRUCT(init, out_buf),
>>                              in->max_readahead, in->flags);
>> +#ifdef CONFIG_LINUX_IO_URING
>> +        /* Set up fuse over io_uring after replying to the first FUSE_INIT */
>> +        if (exp->is_uring) {
>> +            fuse_uring_start(exp, FUSE_OUT_OP_STRUCT(init, out_buf));
>> +        }
>> +#endif
>>           break;
>>       }
>>
>> @@ -1515,6 +1729,173 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
>>       qemu_vfree(spillover_buf);
>>   }
>>
>> +#ifdef CONFIG_LINUX_IO_URING
>> +static void fuse_uring_prep_sqe_commit(struct io_uring_sqe *sqe, void *opaque)
>> +{
>> +    FuseRingEnt *ent = opaque;
>> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
>> +
>> +    fuse_uring_sqe_prepare(sqe, ent, FUSE_IO_URING_CMD_COMMIT_AND_FETCH);
>> +    fuse_uring_sqe_set_req_data(req, ent->q->qid,
>> +                                     ent->req_commit_id);
>> +}
>> +
>> +static void
>> +fuse_uring_write_response(FuseRingEnt *ent, uint32_t req_id, ssize_t ret,
>> +                          const void *out_op_hdr, const void *buf)
>> +{
>> +    struct fuse_uring_req_header *rrh = ent->req_header;
>> +    struct fuse_out_header *out_header = (struct fuse_out_header *)&rrh->in_out;
>> +    struct fuse_uring_ent_in_out *ent_in_out =
>> +        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
>> +
>> +    if (buf) {
>> +        memcpy(ent->op_payload, buf, ret);
>> +    } else if (ret > 0) {
>> +        if (ret > ent->req_payload_sz) {
>> +            warn_report("data size %zu exceeds payload buffer size %zu",
>> +                        ret, ent->req_payload_sz);
>> +                        ret = -EINVAL;
>> +        } else {
>> +            memcpy(ent->op_payload, out_op_hdr, ret);
>> +        }
>> +    }
>> +
>> +    out_header->error  = ret < 0 ? ret : 0;
>> +    out_header->unique = req_id;
>> +    /* out_header->len = ret > 0 ? ret : 0; */
>> +    ent_in_out->payload_sz = ret > 0 ? ret : 0;
>> +
>> +    aio_add_sqe(fuse_uring_prep_sqe_commit, ent,
>> +                    &ent->fuse_cqe_handler);
>> +}
>> +
>> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent)
>> +{
>> +    FuseQueue *q = ent->q;
>> +    FuseExport *exp = q->exp;
>> +    struct fuse_uring_req_header *rrh = ent->req_header;
>> +    struct fuse_uring_ent_in_out *ent_in_out =
>> +        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
>> +
>> +    char out_op_hdr[MAX_CONST(sizeof(struct fuse_init_out),
>> +                 MAX_CONST(sizeof(struct fuse_open_out),
>> +                 MAX_CONST(sizeof(struct fuse_attr_out),
>> +                 MAX_CONST(sizeof(struct fuse_write_out),
>> +                           sizeof(struct fuse_lseek_out)))))];
>
> A union would be cleaner because the casts below could be avoided and
> MAX_CONST() would be unnecessary:
>
>    union {
>        struct fuse_init_out init;
>        struct fuse_open out open;
>        struct fuse_attr_out attr;
>        struct fuse_write_out write;
>        struct fuse_lseek_out lseek;
>    } out_op_hdr;

Yep, makes sense

>> +
>> +    void *out_data_buffer = NULL;
>> +
>> +    uint32_t opcode;
>> +    uint64_t req_id;
>> +
>> +    struct fuse_in_header *in_hdr = (struct fuse_in_header *)&rrh->in_out;
>> +    opcode = in_hdr->opcode;
>> +    req_id = in_hdr->unique;
>> +
>> +    ent->req_commit_id = ent_in_out->commit_id;
>> +
>> +    if (unlikely(ent->req_commit_id == 0)) {
>> +        /*
>> +         * If this happens kernel will not find the response - it will
>> +         * be stuck forever - better to abort immediately.
>> +         */
>> +        error_report("If this happens kernel will not find the response"
>> +        " - it will be stuck forever - better to abort immediately.");
>> +        fuse_export_halt(exp);
>> +        fuse_dec_in_flight(exp);
>> +        return;
>> +    }
>> +
>> +    ssize_t ret;
>> +
>> +    switch (opcode) {
>> +    case FUSE_OPEN:
>> +        ret = fuse_co_open(exp, (struct fuse_open_out *)out_op_hdr);
>> +        break;
>> +
>> +    case FUSE_RELEASE:
>> +        ret = 0;
>> +        break;
>> +
>> +    case FUSE_LOOKUP:
>> +        ret = -ENOENT; /* There is no node but the root node */
>> +        break;
>> +
>> +    case FUSE_GETATTR:
>> +        ret = fuse_co_getattr(exp, (struct fuse_attr_out *)out_op_hdr);
>> +        break;
>> +
>> +    case FUSE_SETATTR: {
>> +        const struct fuse_setattr_in *in =
>> +                        (const struct fuse_setattr_in *)&rrh->op_in;
>> +        ret = fuse_co_setattr(exp, (struct fuse_attr_out *)out_op_hdr,
>> +                              in->valid, in->size, in->mode, in->uid, in->gid);
>> +        break;
>> +    }
>> +
>> +    case FUSE_READ: {
>> +        const struct fuse_read_in *in =
>> +                        (const struct fuse_read_in *)&rrh->op_in;
>> +        ret = fuse_co_read(exp, &out_data_buffer, in->offset, in->size);
>> +        break;
>> +    }
>> +
>> +    case FUSE_WRITE: {
>> +        const struct fuse_write_in *in =
>> +                        (const struct fuse_write_in *)&rrh->op_in;
>> +
>> +        assert(in->size == ent_in_out->payload_sz);
>> +
>> +        /*
>> +         * poll_fuse_fd() has checked that in_hdr->len matches the number of
>> +         * bytes read, which cannot exceed the max_write value we set
>> +         * (FUSE_MAX_WRITE_BYTES).  So we know that FUSE_MAX_WRITE_BYTES >=
>> +         * in_hdr->len >= in->size + X, so this assertion must hold.
>> +         */
>> +        assert(in->size <= FUSE_MAX_WRITE_BYTES);
>> +
>> +        ret = fuse_co_write(exp, (struct fuse_write_out *)out_op_hdr,
>> +                            in->offset, in->size, NULL, ent->op_payload);
>> +        break;
>> +    }
>> +
>> +    case FUSE_FALLOCATE: {
>> +        const struct fuse_fallocate_in *in =
>> +                        (const struct fuse_fallocate_in *)&rrh->op_in;
>> +        ret = fuse_co_fallocate(exp, in->offset, in->length, in->mode);
>> +        break;
>> +    }
>> +
>> +    case FUSE_FSYNC:
>> +        ret = fuse_co_fsync(exp);
>> +        break;
>> +
>> +    case FUSE_FLUSH:
>> +        ret = fuse_co_flush(exp);
>> +        break;
>> +
>> +#ifdef CONFIG_FUSE_LSEEK
>> +    case FUSE_LSEEK: {
>> +        const struct fuse_lseek_in *in =
>> +                        (const struct fuse_lseek_in *)&rrh->op_in;
>> +        ret = fuse_co_lseek(exp, (struct fuse_lseek_out *)out_op_hdr,
>> +                            in->offset, in->whence);
>> +        break;
>> +    }
>> +#endif
>> +
>> +    default:
>> +        ret = -ENOSYS;
>> +    }
>
> It would be nice to reuse the non-io_uring code rather than duplicating
> the switch statement that covers each FUSE opcode. Is the memory layout
> so different that the code cannot be shared?
>

Yes. But I think the main issue is that we have to handle too many
differences when it comes to various operations and the final step of
replying to the request. There would be a lot of #ifdef
CONFIG_LINUX_IO_URING and if statements. So, for simplicity, I made it a
separate function.

In the traditional model:
q->request_buf = fuse_in_header + struct fuse_opsxx_in +
FUSE_IN_PLACE_WRITE_BYTES (used for part of the payload data)
q->spillover_buf is used for the rest of the payload data.

In Fuse-over-io_uring:
FuseRingEnt contains req_header (which includes in_out for
fuse_in/out_header, and op_in for struct fuse_opsxx_in)
op_payload corresponds to the final FUSE_IN_PLACE_WRITE_BYTES bytes from
request_buf + spillover_buf in the traditional model but carries either
the out operation headers (fuse_ops_out) or the file data to be written
or read.


>> +
>> +    fuse_uring_write_response(ent, req_id, ret, out_op_hdr, out_data_buffer);
>> +
>> +    if (out_data_buffer)
>> +        qemu_vfree(out_data_buffer);
>> +}
>> +#endif
>> +
>>   const BlockExportDriver blk_exp_fuse = {
>>       .type               = BLOCK_EXPORT_TYPE_FUSE,
>>       .instance_size      = sizeof(FuseExport),
>> diff --git a/docs/tools/qemu-storage-daemon.rst b/docs/tools/qemu-storage-daemon.rst
>> index 35ab2d7807..4ec0648e95 100644
>> --- a/docs/tools/qemu-storage-daemon.rst
>> +++ b/docs/tools/qemu-storage-daemon.rst
>> @@ -78,7 +78,7 @@ Standard options:
>>   .. option:: --export [type=]nbd,id=<id>,node-name=<node-name>[,name=<export-name>][,writable=on|off][,bitmap=<name>]
>>     --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=unix,addr.path=<socket-path>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>>     --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=fd,addr.str=<fd>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>> -  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto]
>> +  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto][,uring=on|off]
>>     --export [type=]vduse-blk,id=<id>,node-name=<node-name>,name=<vduse-name>[,writable=on|off][,num-queues=<num-queues>][,queue-size=<queue-size>][,logical-block-size=<block-size>][,serial=<serial-number>]
>>
>>     is a block export definition. ``node-name`` is the block node that should be
>> @@ -111,7 +111,13 @@ Standard options:
>>     that enabling this option as a non-root user requires enabling the
>>     user_allow_other option in the global fuse.conf configuration file.  Setting
>>     ``allow-other`` to auto (the default) will try enabling this option, and on
>> -  error fall back to disabling it.
>> +  error fall back to disabling it. Once ``uring`` is enabled
>> +  (off by default), the initialization of FUSE-over-io_uring-related settings
>> +  will be performed in the FUSE_INIT request handler. This setup bypasses
>> +  the traditional /dev/fuse communication mechanism and instead uses io_uring
>> +  for handling FUSE operations.
>> +
>> +
>>
>>     The ``vduse-blk`` export type takes a ``name`` (must be unique across the host)
>>     to create the VDUSE device.
>> diff --git a/qapi/block-export.json b/qapi/block-export.json
>> index 9ae703ad01..7d14f3f1ba 100644
>> --- a/qapi/block-export.json
>> +++ b/qapi/block-export.json
>> @@ -184,12 +184,16 @@
>>   #     mount the export with allow_other, and if that fails, try again
>>   #     without.  (since 6.1; default: auto)
>>   #
>> +# @uring: If we enable uring option, it will enable FUSE over io_uring
>> +#         feature for QEMU FUSE export.  (default: false)
>> +#
>>   # Since: 6.0
>>   ##
>>   { 'struct': 'BlockExportOptionsFuse',
>>     'data': { 'mountpoint': 'str',
>>               '*growable': 'bool',
>> -            '*allow-other': 'FuseExportAllowOther' },
>> +            '*allow-other': 'FuseExportAllowOther',
>> +            '*uring': 'bool' },
>>     'if': 'CONFIG_FUSE' }
>>
>>   ##
>> diff --git a/storage-daemon/qemu-storage-daemon.c b/storage-daemon/qemu-storage-daemon.c
>> index eb72561358..803538db29 100644
>> --- a/storage-daemon/qemu-storage-daemon.c
>> +++ b/storage-daemon/qemu-storage-daemon.c
>> @@ -107,6 +107,7 @@ static void help(void)
>>   #ifdef CONFIG_FUSE
>>   "  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>\n"
>>   "           [,growable=on|off][,writable=on|off][,allow-other=on|off|auto]\n"
>> +"           [,fuse-over-uring=on|off]"
>>   "                         export the specified block node over FUSE\n"
>>   "\n"
>>   #endif /* CONFIG_FUSE */
>> diff --git a/util/fdmon-io_uring.c b/util/fdmon-io_uring.c
>> index d2433d1d99..68d3fe8e01 100644
>> --- a/util/fdmon-io_uring.c
>> +++ b/util/fdmon-io_uring.c
>> @@ -452,10 +452,13 @@ static const FDMonOps fdmon_io_uring_ops = {
>>   void fdmon_io_uring_setup(AioContext *ctx, Error **errp)
>>   {
>>       int ret;
>> +    int flags;
>>
>>       ctx->io_uring_fd_tag = NULL;
>> +    flags = IORING_SETUP_SQE128;
>>
>> -    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES, &ctx->fdmon_io_uring, 0);
>> +    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES,
>> +                            &ctx->fdmon_io_uring, flags);
>>       if (ret != 0) {
>>           error_setg_errno(errp, -ret, "Failed to initialize io_uring");
>>           return;
>> --
>> 2.50.1
>>


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-21 13:51   ` Bernd Schubert
  2025-07-21 18:26     ` Stefan Hajnoczi
@ 2025-07-22 12:00     ` Brian Song
  2025-07-22 14:51       ` Stefan Hajnoczi
  2025-07-24 20:36     ` Stefan Hajnoczi
  2 siblings, 1 reply; 19+ messages in thread
From: Brian Song @ 2025-07-22 12:00 UTC (permalink / raw)
  To: Bernd Schubert, qemu-block@nongnu.org
  Cc: qemu-devel@nongnu.org, armbru@redhat.com, fam@euphon.net,
	hreitz@redhat.com, kwolf@redhat.com, stefanha@redhat.com

On 7/21/25 9:51 AM, Bernd Schubert wrote:
> Hi Brian,
>
> nice work! Just some minor comments from my side below.
>

Really appreciate you taking the time to review my code and many thanks
for the helpful suggestions!


> On 7/16/25 20:38, Brian Song wrote:
>> [You don't often get email from hibriansong@gmail.com. Learn why this is important at https://aka.ms/LearnAboutSenderIdentification ]
>>
>> This work provides an initial implementation of fuse-over-io_uring
>> support for QEMU export. According to the fuse-over-io_uring protocol
>> specification, the userspace side must create the same number of queues
>> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
>> queue contains only a single SQE entry, which is used to validate the
>> correctness of the fuse-over-io_uring functionality.
>>
>> All FUSE read and write operations interact with the kernel via io
>> vectors embedded in the SQE entry during submission and CQE fetching.
>> The req_header and op_payload members of each entry are included as
>> parts of the io vector: req_header carries the FUSE operation header,
>> and op_payload carries the data payload, such as file attributes in a
>> getattr reply, file content in a read reply, or file content being
>> written to the FUSE client in a write operation.
>>
>> At present, multi-threading support is still incomplete. In addition,
>> handling connection termination and managing the "drained" state of a
>> FUSE block export in QEMU remain as pending work.
>>
>> Suggested-by: Kevin Wolf <kwolf@redhat.com>
>> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
>> Signed-off-by: Brian Song <hibriansong@gmail.com>
>>
>> ---
>>   block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>>   docs/tools/qemu-storage-daemon.rst   |  10 +-
>>   qapi/block-export.json               |   6 +-
>>   storage-daemon/qemu-storage-daemon.c |   1 +
>>   util/fdmon-io_uring.c                |   5 +-
>>   5 files changed, 420 insertions(+), 25 deletions(-)
>>
>> diff --git a/block/export/fuse.c b/block/export/fuse.c
>> index c0ad4696ce..637d36186a 100644
>> --- a/block/export/fuse.c
>> +++ b/block/export/fuse.c
>> @@ -48,6 +48,11 @@
>>   #include <linux/fs.h>
>>   #endif
>>
>> +#define FUSE_DEFAULT_MAX_PAGES_PER_REQ 32
>> +
>> +/* room needed in buffer to accommodate header */
>> +#define FUSE_BUFFER_HEADER_SIZE 0x1000
>> +
>>   /* Prevent overly long bounce buffer allocations */
>>   #define FUSE_MAX_READ_BYTES (MIN(BDRV_REQUEST_MAX_BYTES, 1 * 1024 * 1024))
>>   /*
>> @@ -64,6 +69,26 @@
>>
>>   typedef struct FuseExport FuseExport;
>>
>> +struct FuseQueue;
>> +
>> +typedef struct FuseRingEnt {
>> +    /* back pointer */
>> +    struct FuseQueue *q;
>> +
>> +    /* commit id of a fuse request */
>> +    uint64_t req_commit_id;
>> +
>> +    /* fuse request header and payload */
>> +    struct fuse_uring_req_header *req_header;
>> +    void *op_payload;
>> +    size_t req_payload_sz;
>
> Note that this will be needed only later, when we have multiple request sizes per queue. I had added it to libfuse only pro-actively to avoid the need to change too much code later on.
>

I noticed that libfuse uses it the same way I do in QEMU: I use
req_payload_sz to represent the size of op_payload (this is needed by
the kernel to determine how much data should be copied into its buffer),
and ent_in_out->payload_sz as the actual size of the data we are going
to respond with.

>> +
>> +    /* The vector passed to the kernel */
>> +    struct iovec iov[2];
>> +
>> +    CqeHandler fuse_cqe_handler;
>> +} FuseRingEnt;
>> +
>>   /*
>>    * One FUSE "queue", representing one FUSE FD from which requests are fetched
>>    * and processed.  Each queue is tied to an AioContext.
>> @@ -73,6 +98,7 @@ typedef struct FuseQueue {
>>
>>       AioContext *ctx;
>>       int fuse_fd;
>> +    int qid;
>>
>>       /*
>>        * The request buffer must be able to hold a full write, and/or at least
>> @@ -109,6 +135,17 @@ typedef struct FuseQueue {
>>        * Free this buffer with qemu_vfree().
>>        */
>>       void *spillover_buf;
>> +
>> +#ifdef CONFIG_LINUX_IO_URING
>> +    FuseRingEnt ent;
>> +
>> +    /*
>> +     * TODO
>> +     * Support multi-threaded FUSE over io_uring by using eventfd and allocating
>> +     * an extra SQE for each thread to be notified when the connection
>> +     * shuts down.
>> +     */
>> +#endif
>>   } FuseQueue;
>>
>>   /*
>> @@ -148,6 +185,7 @@ struct FuseExport {
>>       bool growable;
>>       /* Whether allow_other was used as a mount option or not */
>>       bool allow_other;
>> +    bool is_uring;
>>
>>       mode_t st_mode;
>>       uid_t st_uid;
>> @@ -257,6 +295,126 @@ static const BlockDevOps fuse_export_blk_dev_ops = {
>>       .drained_poll  = fuse_export_drained_poll,
>>   };
>>
>> +#ifdef CONFIG_LINUX_IO_URING
>> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent);
>> +
>> +static void coroutine_fn co_fuse_uring_queue_handle_cqes(void *opaque)
>> +{
>> +    CqeHandler *cqe_handler = opaque;
>> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
>> +    FuseExport *exp = ent->q->exp;
>> +
>> +    fuse_uring_co_process_request(ent);
>> +
>> +    fuse_dec_in_flight(exp);
>> +}
>> +
>> +static void fuse_uring_cqe_handler(CqeHandler *cqe_handler)
>> +{
>> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
>> +    FuseQueue *q = ent->q;
>> +    Coroutine *co;
>> +    FuseExport *exp = ent->q->exp;
>> +
>> +    int err = cqe_handler->cqe.res;
>> +    if (err != 0) {
>> +        /* TODO end_conn support */
>> +
>> +        /* -ENOTCONN is ok on umount  */
>> +        if (err != -EINTR && err != -EOPNOTSUPP &&
>> +            err != -EAGAIN && err != -ENOTCONN) {
>> +            fuse_export_halt(exp);
>> +        }
>> +    } else {
>> +        co = qemu_coroutine_create(co_fuse_uring_queue_handle_cqes,
>> +                            cqe_handler);
>> +        /* Decremented by co_fuse_uring_queue_handle_cqes() */
>> +        fuse_inc_in_flight(q->exp);
>> +        qemu_coroutine_enter(co);
>> +    }
>> +}
>> +
>> +static void fuse_uring_sqe_set_req_data(struct fuse_uring_cmd_req *req,
>> +                    const unsigned int qid,
>> +                    const unsigned int commit_id)
>> +{
>> +    req->qid = qid;
>> +    req->commit_id = commit_id;
>> +    req->flags = 0;
>> +}
>> +
>> +static void fuse_uring_sqe_prepare(struct io_uring_sqe *sqe, FuseRingEnt *ent,
>> +               __u32 cmd_op)
>> +{
>> +    sqe->opcode = IORING_OP_URING_CMD;
>> +
>> +    sqe->fd = ent->q->fuse_fd;
>> +    sqe->rw_flags = 0;
>> +    sqe->ioprio = 0;
>> +    sqe->off = 0;
>> +
>> +    sqe->cmd_op = cmd_op;
>> +    sqe->__pad1 = 0;
>> +}
>> +
>> +static void fuse_uring_prep_sqe_register(struct io_uring_sqe *sqe, void *opaque)
>> +{
>> +    FuseQueue *q = opaque;
>> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
>> +
>> +    fuse_uring_sqe_prepare(sqe, &q->ent, FUSE_IO_URING_CMD_REGISTER);
>> +
>> +    sqe->addr = (uint64_t)(q->ent.iov);
>> +    sqe->len = 2;
>> +
>> +    fuse_uring_sqe_set_req_data(req, q->qid, 0);
>> +}
>> +
>> +static void fuse_uring_start(FuseExport *exp, struct fuse_init_out *out)
>> +{
>> +    /*
>> +     * Since we didn't enable the FUSE_MAX_PAGES feature, the value of
>> +     * fc->max_pages should be FUSE_DEFAULT_MAX_PAGES_PER_REQ, which is set by
>> +     * the kernel by default. Also, max_write should not exceed
>> +     * FUSE_DEFAULT_MAX_PAGES_PER_REQ * PAGE_SIZE.
>
>  From kernel point of view these things always cause headache, because now it
> is impossible to change defaults. Can't it set FUSE_MAX_PAGES and then define
> the actual limit?
>

We can definitely set FUSE_MAX_PAGES and use a large buffer.
However, in QEMU, it seems that we don't really need this parameter
right now, though it could still be exposed as an option for users.

>> +     */
>> +    size_t bufsize = out->max_write + FUSE_BUFFER_HEADER_SIZE;
>> +
>> +    if (!(out->flags & FUSE_MAX_PAGES)) {
>> +        /*
>> +         * bufsize = MIN(FUSE_DEFAULT_MAX_PAGES_PER_REQ *
>> +         *       qemu_real_host_page_size() + FUSE_BUFFER_HEADER_SIZE, bufsize);
>> +         */
>> +        bufsize = FUSE_DEFAULT_MAX_PAGES_PER_REQ * qemu_real_host_page_size()
>> +                         + FUSE_BUFFER_HEADER_SIZE;
>> +    }
>> +
>> +    for (int i = 0; i < exp->num_queues; i++) {
>> +        FuseQueue *q = &exp->queues[i];
>> +
>> +        q->ent.q = q;
>> +
>> +        q->ent.req_header = g_malloc0(sizeof(struct fuse_uring_req_header));
>> +        q->ent.req_payload_sz = bufsize - FUSE_BUFFER_HEADER_SIZE;
>> +        q->ent.op_payload = g_malloc0(q->ent.req_payload_sz);
>> +
>> +        q->ent.iov[0] = (struct iovec) {
>> +            q->ent.req_header,
>> +            sizeof(struct fuse_uring_req_header)
>> +        };
>> +        q->ent.iov[1] = (struct iovec) {
>> +            q->ent.op_payload,
>> +            q->ent.req_payload_sz
>> +        };
>> +
>> +        exp->queues[i].ent.fuse_cqe_handler.cb = fuse_uring_cqe_handler;
>> +
>> +        aio_add_sqe(fuse_uring_prep_sqe_register, &(exp->queues[i]),
>> +            &(exp->queues[i].ent.fuse_cqe_handler));
>> +    }
>> +}
>> +#endif
>> +
>>   static int fuse_export_create(BlockExport *blk_exp,
>>                                 BlockExportOptions *blk_exp_args,
>>                                 AioContext *const *multithread,
>> @@ -280,6 +438,7 @@ static int fuse_export_create(BlockExport *blk_exp,
>>
>>           for (size_t i = 0; i < mt_count; i++) {
>>               exp->queues[i] = (FuseQueue) {
>> +                .qid = i,
>>                   .exp = exp,
>>                   .ctx = multithread[i],
>>                   .fuse_fd = -1,
>> @@ -293,6 +452,7 @@ static int fuse_export_create(BlockExport *blk_exp,
>>           exp->num_queues = 1;
>>           exp->queues = g_new(FuseQueue, 1);
>>           exp->queues[0] = (FuseQueue) {
>> +            .qid = 0,
>>               .exp = exp,
>>               .ctx = exp->common.ctx,
>>               .fuse_fd = -1,
>> @@ -312,6 +472,8 @@ static int fuse_export_create(BlockExport *blk_exp,
>>           }
>>       }
>>
>> +    exp->is_uring = args->uring ? true : false;
>> +
>>       blk_set_dev_ops(exp->common.blk, &fuse_export_blk_dev_ops, exp);
>>
>>       /*
>> @@ -597,6 +759,22 @@ static void read_from_fuse_fd(void *opaque)
>>       qemu_coroutine_enter(co);
>>   }
>>
>> +#ifdef CONFIG_LINUX_IO_URING
>> +static void fuse_export_delete_uring(FuseExport *exp)
>> +{
>> +    exp->is_uring = false;
>> +
>> +    /*
>> +     * TODO
>> +     * end_conn handling
>> +     */
>> +    for (size_t qid = 0; qid < exp->num_queues; qid++) {
>> +        g_free(exp->queues[qid].ent.req_header);
>> +        g_free(exp->queues[qid].ent.op_payload);
>> +    }
>> +}
>> +#endif
>> +
>>   static void fuse_export_shutdown(BlockExport *blk_exp)
>>   {
>>       FuseExport *exp = container_of(blk_exp, FuseExport, common);
>> @@ -618,6 +796,11 @@ static void fuse_export_delete(BlockExport *blk_exp)
>>   {
>>       FuseExport *exp = container_of(blk_exp, FuseExport, common);
>>
>> +#ifdef CONFIG_LINUX_IO_URING
>> +    if (exp->is_uring)
>> +        fuse_export_delete_uring(exp);
>> +#endif
>> +
>>       for (int i = 0; i < exp->num_queues; i++) {
>>           FuseQueue *q = &exp->queues[i];
>>
>> @@ -687,15 +870,22 @@ static ssize_t coroutine_fn
>>   fuse_co_init(FuseExport *exp, struct fuse_init_out *out,
>>                uint32_t max_readahead, uint32_t flags)
>>   {
>> -    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO;
>> +    const uint32_t supported_flags = FUSE_ASYNC_READ | FUSE_ASYNC_DIO
>> +                                     | FUSE_INIT_EXT;
>> +    uint64_t outargflags = flags;
>> +
>> +#ifdef CONFIG_LINUX_IO_URING
>> +    if (exp->is_uring)
>> +        outargflags |= FUSE_OVER_IO_URING;
>> +#endif
>>
>>       *out = (struct fuse_init_out) {
>>           .major = FUSE_KERNEL_VERSION,
>>           .minor = FUSE_KERNEL_MINOR_VERSION,
>>           .max_readahead = max_readahead,
>>           .max_write = FUSE_MAX_WRITE_BYTES,
>> -        .flags = flags & supported_flags,
>> -        .flags2 = 0,
>> +        .flags = outargflags & supported_flags,
>> +        .flags2 = outargflags >> 32,
>>
>>           /* libfuse maximum: 2^16 - 1 */
>>           .max_background = UINT16_MAX,
>> @@ -943,6 +1133,9 @@ fuse_co_read(FuseExport *exp, void **bufptr, uint64_t offset, uint32_t size)
>>    * Data in @in_place_buf is assumed to be overwritten after yielding, so will
>>    * be copied to a bounce buffer beforehand.  @spillover_buf in contrast is
>>    * assumed to be exclusively owned and will be used as-is.
>> + * In FUSE-over-io_uring mode, the actual op_payload content is stored in
>> + * @spillover_buf. To ensure this buffer is used for writing, @in_place_buf
>> + * is explicitly set to NULL.
>>    * Return the number of bytes written to *out on success, and -errno on error.
>>    */
>>   static ssize_t coroutine_fn
>> @@ -950,8 +1143,8 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>>                 uint64_t offset, uint32_t size,
>>                 const void *in_place_buf, const void *spillover_buf)
>>   {
>> -    size_t in_place_size;
>> -    void *copied;
>> +    size_t in_place_size = 0;
>> +    void *copied = NULL;
>>       int64_t blk_len;
>>       int ret;
>>       struct iovec iov[2];
>> @@ -966,10 +1159,12 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>>           return -EACCES;
>>       }
>>
>> -    /* Must copy to bounce buffer before potentially yielding */
>> -    in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
>> -    copied = blk_blockalign(exp->common.blk, in_place_size);
>> -    memcpy(copied, in_place_buf, in_place_size);
>> +    if (in_place_buf) {
>> +        /* Must copy to bounce buffer before potentially yielding */
>> +        in_place_size = MIN(size, FUSE_IN_PLACE_WRITE_BYTES);
>> +        copied = blk_blockalign(exp->common.blk, in_place_size);
>> +        memcpy(copied, in_place_buf, in_place_size);
>> +    }
>>
>>       /**
>>        * Clients will expect short writes at EOF, so we have to limit
>> @@ -993,26 +1188,37 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>>           }
>>       }
>>
>> -    iov[0] = (struct iovec) {
>> -        .iov_base = copied,
>> -        .iov_len = in_place_size,
>> -    };
>> -    if (size > FUSE_IN_PLACE_WRITE_BYTES) {
>> -        assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
>> -        iov[1] = (struct iovec) {
>> -            .iov_base = (void *)spillover_buf,
>> -            .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
>> +    if (in_place_buf) {
>> +        iov[0] = (struct iovec) {
>> +            .iov_base = copied,
>> +            .iov_len = in_place_size,
>>           };
>> -        qemu_iovec_init_external(&qiov, iov, 2);
>> +        if (size > FUSE_IN_PLACE_WRITE_BYTES) {
>> +            assert(size - FUSE_IN_PLACE_WRITE_BYTES <= FUSE_SPILLOVER_BUF_SIZE);
>> +            iov[1] = (struct iovec) {
>> +                .iov_base = (void *)spillover_buf,
>> +                .iov_len = size - FUSE_IN_PLACE_WRITE_BYTES,
>> +            };
>> +            qemu_iovec_init_external(&qiov, iov, 2);
>> +        } else {
>> +            qemu_iovec_init_external(&qiov, iov, 1);
>> +        }
>>       } else {
>> +        /* fuse over io_uring */
>> +        iov[0] = (struct iovec) {
>> +            .iov_base = (void *)spillover_buf,
>> +            .iov_len = size,
>> +        };
>>           qemu_iovec_init_external(&qiov, iov, 1);
>>       }
>> +
>>       ret = blk_co_pwritev(exp->common.blk, offset, size, &qiov, 0);
>>       if (ret < 0) {
>>           goto fail_free_buffer;
>>       }
>>
>> -    qemu_vfree(copied);
>> +    if (in_place_buf)
>> +        qemu_vfree(copied);
>>
>>       *out = (struct fuse_write_out) {
>>           .size = size,
>> @@ -1020,7 +1226,9 @@ fuse_co_write(FuseExport *exp, struct fuse_write_out *out,
>>       return sizeof(*out);
>>
>>   fail_free_buffer:
>> -    qemu_vfree(copied);
>> +    if (in_place_buf) {
>> +        qemu_vfree(copied);
>> +    }
>>       return ret;
>>   }
>>
>> @@ -1409,6 +1617,12 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
>>           const struct fuse_init_in *in = FUSE_IN_OP_STRUCT(init, q);
>>           ret = fuse_co_init(exp, FUSE_OUT_OP_STRUCT(init, out_buf),
>>                              in->max_readahead, in->flags);
>> +#ifdef CONFIG_LINUX_IO_URING
>> +        /* Set up fuse over io_uring after replying to the first FUSE_INIT */
>> +        if (exp->is_uring) {
>> +            fuse_uring_start(exp, FUSE_OUT_OP_STRUCT(init, out_buf));
>> +        }
>> +#endif
>>           break;
>>       }
>>
>> @@ -1515,6 +1729,173 @@ fuse_co_process_request(FuseQueue *q, void *spillover_buf)
>>       qemu_vfree(spillover_buf);
>>   }
>>
>> +#ifdef CONFIG_LINUX_IO_URING
>> +static void fuse_uring_prep_sqe_commit(struct io_uring_sqe *sqe, void *opaque)
>> +{
>> +    FuseRingEnt *ent = opaque;
>> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
>> +
>> +    fuse_uring_sqe_prepare(sqe, ent, FUSE_IO_URING_CMD_COMMIT_AND_FETCH);
>> +    fuse_uring_sqe_set_req_data(req, ent->q->qid,
>> +                                     ent->req_commit_id);
>> +}
>> +
>> +static void
>> +fuse_uring_write_response(FuseRingEnt *ent, uint32_t req_id, ssize_t ret,
>> +                          const void *out_op_hdr, const void *buf)
>
> Isn't it more a "send_response"? With /dev/fuse there was a write()
> operation, but with io-uring it just fills an SQE?
>

Yep, make sense. Thanks for pointing out.

>> +{
>> +    struct fuse_uring_req_header *rrh = ent->req_header;
>> +    struct fuse_out_header *out_header = (struct fuse_out_header *)&rrh->in_out;
>> +    struct fuse_uring_ent_in_out *ent_in_out =
>> +        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
>> +
>> +    if (buf) {
>> +        memcpy(ent->op_payload, buf, ret);
>> +    } else if (ret > 0) {
>> +        if (ret > ent->req_payload_sz) {
>> +            warn_report("data size %zu exceeds payload buffer size %zu",
>> +                        ret, ent->req_payload_sz);
>> +                        ret = -EINVAL;
>> +        } else {
>> +            memcpy(ent->op_payload, out_op_hdr, ret);
>> +        }
>> +    }
>
> I would try to optimize that away. Like
>
> if (buf && buf != ent->op_payload) {
>       /* copy needed */
> }
>> +
>> +    out_header->error  = ret < 0 ? ret : 0;
>> +    out_header->unique = req_id;
>> +    /* out_header->len = ret > 0 ? ret : 0; */
>> +    ent_in_out->payload_sz = ret > 0 ? ret : 0;
>> +
>> +    aio_add_sqe(fuse_uring_prep_sqe_commit, ent,
>> +                    &ent->fuse_cqe_handler);
>> +}
>> +
>> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent)
>> +{
>> +    FuseQueue *q = ent->q;
>> +    FuseExport *exp = q->exp;
>> +    struct fuse_uring_req_header *rrh = ent->req_header;
>> +    struct fuse_uring_ent_in_out *ent_in_out =
>> +        (struct fuse_uring_ent_in_out *)&rrh->ring_ent_in_out;
>> +
>> +    char out_op_hdr[MAX_CONST(sizeof(struct fuse_init_out),
>> +                 MAX_CONST(sizeof(struct fuse_open_out),
>> +                 MAX_CONST(sizeof(struct fuse_attr_out),
>> +                 MAX_CONST(sizeof(struct fuse_write_out),
>> +                           sizeof(struct fuse_lseek_out)))))];
>> +
>> +    void *out_data_buffer = NULL;
>> +
>> +    uint32_t opcode;
>> +    uint64_t req_id;
>> +
>> +    struct fuse_in_header *in_hdr = (struct fuse_in_header *)&rrh->in_out;
>> +    opcode = in_hdr->opcode;
>> +    req_id = in_hdr->unique;
>> +
>> +    ent->req_commit_id = ent_in_out->commit_id;
>> +
>> +    if (unlikely(ent->req_commit_id == 0)) {
>> +        /*
>> +         * If this happens kernel will not find the response - it will
>> +         * be stuck forever - better to abort immediately.
>> +         */
>> +        error_report("If this happens kernel will not find the response"
>> +        " - it will be stuck forever - better to abort immediately.");
>
> Dunno about qemu style, but isn't it better to keep the string in a single
> line to make it greppable?
>

The actual log output doesn’t include any line breaks. I just broke the
line in the code to keep it under 80 characters per line.

>> +        fuse_export_halt(exp);
>> +        fuse_dec_in_flight(exp);
>> +        return;
>> +    }
>> +
>> +    ssize_t ret;
>> +
>> +    switch (opcode) {
>> +    case FUSE_OPEN:
>> +        ret = fuse_co_open(exp, (struct fuse_open_out *)out_op_hdr);
>> +        break;
>> +
>> +    case FUSE_RELEASE:
>> +        ret = 0;
>> +        break;
>> +
>> +    case FUSE_LOOKUP:
>> +        ret = -ENOENT; /* There is no node but the root node */
>> +        break;
>> +
>> +    case FUSE_GETATTR:
>> +        ret = fuse_co_getattr(exp, (struct fuse_attr_out *)out_op_hdr);
>> +        break;
>> +
>> +    case FUSE_SETATTR: {
>> +        const struct fuse_setattr_in *in =
>> +                        (const struct fuse_setattr_in *)&rrh->op_in;
>> +        ret = fuse_co_setattr(exp, (struct fuse_attr_out *)out_op_hdr,
>> +                              in->valid, in->size, in->mode, in->uid, in->gid);
>> +        break;
>> +    }
>> +
>> +    case FUSE_READ: {
>> +        const struct fuse_read_in *in =
>> +                        (const struct fuse_read_in *)&rrh->op_in;
>> +        ret = fuse_co_read(exp, &out_data_buffer, in->offset, in->size);
>> +        break;
>> +    }
>
> There is room for optimization here - this could just pass the payload
> buffer and let it directly copy the data into it?
>

We ultimately copy either the fuse_ops_out structures or the output data
from read operations into the payload.

Technically, we could just pass the payload pointer directly to each
operation handler in fuse_co_ops, but that might be confusing, it's not
immediately clear to readers what the payload is actually being used for.

For better code readability, I decided to allocate a temporary buffer
for non-read operations to store their fuse_ops_out structures.

This is especially important for write operations, where we may end up
passing the same ent->op_payload as both the output header and the file
data to be written to disk, which could lead to confusion.

In libfuse, there are many more operation types, and they always copy
both the output headers and the file content into the payload anyway.

Maybe we could use differently named pointers to refer to
ent->op_payload, just to make the code more readable and clarify its
role in different contexts.


>> +
>> +    case FUSE_WRITE: {
>> +        const struct fuse_write_in *in =
>> +                        (const struct fuse_write_in *)&rrh->op_in;
>> +
>> +        assert(in->size == ent_in_out->payload_sz);
>> +
>> +        /*
>> +         * poll_fuse_fd() has checked that in_hdr->len matches the number of
>> +         * bytes read, which cannot exceed the max_write value we set
>> +         * (FUSE_MAX_WRITE_BYTES).  So we know that FUSE_MAX_WRITE_BYTES >=
>> +         * in_hdr->len >= in->size + X, so this assertion must hold.
>> +         */
>> +        assert(in->size <= FUSE_MAX_WRITE_BYTES);
>> +
>> +        ret = fuse_co_write(exp, (struct fuse_write_out *)out_op_hdr,
>> +                            in->offset, in->size, NULL, ent->op_payload);
>> +        break;
>> +    }
>> +
>> +    case FUSE_FALLOCATE: {
>> +        const struct fuse_fallocate_in *in =
>> +                        (const struct fuse_fallocate_in *)&rrh->op_in;
>> +        ret = fuse_co_fallocate(exp, in->offset, in->length, in->mode);
>> +        break;
>> +    }
>> +
>> +    case FUSE_FSYNC:
>> +        ret = fuse_co_fsync(exp);
>> +        break;
>> +
>> +    case FUSE_FLUSH:
>> +        ret = fuse_co_flush(exp);
>> +        break;
>> +
>> +#ifdef CONFIG_FUSE_LSEEK
>> +    case FUSE_LSEEK: {
>> +        const struct fuse_lseek_in *in =
>> +                        (const struct fuse_lseek_in *)&rrh->op_in;
>> +        ret = fuse_co_lseek(exp, (struct fuse_lseek_out *)out_op_hdr,
>> +                            in->offset, in->whence);
>> +        break;
>> +    }
>> +#endif
>> +
>> +    default:
>> +        ret = -ENOSYS;
>> +    }
>> +
>> +    fuse_uring_write_response(ent, req_id, ret, out_op_hdr, out_data_buffer);
>> +
>> +    if (out_data_buffer)
>> +        qemu_vfree(out_data_buffer);
>> +}
>> +#endif
>> +
>>   const BlockExportDriver blk_exp_fuse = {
>>       .type               = BLOCK_EXPORT_TYPE_FUSE,
>>       .instance_size      = sizeof(FuseExport),
>> diff --git a/docs/tools/qemu-storage-daemon.rst b/docs/tools/qemu-storage-daemon.rst
>> index 35ab2d7807..4ec0648e95 100644
>> --- a/docs/tools/qemu-storage-daemon.rst
>> +++ b/docs/tools/qemu-storage-daemon.rst
>> @@ -78,7 +78,7 @@ Standard options:
>>   .. option:: --export [type=]nbd,id=<id>,node-name=<node-name>[,name=<export-name>][,writable=on|off][,bitmap=<name>]
>>     --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=unix,addr.path=<socket-path>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>>     --export [type=]vhost-user-blk,id=<id>,node-name=<node-name>,addr.type=fd,addr.str=<fd>[,writable=on|off][,logical-block-size=<block-size>][,num-queues=<num-queues>]
>> -  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto]
>> +  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>[,growable=on|off][,writable=on|off][,allow-other=on|off|auto][,uring=on|off]
>>     --export [type=]vduse-blk,id=<id>,node-name=<node-name>,name=<vduse-name>[,writable=on|off][,num-queues=<num-queues>][,queue-size=<queue-size>][,logical-block-size=<block-size>][,serial=<serial-number>]
>>
>>     is a block export definition. ``node-name`` is the block node that should be
>> @@ -111,7 +111,13 @@ Standard options:
>>     that enabling this option as a non-root user requires enabling the
>>     user_allow_other option in the global fuse.conf configuration file.  Setting
>>     ``allow-other`` to auto (the default) will try enabling this option, and on
>> -  error fall back to disabling it.
>> +  error fall back to disabling it. Once ``uring`` is enabled
>> +  (off by default), the initialization of FUSE-over-io_uring-related settings
>> +  will be performed in the FUSE_INIT request handler. This setup bypasses
>> +  the traditional /dev/fuse communication mechanism and instead uses io_uring
>> +  for handling FUSE operations.
>> +
>> +
>>
>>     The ``vduse-blk`` export type takes a ``name`` (must be unique across the host)
>>     to create the VDUSE device.
>> diff --git a/qapi/block-export.json b/qapi/block-export.json
>> index 9ae703ad01..7d14f3f1ba 100644
>> --- a/qapi/block-export.json
>> +++ b/qapi/block-export.json
>> @@ -184,12 +184,16 @@
>>   #     mount the export with allow_other, and if that fails, try again
>>   #     without.  (since 6.1; default: auto)
>>   #
>> +# @uring: If we enable uring option, it will enable FUSE over io_uring
>> +#         feature for QEMU FUSE export.  (default: false)
>> +#
>>   # Since: 6.0
>>   ##
>>   { 'struct': 'BlockExportOptionsFuse',
>>     'data': { 'mountpoint': 'str',
>>               '*growable': 'bool',
>> -            '*allow-other': 'FuseExportAllowOther' },
>> +            '*allow-other': 'FuseExportAllowOther',
>> +            '*uring': 'bool' },
>>     'if': 'CONFIG_FUSE' }
>>
>>   ##
>> diff --git a/storage-daemon/qemu-storage-daemon.c b/storage-daemon/qemu-storage-daemon.c
>> index eb72561358..803538db29 100644
>> --- a/storage-daemon/qemu-storage-daemon.c
>> +++ b/storage-daemon/qemu-storage-daemon.c
>> @@ -107,6 +107,7 @@ static void help(void)
>>   #ifdef CONFIG_FUSE
>>   "  --export [type=]fuse,id=<id>,node-name=<node-name>,mountpoint=<file>\n"
>>   "           [,growable=on|off][,writable=on|off][,allow-other=on|off|auto]\n"
>> +"           [,fuse-over-uring=on|off]"
>>   "                         export the specified block node over FUSE\n"
>>   "\n"
>>   #endif /* CONFIG_FUSE */
>> diff --git a/util/fdmon-io_uring.c b/util/fdmon-io_uring.c
>> index d2433d1d99..68d3fe8e01 100644
>> --- a/util/fdmon-io_uring.c
>> +++ b/util/fdmon-io_uring.c
>> @@ -452,10 +452,13 @@ static const FDMonOps fdmon_io_uring_ops = {
>>   void fdmon_io_uring_setup(AioContext *ctx, Error **errp)
>>   {
>>       int ret;
>> +    int flags;
>>
>>       ctx->io_uring_fd_tag = NULL;
>> +    flags = IORING_SETUP_SQE128;
>>
>> -    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES, &ctx->fdmon_io_uring, 0);
>> +    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES,
>> +                            &ctx->fdmon_io_uring, flags);
>>       if (ret != 0) {
>>           error_setg_errno(errp, -ret, "Failed to initialize io_uring");
>>           return;
>
> I need test these flags again, might be worthful for qemu as well
>
>       /* These flags should help to increase performance, but actually
>        * make it a bit slower - reason should get investigated.
>        */
>       if (0) {
>               /* Has the main slow down effect */
>               params.flags |= IORING_SETUP_SINGLE_ISSUER;
>
>               // params.flags |= IORING_SETUP_DEFER_TASKRUN;
>               params.flags |= IORING_SETUP_TASKRUN_FLAG;
>
>               /* Second main effect to make it slower */
>               params.flags |= IORING_SETUP_COOP_TASKRUN;
>       }
>
>
>
>
> Thanks,
> Bernd
>


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 0/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-20 16:13 ` [PATCH RFC 0/1] " Stefan Hajnoczi
@ 2025-07-22 12:00   ` Brian Song
  2025-07-22 14:47     ` Stefan Hajnoczi
  0 siblings, 1 reply; 19+ messages in thread
From: Brian Song @ 2025-07-22 12:00 UTC (permalink / raw)
  To: Stefan Hajnoczi
  Cc: qemu-block, qemu-devel, armbru, bschubert, fam, hreitz, kwolf

On 7/20/25 12:13 PM, Stefan Hajnoczi wrote:
> On Wed, Jul 16, 2025 at 02:38:23PM -0400, Brian Song wrote:
>> This RFC patch represents an initial implementation of the FUSE-over-
>> io_uring Exports idea proposed for Google Summer of Code (2025) under
>> the QEMU community:
>> https://wiki.qemu.org/Google_Summer_of_Code_2025#FUSE-over-io_uring_exports
>>
>> The implementation approach is primarily borrowed from how libfuse
>> interacts with the kernel.
>>
>> FUSE-over-io_uring (https://docs.kernel.org/next/filesystems/fuse-io-
>> uring.html) has been officially merged into the Linux kernel. The idea
>> is to replace the traditional /dev/fuse based communication with a more
>> efficient io_uring-based approach. In this model, userspace registers
>> io_uring SQEs via the FUSE_IO_URING_CMD_REGISTER opcode, and then waits
>> for the kernel to forward FUSE requests as CQEs. These are processed by
>> the FUSE exports implementation in userspace and then committed back to
>> the kernel using FUSE_IO_URING_CMD_COMMIT_AND_FETCH.
>>
>> To enable this feature in qemu-export-daemon, simply add the uring=on
>> option to the export configuration.
>
> For consistency with the --blockdev file,aio=io_uring option, please
> name it "io_uring=on" instead of "uring=on".
>

It seems we are not allowed to use '_' here.

In file included from ../storage-daemon/qapi/qapi-schema.json:57:

../storage-daemon/qapi/../../qapi/block-export.json: In struct
'BlockExportOptionsFuse':

../storage-daemon/qapi/../../qapi/block-export.json:191: name of 'data'
member '*io_uring' must not use uppercase or '_'


>>
>> As this patch is still in the RFC stage, it currently supports **only
>> single thread**. Due to protocol requirements in FUSE-over-io_uring,
>> the number of FUSE threads must match the number of CPUs. Therefore,
>> this initial version only works on single-core VMs (i.e., QEMU started
>> with -smp 1) or single core machine.
>>
>> Brian Song (1):
>>    block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
>>
>>   block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>>   docs/tools/qemu-storage-daemon.rst   |  10 +-
>>   qapi/block-export.json               |   6 +-
>>   storage-daemon/qemu-storage-daemon.c |   1 +
>>   util/fdmon-io_uring.c                |   5 +-
>>   5 files changed, 420 insertions(+), 25 deletions(-)
>>
>> --
>> 2.50.1
>>


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-16 18:38 ` [PATCH RFC 1/1] " Brian Song
                     ` (2 preceding siblings ...)
  2025-07-21 13:51   ` Bernd Schubert
@ 2025-07-22 13:32   ` Kevin Wolf
  3 siblings, 0 replies; 19+ messages in thread
From: Kevin Wolf @ 2025-07-22 13:32 UTC (permalink / raw)
  To: Brian Song
  Cc: qemu-block, qemu-devel, armbru, bschubert, fam, hreitz, stefanha

Am 16.07.2025 um 20:38 hat Brian Song geschrieben:
> This work provides an initial implementation of fuse-over-io_uring
> support for QEMU export. According to the fuse-over-io_uring protocol
> specification, the userspace side must create the same number of queues
> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
> queue contains only a single SQE entry, which is used to validate the
> correctness of the fuse-over-io_uring functionality.
> 
> All FUSE read and write operations interact with the kernel via io
> vectors embedded in the SQE entry during submission and CQE fetching.
> The req_header and op_payload members of each entry are included as
> parts of the io vector: req_header carries the FUSE operation header,
> and op_payload carries the data payload, such as file attributes in a
> getattr reply, file content in a read reply, or file content being
> written to the FUSE client in a write operation.
> 
> At present, multi-threading support is still incomplete. In addition,
> handling connection termination and managing the "drained" state of a
> FUSE block export in QEMU remain as pending work.
> 
> Suggested-by: Kevin Wolf <kwolf@redhat.com>
> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
> Signed-off-by: Brian Song <hibriansong@gmail.com>
> 
> ---
>  block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>  docs/tools/qemu-storage-daemon.rst   |  10 +-
>  qapi/block-export.json               |   6 +-
>  storage-daemon/qemu-storage-daemon.c |   1 +
>  util/fdmon-io_uring.c                |   5 +-
>  5 files changed, 420 insertions(+), 25 deletions(-)

You already got a lot of feedback on details. Let me add a more generic
point that should be addressed for a non-RFC submission: You should try
to limit each commit to a single logical change. You'll then send a
patch series instead of just a single patch where each patch works
incrementally towards the final state.

This makes it not only easier to review the changes because there is
less going on in each individual patch, but it will also be helpful if
we ever have to debug a problem and can bisect to a smaller change, or
even just because looking at the commit message in a few years is
likelier to explain why some specific code was written the way it is.

For example, your change to util/fdmon-io_uring.c could be the first
patch, enabling IORING_SETUP_SQE128 is a self-contained logical change.
You can mention in its commit message that it's in preparation for using
IORING_OP_URING_CMD.

Another patch could be refactoring fuse_co_process_request() so that it
works for both /dev/fuse based and io_uring based exports. (I agree with
Stefan that the code should be shared between both.)

And then you could see if adding io_uring support to the FUSE export
itself can be broken down into multiple self-contained changes or if
that part has to stay a single big patch. Maybe you can keep it similar
to how you're actually developing the code: First a patch with a minimal
implementation that processes one request per queue, then another one to
implement parallel I/O, then one for supporting multiple iothreads.


Another thing I wondered is if the large #ifdef'ed section with io_uring
helpers would actually make more sense as a separate source file
block/export/fuse-io_uring.c that can be linked conditionally if
io_uring is available. This would help to minimise the number of #ifdefs
in fuse.c.

Kevin



^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-21  0:53   ` Stefan Hajnoczi
  2025-07-22 12:00     ` Brian Song
@ 2025-07-22 14:06     ` Bernd Schubert
  2025-07-22 15:43       ` Stefan Hajnoczi
  1 sibling, 1 reply; 19+ messages in thread
From: Bernd Schubert @ 2025-07-22 14:06 UTC (permalink / raw)
  To: Stefan Hajnoczi, Brian Song
  Cc: qemu-block@nongnu.org, qemu-devel@nongnu.org, armbru@redhat.com,
	fam@euphon.net, hreitz@redhat.com, kwolf@redhat.com

On 7/21/25 02:53, Stefan Hajnoczi wrote:
> On Wed, Jul 16, 2025 at 02:38:24PM -0400, Brian Song wrote:
>> This work provides an initial implementation of fuse-over-io_uring
>> support for QEMU export. According to the fuse-over-io_uring protocol
>> specification, the userspace side must create the same number of queues
>> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
>> queue contains only a single SQE entry, which is used to validate the
>> correctness of the fuse-over-io_uring functionality.
>>
>> All FUSE read and write operations interact with the kernel via io
>> vectors embedded in the SQE entry during submission and CQE fetching.
>> The req_header and op_payload members of each entry are included as
>> parts of the io vector: req_header carries the FUSE operation header,
>> and op_payload carries the data payload, such as file attributes in a
>> getattr reply, file content in a read reply, or file content being
>> written to the FUSE client in a write operation.
>>
>> At present, multi-threading support is still incomplete. In addition,
>> handling connection termination and managing the "drained" state of a
>> FUSE block export in QEMU remain as pending work.
>>
>> Suggested-by: Kevin Wolf <kwolf@redhat.com>
>> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
>> Signed-off-by: Brian Song <hibriansong@gmail.com>
>>
>> ---
>>  block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>>  docs/tools/qemu-storage-daemon.rst   |  10 +-
>>  qapi/block-export.json               |   6 +-
>>  storage-daemon/qemu-storage-daemon.c |   1 +
>>  util/fdmon-io_uring.c                |   5 +-
>>  5 files changed, 420 insertions(+), 25 deletions(-)
> 
> Here is feedback from a first pass over this patch.
> 
>>
>> diff --git a/block/export/fuse.c b/block/export/fuse.c
>> index c0ad4696ce..637d36186a 100644
>> --- a/block/export/fuse.c
>> +++ b/block/export/fuse.c
>> @@ -48,6 +48,11 @@
>>  #include <linux/fs.h>
>>  #endif
>>  
>> +#define FUSE_DEFAULT_MAX_PAGES_PER_REQ 32
>> +
>> +/* room needed in buffer to accommodate header */
>> +#define FUSE_BUFFER_HEADER_SIZE 0x1000
>> +
>>  /* Prevent overly long bounce buffer allocations */
>>  #define FUSE_MAX_READ_BYTES (MIN(BDRV_REQUEST_MAX_BYTES, 1 * 1024 * 1024))
>>  /*
>> @@ -64,6 +69,26 @@
>>  
>>  typedef struct FuseExport FuseExport;
>>  
>> +struct FuseQueue;
> 
> Use "typedef struct FuseQueue FuseQueue;" here...
> 
>> +
>> +typedef struct FuseRingEnt {
>> +    /* back pointer */
>> +    struct FuseQueue *q;
> 
> ...and then this can be "FuseQueue *q;" so that QEMU coding style is
> followed.
> 
>> +
>> +    /* commit id of a fuse request */
>> +    uint64_t req_commit_id;
>> +
>> +    /* fuse request header and payload */
>> +    struct fuse_uring_req_header *req_header;
>> +    void *op_payload;
>> +    size_t req_payload_sz;
>> +
>> +    /* The vector passed to the kernel */
>> +    struct iovec iov[2];
>> +
>> +    CqeHandler fuse_cqe_handler;
>> +} FuseRingEnt;
>> +
>>  /*
>>   * One FUSE "queue", representing one FUSE FD from which requests are fetched
>>   * and processed.  Each queue is tied to an AioContext.
>> @@ -73,6 +98,7 @@ typedef struct FuseQueue {
>>  
>>      AioContext *ctx;
>>      int fuse_fd;
>> +    int qid;
> 
> Could this go inside #ifdef CONFIG_LINUX_IO_URING? It seems to be
> specific to FUSE-over-io_uring.
> 
>>  
>>      /*
>>       * The request buffer must be able to hold a full write, and/or at least
>> @@ -109,6 +135,17 @@ typedef struct FuseQueue {
>>       * Free this buffer with qemu_vfree().
>>       */
>>      void *spillover_buf;
>> +
>> +#ifdef CONFIG_LINUX_IO_URING
>> +    FuseRingEnt ent;
>> +
>> +    /*
>> +     * TODO
>> +     * Support multi-threaded FUSE over io_uring by using eventfd and allocating
>> +     * an extra SQE for each thread to be notified when the connection
>> +     * shuts down.
>> +     */
> 
> eventfd and the extra SQE won't be necessary because
> aio_bh_schedule_oneshot() can be used to cause threads to execute a
> function.
> 
> (I think this comment effectively says that connection shutdown still
> needs to be implemented. The implementation details don't matter at this
> point.)
> 
>> +#endif
>>  } FuseQueue;
>>  
>>  /*
>> @@ -148,6 +185,7 @@ struct FuseExport {
>>      bool growable;
>>      /* Whether allow_other was used as a mount option or not */
>>      bool allow_other;
>> +    bool is_uring;
>>  
>>      mode_t st_mode;
>>      uid_t st_uid;
>> @@ -257,6 +295,126 @@ static const BlockDevOps fuse_export_blk_dev_ops = {
>>      .drained_poll  = fuse_export_drained_poll,
>>  };
>>  
>> +#ifdef CONFIG_LINUX_IO_URING
>> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent);
>> +
>> +static void coroutine_fn co_fuse_uring_queue_handle_cqes(void *opaque)
>> +{
>> +    CqeHandler *cqe_handler = opaque;
>> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
> 
> Passing ent in opaque instead of cqe_handler would simplify this.
> 
>> +    FuseExport *exp = ent->q->exp;
>> +
>> +    fuse_uring_co_process_request(ent);
>> +
>> +    fuse_dec_in_flight(exp);
>> +}
>> +
>> +static void fuse_uring_cqe_handler(CqeHandler *cqe_handler)
>> +{
>> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
>> +    FuseQueue *q = ent->q;
>> +    Coroutine *co;
>> +    FuseExport *exp = ent->q->exp;
>> +
>> +    int err = cqe_handler->cqe.res;
>> +    if (err != 0) {
>> +        /* TODO end_conn support */
>> +
>> +        /* -ENOTCONN is ok on umount  */
>> +        if (err != -EINTR && err != -EOPNOTSUPP &&
>> +            err != -EAGAIN && err != -ENOTCONN) {
>> +            fuse_export_halt(exp);
>> +        }
>> +    } else {
>> +        co = qemu_coroutine_create(co_fuse_uring_queue_handle_cqes,
>> +                            cqe_handler);
>> +        /* Decremented by co_fuse_uring_queue_handle_cqes() */
>> +        fuse_inc_in_flight(q->exp);
> 
> Can this be moved inside co_fuse_uring_queue_handle_cqes() to avoid
> calling inc/dec from different functions? That would make the code
> easier to understand and more robust against future bugs.
> 
>> +        qemu_coroutine_enter(co);
>> +    }
>> +}
>> +
>> +static void fuse_uring_sqe_set_req_data(struct fuse_uring_cmd_req *req,
>> +                    const unsigned int qid,
>> +                    const unsigned int commit_id)
>> +{
>> +    req->qid = qid;
>> +    req->commit_id = commit_id;
>> +    req->flags = 0;
>> +}
>> +
>> +static void fuse_uring_sqe_prepare(struct io_uring_sqe *sqe, FuseRingEnt *ent,
>> +               __u32 cmd_op)
>> +{
>> +    sqe->opcode = IORING_OP_URING_CMD;
>> +
>> +    sqe->fd = ent->q->fuse_fd;
>> +    sqe->rw_flags = 0;
>> +    sqe->ioprio = 0;
>> +    sqe->off = 0;
>> +
>> +    sqe->cmd_op = cmd_op;
>> +    sqe->__pad1 = 0;
>> +}
>> +
>> +static void fuse_uring_prep_sqe_register(struct io_uring_sqe *sqe, void *opaque)
>> +{
>> +    FuseQueue *q = opaque;
>> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
>> +
>> +    fuse_uring_sqe_prepare(sqe, &q->ent, FUSE_IO_URING_CMD_REGISTER);
>> +
>> +    sqe->addr = (uint64_t)(q->ent.iov);
>> +    sqe->len = 2;
>> +
>> +    fuse_uring_sqe_set_req_data(req, q->qid, 0);
>> +}
>> +
>> +static void fuse_uring_start(FuseExport *exp, struct fuse_init_out *out)
>> +{
>> +    /*
>> +     * Since we didn't enable the FUSE_MAX_PAGES feature, the value of
>> +     * fc->max_pages should be FUSE_DEFAULT_MAX_PAGES_PER_REQ, which is set by
>> +     * the kernel by default. Also, max_write should not exceed
>> +     * FUSE_DEFAULT_MAX_PAGES_PER_REQ * PAGE_SIZE.
>> +     */
>> +    size_t bufsize = out->max_write + FUSE_BUFFER_HEADER_SIZE;
>> +
>> +    if (!(out->flags & FUSE_MAX_PAGES)) {
>> +        /*
>> +         * bufsize = MIN(FUSE_DEFAULT_MAX_PAGES_PER_REQ *
>> +         *       qemu_real_host_page_size() + FUSE_BUFFER_HEADER_SIZE, bufsize);
>> +         */
>> +        bufsize = FUSE_DEFAULT_MAX_PAGES_PER_REQ * qemu_real_host_page_size()
>> +                         + FUSE_BUFFER_HEADER_SIZE;
>> +    }
>> +
>> +    for (int i = 0; i < exp->num_queues; i++) {
>> +        FuseQueue *q = &exp->queues[i];
>> +
>> +        q->ent.q = q;
>> +
>> +        q->ent.req_header = g_malloc0(sizeof(struct fuse_uring_req_header));
> 
> It's probably easier to embed the header as a FuseRingEnt field instead
> of heap allocating it.

Hmm well. So we have two additional patch in the DDN branch for which I 
didn't have time to upstream them yet. These patches allow to pin these
buffers/pages and with that the application doing IO can directly write
into the buffer - saves context swithes. The initial RFC kernel patches
were using mmaped buffers and when I had to switch to userspace buffers,
performance went badly down. I didn't run real benchmarks, but just
xfstests  - with mmapped buffers it was running like 3 times faster than
legacy fuse and that advantage got lost with normal buffers that
get mapped per request. Switching to pinned buffers brought back the
fast xfstest runs.
Issue is that the buffer needs to be page aligned - which is why libfuse
takes an extra allocation here.
In libfuse I should probably make this optional, as pinned buffers
will mostly only work for root (needs locked memory).

In principle I would need to document these details somewhere, I should
probably create blog or so.


Cheers,
Bernd







^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 0/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-22 12:00   ` Brian Song
@ 2025-07-22 14:47     ` Stefan Hajnoczi
  0 siblings, 0 replies; 19+ messages in thread
From: Stefan Hajnoczi @ 2025-07-22 14:47 UTC (permalink / raw)
  To: Brian Song; +Cc: qemu-block, qemu-devel, armbru, bschubert, fam, hreitz, kwolf

[-- Attachment #1: Type: text/plain, Size: 2762 bytes --]

On Tue, Jul 22, 2025 at 08:00:00AM -0400, Brian Song wrote:
> On 7/20/25 12:13 PM, Stefan Hajnoczi wrote:
> > On Wed, Jul 16, 2025 at 02:38:23PM -0400, Brian Song wrote:
> >> This RFC patch represents an initial implementation of the FUSE-over-
> >> io_uring Exports idea proposed for Google Summer of Code (2025) under
> >> the QEMU community:
> >> https://wiki.qemu.org/Google_Summer_of_Code_2025#FUSE-over-io_uring_exports
> >>
> >> The implementation approach is primarily borrowed from how libfuse
> >> interacts with the kernel.
> >>
> >> FUSE-over-io_uring (https://docs.kernel.org/next/filesystems/fuse-io-
> >> uring.html) has been officially merged into the Linux kernel. The idea
> >> is to replace the traditional /dev/fuse based communication with a more
> >> efficient io_uring-based approach. In this model, userspace registers
> >> io_uring SQEs via the FUSE_IO_URING_CMD_REGISTER opcode, and then waits
> >> for the kernel to forward FUSE requests as CQEs. These are processed by
> >> the FUSE exports implementation in userspace and then committed back to
> >> the kernel using FUSE_IO_URING_CMD_COMMIT_AND_FETCH.
> >>
> >> To enable this feature in qemu-export-daemon, simply add the uring=on
> >> option to the export configuration.
> >
> > For consistency with the --blockdev file,aio=io_uring option, please
> > name it "io_uring=on" instead of "uring=on".
> >
> 
> It seems we are not allowed to use '_' here.
> 
> In file included from ../storage-daemon/qapi/qapi-schema.json:57:
> 
> ../storage-daemon/qapi/../../qapi/block-export.json: In struct
> 'BlockExportOptionsFuse':
> 
> ../storage-daemon/qapi/../../qapi/block-export.json:191: name of 'data'
> member '*io_uring' must not use uppercase or '_'

Then "-" is fine, but please use "io<separator>uring" so it is as close
as possible (not "uring", "iouring", etc).

Thanks,
Stefan

> 
> 
> >>
> >> As this patch is still in the RFC stage, it currently supports **only
> >> single thread**. Due to protocol requirements in FUSE-over-io_uring,
> >> the number of FUSE threads must match the number of CPUs. Therefore,
> >> this initial version only works on single-core VMs (i.e., QEMU started
> >> with -smp 1) or single core machine.
> >>
> >> Brian Song (1):
> >>    block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
> >>
> >>   block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
> >>   docs/tools/qemu-storage-daemon.rst   |  10 +-
> >>   qapi/block-export.json               |   6 +-
> >>   storage-daemon/qemu-storage-daemon.c |   1 +
> >>   util/fdmon-io_uring.c                |   5 +-
> >>   5 files changed, 420 insertions(+), 25 deletions(-)
> >>
> >> --
> >> 2.50.1
> >>
> 

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 488 bytes --]

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-22 12:00     ` Brian Song
@ 2025-07-22 14:51       ` Stefan Hajnoczi
  0 siblings, 0 replies; 19+ messages in thread
From: Stefan Hajnoczi @ 2025-07-22 14:51 UTC (permalink / raw)
  To: Brian Song
  Cc: Bernd Schubert, qemu-block@nongnu.org, qemu-devel@nongnu.org,
	armbru@redhat.com, fam@euphon.net, hreitz@redhat.com,
	kwolf@redhat.com

[-- Attachment #1: Type: text/plain, Size: 1273 bytes --]

On Tue, Jul 22, 2025 at 08:00:00AM -0400, Brian Song wrote:
> On 7/21/25 9:51 AM, Bernd Schubert wrote:
> >> +    struct fuse_in_header *in_hdr = (struct fuse_in_header *)&rrh->in_out;
> >> +    opcode = in_hdr->opcode;
> >> +    req_id = in_hdr->unique;
> >> +
> >> +    ent->req_commit_id = ent_in_out->commit_id;
> >> +
> >> +    if (unlikely(ent->req_commit_id == 0)) {
> >> +        /*
> >> +         * If this happens kernel will not find the response - it will
> >> +         * be stuck forever - better to abort immediately.
> >> +         */
> >> +        error_report("If this happens kernel will not find the response"
> >> +        " - it will be stuck forever - better to abort immediately.");
> >
> > Dunno about qemu style, but isn't it better to keep the string in a single
> > line to make it greppable?
> >
> 
> The actual log output doesn’t include any line breaks. I just broke the
> line in the code to keep it under 80 characters per line.

QEMU coding style is stricter about 80 characters per line. Linux allows
long error message strings on a line so it is easier to grep for them in
the source code.

I think it's just a difference in coding style and splitting strings is
generally how QEMU code does it.

Stefan

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 488 bytes --]

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-22 12:00     ` Brian Song
@ 2025-07-22 15:17       ` Kevin Wolf
  0 siblings, 0 replies; 19+ messages in thread
From: Kevin Wolf @ 2025-07-22 15:17 UTC (permalink / raw)
  To: Brian Song
  Cc: Stefan Hajnoczi, qemu-block, qemu-devel, armbru, bschubert, fam,
	hreitz

Am 22.07.2025 um 14:00 hat Brian Song geschrieben:
> On 7/20/25 8:53 PM, Stefan Hajnoczi wrote:
> > On Wed, Jul 16, 2025 at 02:38:24PM -0400, Brian Song wrote:
> >> +    case FUSE_FSYNC:
> >> +        ret = fuse_co_fsync(exp);
> >> +        break;
> >> +
> >> +    case FUSE_FLUSH:
> >> +        ret = fuse_co_flush(exp);
> >> +        break;
> >> +
> >> +#ifdef CONFIG_FUSE_LSEEK
> >> +    case FUSE_LSEEK: {
> >> +        const struct fuse_lseek_in *in =
> >> +                        (const struct fuse_lseek_in *)&rrh->op_in;
> >> +        ret = fuse_co_lseek(exp, (struct fuse_lseek_out *)out_op_hdr,
> >> +                            in->offset, in->whence);
> >> +        break;
> >> +    }
> >> +#endif
> >> +
> >> +    default:
> >> +        ret = -ENOSYS;
> >> +    }
> >
> > It would be nice to reuse the non-io_uring code rather than duplicating
> > the switch statement that covers each FUSE opcode. Is the memory layout
> > so different that the code cannot be shared?
> 
> Yes. But I think the main issue is that we have to handle too many
> differences when it comes to various operations and the final step of
> replying to the request. There would be a lot of #ifdef
> CONFIG_LINUX_IO_URING and if statements. So, for simplicity, I made it a
> separate function.
> 
> In the traditional model:
> q->request_buf = fuse_in_header + struct fuse_opsxx_in +
> FUSE_IN_PLACE_WRITE_BYTES (used for part of the payload data)
> q->spillover_buf is used for the rest of the payload data.
> 
> In Fuse-over-io_uring:
> FuseRingEnt contains req_header (which includes in_out for
> fuse_in/out_header, and op_in for struct fuse_opsxx_in)
> op_payload corresponds to the final FUSE_IN_PLACE_WRITE_BYTES bytes from
> request_buf + spillover_buf in the traditional model but carries either
> the out operation headers (fuse_ops_out) or the file data to be written
> or read.

The individual opcode handlers you call from both functions are already
generic enough to work for both cases, e.g. fuse_co_write() takes both
an in_place_buf and spillover_buf. In the io_uring case you just pass
NULL for in_place_buf.

Why doesn't the same approach work for fuse_co_process_request()? Can't
you just pass three pointers for header, in_place_buf and spillover_buf
to it and have two very small wrappers around it that take care of
passing the right pointers for the respective case?

Kevin



^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-22 14:06     ` Bernd Schubert
@ 2025-07-22 15:43       ` Stefan Hajnoczi
  2025-07-22 16:20         ` Bernd Schubert
  0 siblings, 1 reply; 19+ messages in thread
From: Stefan Hajnoczi @ 2025-07-22 15:43 UTC (permalink / raw)
  To: Bernd Schubert
  Cc: Brian Song, qemu-block@nongnu.org, qemu-devel@nongnu.org,
	armbru@redhat.com, fam@euphon.net, hreitz@redhat.com,
	kwolf@redhat.com

[-- Attachment #1: Type: text/plain, Size: 10748 bytes --]

On Tue, Jul 22, 2025 at 02:06:04PM +0000, Bernd Schubert wrote:
> On 7/21/25 02:53, Stefan Hajnoczi wrote:
> > On Wed, Jul 16, 2025 at 02:38:24PM -0400, Brian Song wrote:
> >> This work provides an initial implementation of fuse-over-io_uring
> >> support for QEMU export. According to the fuse-over-io_uring protocol
> >> specification, the userspace side must create the same number of queues
> >> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
> >> queue contains only a single SQE entry, which is used to validate the
> >> correctness of the fuse-over-io_uring functionality.
> >>
> >> All FUSE read and write operations interact with the kernel via io
> >> vectors embedded in the SQE entry during submission and CQE fetching.
> >> The req_header and op_payload members of each entry are included as
> >> parts of the io vector: req_header carries the FUSE operation header,
> >> and op_payload carries the data payload, such as file attributes in a
> >> getattr reply, file content in a read reply, or file content being
> >> written to the FUSE client in a write operation.
> >>
> >> At present, multi-threading support is still incomplete. In addition,
> >> handling connection termination and managing the "drained" state of a
> >> FUSE block export in QEMU remain as pending work.
> >>
> >> Suggested-by: Kevin Wolf <kwolf@redhat.com>
> >> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
> >> Signed-off-by: Brian Song <hibriansong@gmail.com>
> >>
> >> ---
> >>  block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
> >>  docs/tools/qemu-storage-daemon.rst   |  10 +-
> >>  qapi/block-export.json               |   6 +-
> >>  storage-daemon/qemu-storage-daemon.c |   1 +
> >>  util/fdmon-io_uring.c                |   5 +-
> >>  5 files changed, 420 insertions(+), 25 deletions(-)
> > 
> > Here is feedback from a first pass over this patch.
> > 
> >>
> >> diff --git a/block/export/fuse.c b/block/export/fuse.c
> >> index c0ad4696ce..637d36186a 100644
> >> --- a/block/export/fuse.c
> >> +++ b/block/export/fuse.c
> >> @@ -48,6 +48,11 @@
> >>  #include <linux/fs.h>
> >>  #endif
> >>  
> >> +#define FUSE_DEFAULT_MAX_PAGES_PER_REQ 32
> >> +
> >> +/* room needed in buffer to accommodate header */
> >> +#define FUSE_BUFFER_HEADER_SIZE 0x1000
> >> +
> >>  /* Prevent overly long bounce buffer allocations */
> >>  #define FUSE_MAX_READ_BYTES (MIN(BDRV_REQUEST_MAX_BYTES, 1 * 1024 * 1024))
> >>  /*
> >> @@ -64,6 +69,26 @@
> >>  
> >>  typedef struct FuseExport FuseExport;
> >>  
> >> +struct FuseQueue;
> > 
> > Use "typedef struct FuseQueue FuseQueue;" here...
> > 
> >> +
> >> +typedef struct FuseRingEnt {
> >> +    /* back pointer */
> >> +    struct FuseQueue *q;
> > 
> > ...and then this can be "FuseQueue *q;" so that QEMU coding style is
> > followed.
> > 
> >> +
> >> +    /* commit id of a fuse request */
> >> +    uint64_t req_commit_id;
> >> +
> >> +    /* fuse request header and payload */
> >> +    struct fuse_uring_req_header *req_header;
> >> +    void *op_payload;
> >> +    size_t req_payload_sz;
> >> +
> >> +    /* The vector passed to the kernel */
> >> +    struct iovec iov[2];
> >> +
> >> +    CqeHandler fuse_cqe_handler;
> >> +} FuseRingEnt;
> >> +
> >>  /*
> >>   * One FUSE "queue", representing one FUSE FD from which requests are fetched
> >>   * and processed.  Each queue is tied to an AioContext.
> >> @@ -73,6 +98,7 @@ typedef struct FuseQueue {
> >>  
> >>      AioContext *ctx;
> >>      int fuse_fd;
> >> +    int qid;
> > 
> > Could this go inside #ifdef CONFIG_LINUX_IO_URING? It seems to be
> > specific to FUSE-over-io_uring.
> > 
> >>  
> >>      /*
> >>       * The request buffer must be able to hold a full write, and/or at least
> >> @@ -109,6 +135,17 @@ typedef struct FuseQueue {
> >>       * Free this buffer with qemu_vfree().
> >>       */
> >>      void *spillover_buf;
> >> +
> >> +#ifdef CONFIG_LINUX_IO_URING
> >> +    FuseRingEnt ent;
> >> +
> >> +    /*
> >> +     * TODO
> >> +     * Support multi-threaded FUSE over io_uring by using eventfd and allocating
> >> +     * an extra SQE for each thread to be notified when the connection
> >> +     * shuts down.
> >> +     */
> > 
> > eventfd and the extra SQE won't be necessary because
> > aio_bh_schedule_oneshot() can be used to cause threads to execute a
> > function.
> > 
> > (I think this comment effectively says that connection shutdown still
> > needs to be implemented. The implementation details don't matter at this
> > point.)
> > 
> >> +#endif
> >>  } FuseQueue;
> >>  
> >>  /*
> >> @@ -148,6 +185,7 @@ struct FuseExport {
> >>      bool growable;
> >>      /* Whether allow_other was used as a mount option or not */
> >>      bool allow_other;
> >> +    bool is_uring;
> >>  
> >>      mode_t st_mode;
> >>      uid_t st_uid;
> >> @@ -257,6 +295,126 @@ static const BlockDevOps fuse_export_blk_dev_ops = {
> >>      .drained_poll  = fuse_export_drained_poll,
> >>  };
> >>  
> >> +#ifdef CONFIG_LINUX_IO_URING
> >> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent);
> >> +
> >> +static void coroutine_fn co_fuse_uring_queue_handle_cqes(void *opaque)
> >> +{
> >> +    CqeHandler *cqe_handler = opaque;
> >> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
> > 
> > Passing ent in opaque instead of cqe_handler would simplify this.
> > 
> >> +    FuseExport *exp = ent->q->exp;
> >> +
> >> +    fuse_uring_co_process_request(ent);
> >> +
> >> +    fuse_dec_in_flight(exp);
> >> +}
> >> +
> >> +static void fuse_uring_cqe_handler(CqeHandler *cqe_handler)
> >> +{
> >> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
> >> +    FuseQueue *q = ent->q;
> >> +    Coroutine *co;
> >> +    FuseExport *exp = ent->q->exp;
> >> +
> >> +    int err = cqe_handler->cqe.res;
> >> +    if (err != 0) {
> >> +        /* TODO end_conn support */
> >> +
> >> +        /* -ENOTCONN is ok on umount  */
> >> +        if (err != -EINTR && err != -EOPNOTSUPP &&
> >> +            err != -EAGAIN && err != -ENOTCONN) {
> >> +            fuse_export_halt(exp);
> >> +        }
> >> +    } else {
> >> +        co = qemu_coroutine_create(co_fuse_uring_queue_handle_cqes,
> >> +                            cqe_handler);
> >> +        /* Decremented by co_fuse_uring_queue_handle_cqes() */
> >> +        fuse_inc_in_flight(q->exp);
> > 
> > Can this be moved inside co_fuse_uring_queue_handle_cqes() to avoid
> > calling inc/dec from different functions? That would make the code
> > easier to understand and more robust against future bugs.
> > 
> >> +        qemu_coroutine_enter(co);
> >> +    }
> >> +}
> >> +
> >> +static void fuse_uring_sqe_set_req_data(struct fuse_uring_cmd_req *req,
> >> +                    const unsigned int qid,
> >> +                    const unsigned int commit_id)
> >> +{
> >> +    req->qid = qid;
> >> +    req->commit_id = commit_id;
> >> +    req->flags = 0;
> >> +}
> >> +
> >> +static void fuse_uring_sqe_prepare(struct io_uring_sqe *sqe, FuseRingEnt *ent,
> >> +               __u32 cmd_op)
> >> +{
> >> +    sqe->opcode = IORING_OP_URING_CMD;
> >> +
> >> +    sqe->fd = ent->q->fuse_fd;
> >> +    sqe->rw_flags = 0;
> >> +    sqe->ioprio = 0;
> >> +    sqe->off = 0;
> >> +
> >> +    sqe->cmd_op = cmd_op;
> >> +    sqe->__pad1 = 0;
> >> +}
> >> +
> >> +static void fuse_uring_prep_sqe_register(struct io_uring_sqe *sqe, void *opaque)
> >> +{
> >> +    FuseQueue *q = opaque;
> >> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
> >> +
> >> +    fuse_uring_sqe_prepare(sqe, &q->ent, FUSE_IO_URING_CMD_REGISTER);
> >> +
> >> +    sqe->addr = (uint64_t)(q->ent.iov);
> >> +    sqe->len = 2;
> >> +
> >> +    fuse_uring_sqe_set_req_data(req, q->qid, 0);
> >> +}
> >> +
> >> +static void fuse_uring_start(FuseExport *exp, struct fuse_init_out *out)
> >> +{
> >> +    /*
> >> +     * Since we didn't enable the FUSE_MAX_PAGES feature, the value of
> >> +     * fc->max_pages should be FUSE_DEFAULT_MAX_PAGES_PER_REQ, which is set by
> >> +     * the kernel by default. Also, max_write should not exceed
> >> +     * FUSE_DEFAULT_MAX_PAGES_PER_REQ * PAGE_SIZE.
> >> +     */
> >> +    size_t bufsize = out->max_write + FUSE_BUFFER_HEADER_SIZE;
> >> +
> >> +    if (!(out->flags & FUSE_MAX_PAGES)) {
> >> +        /*
> >> +         * bufsize = MIN(FUSE_DEFAULT_MAX_PAGES_PER_REQ *
> >> +         *       qemu_real_host_page_size() + FUSE_BUFFER_HEADER_SIZE, bufsize);
> >> +         */
> >> +        bufsize = FUSE_DEFAULT_MAX_PAGES_PER_REQ * qemu_real_host_page_size()
> >> +                         + FUSE_BUFFER_HEADER_SIZE;
> >> +    }
> >> +
> >> +    for (int i = 0; i < exp->num_queues; i++) {
> >> +        FuseQueue *q = &exp->queues[i];
> >> +
> >> +        q->ent.q = q;
> >> +
> >> +        q->ent.req_header = g_malloc0(sizeof(struct fuse_uring_req_header));
> > 
> > It's probably easier to embed the header as a FuseRingEnt field instead
> > of heap allocating it.
> 
> Hmm well. So we have two additional patch in the DDN branch for which I 
> didn't have time to upstream them yet. These patches allow to pin these
> buffers/pages and with that the application doing IO can directly write
> into the buffer - saves context swithes. The initial RFC kernel patches
> were using mmaped buffers and when I had to switch to userspace buffers,
> performance went badly down. I didn't run real benchmarks, but just
> xfstests  - with mmapped buffers it was running like 3 times faster than
> legacy fuse and that advantage got lost with normal buffers that
> get mapped per request. Switching to pinned buffers brought back the
> fast xfstest runs.
> Issue is that the buffer needs to be page aligned - which is why libfuse
> takes an extra allocation here.
> In libfuse I should probably make this optional, as pinned buffers
> will mostly only work for root (needs locked memory).
> 
> In principle I would need to document these details somewhere, I should
> probably create blog or so.

That sounds like something we'd like to try out, although we try to
minimize privileges needed for qemu-storage-daemon.

A question about pinning and resource limits: qemu-storage-daemon uses
O_DIRECT for I/O as non-root without hitting mlock resource limits. Is
there a difference between pin_user_pages(FOLL_PIN) (for direct I/O) and
pin_user_pages(FOLL_PIN | FOLL_LONGTERM) (possibly for
FUSE-over-io_uring pinning) in terms of resource limit behavior?

Thanks,
Stefan

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 488 bytes --]

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-22 15:43       ` Stefan Hajnoczi
@ 2025-07-22 16:20         ` Bernd Schubert
  0 siblings, 0 replies; 19+ messages in thread
From: Bernd Schubert @ 2025-07-22 16:20 UTC (permalink / raw)
  To: Stefan Hajnoczi
  Cc: Brian Song, qemu-block@nongnu.org, qemu-devel@nongnu.org,
	armbru@redhat.com, fam@euphon.net, hreitz@redhat.com,
	kwolf@redhat.com



On 7/22/25 17:43, Stefan Hajnoczi wrote:
> On Tue, Jul 22, 2025 at 02:06:04PM +0000, Bernd Schubert wrote:
>> On 7/21/25 02:53, Stefan Hajnoczi wrote:
>>> On Wed, Jul 16, 2025 at 02:38:24PM -0400, Brian Song wrote:
>>>> This work provides an initial implementation of fuse-over-io_uring
>>>> support for QEMU export. According to the fuse-over-io_uring protocol
>>>> specification, the userspace side must create the same number of queues
>>>> as the number of CPUs (nr_cpu), just like the kernel. Currently, each
>>>> queue contains only a single SQE entry, which is used to validate the
>>>> correctness of the fuse-over-io_uring functionality.
>>>>
>>>> All FUSE read and write operations interact with the kernel via io
>>>> vectors embedded in the SQE entry during submission and CQE fetching.
>>>> The req_header and op_payload members of each entry are included as
>>>> parts of the io vector: req_header carries the FUSE operation header,
>>>> and op_payload carries the data payload, such as file attributes in a
>>>> getattr reply, file content in a read reply, or file content being
>>>> written to the FUSE client in a write operation.
>>>>
>>>> At present, multi-threading support is still incomplete. In addition,
>>>> handling connection termination and managing the "drained" state of a
>>>> FUSE block export in QEMU remain as pending work.
>>>>
>>>> Suggested-by: Kevin Wolf <kwolf@redhat.com>
>>>> Suggested-by: Stefan Hajnoczi <stefanha@redhat.com>
>>>> Signed-off-by: Brian Song <hibriansong@gmail.com>
>>>>
>>>> ---
>>>>  block/export/fuse.c                  | 423 +++++++++++++++++++++++++--
>>>>  docs/tools/qemu-storage-daemon.rst   |  10 +-
>>>>  qapi/block-export.json               |   6 +-
>>>>  storage-daemon/qemu-storage-daemon.c |   1 +
>>>>  util/fdmon-io_uring.c                |   5 +-
>>>>  5 files changed, 420 insertions(+), 25 deletions(-)
>>>
>>> Here is feedback from a first pass over this patch.
>>>
>>>>
>>>> diff --git a/block/export/fuse.c b/block/export/fuse.c
>>>> index c0ad4696ce..637d36186a 100644
>>>> --- a/block/export/fuse.c
>>>> +++ b/block/export/fuse.c
>>>> @@ -48,6 +48,11 @@
>>>>  #include <linux/fs.h>
>>>>  #endif
>>>>  
>>>> +#define FUSE_DEFAULT_MAX_PAGES_PER_REQ 32
>>>> +
>>>> +/* room needed in buffer to accommodate header */
>>>> +#define FUSE_BUFFER_HEADER_SIZE 0x1000
>>>> +
>>>>  /* Prevent overly long bounce buffer allocations */
>>>>  #define FUSE_MAX_READ_BYTES (MIN(BDRV_REQUEST_MAX_BYTES, 1 * 1024 * 1024))
>>>>  /*
>>>> @@ -64,6 +69,26 @@
>>>>  
>>>>  typedef struct FuseExport FuseExport;
>>>>  
>>>> +struct FuseQueue;
>>>
>>> Use "typedef struct FuseQueue FuseQueue;" here...
>>>
>>>> +
>>>> +typedef struct FuseRingEnt {
>>>> +    /* back pointer */
>>>> +    struct FuseQueue *q;
>>>
>>> ...and then this can be "FuseQueue *q;" so that QEMU coding style is
>>> followed.
>>>
>>>> +
>>>> +    /* commit id of a fuse request */
>>>> +    uint64_t req_commit_id;
>>>> +
>>>> +    /* fuse request header and payload */
>>>> +    struct fuse_uring_req_header *req_header;
>>>> +    void *op_payload;
>>>> +    size_t req_payload_sz;
>>>> +
>>>> +    /* The vector passed to the kernel */
>>>> +    struct iovec iov[2];
>>>> +
>>>> +    CqeHandler fuse_cqe_handler;
>>>> +} FuseRingEnt;
>>>> +
>>>>  /*
>>>>   * One FUSE "queue", representing one FUSE FD from which requests are fetched
>>>>   * and processed.  Each queue is tied to an AioContext.
>>>> @@ -73,6 +98,7 @@ typedef struct FuseQueue {
>>>>  
>>>>      AioContext *ctx;
>>>>      int fuse_fd;
>>>> +    int qid;
>>>
>>> Could this go inside #ifdef CONFIG_LINUX_IO_URING? It seems to be
>>> specific to FUSE-over-io_uring.
>>>
>>>>  
>>>>      /*
>>>>       * The request buffer must be able to hold a full write, and/or at least
>>>> @@ -109,6 +135,17 @@ typedef struct FuseQueue {
>>>>       * Free this buffer with qemu_vfree().
>>>>       */
>>>>      void *spillover_buf;
>>>> +
>>>> +#ifdef CONFIG_LINUX_IO_URING
>>>> +    FuseRingEnt ent;
>>>> +
>>>> +    /*
>>>> +     * TODO
>>>> +     * Support multi-threaded FUSE over io_uring by using eventfd and allocating
>>>> +     * an extra SQE for each thread to be notified when the connection
>>>> +     * shuts down.
>>>> +     */
>>>
>>> eventfd and the extra SQE won't be necessary because
>>> aio_bh_schedule_oneshot() can be used to cause threads to execute a
>>> function.
>>>
>>> (I think this comment effectively says that connection shutdown still
>>> needs to be implemented. The implementation details don't matter at this
>>> point.)
>>>
>>>> +#endif
>>>>  } FuseQueue;
>>>>  
>>>>  /*
>>>> @@ -148,6 +185,7 @@ struct FuseExport {
>>>>      bool growable;
>>>>      /* Whether allow_other was used as a mount option or not */
>>>>      bool allow_other;
>>>> +    bool is_uring;
>>>>  
>>>>      mode_t st_mode;
>>>>      uid_t st_uid;
>>>> @@ -257,6 +295,126 @@ static const BlockDevOps fuse_export_blk_dev_ops = {
>>>>      .drained_poll  = fuse_export_drained_poll,
>>>>  };
>>>>  
>>>> +#ifdef CONFIG_LINUX_IO_URING
>>>> +static void coroutine_fn fuse_uring_co_process_request(FuseRingEnt *ent);
>>>> +
>>>> +static void coroutine_fn co_fuse_uring_queue_handle_cqes(void *opaque)
>>>> +{
>>>> +    CqeHandler *cqe_handler = opaque;
>>>> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
>>>
>>> Passing ent in opaque instead of cqe_handler would simplify this.
>>>
>>>> +    FuseExport *exp = ent->q->exp;
>>>> +
>>>> +    fuse_uring_co_process_request(ent);
>>>> +
>>>> +    fuse_dec_in_flight(exp);
>>>> +}
>>>> +
>>>> +static void fuse_uring_cqe_handler(CqeHandler *cqe_handler)
>>>> +{
>>>> +    FuseRingEnt *ent = container_of(cqe_handler, FuseRingEnt, fuse_cqe_handler);
>>>> +    FuseQueue *q = ent->q;
>>>> +    Coroutine *co;
>>>> +    FuseExport *exp = ent->q->exp;
>>>> +
>>>> +    int err = cqe_handler->cqe.res;
>>>> +    if (err != 0) {
>>>> +        /* TODO end_conn support */
>>>> +
>>>> +        /* -ENOTCONN is ok on umount  */
>>>> +        if (err != -EINTR && err != -EOPNOTSUPP &&
>>>> +            err != -EAGAIN && err != -ENOTCONN) {
>>>> +            fuse_export_halt(exp);
>>>> +        }
>>>> +    } else {
>>>> +        co = qemu_coroutine_create(co_fuse_uring_queue_handle_cqes,
>>>> +                            cqe_handler);
>>>> +        /* Decremented by co_fuse_uring_queue_handle_cqes() */
>>>> +        fuse_inc_in_flight(q->exp);
>>>
>>> Can this be moved inside co_fuse_uring_queue_handle_cqes() to avoid
>>> calling inc/dec from different functions? That would make the code
>>> easier to understand and more robust against future bugs.
>>>
>>>> +        qemu_coroutine_enter(co);
>>>> +    }
>>>> +}
>>>> +
>>>> +static void fuse_uring_sqe_set_req_data(struct fuse_uring_cmd_req *req,
>>>> +                    const unsigned int qid,
>>>> +                    const unsigned int commit_id)
>>>> +{
>>>> +    req->qid = qid;
>>>> +    req->commit_id = commit_id;
>>>> +    req->flags = 0;
>>>> +}
>>>> +
>>>> +static void fuse_uring_sqe_prepare(struct io_uring_sqe *sqe, FuseRingEnt *ent,
>>>> +               __u32 cmd_op)
>>>> +{
>>>> +    sqe->opcode = IORING_OP_URING_CMD;
>>>> +
>>>> +    sqe->fd = ent->q->fuse_fd;
>>>> +    sqe->rw_flags = 0;
>>>> +    sqe->ioprio = 0;
>>>> +    sqe->off = 0;
>>>> +
>>>> +    sqe->cmd_op = cmd_op;
>>>> +    sqe->__pad1 = 0;
>>>> +}
>>>> +
>>>> +static void fuse_uring_prep_sqe_register(struct io_uring_sqe *sqe, void *opaque)
>>>> +{
>>>> +    FuseQueue *q = opaque;
>>>> +    struct fuse_uring_cmd_req *req = (void *)&sqe->cmd[0];
>>>> +
>>>> +    fuse_uring_sqe_prepare(sqe, &q->ent, FUSE_IO_URING_CMD_REGISTER);
>>>> +
>>>> +    sqe->addr = (uint64_t)(q->ent.iov);
>>>> +    sqe->len = 2;
>>>> +
>>>> +    fuse_uring_sqe_set_req_data(req, q->qid, 0);
>>>> +}
>>>> +
>>>> +static void fuse_uring_start(FuseExport *exp, struct fuse_init_out *out)
>>>> +{
>>>> +    /*
>>>> +     * Since we didn't enable the FUSE_MAX_PAGES feature, the value of
>>>> +     * fc->max_pages should be FUSE_DEFAULT_MAX_PAGES_PER_REQ, which is set by
>>>> +     * the kernel by default. Also, max_write should not exceed
>>>> +     * FUSE_DEFAULT_MAX_PAGES_PER_REQ * PAGE_SIZE.
>>>> +     */
>>>> +    size_t bufsize = out->max_write + FUSE_BUFFER_HEADER_SIZE;
>>>> +
>>>> +    if (!(out->flags & FUSE_MAX_PAGES)) {
>>>> +        /*
>>>> +         * bufsize = MIN(FUSE_DEFAULT_MAX_PAGES_PER_REQ *
>>>> +         *       qemu_real_host_page_size() + FUSE_BUFFER_HEADER_SIZE, bufsize);
>>>> +         */
>>>> +        bufsize = FUSE_DEFAULT_MAX_PAGES_PER_REQ * qemu_real_host_page_size()
>>>> +                         + FUSE_BUFFER_HEADER_SIZE;
>>>> +    }
>>>> +
>>>> +    for (int i = 0; i < exp->num_queues; i++) {
>>>> +        FuseQueue *q = &exp->queues[i];
>>>> +
>>>> +        q->ent.q = q;
>>>> +
>>>> +        q->ent.req_header = g_malloc0(sizeof(struct fuse_uring_req_header));
>>>
>>> It's probably easier to embed the header as a FuseRingEnt field instead
>>> of heap allocating it.
>>
>> Hmm well. So we have two additional patch in the DDN branch for which I 
>> didn't have time to upstream them yet. These patches allow to pin these
>> buffers/pages and with that the application doing IO can directly write
>> into the buffer - saves context swithes. The initial RFC kernel patches
>> were using mmaped buffers and when I had to switch to userspace buffers,
>> performance went badly down. I didn't run real benchmarks, but just
>> xfstests  - with mmapped buffers it was running like 3 times faster than
>> legacy fuse and that advantage got lost with normal buffers that
>> get mapped per request. Switching to pinned buffers brought back the
>> fast xfstest runs.
>> Issue is that the buffer needs to be page aligned - which is why libfuse
>> takes an extra allocation here.
>> In libfuse I should probably make this optional, as pinned buffers
>> will mostly only work for root (needs locked memory).
>>
>> In principle I would need to document these details somewhere, I should
>> probably create blog or so.
> 
> That sounds like something we'd like to try out, although we try to
> minimize privileges needed for qemu-storage-daemon.
> 
> A question about pinning and resource limits: qemu-storage-daemon uses
> O_DIRECT for I/O as non-root without hitting mlock resource limits. Is
> there a difference between pin_user_pages(FOLL_PIN) (for direct I/O) and
> pin_user_pages(FOLL_PIN | FOLL_LONGTERM) (possibly for
> FUSE-over-io_uring pinning) in terms of resource limit behavior?

And O_DIRECT read/write is just pinned during the time of the request and
and from the application doing the request. Pinned fuse req buffers 
(header and payload) are pinned for the live time of fuse-server.
Right now that is (nr-cores * queue-depth * IO-size) + 4K. Patches to be able
to reduce the number queues will go out today.

Btw, going to be on vacation from tomorrow on.



^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [PATCH RFC 1/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports
  2025-07-21 13:51   ` Bernd Schubert
  2025-07-21 18:26     ` Stefan Hajnoczi
  2025-07-22 12:00     ` Brian Song
@ 2025-07-24 20:36     ` Stefan Hajnoczi
  2 siblings, 0 replies; 19+ messages in thread
From: Stefan Hajnoczi @ 2025-07-24 20:36 UTC (permalink / raw)
  To: Bernd Schubert
  Cc: Brian Song, qemu-block@nongnu.org, qemu-devel@nongnu.org,
	armbru@redhat.com, fam@euphon.net, hreitz@redhat.com,
	kwolf@redhat.com

[-- Attachment #1: Type: text/plain, Size: 2028 bytes --]

On Mon, Jul 21, 2025 at 01:51:44PM +0000, Bernd Schubert wrote:
> On 7/16/25 20:38, Brian Song wrote:
> > -    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES, &ctx->fdmon_io_uring, 0);
> > +    ret = io_uring_queue_init(FDMON_IO_URING_ENTRIES,
> > +                            &ctx->fdmon_io_uring, flags);
> >      if (ret != 0) {
> >          error_setg_errno(errp, -ret, "Failed to initialize io_uring");
> >          return;
> 
> I need test these flags again, might be worthful for qemu as well
> 
> 	/* These flags should help to increase performance, but actually
> 	 * make it a bit slower - reason should get investigated.
> 	 */
> 	if (0) {
> 		/* Has the main slow down effect */
> 		params.flags |= IORING_SETUP_SINGLE_ISSUER;
> 
> 		// params.flags |= IORING_SETUP_DEFER_TASKRUN;
> 		params.flags |= IORING_SETUP_TASKRUN_FLAG;
> 
> 		/* Second main effect to make it slower */
> 		params.flags |= IORING_SETUP_COOP_TASKRUN;
> 	}

I measured the following using fio bs=4k in a 4 vCPU guest with 1
IOThread using a virtio-blk disk backed by a local NVMe drive:

                      IOPS               IOPS
Benchmark             SINGLE_ISSUER      SINGLE_ISSUER|COOP_TASKRUN|TASKRUN_FLAG
randread  iodepth=1   54,045 (+1.2%)     54,189 (+1.5%)
randread  iodepth=64  318,135 (+0.1%)    315,632 (-0.68%)
randwrite iodepth=1   141,918 (-0.44%)   143,337 (+0.55%)
randwrite iodepth=64  323,948 (-0.015%)  322,755 (-0.38%)

The differences are so small that I'd rather not merge the code changes.
I will post an RFC series just to archive the patches in case they are
needed again in the future.

You can find detailed benchmarking results here including the fio
output, fio command-line, and guest libvirt domain XML:
https://gitlab.com/stefanha/virt-playbooks/-/tree/io_uring-flags/notebook/fio-output
https://gitlab.com/stefanha/virt-playbooks/-/blob/io_uring-flags/files/fio.sh
https://gitlab.com/stefanha/virt-playbooks/-/blob/io_uring-flags/files/test.xml.j2

Stefan

[-- Attachment #2: signature.asc --]
[-- Type: application/pgp-signature, Size: 488 bytes --]

^ permalink raw reply	[flat|nested] 19+ messages in thread

end of thread, other threads:[~2025-07-24 20:37 UTC | newest]

Thread overview: 19+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2025-07-16 18:38 [PATCH RFC 0/1] block/export: FUSE-over-io_uring Support for QEMU FUSE Exports Brian Song
2025-07-16 18:38 ` [PATCH RFC 1/1] " Brian Song
2025-07-17  6:03   ` Markus Armbruster
2025-07-22 12:00     ` Brian Song
2025-07-21  0:53   ` Stefan Hajnoczi
2025-07-22 12:00     ` Brian Song
2025-07-22 15:17       ` Kevin Wolf
2025-07-22 14:06     ` Bernd Schubert
2025-07-22 15:43       ` Stefan Hajnoczi
2025-07-22 16:20         ` Bernd Schubert
2025-07-21 13:51   ` Bernd Schubert
2025-07-21 18:26     ` Stefan Hajnoczi
2025-07-22 12:00     ` Brian Song
2025-07-22 14:51       ` Stefan Hajnoczi
2025-07-24 20:36     ` Stefan Hajnoczi
2025-07-22 13:32   ` Kevin Wolf
2025-07-20 16:13 ` [PATCH RFC 0/1] " Stefan Hajnoczi
2025-07-22 12:00   ` Brian Song
2025-07-22 14:47     ` Stefan Hajnoczi

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).