]> Git Repo - qemu.git/blobdiff - block/file-posix.c
avoid TABs in files that only contain a few
[qemu.git] / block / file-posix.c
index 074f0ce2b375c330321e6b5aa7e50439ecfb42ea..8aee7a3fb8be591aa4b883182ebf0fa01cb35491 100644 (file)
@@ -142,7 +142,6 @@ do { \
 
 typedef struct BDRVRawState {
     int fd;
-    int lock_fd;
     bool use_lock;
     int type;
     int open_flags;
@@ -152,6 +151,11 @@ typedef struct BDRVRawState {
     uint64_t perm;
     uint64_t shared_perm;
 
+    /* The perms bits whose corresponding bytes are already locked in
+     * s->fd. */
+    uint64_t locked_perm;
+    uint64_t locked_shared_perm;
+
 #ifdef CONFIG_XFS
     bool is_xfs:1;
 #endif
@@ -178,25 +182,29 @@ static int64_t raw_getlength(BlockDriverState *bs);
 
 typedef struct RawPosixAIOData {
     BlockDriverState *bs;
+    int aio_type;
     int aio_fildes;
-    union {
-        struct iovec *aio_iov;
-        void *aio_ioctl_buf;
-    };
-    int aio_niov;
-    uint64_t aio_nbytes;
-#define aio_ioctl_cmd   aio_nbytes /* for QEMU_AIO_IOCTL */
+
     off_t aio_offset;
-    int aio_type;
+    uint64_t aio_nbytes;
+
     union {
+        struct {
+            struct iovec *iov;
+            int niov;
+        } io;
+        struct {
+            uint64_t cmd;
+            void *buf;
+        } ioctl;
         struct {
             int aio_fd2;
             off_t aio_offset2;
-        };
+        } copy_range;
         struct {
             PreallocMode prealloc;
             Error **errp;
-        };
+        } truncate;
     };
 } RawPosixAIOData;
 
@@ -546,18 +554,6 @@ static int raw_open_common(BlockDriverState *bs, QDict *options,
     }
     s->fd = fd;
 
-    s->lock_fd = -1;
-    if (s->use_lock) {
-        fd = qemu_open(filename, s->open_flags);
-        if (fd < 0) {
-            ret = -errno;
-            error_setg_errno(errp, errno, "Could not open '%s' for locking",
-                             filename);
-            qemu_close(s->fd);
-            goto fail;
-        }
-        s->lock_fd = fd;
-    }
     s->perm = 0;
     s->shared_perm = BLK_PERM_ALL;
 
@@ -689,43 +685,72 @@ typedef enum {
  * file; if @unlock == true, also unlock the unneeded bytes.
  * @shared_perm_lock_bits is the mask of all permissions that are NOT shared.
  */
-static int raw_apply_lock_bytes(int fd,
+static int raw_apply_lock_bytes(BDRVRawState *s, int fd,
                                 uint64_t perm_lock_bits,
                                 uint64_t shared_perm_lock_bits,
                                 bool unlock, Error **errp)
 {
     int ret;
     int i;
+    uint64_t locked_perm, locked_shared_perm;
+
+    if (s) {
+        locked_perm = s->locked_perm;
+        locked_shared_perm = s->locked_shared_perm;
+    } else {
+        /*
+         * We don't have the previous bits, just lock/unlock for each of the
+         * requested bits.
+         */
+        if (unlock) {
+            locked_perm = BLK_PERM_ALL;
+            locked_shared_perm = BLK_PERM_ALL;
+        } else {
+            locked_perm = 0;
+            locked_shared_perm = 0;
+        }
+    }
 
     PERM_FOREACH(i) {
         int off = RAW_LOCK_PERM_BASE + i;
-        if (perm_lock_bits & (1ULL << i)) {
+        uint64_t bit = (1ULL << i);
+        if ((perm_lock_bits & bit) && !(locked_perm & bit)) {
             ret = qemu_lock_fd(fd, off, 1, false);
             if (ret) {
                 error_setg(errp, "Failed to lock byte %d", off);
                 return ret;
+            } else if (s) {
+                s->locked_perm |= bit;
             }
-        } else if (unlock) {
+        } else if (unlock && (locked_perm & bit) && !(perm_lock_bits & bit)) {
             ret = qemu_unlock_fd(fd, off, 1);
             if (ret) {
                 error_setg(errp, "Failed to unlock byte %d", off);
                 return ret;
+            } else if (s) {
+                s->locked_perm &= ~bit;
             }
         }
     }
     PERM_FOREACH(i) {
         int off = RAW_LOCK_SHARED_BASE + i;
-        if (shared_perm_lock_bits & (1ULL << i)) {
+        uint64_t bit = (1ULL << i);
+        if ((shared_perm_lock_bits & bit) && !(locked_shared_perm & bit)) {
             ret = qemu_lock_fd(fd, off, 1, false);
             if (ret) {
                 error_setg(errp, "Failed to lock byte %d", off);
                 return ret;
+            } else if (s) {
+                s->locked_shared_perm |= bit;
             }
-        } else if (unlock) {
+        } else if (unlock && (locked_shared_perm & bit) &&
+                   !(shared_perm_lock_bits & bit)) {
             ret = qemu_unlock_fd(fd, off, 1);
             if (ret) {
                 error_setg(errp, "Failed to unlock byte %d", off);
                 return ret;
+            } else if (s) {
+                s->locked_shared_perm &= ~bit;
             }
         }
     }
@@ -789,15 +814,13 @@ static int raw_handle_perm_lock(BlockDriverState *bs,
         return 0;
     }
 
-    assert(s->lock_fd > 0);
-
     switch (op) {
     case RAW_PL_PREPARE:
-        ret = raw_apply_lock_bytes(s->lock_fd, s->perm | new_perm,
+        ret = raw_apply_lock_bytes(s, s->fd, s->perm | new_perm,
                                    ~s->shared_perm | ~new_shared,
                                    false, errp);
         if (!ret) {
-            ret = raw_check_lock_bytes(s->lock_fd, new_perm, new_shared, errp);
+            ret = raw_check_lock_bytes(s->fd, new_perm, new_shared, errp);
             if (!ret) {
                 return 0;
             }
@@ -808,7 +831,7 @@ static int raw_handle_perm_lock(BlockDriverState *bs,
         op = RAW_PL_ABORT;
         /* fall through to unlock bytes. */
     case RAW_PL_ABORT:
-        raw_apply_lock_bytes(s->lock_fd, s->perm, ~s->shared_perm,
+        raw_apply_lock_bytes(s, s->fd, s->perm, ~s->shared_perm,
                              true, &local_err);
         if (local_err) {
             /* Theoretically the above call only unlocks bytes and it cannot
@@ -818,7 +841,7 @@ static int raw_handle_perm_lock(BlockDriverState *bs,
         }
         break;
     case RAW_PL_COMMIT:
-        raw_apply_lock_bytes(s->lock_fd, new_perm, ~new_shared,
+        raw_apply_lock_bytes(s, s->fd, new_perm, ~new_shared,
                              true, &local_err);
         if (local_err) {
             /* Theoretically the above call only unlocks bytes and it cannot
@@ -933,10 +956,18 @@ static void raw_reopen_commit(BDRVReopenState *state)
 {
     BDRVRawReopenState *rs = state->opaque;
     BDRVRawState *s = state->bs->opaque;
+    Error *local_err = NULL;
 
     s->check_cache_dropped = rs->check_cache_dropped;
     s->open_flags = rs->open_flags;
 
+    /* Copy locks to the new fd before closing the old one. */
+    raw_apply_lock_bytes(NULL, rs->fd, s->locked_perm,
+                         s->locked_shared_perm, false, &local_err);
+    if (local_err) {
+        /* shouldn't fail in a sane host, but report it just in case. */
+        error_report_err(local_err);
+    }
     qemu_close(s->fd);
     s->fd = rs->fd;
 
@@ -1121,20 +1152,24 @@ static int hdev_probe_geometry(BlockDriverState *bs, HDGeometry *geo)
 }
 #endif
 
-static ssize_t handle_aiocb_ioctl(RawPosixAIOData *aiocb)
+#if defined(__linux__)
+static int handle_aiocb_ioctl(void *opaque)
 {
+    RawPosixAIOData *aiocb = opaque;
     int ret;
 
-    ret = ioctl(aiocb->aio_fildes, aiocb->aio_ioctl_cmd, aiocb->aio_ioctl_buf);
+    ret = ioctl(aiocb->aio_fildes, aiocb->ioctl.cmd, aiocb->ioctl.buf);
     if (ret == -1) {
         return -errno;
     }
 
     return 0;
 }
+#endif /* linux */
 
-static ssize_t handle_aiocb_flush(RawPosixAIOData *aiocb)
+static int handle_aiocb_flush(void *opaque)
 {
+    RawPosixAIOData *aiocb = opaque;
     BDRVRawState *s = aiocb->bs->opaque;
     int ret;
 
@@ -1206,13 +1241,13 @@ static ssize_t handle_aiocb_rw_vector(RawPosixAIOData *aiocb)
     do {
         if (aiocb->aio_type & QEMU_AIO_WRITE)
             len = qemu_pwritev(aiocb->aio_fildes,
-                               aiocb->aio_iov,
-                               aiocb->aio_niov,
+                               aiocb->io.iov,
+                               aiocb->io.niov,
                                aiocb->aio_offset);
          else
             len = qemu_preadv(aiocb->aio_fildes,
-                              aiocb->aio_iov,
-                              aiocb->aio_niov,
+                              aiocb->io.iov,
+                              aiocb->io.niov,
                               aiocb->aio_offset);
     } while (len == -1 && errno == EINTR);
 
@@ -1268,8 +1303,9 @@ static ssize_t handle_aiocb_rw_linear(RawPosixAIOData *aiocb, char *buf)
     return offset;
 }
 
-static ssize_t handle_aiocb_rw(RawPosixAIOData *aiocb)
+static int handle_aiocb_rw(void *opaque)
 {
+    RawPosixAIOData *aiocb = opaque;
     ssize_t nbytes;
     char *buf;
 
@@ -1278,8 +1314,9 @@ static ssize_t handle_aiocb_rw(RawPosixAIOData *aiocb)
          * If there is just a single buffer, and it is properly aligned
          * we can just use plain pread/pwrite without any problems.
          */
-        if (aiocb->aio_niov == 1) {
-             return handle_aiocb_rw_linear(aiocb, aiocb->aio_iov->iov_base);
+        if (aiocb->io.niov == 1) {
+            nbytes = handle_aiocb_rw_linear(aiocb, aiocb->io.iov->iov_base);
+            goto out;
         }
         /*
          * We have more than one iovec, and all are properly aligned.
@@ -1291,7 +1328,7 @@ static ssize_t handle_aiocb_rw(RawPosixAIOData *aiocb)
             nbytes = handle_aiocb_rw_vector(aiocb);
             if (nbytes == aiocb->aio_nbytes ||
                 (nbytes < 0 && nbytes != -ENOSYS)) {
-                return nbytes;
+                goto out;
             }
             preadv_present = false;
         }
@@ -1309,16 +1346,17 @@ static ssize_t handle_aiocb_rw(RawPosixAIOData *aiocb)
      */
     buf = qemu_try_blockalign(aiocb->bs, aiocb->aio_nbytes);
     if (buf == NULL) {
-        return -ENOMEM;
+        nbytes = -ENOMEM;
+        goto out;
     }
 
     if (aiocb->aio_type & QEMU_AIO_WRITE) {
         char *p = buf;
         int i;
 
-        for (i = 0; i < aiocb->aio_niov; ++i) {
-            memcpy(p, aiocb->aio_iov[i].iov_base, aiocb->aio_iov[i].iov_len);
-            p += aiocb->aio_iov[i].iov_len;
+        for (i = 0; i < aiocb->io.niov; ++i) {
+            memcpy(p, aiocb->io.iov[i].iov_base, aiocb->io.iov[i].iov_len);
+            p += aiocb->io.iov[i].iov_len;
         }
         assert(p - buf == aiocb->aio_nbytes);
     }
@@ -1329,12 +1367,12 @@ static ssize_t handle_aiocb_rw(RawPosixAIOData *aiocb)
         size_t count = aiocb->aio_nbytes, copy;
         int i;
 
-        for (i = 0; i < aiocb->aio_niov && count; ++i) {
+        for (i = 0; i < aiocb->io.niov && count; ++i) {
             copy = count;
-            if (copy > aiocb->aio_iov[i].iov_len) {
-                copy = aiocb->aio_iov[i].iov_len;
+            if (copy > aiocb->io.iov[i].iov_len) {
+                copy = aiocb->io.iov[i].iov_len;
             }
-            memcpy(aiocb->aio_iov[i].iov_base, p, copy);
+            memcpy(aiocb->io.iov[i].iov_base, p, copy);
             assert(count >= copy);
             p     += copy;
             count -= copy;
@@ -1343,7 +1381,21 @@ static ssize_t handle_aiocb_rw(RawPosixAIOData *aiocb)
     }
     qemu_vfree(buf);
 
-    return nbytes;
+out:
+    if (nbytes == aiocb->aio_nbytes) {
+        return 0;
+    } else if (nbytes >= 0 && nbytes < aiocb->aio_nbytes) {
+        if (aiocb->aio_type & QEMU_AIO_WRITE) {
+            return -EINVAL;
+        } else {
+            iov_memset(aiocb->io.iov, aiocb->io.niov, nbytes,
+                      0, aiocb->aio_nbytes - nbytes);
+            return 0;
+        }
+    } else {
+        assert(nbytes < 0);
+        return nbytes;
+    }
 }
 
 #ifdef CONFIG_XFS
@@ -1433,8 +1485,9 @@ static ssize_t handle_aiocb_write_zeroes_block(RawPosixAIOData *aiocb)
     return ret;
 }
 
-static ssize_t handle_aiocb_write_zeroes(RawPosixAIOData *aiocb)
+static int handle_aiocb_write_zeroes(void *opaque)
 {
+    RawPosixAIOData *aiocb = opaque;
 #if defined(CONFIG_FALLOCATE) || defined(CONFIG_XFS)
     BDRVRawState *s = aiocb->bs->opaque;
 #endif
@@ -1498,8 +1551,9 @@ static ssize_t handle_aiocb_write_zeroes(RawPosixAIOData *aiocb)
     return -ENOTSUP;
 }
 
-static ssize_t handle_aiocb_write_zeroes_unmap(RawPosixAIOData *aiocb)
+static int handle_aiocb_write_zeroes_unmap(void *opaque)
 {
+    RawPosixAIOData *aiocb = opaque;
     BDRVRawState *s G_GNUC_UNUSED = aiocb->bs->opaque;
     int ret;
 
@@ -1541,18 +1595,20 @@ static off_t copy_file_range(int in_fd, off_t *in_off, int out_fd,
 }
 #endif
 
-static ssize_t handle_aiocb_copy_range(RawPosixAIOData *aiocb)
+static int handle_aiocb_copy_range(void *opaque)
 {
+    RawPosixAIOData *aiocb = opaque;
     uint64_t bytes = aiocb->aio_nbytes;
     off_t in_off = aiocb->aio_offset;
-    off_t out_off = aiocb->aio_offset2;
+    off_t out_off = aiocb->copy_range.aio_offset2;
 
     while (bytes) {
         ssize_t ret = copy_file_range(aiocb->aio_fildes, &in_off,
-                                      aiocb->aio_fd2, &out_off,
+                                      aiocb->copy_range.aio_fd2, &out_off,
                                       bytes, 0);
         trace_file_copy_file_range(aiocb->bs, aiocb->aio_fildes, in_off,
-                                   aiocb->aio_fd2, out_off, bytes, 0, ret);
+                                   aiocb->copy_range.aio_fd2, out_off, bytes,
+                                   0, ret);
         if (ret == 0) {
             /* No progress (e.g. when beyond EOF), let the caller fall back to
              * buffer I/O. */
@@ -1573,8 +1629,9 @@ static ssize_t handle_aiocb_copy_range(RawPosixAIOData *aiocb)
     return 0;
 }
 
-static ssize_t handle_aiocb_discard(RawPosixAIOData *aiocb)
+static int handle_aiocb_discard(void *opaque)
 {
+    RawPosixAIOData *aiocb = opaque;
     int ret = -EOPNOTSUPP;
     BDRVRawState *s = aiocb->bs->opaque;
 
@@ -1613,15 +1670,17 @@ static ssize_t handle_aiocb_discard(RawPosixAIOData *aiocb)
     return ret;
 }
 
-static int handle_aiocb_truncate(RawPosixAIOData *aiocb)
+static int handle_aiocb_truncate(void *opaque)
 {
+    RawPosixAIOData *aiocb = opaque;
     int result = 0;
     int64_t current_length = 0;
     char *buf = NULL;
     struct stat st;
     int fd = aiocb->aio_fildes;
     int64_t offset = aiocb->aio_offset;
-    Error **errp = aiocb->errp;
+    PreallocMode prealloc = aiocb->truncate.prealloc;
+    Error **errp = aiocb->truncate.errp;
 
     if (fstat(fd, &st) < 0) {
         result = -errno;
@@ -1630,12 +1689,12 @@ static int handle_aiocb_truncate(RawPosixAIOData *aiocb)
     }
 
     current_length = st.st_size;
-    if (current_length > offset && aiocb->prealloc != PREALLOC_MODE_OFF) {
+    if (current_length > offset && prealloc != PREALLOC_MODE_OFF) {
         error_setg(errp, "Cannot use preallocation for shrinking files");
         return -ENOTSUP;
     }
 
-    switch (aiocb->prealloc) {
+    switch (prealloc) {
 #ifdef CONFIG_POSIX_FALLOCATE
     case PREALLOC_MODE_FALLOC:
         /*
@@ -1716,7 +1775,7 @@ static int handle_aiocb_truncate(RawPosixAIOData *aiocb)
     default:
         result = -ENOTSUP;
         error_setg(errp, "Unsupported preallocation mode: %s",
-                   PreallocMode_str(aiocb->prealloc));
+                   PreallocMode_str(prealloc));
         return result;
     }
 
@@ -1732,104 +1791,19 @@ out:
     return result;
 }
 
-static int aio_worker(void *arg)
-{
-    RawPosixAIOData *aiocb = arg;
-    ssize_t ret = 0;
-
-    switch (aiocb->aio_type & QEMU_AIO_TYPE_MASK) {
-    case QEMU_AIO_READ:
-        ret = handle_aiocb_rw(aiocb);
-        if (ret >= 0 && ret < aiocb->aio_nbytes) {
-            iov_memset(aiocb->aio_iov, aiocb->aio_niov, ret,
-                      0, aiocb->aio_nbytes - ret);
-
-            ret = aiocb->aio_nbytes;
-        }
-        if (ret == aiocb->aio_nbytes) {
-            ret = 0;
-        } else if (ret >= 0 && ret < aiocb->aio_nbytes) {
-            ret = -EINVAL;
-        }
-        break;
-    case QEMU_AIO_WRITE:
-        ret = handle_aiocb_rw(aiocb);
-        if (ret == aiocb->aio_nbytes) {
-            ret = 0;
-        } else if (ret >= 0 && ret < aiocb->aio_nbytes) {
-            ret = -EINVAL;
-        }
-        break;
-    case QEMU_AIO_FLUSH:
-        ret = handle_aiocb_flush(aiocb);
-        break;
-    case QEMU_AIO_IOCTL:
-        ret = handle_aiocb_ioctl(aiocb);
-        break;
-    case QEMU_AIO_DISCARD:
-        ret = handle_aiocb_discard(aiocb);
-        break;
-    case QEMU_AIO_WRITE_ZEROES:
-        ret = handle_aiocb_write_zeroes(aiocb);
-        break;
-    case QEMU_AIO_WRITE_ZEROES | QEMU_AIO_DISCARD:
-        ret = handle_aiocb_write_zeroes_unmap(aiocb);
-        break;
-    case QEMU_AIO_COPY_RANGE:
-        ret = handle_aiocb_copy_range(aiocb);
-        break;
-    case QEMU_AIO_TRUNCATE:
-        ret = handle_aiocb_truncate(aiocb);
-        break;
-    default:
-        error_report("invalid aio request (0x%x)", aiocb->aio_type);
-        ret = -EINVAL;
-        break;
-    }
-
-    g_free(aiocb);
-    return ret;
-}
-
-static int paio_submit_co_full(BlockDriverState *bs, int fd,
-                               int64_t offset, int fd2, int64_t offset2,
-                               QEMUIOVector *qiov,
-                               int bytes, int type)
+static int coroutine_fn raw_thread_pool_submit(BlockDriverState *bs,
+                                               ThreadPoolFunc func, void *arg)
 {
-    RawPosixAIOData *acb = g_new(RawPosixAIOData, 1);
-    ThreadPool *pool;
-
-    acb->bs = bs;
-    acb->aio_type = type;
-    acb->aio_fildes = fd;
-    acb->aio_fd2 = fd2;
-    acb->aio_offset2 = offset2;
-
-    acb->aio_nbytes = bytes;
-    acb->aio_offset = offset;
-
-    if (qiov) {
-        acb->aio_iov = qiov->iov;
-        acb->aio_niov = qiov->niov;
-        assert(qiov->size == bytes);
-    }
-
-    trace_file_paio_submit_co(offset, bytes, type);
-    pool = aio_get_thread_pool(bdrv_get_aio_context(bs));
-    return thread_pool_submit_co(pool, aio_worker, acb);
-}
-
-static inline int paio_submit_co(BlockDriverState *bs, int fd,
-                                 int64_t offset, QEMUIOVector *qiov,
-                                 int bytes, int type)
-{
-    return paio_submit_co_full(bs, fd, offset, -1, 0, qiov, bytes, type);
+    /* @bs can be NULL, bdrv_get_aio_context() returns the main context then */
+    ThreadPool *pool = aio_get_thread_pool(bdrv_get_aio_context(bs));
+    return thread_pool_submit_co(pool, func, arg);
 }
 
 static int coroutine_fn raw_co_prw(BlockDriverState *bs, uint64_t offset,
                                    uint64_t bytes, QEMUIOVector *qiov, int type)
 {
     BDRVRawState *s = bs->opaque;
+    RawPosixAIOData acb;
 
     if (fd_open(bs) < 0)
         return -EIO;
@@ -1852,7 +1826,20 @@ static int coroutine_fn raw_co_prw(BlockDriverState *bs, uint64_t offset,
         }
     }
 
-    return paio_submit_co(bs, s->fd, offset, qiov, bytes, type);
+    acb = (RawPosixAIOData) {
+        .bs             = bs,
+        .aio_fildes     = s->fd,
+        .aio_type       = type,
+        .aio_offset     = offset,
+        .aio_nbytes     = bytes,
+        .io             = {
+            .iov            = qiov->iov,
+            .niov           = qiov->niov,
+        },
+    };
+
+    assert(qiov->size == bytes);
+    return raw_thread_pool_submit(bs, handle_aiocb_rw, &acb);
 }
 
 static int coroutine_fn raw_co_preadv(BlockDriverState *bs, uint64_t offset,
@@ -1895,6 +1882,7 @@ static void raw_aio_unplug(BlockDriverState *bs)
 static int raw_co_flush_to_disk(BlockDriverState *bs)
 {
     BDRVRawState *s = bs->opaque;
+    RawPosixAIOData acb;
     int ret;
 
     ret = fd_open(bs);
@@ -1902,7 +1890,13 @@ static int raw_co_flush_to_disk(BlockDriverState *bs)
         return ret;
     }
 
-    return paio_submit_co(bs, s->fd, 0, NULL, 0, QEMU_AIO_FLUSH);
+    acb = (RawPosixAIOData) {
+        .bs             = bs,
+        .aio_fildes     = s->fd,
+        .aio_type       = QEMU_AIO_FLUSH,
+    };
+
+    return raw_thread_pool_submit(bs, handle_aiocb_flush, &acb);
 }
 
 static void raw_aio_attach_aio_context(BlockDriverState *bs,
@@ -1929,10 +1923,6 @@ static void raw_close(BlockDriverState *bs)
         qemu_close(s->fd);
         s->fd = -1;
     }
-    if (s->lock_fd >= 0) {
-        qemu_close(s->lock_fd);
-        s->lock_fd = -1;
-    }
 }
 
 /**
@@ -1945,21 +1935,20 @@ static int coroutine_fn
 raw_regular_truncate(BlockDriverState *bs, int fd, int64_t offset,
                      PreallocMode prealloc, Error **errp)
 {
-    RawPosixAIOData *acb = g_new(RawPosixAIOData, 1);
-    ThreadPool *pool;
+    RawPosixAIOData acb;
 
-    *acb = (RawPosixAIOData) {
+    acb = (RawPosixAIOData) {
         .bs             = bs,
         .aio_fildes     = fd,
         .aio_type       = QEMU_AIO_TRUNCATE,
         .aio_offset     = offset,
-        .prealloc       = prealloc,
-        .errp           = errp,
+        .truncate       = {
+            .prealloc       = prealloc,
+            .errp           = errp,
+        },
     };
 
-    /* @bs can be NULL, bdrv_get_aio_context() returns the main context then */
-    pool = aio_get_thread_pool(bdrv_get_aio_context(bs));
-    return thread_pool_submit_co(pool, aio_worker, acb);
+    return raw_thread_pool_submit(bs, handle_aiocb_truncate, &acb);
 }
 
 static int coroutine_fn raw_co_truncate(BlockDriverState *bs, int64_t offset,
@@ -2094,7 +2083,7 @@ again:
 #endif
     if (!fstat(fd, &sb) && (S_IFCHR & sb.st_mode)) {
 #ifdef DIOCGMEDIASIZE
-       if (ioctl(fd, DIOCGMEDIASIZE, (off_t *)&size))
+        if (ioctl(fd, DIOCGMEDIASIZE, (off_t *)&size))
 #elif defined(DIOCGPART)
         {
                 struct partinfo pi;
@@ -2220,7 +2209,7 @@ raw_co_create(BlockdevCreateOptions *options, Error **errp)
     shared = BLK_PERM_ALL & ~BLK_PERM_RESIZE;
 
     /* Step one: Take locks */
-    result = raw_apply_lock_bytes(fd, perm, ~shared, false, errp);
+    result = raw_apply_lock_bytes(NULL, fd, perm, ~shared, false, errp);
     if (result < 0) {
         goto out_close;
     }
@@ -2264,7 +2253,7 @@ raw_co_create(BlockdevCreateOptions *options, Error **errp)
     }
 
 out_unlock:
-    raw_apply_lock_bytes(fd, 0, 0, true, &local_err);
+    raw_apply_lock_bytes(NULL, fd, 0, 0, true, &local_err);
     if (local_err) {
         /* The above call should not fail, and if it does, that does
          * not mean the whole creation operation has failed.  So
@@ -2590,25 +2579,67 @@ static void coroutine_fn raw_co_invalidate_cache(BlockDriverState *bs,
 }
 
 static coroutine_fn int
-raw_co_pdiscard(BlockDriverState *bs, int64_t offset, int bytes)
+raw_do_pdiscard(BlockDriverState *bs, int64_t offset, int bytes, bool blkdev)
 {
     BDRVRawState *s = bs->opaque;
+    RawPosixAIOData acb;
+
+    acb = (RawPosixAIOData) {
+        .bs             = bs,
+        .aio_fildes     = s->fd,
+        .aio_type       = QEMU_AIO_DISCARD,
+        .aio_offset     = offset,
+        .aio_nbytes     = bytes,
+    };
 
-    return paio_submit_co(bs, s->fd, offset, NULL, bytes, QEMU_AIO_DISCARD);
+    if (blkdev) {
+        acb.aio_type |= QEMU_AIO_BLKDEV;
+    }
+
+    return raw_thread_pool_submit(bs, handle_aiocb_discard, &acb);
 }
 
-static int coroutine_fn raw_co_pwrite_zeroes(
-    BlockDriverState *bs, int64_t offset,
-    int bytes, BdrvRequestFlags flags)
+static coroutine_fn int
+raw_co_pdiscard(BlockDriverState *bs, int64_t offset, int bytes)
+{
+    return raw_do_pdiscard(bs, offset, bytes, false);
+}
+
+static int coroutine_fn
+raw_do_pwrite_zeroes(BlockDriverState *bs, int64_t offset, int bytes,
+                     BdrvRequestFlags flags, bool blkdev)
 {
     BDRVRawState *s = bs->opaque;
-    int operation = QEMU_AIO_WRITE_ZEROES;
+    RawPosixAIOData acb;
+    ThreadPoolFunc *handler;
+
+    acb = (RawPosixAIOData) {
+        .bs             = bs,
+        .aio_fildes     = s->fd,
+        .aio_type       = QEMU_AIO_WRITE_ZEROES,
+        .aio_offset     = offset,
+        .aio_nbytes     = bytes,
+    };
+
+    if (blkdev) {
+        acb.aio_type |= QEMU_AIO_BLKDEV;
+    }
 
     if (flags & BDRV_REQ_MAY_UNMAP) {
-        operation |= QEMU_AIO_DISCARD;
+        acb.aio_type |= QEMU_AIO_DISCARD;
+        handler = handle_aiocb_write_zeroes_unmap;
+    } else {
+        handler = handle_aiocb_write_zeroes;
     }
 
-    return paio_submit_co(bs, s->fd, offset, NULL, bytes, operation);
+    return raw_thread_pool_submit(bs, handler, &acb);
+}
+
+static int coroutine_fn raw_co_pwrite_zeroes(
+    BlockDriverState *bs, int64_t offset,
+    int bytes, BdrvRequestFlags flags)
+{
+    return raw_do_pwrite_zeroes(bs, offset, bytes, flags, false);
 }
 
 static int raw_get_info(BlockDriverState *bs, BlockDriverInfo *bdi)
@@ -2679,6 +2710,7 @@ static int coroutine_fn raw_co_copy_range_to(BlockDriverState *bs,
                                              BdrvRequestFlags read_flags,
                                              BdrvRequestFlags write_flags)
 {
+    RawPosixAIOData acb;
     BDRVRawState *s = bs->opaque;
     BDRVRawState *src_s;
 
@@ -2691,8 +2723,20 @@ static int coroutine_fn raw_co_copy_range_to(BlockDriverState *bs,
     if (fd_open(src->bs) < 0 || fd_open(dst->bs) < 0) {
         return -EIO;
     }
-    return paio_submit_co_full(bs, src_s->fd, src_offset, s->fd, dst_offset,
-                               NULL, bytes, QEMU_AIO_COPY_RANGE);
+
+    acb = (RawPosixAIOData) {
+        .bs             = bs,
+        .aio_type       = QEMU_AIO_COPY_RANGE,
+        .aio_fildes     = src_s->fd,
+        .aio_offset     = src_offset,
+        .aio_nbytes     = bytes,
+        .copy_range     = {
+            .aio_fd2        = s->fd,
+            .aio_offset2    = dst_offset,
+        },
+    };
+
+    return raw_thread_pool_submit(bs, handle_aiocb_copy_range, &acb);
 }
 
 BlockDriver bdrv_file = {
@@ -3040,36 +3084,39 @@ hdev_open_Mac_error:
 }
 
 #if defined(__linux__)
-
-static BlockAIOCB *hdev_aio_ioctl(BlockDriverState *bs,
-        unsigned long int req, void *buf,
-        BlockCompletionFunc *cb, void *opaque)
+static int coroutine_fn
+hdev_co_ioctl(BlockDriverState *bs, unsigned long int req, void *buf)
 {
     BDRVRawState *s = bs->opaque;
-    RawPosixAIOData *acb;
-    ThreadPool *pool;
+    RawPosixAIOData acb;
+    int ret;
 
-    if (fd_open(bs) < 0)
-        return NULL;
+    ret = fd_open(bs);
+    if (ret < 0) {
+        return ret;
+    }
 
     if (req == SG_IO && s->pr_mgr) {
         struct sg_io_hdr *io_hdr = buf;
         if (io_hdr->cmdp[0] == PERSISTENT_RESERVE_OUT ||
             io_hdr->cmdp[0] == PERSISTENT_RESERVE_IN) {
             return pr_manager_execute(s->pr_mgr, bdrv_get_aio_context(bs),
-                                      s->fd, io_hdr, cb, opaque);
+                                      s->fd, io_hdr);
         }
     }
 
-    acb = g_new(RawPosixAIOData, 1);
-    acb->bs = bs;
-    acb->aio_type = QEMU_AIO_IOCTL;
-    acb->aio_fildes = s->fd;
-    acb->aio_offset = 0;
-    acb->aio_ioctl_buf = buf;
-    acb->aio_ioctl_cmd = req;
-    pool = aio_get_thread_pool(bdrv_get_aio_context(bs));
-    return thread_pool_submit_aio(pool, aio_worker, acb, cb, opaque);
+    acb = (RawPosixAIOData) {
+        .bs         = bs,
+        .aio_type   = QEMU_AIO_IOCTL,
+        .aio_fildes = s->fd,
+        .aio_offset = 0,
+        .ioctl      = {
+            .buf        = buf,
+            .cmd        = req,
+        },
+    };
+
+    return raw_thread_pool_submit(bs, handle_aiocb_ioctl, &acb);
 }
 #endif /* linux */
 
@@ -3086,22 +3133,18 @@ static int fd_open(BlockDriverState *bs)
 static coroutine_fn int
 hdev_co_pdiscard(BlockDriverState *bs, int64_t offset, int bytes)
 {
-    BDRVRawState *s = bs->opaque;
     int ret;
 
     ret = fd_open(bs);
     if (ret < 0) {
         return ret;
     }
-    return paio_submit_co(bs, s->fd, offset, NULL, bytes,
-                          QEMU_AIO_DISCARD | QEMU_AIO_BLKDEV);
+    return raw_do_pdiscard(bs, offset, bytes, true);
 }
 
 static coroutine_fn int hdev_co_pwrite_zeroes(BlockDriverState *bs,
     int64_t offset, int bytes, BdrvRequestFlags flags)
 {
-    BDRVRawState *s = bs->opaque;
-    int operation = QEMU_AIO_WRITE_ZEROES | QEMU_AIO_BLKDEV;
     int rc;
 
     rc = fd_open(bs);
@@ -3109,11 +3152,7 @@ static coroutine_fn int hdev_co_pwrite_zeroes(BlockDriverState *bs,
         return rc;
     }
 
-    if (flags & BDRV_REQ_MAY_UNMAP) {
-        operation |= QEMU_AIO_DISCARD;
-    }
-
-    return paio_submit_co(bs, s->fd, offset, NULL, bytes, operation);
+    return raw_do_pwrite_zeroes(bs, offset, bytes, flags, true);
 }
 
 static int coroutine_fn hdev_co_create_opts(const char *filename, QemuOpts *opts,
@@ -3218,7 +3257,7 @@ static BlockDriver bdrv_host_device = {
 
     /* generic scsi device */
 #ifdef __linux__
-    .bdrv_aio_ioctl     = hdev_aio_ioctl,
+    .bdrv_co_ioctl          = hdev_co_ioctl,
 #endif
 };
 
@@ -3340,7 +3379,7 @@ static BlockDriver bdrv_host_cdrom = {
     .bdrv_lock_medium   = cdrom_lock_medium,
 
     /* generic scsi device */
-    .bdrv_aio_ioctl     = hdev_aio_ioctl,
+    .bdrv_co_ioctl      = hdev_co_ioctl,
 };
 #endif /* __linux__ */
 
This page took 0.058251 seconds and 4 git commands to generate.