]> Git Repo - qemu.git/blobdiff - migration/ram.c
block/backup: add backup_is_cluster_allocated
[qemu.git] / migration / ram.c
index 908517fc2bd0a391ecd6a5aae37f9bcb99fbb5de..35552c090b5f27ef307706fa42b23f5ad7ce21e8 100644 (file)
@@ -661,6 +661,8 @@ typedef struct {
     uint64_t num_packets;
     /* pages sent through this channel */
     uint64_t num_pages;
+    /* syncs main thread and channels */
+    QemuSemaphore sem_sync;
 }  MultiFDSendParams;
 
 typedef struct {
@@ -677,6 +679,8 @@ typedef struct {
     QemuMutex mutex;
     /* is this channel thread running */
     bool running;
+    /* should this thread finish */
+    bool quit;
     /* array of pages to receive */
     MultiFDPages_t *pages;
     /* packet allocated len */
@@ -894,8 +898,6 @@ struct {
     MultiFDSendParams *params;
     /* array of pages to sent */
     MultiFDPages_t *pages;
-    /* syncs main thread and channels */
-    QemuSemaphore sem_sync;
     /* global number of generated multifd packets */
     uint64_t packet_num;
     /* send channels ready */
@@ -920,7 +922,7 @@ struct {
  * false.
  */
 
-static void multifd_send_pages(void)
+static int multifd_send_pages(RAMState *rs)
 {
     int i;
     static int next_channel;
@@ -933,6 +935,11 @@ static void multifd_send_pages(void)
         p = &multifd_send_state->params[i];
 
         qemu_mutex_lock(&p->mutex);
+        if (p->quit) {
+            error_report("%s: channel %d has already quit!", __func__, i);
+            qemu_mutex_unlock(&p->mutex);
+            return -1;
+        }
         if (!p->pending_job) {
             p->pending_job++;
             next_channel = (i + 1) % migrate_multifd_channels();
@@ -947,13 +954,16 @@ static void multifd_send_pages(void)
     multifd_send_state->pages = p->pages;
     p->pages = pages;
     transferred = ((uint64_t) pages->used) * TARGET_PAGE_SIZE + p->packet_len;
+    qemu_file_update_transfer(rs->f, transferred);
     ram_counters.multifd_bytes += transferred;
     ram_counters.transferred += transferred;;
     qemu_mutex_unlock(&p->mutex);
     qemu_sem_post(&p->sem);
+
+    return 1;
 }
 
-static void multifd_queue_page(RAMBlock *block, ram_addr_t offset)
+static int multifd_queue_page(RAMState *rs, RAMBlock *block, ram_addr_t offset)
 {
     MultiFDPages_t *pages = multifd_send_state->pages;
 
@@ -968,21 +978,27 @@ static void multifd_queue_page(RAMBlock *block, ram_addr_t offset)
         pages->used++;
 
         if (pages->used < pages->allocated) {
-            return;
+            return 1;
         }
     }
 
-    multifd_send_pages();
+    if (multifd_send_pages(rs) < 0) {
+        return -1;
+    }
 
     if (pages->block != block) {
-        multifd_queue_page(block, offset);
+        return  multifd_queue_page(rs, block, offset);
     }
+
+    return 1;
 }
 
 static void multifd_send_terminate_threads(Error *err)
 {
     int i;
 
+    trace_multifd_send_terminate_threads(err != NULL);
+
     if (err) {
         MigrationState *s = migrate_get_current();
         migrate_set_error(s, err);
@@ -1023,6 +1039,7 @@ void multifd_save_cleanup(void)
         p->c = NULL;
         qemu_mutex_destroy(&p->mutex);
         qemu_sem_destroy(&p->sem);
+        qemu_sem_destroy(&p->sem_sync);
         g_free(p->name);
         p->name = NULL;
         multifd_pages_clear(p->pages);
@@ -1032,7 +1049,6 @@ void multifd_save_cleanup(void)
         p->packet = NULL;
     }
     qemu_sem_destroy(&multifd_send_state->channels_ready);
-    qemu_sem_destroy(&multifd_send_state->sem_sync);
     g_free(multifd_send_state->params);
     multifd_send_state->params = NULL;
     multifd_pages_clear(multifd_send_state->pages);
@@ -1041,7 +1057,7 @@ void multifd_save_cleanup(void)
     multifd_send_state = NULL;
 }
 
-static void multifd_send_sync_main(void)
+static void multifd_send_sync_main(RAMState *rs)
 {
     int i;
 
@@ -1049,7 +1065,10 @@ static void multifd_send_sync_main(void)
         return;
     }
     if (multifd_send_state->pages->used) {
-        multifd_send_pages();
+        if (multifd_send_pages(rs) < 0) {
+            error_report("%s: multifd_send_pages fail", __func__);
+            return;
+        }
     }
     for (i = 0; i < migrate_multifd_channels(); i++) {
         MultiFDSendParams *p = &multifd_send_state->params[i];
@@ -1058,9 +1077,18 @@ static void multifd_send_sync_main(void)
 
         qemu_mutex_lock(&p->mutex);
 
+        if (p->quit) {
+            error_report("%s: channel %d has already quit", __func__, i);
+            qemu_mutex_unlock(&p->mutex);
+            return;
+        }
+
         p->packet_num = multifd_send_state->packet_num++;
         p->flags |= MULTIFD_FLAG_SYNC;
         p->pending_job++;
+        qemu_file_update_transfer(rs->f, p->packet_len);
+        ram_counters.multifd_bytes += p->packet_len;
+        ram_counters.transferred += p->packet_len;
         qemu_mutex_unlock(&p->mutex);
         qemu_sem_post(&p->sem);
     }
@@ -1068,7 +1096,7 @@ static void multifd_send_sync_main(void)
         MultiFDSendParams *p = &multifd_send_state->params[i];
 
         trace_multifd_send_sync_main_wait(p->id);
-        qemu_sem_wait(&multifd_send_state->sem_sync);
+        qemu_sem_wait(&p->sem_sync);
     }
     trace_multifd_send_sync_main(multifd_send_state->packet_num);
 }
@@ -1077,7 +1105,8 @@ static void *multifd_send_thread(void *opaque)
 {
     MultiFDSendParams *p = opaque;
     Error *local_err = NULL;
-    int ret;
+    int ret = 0;
+    uint32_t flags = 0;
 
     trace_multifd_send_thread_start(p->id);
     rcu_register_thread();
@@ -1095,7 +1124,7 @@ static void *multifd_send_thread(void *opaque)
         if (p->pending_job) {
             uint32_t used = p->pages->used;
             uint64_t packet_num = p->packet_num;
-            uint32_t flags = p->flags;
+            flags = p->flags;
 
             p->next_packet_size = used * qemu_target_page_size();
             multifd_send_fill_packet(p);
@@ -1127,7 +1156,7 @@ static void *multifd_send_thread(void *opaque)
             qemu_mutex_unlock(&p->mutex);
 
             if (flags & MULTIFD_FLAG_SYNC) {
-                qemu_sem_post(&multifd_send_state->sem_sync);
+                qemu_sem_post(&p->sem_sync);
             }
             qemu_sem_post(&multifd_send_state->channels_ready);
         } else if (p->quit) {
@@ -1141,9 +1170,21 @@ static void *multifd_send_thread(void *opaque)
 
 out:
     if (local_err) {
+        trace_multifd_send_error(p->id);
         multifd_send_terminate_threads(local_err);
     }
 
+    /*
+     * Error happen, I will exit, but I can't just leave, tell
+     * who pay attention to me.
+     */
+    if (ret != 0) {
+        if (flags & MULTIFD_FLAG_SYNC) {
+            qemu_sem_post(&p->sem_sync);
+        }
+        qemu_sem_post(&multifd_send_state->channels_ready);
+    }
+
     qemu_mutex_lock(&p->mutex);
     p->running = false;
     qemu_mutex_unlock(&p->mutex);
@@ -1160,6 +1201,7 @@ static void multifd_new_send_channel_async(QIOTask *task, gpointer opaque)
     QIOChannel *sioc = QIO_CHANNEL(qio_task_get_source(task));
     Error *local_err = NULL;
 
+    trace_multifd_new_send_channel_async(p->id);
     if (qio_task_propagate_error(task, &local_err)) {
         migrate_set_error(migrate_get_current(), local_err);
         multifd_save_cleanup();
@@ -1185,7 +1227,6 @@ int multifd_save_setup(void)
     multifd_send_state = g_malloc0(sizeof(*multifd_send_state));
     multifd_send_state->params = g_new0(MultiFDSendParams, thread_count);
     multifd_send_state->pages = multifd_pages_init(page_count);
-    qemu_sem_init(&multifd_send_state->sem_sync, 0);
     qemu_sem_init(&multifd_send_state->channels_ready, 0);
 
     for (i = 0; i < thread_count; i++) {
@@ -1193,6 +1234,7 @@ int multifd_save_setup(void)
 
         qemu_mutex_init(&p->mutex);
         qemu_sem_init(&p->sem, 0);
+        qemu_sem_init(&p->sem_sync, 0);
         p->quit = false;
         p->pending_job = 0;
         p->id = i;
@@ -1220,6 +1262,8 @@ static void multifd_recv_terminate_threads(Error *err)
 {
     int i;
 
+    trace_multifd_recv_terminate_threads(err != NULL);
+
     if (err) {
         MigrationState *s = migrate_get_current();
         migrate_set_error(s, err);
@@ -1234,6 +1278,7 @@ static void multifd_recv_terminate_threads(Error *err)
         MultiFDRecvParams *p = &multifd_recv_state->params[i];
 
         qemu_mutex_lock(&p->mutex);
+        p->quit = true;
         /* We could arrive here for two reasons:
            - normal quit, i.e. everything went fine, just finished
            - error quit: We close the channels so the channel threads
@@ -1256,6 +1301,12 @@ int multifd_load_cleanup(Error **errp)
         MultiFDRecvParams *p = &multifd_recv_state->params[i];
 
         if (p->running) {
+            p->quit = true;
+            /*
+             * multifd_recv_thread may hung at MULTIFD_FLAG_SYNC handle code,
+             * however try to wakeup it without harm in cleanup phase.
+             */
+            qemu_sem_post(&p->sem_sync);
             qemu_thread_join(&p->thread);
         }
         object_unref(OBJECT(p->c));
@@ -1291,15 +1342,15 @@ static void multifd_recv_sync_main(void)
 
         trace_multifd_recv_sync_main_wait(p->id);
         qemu_sem_wait(&multifd_recv_state->sem_sync);
+    }
+    for (i = 0; i < migrate_multifd_channels(); i++) {
+        MultiFDRecvParams *p = &multifd_recv_state->params[i];
+
         qemu_mutex_lock(&p->mutex);
         if (multifd_recv_state->packet_num < p->packet_num) {
             multifd_recv_state->packet_num = p->packet_num;
         }
         qemu_mutex_unlock(&p->mutex);
-    }
-    for (i = 0; i < migrate_multifd_channels(); i++) {
-        MultiFDRecvParams *p = &multifd_recv_state->params[i];
-
         trace_multifd_recv_sync_main_signal(p->id);
         qemu_sem_post(&p->sem_sync);
     }
@@ -1319,6 +1370,10 @@ static void *multifd_recv_thread(void *opaque)
         uint32_t used;
         uint32_t flags;
 
+        if (p->quit) {
+            break;
+        }
+
         ret = qio_channel_read_all_eof(p->c, (void *)p->packet,
                                        p->packet_len, &local_err);
         if (ret == 0) {   /* EOF */
@@ -1390,6 +1445,7 @@ int multifd_load_setup(void)
 
         qemu_mutex_init(&p->mutex);
         qemu_sem_init(&p->sem_sync, 0);
+        p->quit = false;
         p->id = i;
         p->pages = multifd_pages_init(page_count);
         p->packet_len = sizeof(MultiFDPacket_t)
@@ -1432,6 +1488,7 @@ bool multifd_recv_new_channel(QIOChannel *ioc, Error **errp)
                                 atomic_read(&multifd_recv_state->count));
         return false;
     }
+    trace_multifd_recv_new_channel(id);
 
     p = &multifd_recv_state->params[id];
     if (p->c != NULL) {
@@ -1585,25 +1642,30 @@ static int save_xbzrle_page(RAMState *rs, uint8_t **current_data,
     encoded_len = xbzrle_encode_buffer(prev_cached_page, XBZRLE.current_buf,
                                        TARGET_PAGE_SIZE, XBZRLE.encoded_buf,
                                        TARGET_PAGE_SIZE);
+
+    /*
+     * Update the cache contents, so that it corresponds to the data
+     * sent, in all cases except where we skip the page.
+     */
+    if (!last_stage && encoded_len != 0) {
+        memcpy(prev_cached_page, XBZRLE.current_buf, TARGET_PAGE_SIZE);
+        /*
+         * In the case where we couldn't compress, ensure that the caller
+         * sends the data from the cache, since the guest might have
+         * changed the RAM since we copied it.
+         */
+        *current_data = prev_cached_page;
+    }
+
     if (encoded_len == 0) {
         trace_save_xbzrle_page_skipping();
         return 0;
     } else if (encoded_len == -1) {
         trace_save_xbzrle_page_overflow();
         xbzrle_counters.overflow++;
-        /* update data in the cache */
-        if (!last_stage) {
-            memcpy(prev_cached_page, *current_data, TARGET_PAGE_SIZE);
-            *current_data = prev_cached_page;
-        }
         return -1;
     }
 
-    /* we need to update the data in the cache, in order to get the same data */
-    if (!last_stage) {
-        memcpy(prev_cached_page, XBZRLE.current_buf, TARGET_PAGE_SIZE);
-    }
-
     /* Send XBZRLE based compressed page */
     bytes_xbzrle = save_page_header(rs, rs->f, block,
                                     offset | RAM_SAVE_FLAG_XBZRLE);
@@ -1659,6 +1721,33 @@ static inline bool migration_bitmap_clear_dirty(RAMState *rs,
     bool ret;
 
     qemu_mutex_lock(&rs->bitmap_mutex);
+
+    /*
+     * Clear dirty bitmap if needed.  This _must_ be called before we
+     * send any of the page in the chunk because we need to make sure
+     * we can capture further page content changes when we sync dirty
+     * log the next time.  So as long as we are going to send any of
+     * the page in the chunk we clear the remote dirty bitmap for all.
+     * Clearing it earlier won't be a problem, but too late will.
+     */
+    if (rb->clear_bmap && clear_bmap_test_and_clear(rb, page)) {
+        uint8_t shift = rb->clear_bmap_shift;
+        hwaddr size = 1ULL << (TARGET_PAGE_BITS + shift);
+        hwaddr start = (page << TARGET_PAGE_BITS) & (-size);
+
+        /*
+         * CLEAR_BITMAP_SHIFT_MIN should always guarantee this... this
+         * can make things easier sometimes since then start address
+         * of the small chunk will always be 64 pages aligned so the
+         * bitmap will always be aligned to unsigned long.  We should
+         * even be able to remove this restriction but I'm simply
+         * keeping it.
+         */
+        assert(shift >= 6);
+        trace_migration_bitmap_clear_dirty(rb->idstr, start, size, page);
+        memory_region_clear_dirty_bitmap(rb->mr, start, size);
+    }
+
     ret = test_and_clear_bit(page, rb->bmap);
 
     if (ret) {
@@ -1669,11 +1758,11 @@ static inline bool migration_bitmap_clear_dirty(RAMState *rs,
     return ret;
 }
 
-static void migration_bitmap_sync_range(RAMState *rs, RAMBlock *rb,
-                                        ram_addr_t length)
+/* Called with RCU critical section */
+static void ramblock_sync_dirty_bitmap(RAMState *rs, RAMBlock *rb)
 {
     rs->migration_dirty_pages +=
-        cpu_physical_memory_sync_dirty_bitmap(rb, 0, length,
+        cpu_physical_memory_sync_dirty_bitmap(rb, 0, rb->used_length,
                                               &rs->num_dirty_pages_period);
 }
 
@@ -1762,7 +1851,7 @@ static void migration_bitmap_sync(RAMState *rs)
     qemu_mutex_lock(&rs->bitmap_mutex);
     rcu_read_lock();
     RAMBLOCK_FOREACH_NOT_IGNORED(block) {
-        migration_bitmap_sync_range(rs, block, block->used_length);
+        ramblock_sync_dirty_bitmap(rs, block);
     }
     ram_counters.remaining = ram_bytes_remaining();
     rcu_read_unlock();
@@ -2000,7 +2089,9 @@ static int ram_save_page(RAMState *rs, PageSearchStatus *pss, bool last_stage)
 static int ram_save_multifd_page(RAMState *rs, RAMBlock *block,
                                  ram_addr_t offset)
 {
-    multifd_queue_page(block, offset);
+    if (multifd_queue_page(rs, block, offset) < 0) {
+        return -1;
+    }
     ram_counters.normal++;
 
     return 1;
@@ -2281,6 +2372,12 @@ static bool get_queued_page(RAMState *rs, PageSearchStatus *pss)
          */
         pss->block = block;
         pss->page = offset >> TARGET_PAGE_BITS;
+
+        /*
+         * This unqueued page would break the "one round" check, even is
+         * really rare.
+         */
+        pss->complete_round = false;
     }
 
     return !!block;
@@ -2675,6 +2772,8 @@ static void ram_save_cleanup(void *opaque)
     memory_global_dirty_log_stop();
 
     RAMBLOCK_FOREACH_NOT_IGNORED(block) {
+        g_free(block->clear_bmap);
+        block->clear_bmap = NULL;
         g_free(block->bmap);
         block->bmap = NULL;
         g_free(block->unsentmap);
@@ -2762,13 +2861,9 @@ void ram_postcopy_migrated_memory_release(MigrationState *ms)
  *       with the dirtymap; so a '1' means it's either dirty or unsent.
  *
  * @ms: current migration state
- * @pds: state for postcopy
- * @start: RAMBlock starting page
- * @length: RAMBlock size
+ * @block: RAMBlock to discard
  */
-static int postcopy_send_discard_bm_ram(MigrationState *ms,
-                                        PostcopyDiscardState *pds,
-                                        RAMBlock *block)
+static int postcopy_send_discard_bm_ram(MigrationState *ms, RAMBlock *block)
 {
     unsigned long end = block->used_length >> TARGET_PAGE_BITS;
     unsigned long current;
@@ -2776,23 +2871,21 @@ static int postcopy_send_discard_bm_ram(MigrationState *ms,
 
     for (current = 0; current < end; ) {
         unsigned long one = find_next_bit(unsentmap, end, current);
+        unsigned long zero, discard_length;
 
-        if (one <= end) {
-            unsigned long zero = find_next_zero_bit(unsentmap, end, one + 1);
-            unsigned long discard_length;
+        if (one >= end) {
+            break;
+        }
 
-            if (zero >= end) {
-                discard_length = end - one;
-            } else {
-                discard_length = zero - one;
-            }
-            if (discard_length) {
-                postcopy_discard_send_range(ms, pds, one, discard_length);
-            }
-            current = one + discard_length;
+        zero = find_next_zero_bit(unsentmap, end, one + 1);
+
+        if (zero >= end) {
+            discard_length = end - one;
         } else {
-            current = one;
+            discard_length = zero - one;
         }
+        postcopy_discard_send_range(ms, one, discard_length);
+        current = one + discard_length;
     }
 
     return 0;
@@ -2817,16 +2910,15 @@ static int postcopy_each_ram_send_discard(MigrationState *ms)
     int ret;
 
     RAMBLOCK_FOREACH_NOT_IGNORED(block) {
-        PostcopyDiscardState *pds =
-            postcopy_discard_send_init(ms, block->idstr);
+        postcopy_discard_send_init(ms, block->idstr);
 
         /*
          * Postcopy sends chunks of bitmap over the wire, but it
          * just needs indexes at this point, avoids it having
          * target page specific code.
          */
-        ret = postcopy_send_discard_bm_ram(ms, pds, block);
-        postcopy_discard_send_finish(ms, pds);
+        ret = postcopy_send_discard_bm_ram(ms, block);
+        postcopy_discard_send_finish(ms);
         if (ret) {
             return ret;
         }
@@ -2849,11 +2941,9 @@ static int postcopy_each_ram_send_discard(MigrationState *ms)
  * @unsent_pass: if true we need to canonicalize partially unsent host pages
  *               otherwise we need to canonicalize partially dirty host pages
  * @block: block that contains the page we want to canonicalize
- * @pds: state for postcopy
  */
 static void postcopy_chunk_hostpages_pass(MigrationState *ms, bool unsent_pass,
-                                          RAMBlock *block,
-                                          PostcopyDiscardState *pds)
+                                          RAMBlock *block)
 {
     RAMState *rs = ram_state;
     unsigned long *bitmap = block->bmap;
@@ -2876,54 +2966,30 @@ static void postcopy_chunk_hostpages_pass(MigrationState *ms, bool unsent_pass,
     }
 
     while (run_start < pages) {
-        bool do_fixup = false;
-        unsigned long fixup_start_addr;
-        unsigned long host_offset;
 
         /*
          * If the start of this run of pages is in the middle of a host
          * page, then we need to fixup this host page.
          */
-        host_offset = run_start % host_ratio;
-        if (host_offset) {
-            do_fixup = true;
-            run_start -= host_offset;
-            fixup_start_addr = run_start;
-            /* For the next pass */
-            run_start = run_start + host_ratio;
-        } else {
+        if (QEMU_IS_ALIGNED(run_start, host_ratio)) {
             /* Find the end of this run */
-            unsigned long run_end;
             if (unsent_pass) {
-                run_end = find_next_bit(unsentmap, pages, run_start + 1);
+                run_start = find_next_bit(unsentmap, pages, run_start + 1);
             } else {
-                run_end = find_next_zero_bit(bitmap, pages, run_start + 1);
+                run_start = find_next_zero_bit(bitmap, pages, run_start + 1);
             }
             /*
              * If the end isn't at the start of a host page, then the
              * run doesn't finish at the end of a host page
              * and we need to discard.
              */
-            host_offset = run_end % host_ratio;
-            if (host_offset) {
-                do_fixup = true;
-                fixup_start_addr = run_end - host_offset;
-                /*
-                 * This host page has gone, the next loop iteration starts
-                 * from after the fixup
-                 */
-                run_start = fixup_start_addr + host_ratio;
-            } else {
-                /*
-                 * No discards on this iteration, next loop starts from
-                 * next sent/dirty page
-                 */
-                run_start = run_end + 1;
-            }
         }
 
-        if (do_fixup) {
+        if (!QEMU_IS_ALIGNED(run_start, host_ratio)) {
             unsigned long page;
+            unsigned long fixup_start_addr = QEMU_ALIGN_DOWN(run_start,
+                                                             host_ratio);
+            run_start = QEMU_ALIGN_UP(run_start, host_ratio);
 
             /* Tell the destination to discard this page */
             if (unsent_pass || !test_bit(fixup_start_addr, unsentmap)) {
@@ -2934,8 +3000,7 @@ static void postcopy_chunk_hostpages_pass(MigrationState *ms, bool unsent_pass,
                  *     (any partially sent pages were already discarded
                  *     by the previous unsent_pass)
                  */
-                postcopy_discard_send_range(ms, pds, fixup_start_addr,
-                                            host_ratio);
+                postcopy_discard_send_range(ms, fixup_start_addr, host_ratio);
             }
 
             /* Clean up the bitmap */
@@ -2963,7 +3028,7 @@ static void postcopy_chunk_hostpages_pass(MigrationState *ms, bool unsent_pass,
 }
 
 /**
- * postcopy_chuck_hostpages: discrad any partially sent host page
+ * postcopy_chunk_hostpages: discard any partially sent host page
  *
  * Utility for the outgoing postcopy code.
  *
@@ -2978,18 +3043,17 @@ static void postcopy_chunk_hostpages_pass(MigrationState *ms, bool unsent_pass,
  */
 static int postcopy_chunk_hostpages(MigrationState *ms, RAMBlock *block)
 {
-    PostcopyDiscardState *pds =
-        postcopy_discard_send_init(ms, block->idstr);
+    postcopy_discard_send_init(ms, block->idstr);
 
     /* First pass: Discard all partially sent host pages */
-    postcopy_chunk_hostpages_pass(ms, true, block, pds);
+    postcopy_chunk_hostpages_pass(ms, true, block);
     /*
      * Second pass: Ensure that all partially dirty host pages are made
      * fully dirty.
      */
-    postcopy_chunk_hostpages_pass(ms, false, block, pds);
+    postcopy_chunk_hostpages_pass(ms, false, block);
 
-    postcopy_discard_send_finish(ms, pds);
+    postcopy_discard_send_finish(ms);
     return 0;
 }
 
@@ -3173,11 +3237,11 @@ static int ram_state_init(RAMState **rsp)
     QSIMPLEQ_INIT(&(*rsp)->src_page_requests);
 
     /*
+     * Count the total number of pages used by ram blocks not including any
+     * gaps due to alignment or unplugs.
      * This must match with the initial values of dirty bitmap.
-     * Currently we initialize the dirty bitmap to all zeros so
-     * here the total dirty page count is zero.
      */
-    (*rsp)->migration_dirty_pages = 0;
+    (*rsp)->migration_dirty_pages = ram_bytes_total() >> TARGET_PAGE_BITS;
     ram_state_reset(*rsp);
 
     return 0;
@@ -3185,23 +3249,39 @@ static int ram_state_init(RAMState **rsp)
 
 static void ram_list_init_bitmaps(void)
 {
+    MigrationState *ms = migrate_get_current();
     RAMBlock *block;
     unsigned long pages;
+    uint8_t shift;
 
     /* Skip setting bitmap if there is no RAM */
     if (ram_bytes_total()) {
+        shift = ms->clear_bitmap_shift;
+        if (shift > CLEAR_BITMAP_SHIFT_MAX) {
+            error_report("clear_bitmap_shift (%u) too big, using "
+                         "max value (%u)", shift, CLEAR_BITMAP_SHIFT_MAX);
+            shift = CLEAR_BITMAP_SHIFT_MAX;
+        } else if (shift < CLEAR_BITMAP_SHIFT_MIN) {
+            error_report("clear_bitmap_shift (%u) too small, using "
+                         "min value (%u)", shift, CLEAR_BITMAP_SHIFT_MIN);
+            shift = CLEAR_BITMAP_SHIFT_MIN;
+        }
+
         RAMBLOCK_FOREACH_NOT_IGNORED(block) {
             pages = block->max_length >> TARGET_PAGE_BITS;
             /*
              * The initial dirty bitmap for migration must be set with all
              * ones to make sure we'll migrate every guest RAM page to
              * destination.
-             * Here we didn't set RAMBlock.bmap simply because it is already
-             * set in ram_list.dirty_memory[DIRTY_MEMORY_MIGRATION] in
-             * ram_block_add, and that's where we'll sync the dirty bitmaps.
-             * Here setting RAMBlock.bmap would be fine too but not necessary.
+             * Here we set RAMBlock.bmap all to 1 because when rebegin a
+             * new migration after a failed migration, ram_list.
+             * dirty_memory[DIRTY_MEMORY_MIGRATION] don't include the whole
+             * guest memory.
              */
             block->bmap = bitmap_new(pages);
+            bitmap_set(block->bmap, 0, pages);
+            block->clear_bmap_shift = shift;
+            block->clear_bmap = bitmap_new(clear_bmap_size(pages, shift));
             if (migrate_postcopy_ram()) {
                 block->unsentmap = bitmap_new(pages);
                 bitmap_set(block->unsentmap, 0, pages);
@@ -3370,7 +3450,6 @@ static int ram_save_setup(QEMUFile *f, void *opaque)
         }
         if (migrate_ignore_shared()) {
             qemu_put_be64(f, block->mr->addr);
-            qemu_put_byte(f, ramblock_is_ignored(block) ? 1 : 0);
         }
     }
 
@@ -3379,7 +3458,7 @@ static int ram_save_setup(QEMUFile *f, void *opaque)
     ram_control_before_iterate(f, RAM_CONTROL_SETUP);
     ram_control_after_iterate(f, RAM_CONTROL_SETUP);
 
-    multifd_send_sync_main();
+    multifd_send_sync_main(*rsp);
     qemu_put_be64(f, RAM_SAVE_FLAG_EOS);
     qemu_fflush(f);
 
@@ -3466,8 +3545,8 @@ static int ram_save_iterate(QEMUFile *f, void *opaque)
      */
     ram_control_after_iterate(f, RAM_CONTROL_ROUND);
 
-    multifd_send_sync_main();
 out:
+    multifd_send_sync_main(rs);
     qemu_put_be64(f, RAM_SAVE_FLAG_EOS);
     qemu_fflush(f);
     ram_counters.transferred += 8;
@@ -3526,7 +3605,7 @@ static int ram_save_complete(QEMUFile *f, void *opaque)
 
     rcu_read_unlock();
 
-    multifd_send_sync_main();
+    multifd_send_sync_main(rs);
     qemu_put_be64(f, RAM_SAVE_FLAG_EOS);
     qemu_fflush(f);
 
@@ -4193,7 +4272,7 @@ static void colo_flush_ram_cache(void)
     memory_global_dirty_log_sync();
     rcu_read_lock();
     RAMBLOCK_FOREACH_NOT_IGNORED(block) {
-        migration_bitmap_sync_range(ram_state, block, block->used_length);
+        ramblock_sync_dirty_bitmap(ram_state, block);
     }
     rcu_read_unlock();
 
@@ -4219,40 +4298,26 @@ static void colo_flush_ram_cache(void)
     trace_colo_flush_ram_cache_end();
 }
 
-static int ram_load(QEMUFile *f, void *opaque, int version_id)
+/**
+ * ram_load_precopy: load pages in precopy case
+ *
+ * Returns 0 for success or -errno in case of error
+ *
+ * Called in precopy mode by ram_load().
+ * rcu_read_lock is taken prior to this being called.
+ *
+ * @f: QEMUFile where to send the data
+ */
+static int ram_load_precopy(QEMUFile *f)
 {
-    int flags = 0, ret = 0, invalid_flags = 0;
-    static uint64_t seq_iter;
-    int len = 0;
-    /*
-     * If system is running in postcopy mode, page inserts to host memory must
-     * be atomic
-     */
-    bool postcopy_running = postcopy_is_running();
+    int flags = 0, ret = 0, invalid_flags = 0, len = 0;
     /* ADVISE is earlier, it shows the source has the postcopy capability on */
     bool postcopy_advised = postcopy_is_advised();
-
-    seq_iter++;
-
-    if (version_id != 4) {
-        ret = -EINVAL;
-    }
-
     if (!migrate_use_compression()) {
         invalid_flags |= RAM_SAVE_FLAG_COMPRESS_PAGE;
     }
-    /* This RCU critical section can be very long running.
-     * When RCU reclaims in the code start to become numerous,
-     * it will be necessary to reduce the granularity of this
-     * critical section.
-     */
-    rcu_read_lock();
-
-    if (postcopy_running) {
-        ret = ram_load_postcopy(f);
-    }
 
-    while (!postcopy_running && !ret && !(flags & RAM_SAVE_FLAG_EOS)) {
+    while (!ret && !(flags & RAM_SAVE_FLAG_EOS)) {
         ram_addr_t addr, total_ram_bytes;
         void *host = NULL;
         uint8_t ch;
@@ -4337,12 +4402,6 @@ static int ram_load(QEMUFile *f, void *opaque, int version_id)
                     }
                     if (migrate_ignore_shared()) {
                         hwaddr addr = qemu_get_be64(f);
-                        bool ignored = qemu_get_byte(f);
-                        if (ignored != ramblock_is_ignored(block)) {
-                            error_report("RAM block %s should %s be migrated",
-                                         id, ignored ? "" : "not");
-                            ret = -EINVAL;
-                        }
                         if (ramblock_is_ignored(block) &&
                             block->mr->addr != addr) {
                             error_report("Mismatched GPAs for block %s "
@@ -4409,6 +4468,39 @@ static int ram_load(QEMUFile *f, void *opaque, int version_id)
         }
     }
 
+    return ret;
+}
+
+static int ram_load(QEMUFile *f, void *opaque, int version_id)
+{
+    int ret = 0;
+    static uint64_t seq_iter;
+    /*
+     * If system is running in postcopy mode, page inserts to host memory must
+     * be atomic
+     */
+    bool postcopy_running = postcopy_is_running();
+
+    seq_iter++;
+
+    if (version_id != 4) {
+        return -EINVAL;
+    }
+
+    /*
+     * This RCU critical section can be very long running.
+     * When RCU reclaims in the code start to become numerous,
+     * it will be necessary to reduce the granularity of this
+     * critical section.
+     */
+    rcu_read_lock();
+
+    if (postcopy_running) {
+        ret = ram_load_postcopy(f);
+    } else {
+        ret = ram_load_precopy(f);
+    }
+
     ret |= wait_for_decompress_done();
     rcu_read_unlock();
     trace_ram_load_complete(ret, seq_iter);
This page took 0.051273 seconds and 4 git commands to generate.