]> git.proxmox.com Git - mirror_qemu.git/blobdiff - block/qcow2-cluster.c
Merge tag 'pull-parallels-2023-09-20-v2' of https://src.openvz.org/scm/~den/qemu...
[mirror_qemu.git] / block / qcow2-cluster.c
index fcdf7af8e6576711da373eed58f9bfe298f95e63..f4f6cd6ad020e63a6166d3389720ecfb2de30f48 100644 (file)
 #include "qemu/osdep.h"
 #include <zlib.h>
 
+#include "block/block-io.h"
 #include "qapi/error.h"
 #include "qcow2.h"
 #include "qemu/bswap.h"
+#include "qemu/memalign.h"
 #include "trace.h"
 
-int qcow2_shrink_l1_table(BlockDriverState *bs, uint64_t exact_size)
+int coroutine_fn qcow2_shrink_l1_table(BlockDriverState *bs,
+                                       uint64_t exact_size)
 {
     BDRVQcow2State *s = bs->opaque;
     int new_l1_size, i, ret;
@@ -45,20 +48,20 @@ int qcow2_shrink_l1_table(BlockDriverState *bs, uint64_t exact_size)
     fprintf(stderr, "shrink l1_table from %d to %d\n", s->l1_size, new_l1_size);
 #endif
 
-    BLKDBG_EVENT(bs->file, BLKDBG_L1_SHRINK_WRITE_TABLE);
-    ret = bdrv_pwrite_zeroes(bs->file, s->l1_table_offset +
-                                       new_l1_size * sizeof(uint64_t),
-                             (s->l1_size - new_l1_size) * sizeof(uint64_t), 0);
+    BLKDBG_CO_EVENT(bs->file, BLKDBG_L1_SHRINK_WRITE_TABLE);
+    ret = bdrv_co_pwrite_zeroes(bs->file,
+                                s->l1_table_offset + new_l1_size * L1E_SIZE,
+                                (s->l1_size - new_l1_size) * L1E_SIZE, 0);
     if (ret < 0) {
         goto fail;
     }
 
-    ret = bdrv_flush(bs->file->bs);
+    ret = bdrv_co_flush(bs->file->bs);
     if (ret < 0) {
         goto fail;
     }
 
-    BLKDBG_EVENT(bs->file, BLKDBG_L1_SHRINK_FREE_L2_CLUSTERS);
+    BLKDBG_CO_EVENT(bs->file, BLKDBG_L1_SHRINK_FREE_L2_CLUSTERS);
     for (i = s->l1_size - 1; i > new_l1_size - 1; i--) {
         if ((s->l1_table[i] & L1E_OFFSET_MASK) == 0) {
             continue;
@@ -76,7 +79,7 @@ fail:
      * l1_table in memory to avoid possible image corruption.
      */
     memset(s->l1_table + new_l1_size, 0,
-           (s->l1_size - new_l1_size) * sizeof(uint64_t));
+           (s->l1_size - new_l1_size) * L1E_SIZE);
     return ret;
 }
 
@@ -96,7 +99,7 @@ int qcow2_grow_l1_table(BlockDriverState *bs, uint64_t min_size,
     /* Do a sanity check on min_size before trying to calculate new_l1_size
      * (this prevents overflows during the while loop for the calculation of
      * new_l1_size) */
-    if (min_size > INT_MAX / sizeof(uint64_t)) {
+    if (min_size > INT_MAX / L1E_SIZE) {
         return -EFBIG;
     }
 
@@ -114,7 +117,7 @@ int qcow2_grow_l1_table(BlockDriverState *bs, uint64_t min_size,
     }
 
     QEMU_BUILD_BUG_ON(QCOW_MAX_L1_SIZE > INT_MAX);
-    if (new_l1_size > QCOW_MAX_L1_SIZE / sizeof(uint64_t)) {
+    if (new_l1_size > QCOW_MAX_L1_SIZE / L1E_SIZE) {
         return -EFBIG;
     }
 
@@ -123,7 +126,7 @@ int qcow2_grow_l1_table(BlockDriverState *bs, uint64_t min_size,
             s->l1_size, new_l1_size);
 #endif
 
-    new_l1_size2 = sizeof(uint64_t) * new_l1_size;
+    new_l1_size2 = L1E_SIZE * new_l1_size;
     new_l1_table = qemu_try_blockalign(bs->file->bs, new_l1_size2);
     if (new_l1_table == NULL) {
         return -ENOMEM;
@@ -131,7 +134,7 @@ int qcow2_grow_l1_table(BlockDriverState *bs, uint64_t min_size,
     memset(new_l1_table, 0, new_l1_size2);
 
     if (s->l1_size) {
-        memcpy(new_l1_table, s->l1_table, s->l1_size * sizeof(uint64_t));
+        memcpy(new_l1_table, s->l1_table, s->l1_size * L1E_SIZE);
     }
 
     /* write new table (align to cluster) */
@@ -158,8 +161,8 @@ int qcow2_grow_l1_table(BlockDriverState *bs, uint64_t min_size,
     BLKDBG_EVENT(bs->file, BLKDBG_L1_GROW_WRITE_TABLE);
     for(i = 0; i < s->l1_size; i++)
         new_l1_table[i] = cpu_to_be64(new_l1_table[i]);
-    ret = bdrv_pwrite_sync(bs->file, new_l1_table_offset,
-                           new_l1_table, new_l1_size2);
+    ret = bdrv_pwrite_sync(bs->file, new_l1_table_offset, new_l1_size2,
+                           new_l1_table, 0);
     if (ret < 0)
         goto fail;
     for(i = 0; i < s->l1_size; i++)
@@ -170,7 +173,7 @@ int qcow2_grow_l1_table(BlockDriverState *bs, uint64_t min_size,
     stl_be_p(data, new_l1_size);
     stq_be_p(data + 4, new_l1_table_offset);
     ret = bdrv_pwrite_sync(bs->file, offsetof(QCowHeader, l1_size),
-                           data, sizeof(data));
+                           sizeof(data), data, 0);
     if (ret < 0) {
         goto fail;
     }
@@ -180,7 +183,7 @@ int qcow2_grow_l1_table(BlockDriverState *bs, uint64_t min_size,
     s->l1_table = new_l1_table;
     old_l1_size = s->l1_size;
     s->l1_size = new_l1_size;
-    qcow2_free_clusters(bs, old_l1_table_offset, old_l1_size * sizeof(uint64_t),
+    qcow2_free_clusters(bs, old_l1_table_offset, old_l1_size * L1E_SIZE,
                         QCOW2_DISCARD_OTHER);
     return 0;
  fail:
@@ -225,9 +228,9 @@ int qcow2_write_l1_entry(BlockDriverState *bs, int l1_index)
     BDRVQcow2State *s = bs->opaque;
     int l1_start_index;
     int i, ret;
-    int bufsize = MAX(sizeof(uint64_t),
+    int bufsize = MAX(L1E_SIZE,
                       MIN(bs->file->bs->bl.request_alignment, s->cluster_size));
-    int nentries = bufsize / sizeof(uint64_t);
+    int nentries = bufsize / L1E_SIZE;
     g_autofree uint64_t *buf = g_try_new0(uint64_t, nentries);
 
     if (buf == NULL) {
@@ -240,15 +243,15 @@ int qcow2_write_l1_entry(BlockDriverState *bs, int l1_index)
     }
 
     ret = qcow2_pre_write_overlap_check(bs, QCOW2_OL_ACTIVE_L1,
-            s->l1_table_offset + 8 * l1_start_index, bufsize, false);
+            s->l1_table_offset + L1E_SIZE * l1_start_index, bufsize, false);
     if (ret < 0) {
         return ret;
     }
 
     BLKDBG_EVENT(bs->file, BLKDBG_L1_UPDATE);
     ret = bdrv_pwrite_sync(bs->file,
-                           s->l1_table_offset + 8 * l1_start_index,
-                           buf, bufsize);
+                           s->l1_table_offset + L1E_SIZE * l1_start_index,
+                           bufsize, buf, 0);
     if (ret < 0) {
         return ret;
     }
@@ -488,10 +491,9 @@ static int count_contiguous_subclusters(BlockDriverState *bs, int nb_clusters,
     return count;
 }
 
-static int coroutine_fn do_perform_cow_read(BlockDriverState *bs,
-                                            uint64_t src_cluster_offset,
-                                            unsigned offset_in_cluster,
-                                            QEMUIOVector *qiov)
+static int coroutine_fn GRAPH_RDLOCK
+do_perform_cow_read(BlockDriverState *bs, uint64_t src_cluster_offset,
+                    unsigned offset_in_cluster, QEMUIOVector *qiov)
 {
     int ret;
 
@@ -499,13 +501,26 @@ static int coroutine_fn do_perform_cow_read(BlockDriverState *bs,
         return 0;
     }
 
-    BLKDBG_EVENT(bs->file, BLKDBG_COW_READ);
+    BLKDBG_CO_EVENT(bs->file, BLKDBG_COW_READ);
 
     if (!bs->drv) {
         return -ENOMEDIUM;
     }
 
-    /* Call .bdrv_co_readv() directly instead of using the public block-layer
+    /*
+     * We never deal with requests that don't satisfy
+     * bdrv_check_qiov_request(), and aligning requests to clusters never
+     * breaks this condition. So, do some assertions before calling
+     * bs->drv->bdrv_co_preadv_part() which has int64_t arguments.
+     */
+    assert(src_cluster_offset <= INT64_MAX);
+    assert(src_cluster_offset + offset_in_cluster <= INT64_MAX);
+    /* Cast qiov->size to uint64_t to silence a compiler warning on -m32 */
+    assert((uint64_t)qiov->size <= INT64_MAX);
+    bdrv_check_qiov_request(src_cluster_offset + offset_in_cluster, qiov->size,
+                            qiov, 0, &error_abort);
+    /*
+     * Call .bdrv_co_readv() directly instead of using the public block-layer
      * interface.  This avoids double I/O throttling and request tracking,
      * which can lead to deadlock when block layer copy-on-read is enabled.
      */
@@ -519,10 +534,9 @@ static int coroutine_fn do_perform_cow_read(BlockDriverState *bs,
     return 0;
 }
 
-static int coroutine_fn do_perform_cow_write(BlockDriverState *bs,
-                                             uint64_t cluster_offset,
-                                             unsigned offset_in_cluster,
-                                             QEMUIOVector *qiov)
+static int coroutine_fn GRAPH_RDLOCK
+do_perform_cow_write(BlockDriverState *bs, uint64_t cluster_offset,
+                     unsigned offset_in_cluster, QEMUIOVector *qiov)
 {
     BDRVQcow2State *s = bs->opaque;
     int ret;
@@ -537,7 +551,7 @@ static int coroutine_fn do_perform_cow_write(BlockDriverState *bs,
         return ret;
     }
 
-    BLKDBG_EVENT(bs->file, BLKDBG_COW_WRITE);
+    BLKDBG_CO_EVENT(bs->file, BLKDBG_COW_WRITE);
     ret = bdrv_co_pwritev(s->data_file, cluster_offset + offset_in_cluster,
                           qiov->size, qiov, 0);
     if (ret < 0) {
@@ -556,8 +570,7 @@ static int coroutine_fn do_perform_cow_write(BlockDriverState *bs,
  * offset needs to be aligned to a cluster boundary.
  *
  * If the cluster is unallocated then *host_offset will be 0.
- * If the cluster is compressed then *host_offset will contain the
- * complete compressed cluster descriptor.
+ * If the cluster is compressed then *host_offset will contain the l2 entry.
  *
  * On entry, *bytes is the maximum number of contiguous bytes starting at
  * offset that we are interested in.
@@ -660,7 +673,7 @@ int qcow2_get_host_offset(BlockDriverState *bs, uint64_t offset,
             ret = -EIO;
             goto fail;
         }
-        *host_offset = l2_entry & L2E_COMPRESSED_OFFSET_SIZE_MASK;
+        *host_offset = l2_entry;
         break;
     case QCOW2_SUBCLUSTER_ZERO_PLAIN:
     case QCOW2_SUBCLUSTER_UNALLOCATED_PLAIN:
@@ -810,10 +823,9 @@ static int get_cluster_table(BlockDriverState *bs, uint64_t offset,
  *
  * Return 0 on success and -errno in error cases
  */
-int qcow2_alloc_compressed_cluster_offset(BlockDriverState *bs,
-                                          uint64_t offset,
-                                          int compressed_size,
-                                          uint64_t *host_offset)
+int coroutine_fn GRAPH_RDLOCK
+qcow2_alloc_compressed_cluster_offset(BlockDriverState *bs, uint64_t offset,
+                                      int compressed_size, uint64_t *host_offset)
 {
     BDRVQcow2State *s = bs->opaque;
     int l2_index, ret;
@@ -859,7 +871,7 @@ int qcow2_alloc_compressed_cluster_offset(BlockDriverState *bs,
 
     /* compressed clusters never have the copied flag */
 
-    BLKDBG_EVENT(bs->file, BLKDBG_L2_UPDATE_COMPRESSED);
+    BLKDBG_CO_EVENT(bs->file, BLKDBG_L2_UPDATE_COMPRESSED);
     qcow2_cache_entry_mark_dirty(s->l2_table_cache, l2_slice);
     set_l2_entry(s, l2_slice, l2_index, cluster_offset);
     if (has_subclusters(s)) {
@@ -871,7 +883,8 @@ int qcow2_alloc_compressed_cluster_offset(BlockDriverState *bs,
     return 0;
 }
 
-static int perform_cow(BlockDriverState *bs, QCowL2Meta *m)
+static int coroutine_fn GRAPH_RDLOCK
+perform_cow(BlockDriverState *bs, QCowL2Meta *m)
 {
     BDRVQcow2State *s = bs->opaque;
     Qcow2COWRegion *start = &m->cow_start;
@@ -978,7 +991,7 @@ static int perform_cow(BlockDriverState *bs, QCowL2Meta *m)
         /* NOTE: we have a write_aio blkdebug event here followed by
          * a cow_write one in do_perform_cow_write(), but there's only
          * one single I/O operation */
-        BLKDBG_EVENT(bs->file, BLKDBG_WRITE_AIO);
+        BLKDBG_CO_EVENT(bs->file, BLKDBG_WRITE_AIO);
         ret = do_perform_cow_write(bs, m->alloc_offset, start->offset, &qiov);
     } else {
         /* If there's no guest data then write both COW regions separately */
@@ -1011,7 +1024,8 @@ fail:
     return ret;
 }
 
-int qcow2_alloc_cluster_link_l2(BlockDriverState *bs, QCowL2Meta *m)
+int coroutine_fn qcow2_alloc_cluster_link_l2(BlockDriverState *bs,
+                                             QCowL2Meta *m)
 {
     BDRVQcow2State *s = bs->opaque;
     int i, j = 0, l2_index, ret;
@@ -1049,6 +1063,8 @@ int qcow2_alloc_cluster_link_l2(BlockDriverState *bs, QCowL2Meta *m)
     qcow2_cache_entry_mark_dirty(s->l2_table_cache, l2_slice);
 
     assert(l2_index + m->nb_clusters <= s->l2_slice_size);
+    assert(m->cow_end.offset + m->cow_end.nb_bytes <=
+           m->nb_clusters << s->cluster_bits);
     for (i = 0; i < m->nb_clusters; i++) {
         uint64_t offset = cluster_offset + ((uint64_t)i << s->cluster_bits);
         /* if two concurrent writes happen to the same unallocated cluster
@@ -1070,8 +1086,7 @@ int qcow2_alloc_cluster_link_l2(BlockDriverState *bs, QCowL2Meta *m)
         if (has_subclusters(s) && !m->prealloc) {
             uint64_t l2_bitmap = get_l2_bitmap(s, l2_slice, l2_index + i);
             unsigned written_from = m->cow_start.offset;
-            unsigned written_to = m->cow_end.offset + m->cow_end.nb_bytes ?:
-                m->nb_clusters << s->cluster_bits;
+            unsigned written_to = m->cow_end.offset + m->cow_end.nb_bytes;
             int first_sc, last_sc;
             /* Narrow written_from and written_to down to the current cluster */
             written_from = MAX(written_from, i << s->cluster_bits);
@@ -1096,7 +1111,7 @@ int qcow2_alloc_cluster_link_l2(BlockDriverState *bs, QCowL2Meta *m)
      */
     if (!m->keep_old_clusters && j != 0) {
         for (i = 0; i < j; i++) {
-            qcow2_free_any_clusters(bs, old_cluster[i], 1, QCOW2_DISCARD_NEVER);
+            qcow2_free_any_cluster(bs, old_cluster[i], QCOW2_DISCARD_NEVER);
         }
     }
 
@@ -1110,7 +1125,7 @@ err:
  * Frees the allocated clusters because the request failed and they won't
  * actually be linked.
  */
-void qcow2_alloc_cluster_abort(BlockDriverState *bs, QCowL2Meta *m)
+void coroutine_fn qcow2_alloc_cluster_abort(BlockDriverState *bs, QCowL2Meta *m)
 {
     BDRVQcow2State *s = bs->opaque;
     if (!has_data_file(bs) && !m->keep_old_clusters) {
@@ -1140,9 +1155,11 @@ void qcow2_alloc_cluster_abort(BlockDriverState *bs, QCowL2Meta *m)
  *
  * Returns 0 on success, -errno on failure.
  */
-static int calculate_l2_meta(BlockDriverState *bs, uint64_t host_cluster_offset,
-                             uint64_t guest_offset, unsigned bytes,
-                             uint64_t *l2_slice, QCowL2Meta **m, bool keep_old)
+static int coroutine_fn calculate_l2_meta(BlockDriverState *bs,
+                                          uint64_t host_cluster_offset,
+                                          uint64_t guest_offset, unsigned bytes,
+                                          uint64_t *l2_slice, QCowL2Meta **m,
+                                          bool keep_old)
 {
     BDRVQcow2State *s = bs->opaque;
     int sc_index, l2_index = offset_to_l2_slice_index(s, guest_offset);
@@ -1383,8 +1400,9 @@ static int count_single_write_clusters(BlockDriverState *bs, int nb_clusters,
  *           information on cluster allocation may be invalid now. The caller
  *           must start over anyway, so consider *cur_bytes undefined.
  */
-static int handle_dependencies(BlockDriverState *bs, uint64_t guest_offset,
-    uint64_t *cur_bytes, QCowL2Meta **m)
+static int coroutine_fn handle_dependencies(BlockDriverState *bs,
+                                            uint64_t guest_offset,
+                                            uint64_t *cur_bytes, QCowL2Meta **m)
 {
     BDRVQcow2State *s = bs->opaque;
     QCowL2Meta *old_alloc;
@@ -1399,29 +1417,47 @@ static int handle_dependencies(BlockDriverState *bs, uint64_t guest_offset,
 
         if (end <= old_start || start >= old_end) {
             /* No intersection */
+            continue;
+        }
+
+        if (old_alloc->keep_old_clusters &&
+            (end <= l2meta_cow_start(old_alloc) ||
+             start >= l2meta_cow_end(old_alloc)))
+        {
+            /*
+             * Clusters intersect but COW areas don't. And cluster itself is
+             * already allocated. So, there is no actual conflict.
+             */
+            continue;
+        }
+
+        /* Conflict */
+
+        if (start < old_start) {
+            /* Stop at the start of a running allocation */
+            bytes = old_start - start;
         } else {
-            if (start < old_start) {
-                /* Stop at the start of a running allocation */
-                bytes = old_start - start;
-            } else {
-                bytes = 0;
-            }
+            bytes = 0;
+        }
 
-            /* Stop if already an l2meta exists. After yielding, it wouldn't
-             * be valid any more, so we'd have to clean up the old L2Metas
-             * and deal with requests depending on them before starting to
-             * gather new ones. Not worth the trouble. */
-            if (bytes == 0 && *m) {
-                *cur_bytes = 0;
-                return 0;
-            }
+        /*
+         * Stop if an l2meta already exists. After yielding, it wouldn't
+         * be valid any more, so we'd have to clean up the old L2Metas
+         * and deal with requests depending on them before starting to
+         * gather new ones. Not worth the trouble.
+         */
+        if (bytes == 0 && *m) {
+            *cur_bytes = 0;
+            return 0;
+        }
 
-            if (bytes == 0) {
-                /* Wait for the dependency to complete. We need to recheck
-                 * the free/allocated clusters when we continue. */
-                qemu_co_queue_wait(&old_alloc->dependent_requests, &s->lock);
-                return -EAGAIN;
-            }
+        if (bytes == 0) {
+            /*
+             * Wait for the dependency to complete. We need to recheck
+             * the free/allocated clusters when we continue.
+             */
+            qemu_co_queue_wait(&old_alloc->dependent_requests, &s->lock);
+            return -EAGAIN;
         }
     }
 
@@ -1454,8 +1490,9 @@ static int handle_dependencies(BlockDriverState *bs, uint64_t guest_offset,
  *
  *  -errno: in error cases
  */
-static int handle_copied(BlockDriverState *bs, uint64_t guest_offset,
-    uint64_t *host_offset, uint64_t *bytes, QCowL2Meta **m)
+static int coroutine_fn handle_copied(BlockDriverState *bs,
+    uint64_t guest_offset, uint64_t *host_offset, uint64_t *bytes,
+    QCowL2Meta **m)
 {
     BDRVQcow2State *s = bs->opaque;
     int l2_index;
@@ -1563,8 +1600,10 @@ out:
  * function has been waiting for another request and the allocation must be
  * restarted, but the whole request should not be failed.
  */
-static int do_alloc_cluster_offset(BlockDriverState *bs, uint64_t guest_offset,
-                                   uint64_t *host_offset, uint64_t *nb_clusters)
+static int coroutine_fn do_alloc_cluster_offset(BlockDriverState *bs,
+                                                uint64_t guest_offset,
+                                                uint64_t *host_offset,
+                                                uint64_t *nb_clusters)
 {
     BDRVQcow2State *s = bs->opaque;
 
@@ -1619,8 +1658,9 @@ static int do_alloc_cluster_offset(BlockDriverState *bs, uint64_t guest_offset,
  *
  *  -errno: in error cases
  */
-static int handle_alloc(BlockDriverState *bs, uint64_t guest_offset,
-    uint64_t *host_offset, uint64_t *bytes, QCowL2Meta **m)
+static int coroutine_fn handle_alloc(BlockDriverState *bs,
+    uint64_t guest_offset, uint64_t *host_offset, uint64_t *bytes,
+    QCowL2Meta **m)
 {
     BDRVQcow2State *s = bs->opaque;
     int l2_index;
@@ -1710,34 +1750,40 @@ static int handle_alloc(BlockDriverState *bs, uint64_t guest_offset,
 
 out:
     qcow2_cache_put(s->l2_table_cache, (void **) &l2_slice);
-    if (ret < 0 && *m && (*m)->nb_clusters > 0) {
-        QLIST_REMOVE(*m, next_in_flight);
-    }
     return ret;
 }
 
 /*
- * alloc_cluster_offset
+ * For a given area on the virtual disk defined by @offset and @bytes,
+ * find the corresponding area on the qcow2 image, allocating new
+ * clusters (or subclusters) if necessary. The result can span a
+ * combination of allocated and previously unallocated clusters.
  *
- * For a given offset on the virtual disk, find the cluster offset in qcow2
- * file. If the offset is not found, allocate a new cluster.
+ * Note that offset may not be cluster aligned. In this case, the returned
+ * *host_offset points to exact byte referenced by offset and therefore
+ * isn't cluster aligned as well.
  *
- * If the cluster was already allocated, m->nb_clusters is set to 0 and
- * other fields in m are meaningless.
+ * On return, @host_offset is set to the beginning of the requested
+ * area. This area is guaranteed to be contiguous on the qcow2 file
+ * but it can be smaller than initially requested. In this case @bytes
+ * is updated with the actual size.
  *
- * If the cluster is newly allocated, m->nb_clusters is set to the number of
- * contiguous clusters that have been allocated. In this case, the other
- * fields of m are valid and contain information about the first allocated
- * cluster.
+ * If any clusters or subclusters were allocated then @m contains a
+ * list with the information of all the affected regions. Note that
+ * this can happen regardless of whether this function succeeds or
+ * not. The caller is responsible for updating the L2 metadata of the
+ * allocated clusters (on success) or freeing them (on failure), and
+ * for clearing the contents of @m afterwards in both cases.
  *
  * If the request conflicts with another write request in flight, the coroutine
  * is queued and will be reentered when the dependency has completed.
  *
  * Return 0 on success and -errno in error cases
  */
-int qcow2_alloc_cluster_offset(BlockDriverState *bs, uint64_t offset,
-                               unsigned int *bytes, uint64_t *host_offset,
-                               QCowL2Meta **m)
+int coroutine_fn qcow2_alloc_host_offset(BlockDriverState *bs, uint64_t offset,
+                                         unsigned int *bytes,
+                                         uint64_t *host_offset,
+                                         QCowL2Meta **m)
 {
     BDRVQcow2State *s = bs->opaque;
     uint64_t start, remaining;
@@ -1758,7 +1804,7 @@ again:
     while (true) {
 
         if (*host_offset == INV_OFFSET && cluster_offset != INV_OFFSET) {
-            *host_offset = start_of_cluster(s, cluster_offset);
+            *host_offset = cluster_offset;
         }
 
         assert(remaining >= cur_bytes);
@@ -1841,6 +1887,8 @@ again:
     *bytes -= remaining;
     assert(*bytes > 0);
     assert(*host_offset != INV_OFFSET);
+    assert(offset_into_cluster(s, *host_offset) ==
+           offset_into_cluster(s, offset));
 
     return 0;
 }
@@ -1876,6 +1924,10 @@ static int discard_in_l2_slice(BlockDriverState *bs, uint64_t offset,
         uint64_t new_l2_bitmap = old_l2_bitmap;
         QCow2ClusterType cluster_type =
             qcow2_get_cluster_type(bs, old_l2_entry);
+        bool keep_reference = (cluster_type != QCOW2_CLUSTER_COMPRESSED) &&
+                              !full_discard &&
+                              (s->discard_no_unref &&
+                               type == QCOW2_DISCARD_REQUEST);
 
         /*
          * If full_discard is true, the cluster should not read back as zeroes,
@@ -1894,10 +1946,22 @@ static int discard_in_l2_slice(BlockDriverState *bs, uint64_t offset,
             new_l2_entry = new_l2_bitmap = 0;
         } else if (bs->backing || qcow2_cluster_is_allocated(cluster_type)) {
             if (has_subclusters(s)) {
-                new_l2_entry = 0;
+                if (keep_reference) {
+                    new_l2_entry = old_l2_entry;
+                } else {
+                    new_l2_entry = 0;
+                }
                 new_l2_bitmap = QCOW_L2_BITMAP_ALL_ZEROES;
             } else {
-                new_l2_entry = s->qcow_version >= 3 ? QCOW_OFLAG_ZERO : 0;
+                if (s->qcow_version >= 3) {
+                    if (keep_reference) {
+                        new_l2_entry |= QCOW_OFLAG_ZERO;
+                    } else {
+                        new_l2_entry = QCOW_OFLAG_ZERO;
+                    }
+                } else {
+                    new_l2_entry = 0;
+                }
             }
         }
 
@@ -1911,8 +1975,16 @@ static int discard_in_l2_slice(BlockDriverState *bs, uint64_t offset,
         if (has_subclusters(s)) {
             set_l2_bitmap(s, l2_slice, l2_index + i, new_l2_bitmap);
         }
-        /* Then decrease the refcount */
-        qcow2_free_any_clusters(bs, old_l2_entry, 1, type);
+        if (!keep_reference) {
+            /* Then decrease the refcount */
+            qcow2_free_any_cluster(bs, old_l2_entry, type);
+        } else if (s->discard_passthrough[type] &&
+                   (cluster_type == QCOW2_CLUSTER_NORMAL ||
+                    cluster_type == QCOW2_CLUSTER_ZERO_ALLOC)) {
+            /* If we keep the reference, pass on the discard still */
+            bdrv_pdiscard(s->data_file, old_l2_entry & L2E_OFFSET_MASK,
+                          s->cluster_size);
+       }
     }
 
     qcow2_cache_put(s->l2_table_cache, (void **) &l2_slice);
@@ -1965,8 +2037,9 @@ fail:
  * all clusters in the same L2 slice) and returns the number of zeroed
  * clusters.
  */
-static int zero_in_l2_slice(BlockDriverState *bs, uint64_t offset,
-                            uint64_t nb_clusters, int flags)
+static int coroutine_fn
+zero_in_l2_slice(BlockDriverState *bs, uint64_t offset,
+                 uint64_t nb_clusters, int flags)
 {
     BDRVQcow2State *s = bs->opaque;
     uint64_t *l2_slice;
@@ -2002,14 +2075,17 @@ static int zero_in_l2_slice(BlockDriverState *bs, uint64_t offset,
             continue;
         }
 
+        /* First update L2 entries */
         qcow2_cache_entry_mark_dirty(s->l2_table_cache, l2_slice);
-        if (unmap) {
-            qcow2_free_any_clusters(bs, old_l2_entry, 1, QCOW2_DISCARD_REQUEST);
-        }
         set_l2_entry(s, l2_slice, l2_index + i, new_l2_entry);
         if (has_subclusters(s)) {
             set_l2_bitmap(s, l2_slice, l2_index + i, new_l2_bitmap);
         }
+
+        /* Then decrease the refcount */
+        if (unmap) {
+            qcow2_free_any_cluster(bs, old_l2_entry, QCOW2_DISCARD_REQUEST);
+        }
     }
 
     qcow2_cache_put(s->l2_table_cache, (void **) &l2_slice);
@@ -2017,8 +2093,9 @@ static int zero_in_l2_slice(BlockDriverState *bs, uint64_t offset,
     return nb_clusters;
 }
 
-static int zero_l2_subclusters(BlockDriverState *bs, uint64_t offset,
-                               unsigned nb_subclusters)
+static int coroutine_fn
+zero_l2_subclusters(BlockDriverState *bs, uint64_t offset,
+                    unsigned nb_subclusters)
 {
     BDRVQcow2State *s = bs->opaque;
     uint64_t *l2_slice;
@@ -2063,8 +2140,8 @@ out:
     return ret;
 }
 
-int qcow2_subcluster_zeroize(BlockDriverState *bs, uint64_t offset,
-                             uint64_t bytes, int flags)
+int coroutine_fn qcow2_subcluster_zeroize(BlockDriverState *bs, uint64_t offset,
+                                          uint64_t bytes, int flags)
 {
     BDRVQcow2State *s = bs->opaque;
     uint64_t end_offset = offset + bytes;
@@ -2218,7 +2295,8 @@ static int expand_zero_clusters_in_l1(BlockDriverState *bs, uint64_t *l1_table,
                                       (void **)&l2_slice);
             } else {
                 /* load inactive L2 tables from disk */
-                ret = bdrv_pread(bs->file, slice_offset, l2_slice, slice_size2);
+                ret = bdrv_pread(bs->file, slice_offset, slice_size2,
+                                 l2_slice, 0);
             }
             if (ret < 0) {
                 goto fail;
@@ -2334,8 +2412,8 @@ static int expand_zero_clusters_in_l1(BlockDriverState *bs, uint64_t *l1_table,
                         goto fail;
                     }
 
-                    ret = bdrv_pwrite(bs->file, slice_offset,
-                                      l2_slice, slice_size2);
+                    ret = bdrv_pwrite(bs->file, slice_offset, slice_size2,
+                                      l2_slice, 0);
                     if (ret < 0) {
                         goto fail;
                     }
@@ -2410,7 +2488,7 @@ int qcow2_expand_zero_clusters(BlockDriverState *bs,
         Error *local_err = NULL;
 
         ret = qcow2_validate_table(bs, s->snapshots[i].l1_table_offset,
-                                   s->snapshots[i].l1_size, sizeof(uint64_t),
+                                   s->snapshots[i].l1_size, L1E_SIZE,
                                    QCOW_MAX_L1_SIZE, "Snapshot L1 table",
                                    &local_err);
         if (ret < 0) {
@@ -2418,7 +2496,7 @@ int qcow2_expand_zero_clusters(BlockDriverState *bs,
             goto fail;
         }
 
-        l1_size2 = s->snapshots[i].l1_size * sizeof(uint64_t);
+        l1_size2 = s->snapshots[i].l1_size * L1E_SIZE;
         new_l1_table = g_try_realloc(l1_table, l1_size2);
 
         if (!new_l1_table) {
@@ -2428,8 +2506,8 @@ int qcow2_expand_zero_clusters(BlockDriverState *bs,
 
         l1_table = new_l1_table;
 
-        ret = bdrv_pread(bs->file, s->snapshots[i].l1_table_offset,
-                         l1_table, l1_size2);
+        ret = bdrv_pread(bs->file, s->snapshots[i].l1_table_offset, l1_size2,
+                         l1_table, 0);
         if (ret < 0) {
             goto fail;
         }
@@ -2452,3 +2530,18 @@ fail:
     g_free(l1_table);
     return ret;
 }
+
+void qcow2_parse_compressed_l2_entry(BlockDriverState *bs, uint64_t l2_entry,
+                                     uint64_t *coffset, int *csize)
+{
+    BDRVQcow2State *s = bs->opaque;
+    int nb_csectors;
+
+    assert(qcow2_get_cluster_type(bs, l2_entry) == QCOW2_CLUSTER_COMPRESSED);
+
+    *coffset = l2_entry & s->cluster_offset_mask;
+
+    nb_csectors = ((l2_entry >> s->csize_shift) & s->csize_mask) + 1;
+    *csize = nb_csectors * QCOW2_COMPRESSED_SECTOR_SIZE -
+        (*coffset & (QCOW2_COMPRESSED_SECTOR_SIZE - 1));
+}