diff options
Diffstat (limited to 'migration')
-rw-r--r-- | migration/migration.c | 34 | ||||
-rw-r--r-- | migration/qemu-file.c | 29 | ||||
-rw-r--r-- | migration/ram.c | 1628 | ||||
-rw-r--r-- | migration/rdma.c | 78 | ||||
-rw-r--r-- | migration/savevm.c | 1508 | ||||
-rw-r--r-- | migration/vmstate.c | 21 |
6 files changed, 3244 insertions, 54 deletions
diff --git a/migration/migration.c b/migration/migration.c index 732d229708..b04b4571a8 100644 --- a/migration/migration.c +++ b/migration/migration.c @@ -53,6 +53,7 @@ static bool deferred_incoming; migrations at once. For now we don't need to add dynamic creation of migration */ +/* For outgoing */ MigrationState *migrate_get_current(void) { static MigrationState current_migration = { @@ -71,6 +72,30 @@ MigrationState *migrate_get_current(void) return ¤t_migration; } +/* For incoming */ +static MigrationIncomingState *mis_current; + +MigrationIncomingState *migration_incoming_get_current(void) +{ + return mis_current; +} + +MigrationIncomingState *migration_incoming_state_new(QEMUFile* f) +{ + mis_current = g_malloc0(sizeof(MigrationIncomingState)); + mis_current->file = f; + QLIST_INIT(&mis_current->loadvm_handlers); + + return mis_current; +} + +void migration_incoming_state_destroy(void) +{ + loadvm_free_handlers(mis_current); + g_free(mis_current); + mis_current = NULL; +} + /* * Called on -incoming with a defer: uri. * The migration can be started later after any parameters have been @@ -115,9 +140,14 @@ static void process_incoming_migration_co(void *opaque) Error *local_err = NULL; int ret; + migration_incoming_state_new(f); + ret = qemu_loadvm_state(f); + qemu_fclose(f); free_xbzrle_decoded_buf(); + migration_incoming_state_destroy(); + if (ret < 0) { error_report("load of migration failed: %s", strerror(-ret)); migrate_decompress_threads_join(); @@ -738,6 +768,7 @@ static void *migration_thread(void *opaque) int64_t start_time = initial_time; bool old_vm_running = false; + qemu_savevm_state_header(s->file); qemu_savevm_state_begin(s->file, &s->params); s->setup_time = qemu_clock_get_ms(QEMU_CLOCK_HOST) - setup_start; @@ -838,9 +869,6 @@ static void *migration_thread(void *opaque) void migrate_fd_connect(MigrationState *s) { - s->state = MIGRATION_STATUS_SETUP; - trace_migrate_set_state(MIGRATION_STATUS_SETUP); - /* This is a best 1st approximation. ns to ms */ s->expected_downtime = max_downtime/1000000; s->cleanup_bh = qemu_bh_new(migrate_fd_cleanup, s); diff --git a/migration/qemu-file.c b/migration/qemu-file.c index 2750365a7e..965a757772 100644 --- a/migration/qemu-file.c +++ b/migration/qemu-file.c @@ -349,14 +349,14 @@ void qemu_file_skip(QEMUFile *f, int size) } /* - * Read 'size' bytes from file (at 'offset') into buf without moving the - * pointer. + * Read 'size' bytes from file (at 'offset') without moving the + * pointer and set 'buf' to point to that data. * * It will return size bytes unless there was an error, in which case it will * return as many as it managed to read (assuming blocking fd's which * all current QEMUFile are) */ -int qemu_peek_buffer(QEMUFile *f, uint8_t *buf, int size, size_t offset) +int qemu_peek_buffer(QEMUFile *f, uint8_t **buf, int size, size_t offset) { int pending; int index; @@ -392,7 +392,7 @@ int qemu_peek_buffer(QEMUFile *f, uint8_t *buf, int size, size_t offset) size = pending; } - memcpy(buf, f->buf + index, size); + *buf = f->buf + index; return size; } @@ -411,11 +411,13 @@ int qemu_get_buffer(QEMUFile *f, uint8_t *buf, int size) while (pending > 0) { int res; + uint8_t *src; - res = qemu_peek_buffer(f, buf, MIN(pending, IO_BUF_SIZE), 0); + res = qemu_peek_buffer(f, &src, MIN(pending, IO_BUF_SIZE), 0); if (res == 0) { return done; } + memcpy(buf, src, res); qemu_file_skip(f, res); buf += res; pending -= res; @@ -585,3 +587,20 @@ int qemu_put_qemu_file(QEMUFile *f_des, QEMUFile *f_src) } return len; } + +/* + * Get a string whose length is determined by a single preceding byte + * A preallocated 256 byte buffer must be passed in. + * Returns: len on success and a 0 terminated string in the buffer + * else 0 + * (Note a 0 length string will return 0 either way) + */ +size_t qemu_get_counted_string(QEMUFile *f, char buf[256]) +{ + size_t len = qemu_get_byte(f); + size_t res = qemu_get_buffer(f, (uint8_t *)buf, len); + + buf[res] = 0; + + return res == len ? res : 0; +} diff --git a/migration/ram.c b/migration/ram.c new file mode 100644 index 0000000000..57368e1575 --- /dev/null +++ b/migration/ram.c @@ -0,0 +1,1628 @@ +/* + * QEMU System Emulator + * + * Copyright (c) 2003-2008 Fabrice Bellard + * Copyright (c) 2011-2015 Red Hat Inc + * + * Authors: + * Juan Quintela <quintela@redhat.com> + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL + * THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + */ +#include <stdint.h> +#include <zlib.h> +#include "qemu/bitops.h" +#include "qemu/bitmap.h" +#include "qemu/timer.h" +#include "qemu/main-loop.h" +#include "migration/migration.h" +#include "exec/address-spaces.h" +#include "migration/page_cache.h" +#include "qemu/error-report.h" +#include "trace.h" +#include "exec/ram_addr.h" +#include "qemu/rcu_queue.h" + +#ifdef DEBUG_MIGRATION_RAM +#define DPRINTF(fmt, ...) \ + do { fprintf(stdout, "migration_ram: " fmt, ## __VA_ARGS__); } while (0) +#else +#define DPRINTF(fmt, ...) \ + do { } while (0) +#endif + +static bool mig_throttle_on; +static int dirty_rate_high_cnt; +static void check_guest_throttling(void); + +static uint64_t bitmap_sync_count; + +/***********************************************************/ +/* ram save/restore */ + +#define RAM_SAVE_FLAG_FULL 0x01 /* Obsolete, not used anymore */ +#define RAM_SAVE_FLAG_COMPRESS 0x02 +#define RAM_SAVE_FLAG_MEM_SIZE 0x04 +#define RAM_SAVE_FLAG_PAGE 0x08 +#define RAM_SAVE_FLAG_EOS 0x10 +#define RAM_SAVE_FLAG_CONTINUE 0x20 +#define RAM_SAVE_FLAG_XBZRLE 0x40 +/* 0x80 is reserved in migration.h start with 0x100 next */ +#define RAM_SAVE_FLAG_COMPRESS_PAGE 0x100 + +static const uint8_t ZERO_TARGET_PAGE[TARGET_PAGE_SIZE]; + +static inline bool is_zero_range(uint8_t *p, uint64_t size) +{ + return buffer_find_nonzero_offset(p, size) == size; +} + +/* struct contains XBZRLE cache and a static page + used by the compression */ +static struct { + /* buffer used for XBZRLE encoding */ + uint8_t *encoded_buf; + /* buffer for storing page content */ + uint8_t *current_buf; + /* Cache for XBZRLE, Protected by lock. */ + PageCache *cache; + QemuMutex lock; +} XBZRLE; + +/* buffer used for XBZRLE decoding */ +static uint8_t *xbzrle_decoded_buf; + +static void XBZRLE_cache_lock(void) +{ + if (migrate_use_xbzrle()) + qemu_mutex_lock(&XBZRLE.lock); +} + +static void XBZRLE_cache_unlock(void) +{ + if (migrate_use_xbzrle()) + qemu_mutex_unlock(&XBZRLE.lock); +} + +/* + * called from qmp_migrate_set_cache_size in main thread, possibly while + * a migration is in progress. + * A running migration maybe using the cache and might finish during this + * call, hence changes to the cache are protected by XBZRLE.lock(). + */ +int64_t xbzrle_cache_resize(int64_t new_size) +{ + PageCache *new_cache; + int64_t ret; + + if (new_size < TARGET_PAGE_SIZE) { + return -1; + } + + XBZRLE_cache_lock(); + + if (XBZRLE.cache != NULL) { + if (pow2floor(new_size) == migrate_xbzrle_cache_size()) { + goto out_new_size; + } + new_cache = cache_init(new_size / TARGET_PAGE_SIZE, + TARGET_PAGE_SIZE); + if (!new_cache) { + error_report("Error creating cache"); + ret = -1; + goto out; + } + + cache_fini(XBZRLE.cache); + XBZRLE.cache = new_cache; + } + +out_new_size: + ret = pow2floor(new_size); +out: + XBZRLE_cache_unlock(); + return ret; +} + +/* accounting for migration statistics */ +typedef struct AccountingInfo { + uint64_t dup_pages; + uint64_t skipped_pages; + uint64_t norm_pages; + uint64_t iterations; + uint64_t xbzrle_bytes; + uint64_t xbzrle_pages; + uint64_t xbzrle_cache_miss; + double xbzrle_cache_miss_rate; + uint64_t xbzrle_overflows; +} AccountingInfo; + +static AccountingInfo acct_info; + +static void acct_clear(void) +{ + memset(&acct_info, 0, sizeof(acct_info)); +} + +uint64_t dup_mig_bytes_transferred(void) +{ + return acct_info.dup_pages * TARGET_PAGE_SIZE; +} + +uint64_t dup_mig_pages_transferred(void) +{ + return acct_info.dup_pages; +} + +uint64_t skipped_mig_bytes_transferred(void) +{ + return acct_info.skipped_pages * TARGET_PAGE_SIZE; +} + +uint64_t skipped_mig_pages_transferred(void) +{ + return acct_info.skipped_pages; +} + +uint64_t norm_mig_bytes_transferred(void) +{ + return acct_info.norm_pages * TARGET_PAGE_SIZE; +} + +uint64_t norm_mig_pages_transferred(void) +{ + return acct_info.norm_pages; +} + +uint64_t xbzrle_mig_bytes_transferred(void) +{ + return acct_info.xbzrle_bytes; +} + +uint64_t xbzrle_mig_pages_transferred(void) +{ + return acct_info.xbzrle_pages; +} + +uint64_t xbzrle_mig_pages_cache_miss(void) +{ + return acct_info.xbzrle_cache_miss; +} + +double xbzrle_mig_cache_miss_rate(void) +{ + return acct_info.xbzrle_cache_miss_rate; +} + +uint64_t xbzrle_mig_pages_overflow(void) +{ + return acct_info.xbzrle_overflows; +} + +/* This is the last block that we have visited serching for dirty pages + */ +static RAMBlock *last_seen_block; +/* This is the last block from where we have sent data */ +static RAMBlock *last_sent_block; +static ram_addr_t last_offset; +static unsigned long *migration_bitmap; +static uint64_t migration_dirty_pages; +static uint32_t last_version; +static bool ram_bulk_stage; + +struct CompressParam { + bool start; + bool done; + QEMUFile *file; + QemuMutex mutex; + QemuCond cond; + RAMBlock *block; + ram_addr_t offset; +}; +typedef struct CompressParam CompressParam; + +struct DecompressParam { + bool start; + QemuMutex mutex; + QemuCond cond; + void *des; + uint8 *compbuf; + int len; +}; +typedef struct DecompressParam DecompressParam; + +static CompressParam *comp_param; +static QemuThread *compress_threads; +/* comp_done_cond is used to wake up the migration thread when + * one of the compression threads has finished the compression. + * comp_done_lock is used to co-work with comp_done_cond. + */ +static QemuMutex *comp_done_lock; +static QemuCond *comp_done_cond; +/* The empty QEMUFileOps will be used by file in CompressParam */ +static const QEMUFileOps empty_ops = { }; + +static bool compression_switch; +static bool quit_comp_thread; +static bool quit_decomp_thread; +static DecompressParam *decomp_param; +static QemuThread *decompress_threads; +static uint8_t *compressed_data_buf; + +static int do_compress_ram_page(CompressParam *param); + +static void *do_data_compress(void *opaque) +{ + CompressParam *param = opaque; + + while (!quit_comp_thread) { + qemu_mutex_lock(¶m->mutex); + /* Re-check the quit_comp_thread in case of + * terminate_compression_threads is called just before + * qemu_mutex_lock(¶m->mutex) and after + * while(!quit_comp_thread), re-check it here can make + * sure the compression thread terminate as expected. + */ + while (!param->start && !quit_comp_thread) { + qemu_cond_wait(¶m->cond, ¶m->mutex); + } + if (!quit_comp_thread) { + do_compress_ram_page(param); + } + param->start = false; + qemu_mutex_unlock(¶m->mutex); + + qemu_mutex_lock(comp_done_lock); + param->done = true; + qemu_cond_signal(comp_done_cond); + qemu_mutex_unlock(comp_done_lock); + } + + return NULL; +} + +static inline void terminate_compression_threads(void) +{ + int idx, thread_count; + + thread_count = migrate_compress_threads(); + quit_comp_thread = true; + for (idx = 0; idx < thread_count; idx++) { + qemu_mutex_lock(&comp_param[idx].mutex); + qemu_cond_signal(&comp_param[idx].cond); + qemu_mutex_unlock(&comp_param[idx].mutex); + } +} + +void migrate_compress_threads_join(void) +{ + int i, thread_count; + + if (!migrate_use_compression()) { + return; + } + terminate_compression_threads(); + thread_count = migrate_compress_threads(); + for (i = 0; i < thread_count; i++) { + qemu_thread_join(compress_threads + i); + qemu_fclose(comp_param[i].file); + qemu_mutex_destroy(&comp_param[i].mutex); + qemu_cond_destroy(&comp_param[i].cond); + } + qemu_mutex_destroy(comp_done_lock); + qemu_cond_destroy(comp_done_cond); + g_free(compress_threads); + g_free(comp_param); + g_free(comp_done_cond); + g_free(comp_done_lock); + compress_threads = NULL; + comp_param = NULL; + comp_done_cond = NULL; + comp_done_lock = NULL; +} + +void migrate_compress_threads_create(void) +{ + int i, thread_count; + + if (!migrate_use_compression()) { + return; + } + quit_comp_thread = false; + compression_switch = true; + thread_count = migrate_compress_threads(); + compress_threads = g_new0(QemuThread, thread_count); + comp_param = g_new0(CompressParam, thread_count); + comp_done_cond = g_new0(QemuCond, 1); + comp_done_lock = g_new0(QemuMutex, 1); + qemu_cond_init(comp_done_cond); + qemu_mutex_init(comp_done_lock); + for (i = 0; i < thread_count; i++) { + /* com_param[i].file is just used as a dummy buffer to save data, set + * it's ops to empty. + */ + comp_param[i].file = qemu_fopen_ops(NULL, &empty_ops); + comp_param[i].done = true; + qemu_mutex_init(&comp_param[i].mutex); + qemu_cond_init(&comp_param[i].cond); + qemu_thread_create(compress_threads + i, "compress", + do_data_compress, comp_param + i, + QEMU_THREAD_JOINABLE); + } +} + +/** + * save_page_header: Write page header to wire + * + * If this is the 1st block, it also writes the block identification + * + * Returns: Number of bytes written + * + * @f: QEMUFile where to send the data + * @block: block that contains the page we want to send + * @offset: offset inside the block for the page + * in the lower bits, it contains flags + */ +static size_t save_page_header(QEMUFile *f, RAMBlock *block, ram_addr_t offset) +{ + size_t size; + + qemu_put_be64(f, offset); + size = 8; + + if (!(offset & RAM_SAVE_FLAG_CONTINUE)) { + qemu_put_byte(f, strlen(block->idstr)); + qemu_put_buffer(f, (uint8_t *)block->idstr, + strlen(block->idstr)); + size += 1 + strlen(block->idstr); + } + return size; +} + +/* Update the xbzrle cache to reflect a page that's been sent as all 0. + * The important thing is that a stale (not-yet-0'd) page be replaced + * by the new data. + * As a bonus, if the page wasn't in the cache it gets added so that + * when a small write is made into the 0'd page it gets XBZRLE sent + */ +static void xbzrle_cache_zero_page(ram_addr_t current_addr) +{ + if (ram_bulk_stage || !migrate_use_xbzrle()) { + return; + } + + /* We don't care if this fails to allocate a new cache page + * as long as it updated an old one */ + cache_insert(XBZRLE.cache, current_addr, ZERO_TARGET_PAGE, + bitmap_sync_count); +} + +#define ENCODING_FLAG_XBZRLE 0x1 + +/** + * save_xbzrle_page: compress and send current page + * + * Returns: 1 means that we wrote the page + * 0 means that page is identical to the one already sent + * -1 means that xbzrle would be longer than normal + * + * @f: QEMUFile where to send the data + * @current_data: + * @current_addr: + * @block: block that contains the page we want to send + * @offset: offset inside the block for the page + * @last_stage: if we are at the completion stage + * @bytes_transferred: increase it with the number of transferred bytes + */ +static int save_xbzrle_page(QEMUFile *f, uint8_t **current_data, + ram_addr_t current_addr, RAMBlock *block, + ram_addr_t offset, bool last_stage, + uint64_t *bytes_transferred) +{ + int encoded_len = 0, bytes_xbzrle; + uint8_t *prev_cached_page; + + if (!cache_is_cached(XBZRLE.cache, current_addr, bitmap_sync_count)) { + acct_info.xbzrle_cache_miss++; + if (!last_stage) { + if (cache_insert(XBZRLE.cache, current_addr, *current_data, + bitmap_sync_count) == -1) { + return -1; + } else { + /* update *current_data when the page has been + inserted into cache */ + *current_data = get_cached_data(XBZRLE.cache, current_addr); + } + } + return -1; + } + + prev_cached_page = get_cached_data(XBZRLE.cache, current_addr); + + /* save current buffer into memory */ + memcpy(XBZRLE.current_buf, *current_data, TARGET_PAGE_SIZE); + + /* XBZRLE encoding (if there is no overflow) */ + encoded_len = xbzrle_encode_buffer(prev_cached_page, XBZRLE.current_buf, + TARGET_PAGE_SIZE, XBZRLE.encoded_buf, + TARGET_PAGE_SIZE); + if (encoded_len == 0) { + DPRINTF("Skipping unmodified page\n"); + return 0; + } else if (encoded_len == -1) { + DPRINTF("Overflow\n"); + acct_info.xbzrle_overflows++; + /* 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(f, block, offset | RAM_SAVE_FLAG_XBZRLE); + qemu_put_byte(f, ENCODING_FLAG_XBZRLE); + qemu_put_be16(f, encoded_len); + qemu_put_buffer(f, XBZRLE.encoded_buf, encoded_len); + bytes_xbzrle += encoded_len + 1 + 2; + acct_info.xbzrle_pages++; + acct_info.xbzrle_bytes += bytes_xbzrle; + *bytes_transferred += bytes_xbzrle; + + return 1; +} + +static inline +ram_addr_t migration_bitmap_find_and_reset_dirty(MemoryRegion *mr, + ram_addr_t start) +{ + unsigned long base = mr->ram_addr >> TARGET_PAGE_BITS; + unsigned long nr = base + (start >> TARGET_PAGE_BITS); + uint64_t mr_size = TARGET_PAGE_ALIGN(memory_region_size(mr)); + unsigned long size = base + (mr_size >> TARGET_PAGE_BITS); + + unsigned long next; + + if (ram_bulk_stage && nr > base) { + next = nr + 1; + } else { + next = find_next_bit(migration_bitmap, size, nr); + } + + if (next < size) { + clear_bit(next, migration_bitmap); + migration_dirty_pages--; + } + return (next - base) << TARGET_PAGE_BITS; +} + +static void migration_bitmap_sync_range(ram_addr_t start, ram_addr_t length) +{ + migration_dirty_pages += + cpu_physical_memory_sync_dirty_bitmap(migration_bitmap, start, length); +} + + +/* Fix me: there are too many global variables used in migration process. */ +static int64_t start_time; +static int64_t bytes_xfer_prev; +static int64_t num_dirty_pages_period; +static uint64_t xbzrle_cache_miss_prev; +static uint64_t iterations_prev; + +static void migration_bitmap_sync_init(void) +{ + start_time = 0; + bytes_xfer_prev = 0; + num_dirty_pages_period = 0; + xbzrle_cache_miss_prev = 0; + iterations_prev = 0; +} + +/* Called with iothread lock held, to protect ram_list.dirty_memory[] */ +static void migration_bitmap_sync(void) +{ + RAMBlock *block; + uint64_t num_dirty_pages_init = migration_dirty_pages; + MigrationState *s = migrate_get_current(); + int64_t end_time; + int64_t bytes_xfer_now; + + bitmap_sync_count++; + + if (!bytes_xfer_prev) { + bytes_xfer_prev = ram_bytes_transferred(); + } + + if (!start_time) { + start_time = qemu_clock_get_ms(QEMU_CLOCK_REALTIME); + } + + trace_migration_bitmap_sync_start(); + address_space_sync_dirty_bitmap(&address_space_memory); + + rcu_read_lock(); + QLIST_FOREACH_RCU(block, &ram_list.blocks, next) { + migration_bitmap_sync_range(block->mr->ram_addr, block->used_length); + } + rcu_read_unlock(); + + trace_migration_bitmap_sync_end(migration_dirty_pages + - num_dirty_pages_init); + num_dirty_pages_period += migration_dirty_pages - num_dirty_pages_init; + end_time = qemu_clock_get_ms(QEMU_CLOCK_REALTIME); + + /* more than 1 second = 1000 millisecons */ + if (end_time > start_time + 1000) { + if (migrate_auto_converge()) { + /* The following detection logic can be refined later. For now: + Check to see if the dirtied bytes is 50% more than the approx. + amount of bytes that just got transferred since the last time we + were in this routine. If that happens >N times (for now N==4) + we turn on the throttle down logic */ + bytes_xfer_now = ram_bytes_transferred(); + if (s->dirty_pages_rate && + (num_dirty_pages_period * TARGET_PAGE_SIZE > + (bytes_xfer_now - bytes_xfer_prev)/2) && + (dirty_rate_high_cnt++ > 4)) { + trace_migration_throttle(); + mig_throttle_on = true; + dirty_rate_high_cnt = 0; + } + bytes_xfer_prev = bytes_xfer_now; + } else { + mig_throttle_on = false; + } + if (migrate_use_xbzrle()) { + if (iterations_prev != acct_info.iterations) { + acct_info.xbzrle_cache_miss_rate = + (double)(acct_info.xbzrle_cache_miss - + xbzrle_cache_miss_prev) / + (acct_info.iterations - iterations_prev); + } + iterations_prev = acct_info.iterations; + xbzrle_cache_miss_prev = acct_info.xbzrle_cache_miss; + } + s->dirty_pages_rate = num_dirty_pages_period * 1000 + / (end_time - start_time); + s->dirty_bytes_rate = s->dirty_pages_rate * TARGET_PAGE_SIZE; + start_time = end_time; + num_dirty_pages_period = 0; + } + s->dirty_sync_count = bitmap_sync_count; +} + +/** + * save_zero_page: Send the zero page to the stream + * + * Returns: Number of pages written. + * + * @f: QEMUFile where to send the data + * @block: block that contains the page we want to send + * @offset: offset inside the block for the page + * @p: pointer to the page + * @bytes_transferred: increase it with the number of transferred bytes + */ +static int save_zero_page(QEMUFile *f, RAMBlock *block, ram_addr_t offset, + uint8_t *p, uint64_t *bytes_transferred) +{ + int pages = -1; + + if (is_zero_range(p, TARGET_PAGE_SIZE)) { + acct_info.dup_pages++; + *bytes_transferred += save_page_header(f, block, + offset | RAM_SAVE_FLAG_COMPRESS); + qemu_put_byte(f, 0); + *bytes_transferred += 1; + pages = 1; + } + + return pages; +} + +/** + * ram_save_page: Send the given page to the stream + * + * Returns: Number of pages written. + * + * @f: QEMUFile where to send the data + * @block: block that contains the page we want to send + * @offset: offset inside the block for the page + * @last_stage: if we are at the completion stage + * @bytes_transferred: increase it with the number of transferred bytes + */ +static int ram_save_page(QEMUFile *f, RAMBlock* block, ram_addr_t offset, + bool last_stage, uint64_t *bytes_transferred) +{ + int pages = -1; + uint64_t bytes_xmit; + ram_addr_t current_addr; + MemoryRegion *mr = block->mr; + uint8_t *p; + int ret; + bool send_async = true; + + p = memory_region_get_ram_ptr(mr) + offset; + + /* In doubt sent page as normal */ + bytes_xmit = 0; + ret = ram_control_save_page(f, block->offset, + offset, TARGET_PAGE_SIZE, &bytes_xmit); + if (bytes_xmit) { + *bytes_transferred += bytes_xmit; + pages = 1; + } + + XBZRLE_cache_lock(); + + current_addr = block->offset + offset; + + if (block == last_sent_block) { + offset |= RAM_SAVE_FLAG_CONTINUE; + } + if (ret != RAM_SAVE_CONTROL_NOT_SUPP) { + if (ret != RAM_SAVE_CONTROL_DELAYED) { + if (bytes_xmit > 0) { + acct_info.norm_pages++; + } else if (bytes_xmit == 0) { + acct_info.dup_pages++; + } + } + } else { + pages = save_zero_page(f, block, offset, p, bytes_transferred); + if (pages > 0) { + /* Must let xbzrle know, otherwise a previous (now 0'd) cached + * page would be stale + */ + xbzrle_cache_zero_page(current_addr); + } else if (!ram_bulk_stage && migrate_use_xbzrle()) { + pages = save_xbzrle_page(f, &p, current_addr, block, + offset, last_stage, bytes_transferred); + if (!last_stage) { + /* Can't send this cached data async, since the cache page + * might get updated before it gets to the wire + */ + send_async = false; + } + } + } + + /* XBZRLE overflow or normal page */ + if (pages == -1) { + *bytes_transferred += save_page_header(f, block, + offset | RAM_SAVE_FLAG_PAGE); + if (send_async) { + qemu_put_buffer_async(f, p, TARGET_PAGE_SIZE); + } else { + qemu_put_buffer(f, p, TARGET_PAGE_SIZE); + } + *bytes_transferred += TARGET_PAGE_SIZE; + pages = 1; + acct_info.norm_pages++; + } + + XBZRLE_cache_unlock(); + + return pages; +} + +static int do_compress_ram_page(CompressParam *param) +{ + int bytes_sent, blen; + uint8_t *p; + RAMBlock *block = param->block; + ram_addr_t offset = param->offset; + + p = memory_region_get_ram_ptr(block->mr) + (offset & TARGET_PAGE_MASK); + + bytes_sent = save_page_header(param->file, block, offset | + RAM_SAVE_FLAG_COMPRESS_PAGE); + blen = qemu_put_compression_data(param->file, p, TARGET_PAGE_SIZE, + migrate_compress_level()); + bytes_sent += blen; + + return bytes_sent; +} + +static inline void start_compression(CompressParam *param) +{ + param->done = false; + qemu_mutex_lock(¶m->mutex); + param->start = true; + qemu_cond_signal(¶m->cond); + qemu_mutex_unlock(¶m->mutex); +} + +static inline void start_decompression(DecompressParam *param) +{ + qemu_mutex_lock(¶m->mutex); + param->start = true; + qemu_cond_signal(¶m->cond); + qemu_mutex_unlock(¶m->mutex); +} + +static uint64_t bytes_transferred; + +static void flush_compressed_data(QEMUFile *f) +{ + int idx, len, thread_count; + + if (!migrate_use_compression()) { + return; + } + thread_count = migrate_compress_threads(); + for (idx = 0; idx < thread_count; idx++) { + if (!comp_param[idx].done) { + qemu_mutex_lock(comp_done_lock); + while (!comp_param[idx].done && !quit_comp_thread) { + qemu_cond_wait(comp_done_cond, comp_done_lock); + } + qemu_mutex_unlock(comp_done_lock); + } + if (!quit_comp_thread) { + len = qemu_put_qemu_file(f, comp_param[idx].file); + bytes_transferred += len; + } + } +} + +static inline void set_compress_params(CompressParam *param, RAMBlock *block, + ram_addr_t offset) +{ + param->block = block; + param->offset = offset; +} + +static int compress_page_with_multi_thread(QEMUFile *f, RAMBlock *block, + ram_addr_t offset, + uint64_t *bytes_transferred) +{ + int idx, thread_count, bytes_xmit = -1, pages = -1; + + thread_count = migrate_compress_threads(); + qemu_mutex_lock(comp_done_lock); + while (true) { + for (idx = 0; idx < thread_count; idx++) { + if (comp_param[idx].done) { + bytes_xmit = qemu_put_qemu_file(f, comp_param[idx].file); + set_compress_params(&comp_param[idx], block, offset); + start_compression(&comp_param[idx]); + pages = 1; + acct_info.norm_pages++; + *bytes_transferred += bytes_xmit; + break; + } + } + if (pages > 0) { + break; + } else { + qemu_cond_wait(comp_done_cond, comp_done_lock); + } + } + qemu_mutex_unlock(comp_done_lock); + + return pages; +} + +/** + * ram_save_compressed_page: compress the given page and send it to the stream + * + * Returns: Number of pages written. + * + * @f: QEMUFile where to send the data + * @block: block that contains the page we want to send + * @offset: offset inside the block for the page + * @last_stage: if we are at the completion stage + * @bytes_transferred: increase it with the number of transferred bytes + */ +static int ram_save_compressed_page(QEMUFile *f, RAMBlock *block, + ram_addr_t offset, bool last_stage, + uint64_t *bytes_transferred) +{ + int pages = -1; + uint64_t bytes_xmit; + MemoryRegion *mr = block->mr; + uint8_t *p; + int ret; + + p = memory_region_get_ram_ptr(mr) + offset; + + bytes_xmit = 0; + ret = ram_control_save_page(f, block->offset, + offset, TARGET_PAGE_SIZE, &bytes_xmit); + if (bytes_xmit) { + *bytes_transferred += bytes_xmit; + pages = 1; + } + if (block == last_sent_block) { + offset |= RAM_SAVE_FLAG_CONTINUE; + } + if (ret != RAM_SAVE_CONTROL_NOT_SUPP) { + if (ret != RAM_SAVE_CONTROL_DELAYED) { + if (bytes_xmit > 0) { + acct_info.norm_pages++; + } else if (bytes_xmit == 0) { + acct_info.dup_pages++; + } + } + } else { + /* When starting the process of a new block, the first page of + * the block should be sent out before other pages in the same + * block, and all the pages in last block should have been sent + * out, keeping this order is important, because the 'cont' flag + * is used to avoid resending the block name. + */ + if (block != last_sent_block) { + flush_compressed_data(f); + pages = save_zero_page(f, block, offset, p, bytes_transferred); + if (pages == -1) { + set_compress_params(&comp_param[0], block, offset); + /* Use the qemu thread to compress the data to make sure the + * first page is sent out before other pages + */ + bytes_xmit = do_compress_ram_page(&comp_param[0]); + acct_info.norm_pages++; + qemu_put_qemu_file(f, comp_param[0].file); + *bytes_transferred += bytes_xmit; + pages = 1; + } + } else { + pages = save_zero_page(f, block, offset, p, bytes_transferred); + if (pages == -1) { + pages = compress_page_with_multi_thread(f, block, offset, + bytes_transferred); + } + } + } + + return pages; +} + +/** + * ram_find_and_save_block: Finds a dirty page and sends it to f + * + * Called within an RCU critical section. + * + * Returns: The number of pages written + * 0 means no dirty pages + * + * @f: QEMUFile where to send the data + * @last_stage: if we are at the completion stage + * @bytes_transferred: increase it with the number of transferred bytes + */ + +static int ram_find_and_save_block(QEMUFile *f, bool last_stage, + uint64_t *bytes_transferred) +{ + RAMBlock *block = last_seen_block; + ram_addr_t offset = last_offset; + bool complete_round = false; + int pages = 0; + MemoryRegion *mr; + + if (!block) + block = QLIST_FIRST_RCU(&ram_list.blocks); + + while (true) { + mr = block->mr; + offset = migration_bitmap_find_and_reset_dirty(mr, offset); + if (complete_round && block == last_seen_block && + offset >= last_offset) { + break; + } + if (offset >= block->used_length) { + offset = 0; + block = QLIST_NEXT_RCU(block, next); + if (!block) { + block = QLIST_FIRST_RCU(&ram_list.blocks); + complete_round = true; + ram_bulk_stage = false; + if (migrate_use_xbzrle()) { + /* If xbzrle is on, stop using the data compression at this + * point. In theory, xbzrle can do better than compression. + */ + flush_compressed_data(f); + compression_switch = false; + } + } + } else { + if (compression_switch && migrate_use_compression()) { + pages = ram_save_compressed_page(f, block, offset, last_stage, + bytes_transferred); + } else { + pages = ram_save_page(f, block, offset, last_stage, + bytes_transferred); + } + + /* if page is unmodified, continue to the next */ + if (pages > 0) { + last_sent_block = block; + break; + } + } + } + + last_seen_block = block; + last_offset = offset; + + return pages; +} + +void acct_update_position(QEMUFile *f, size_t size, bool zero) +{ + uint64_t pages = size / TARGET_PAGE_SIZE; + if (zero) { + acct_info.dup_pages += pages; + } else { + acct_info.norm_pages += pages; + bytes_transferred += size; + qemu_update_position(f, size); + } +} + +static ram_addr_t ram_save_remaining(void) +{ + return migration_dirty_pages; +} + +uint64_t ram_bytes_remaining(void) +{ + return ram_save_remaining() * TARGET_PAGE_SIZE; +} + +uint64_t ram_bytes_transferred(void) +{ + return bytes_transferred; +} + +uint64_t ram_bytes_total(void) +{ + RAMBlock *block; + uint64_t total = 0; + + rcu_read_lock(); + QLIST_FOREACH_RCU(block, &ram_list.blocks, next) + total += block->used_length; + rcu_read_unlock(); + return total; +} + +void free_xbzrle_decoded_buf(void) +{ + g_free(xbzrle_decoded_buf); + xbzrle_decoded_buf = NULL; +} + +static void migration_end(void) +{ + if (migration_bitmap) { + memory_global_dirty_log_stop(); + g_free(migration_bitmap); + migration_bitmap = NULL; + } + + XBZRLE_cache_lock(); + if (XBZRLE.cache) { + cache_fini(XBZRLE.cache); + g_free(XBZRLE.encoded_buf); + g_free(XBZRLE.current_buf); + XBZRLE.cache = NULL; + XBZRLE.encoded_buf = NULL; + XBZRLE.current_buf = NULL; + } + XBZRLE_cache_unlock(); +} + +static void ram_migration_cancel(void *opaque) +{ + migration_end(); +} + +static void reset_ram_globals(void) +{ + last_seen_block = NULL; + last_sent_block = NULL; + last_offset = 0; + last_version = ram_list.version; + ram_bulk_stage = true; +} + +#define MAX_WAIT 50 /* ms, half buffered_file limit */ + + +/* Each of ram_save_setup, ram_save_iterate and ram_save_complete has + * long-running RCU critical section. When rcu-reclaims in the code + * start to become numerous it will be necessary to reduce the + * granularity of these critical sections. + */ + +static int ram_save_setup(QEMUFile *f, void *opaque) +{ + RAMBlock *block; + int64_t ram_bitmap_pages; /* Size of bitmap in pages, including gaps */ + + mig_throttle_on = false; + dirty_rate_high_cnt = 0; + bitmap_sync_count = 0; + migration_bitmap_sync_init(); + + if (migrate_use_xbzrle()) { + XBZRLE_cache_lock(); + XBZRLE.cache = cache_init(migrate_xbzrle_cache_size() / + TARGET_PAGE_SIZE, + TARGET_PAGE_SIZE); + if (!XBZRLE.cache) { + XBZRLE_cache_unlock(); + error_report("Error creating cache"); + return -1; + } + XBZRLE_cache_unlock(); + + /* We prefer not to abort if there is no memory */ + XBZRLE.encoded_buf = g_try_malloc0(TARGET_PAGE_SIZE); + if (!XBZRLE.encoded_buf) { + error_report("Error allocating encoded_buf"); + return -1; + } + + XBZRLE.current_buf = g_try_malloc(TARGET_PAGE_SIZE); + if (!XBZRLE.current_buf) { + error_report("Error allocating current_buf"); + g_free(XBZRLE.encoded_buf); + XBZRLE.encoded_buf = NULL; + return -1; + } + + acct_clear(); + } + + /* iothread lock needed for ram_list.dirty_memory[] */ + qemu_mutex_lock_iothread(); + qemu_mutex_lock_ramlist(); + rcu_read_lock(); + bytes_transferred = 0; + reset_ram_globals(); + + ram_bitmap_pages = last_ram_offset() >> TARGET_PAGE_BITS; + migration_bitmap = bitmap_new(ram_bitmap_pages); + bitmap_set(migration_bitmap, 0, ram_bitmap_pages); + + /* + * Count the total number of pages used by ram blocks not including any + * gaps due to alignment or unplugs. + */ + migration_dirty_pages = ram_bytes_total() >> TARGET_PAGE_BITS; + + memory_global_dirty_log_start(); + migration_bitmap_sync(); + qemu_mutex_unlock_ramlist(); + qemu_mutex_unlock_iothread(); + + qemu_put_be64(f, ram_bytes_total() | RAM_SAVE_FLAG_MEM_SIZE); + + QLIST_FOREACH_RCU(block, &ram_list.blocks, next) { + qemu_put_byte(f, strlen(block->idstr)); + qemu_put_buffer(f, (uint8_t *)block->idstr, strlen(block->idstr)); + qemu_put_be64(f, block->used_length); + } + + rcu_read_unlock(); + + ram_control_before_iterate(f, RAM_CONTROL_SETUP); + ram_control_after_iterate(f, RAM_CONTROL_SETUP); + + qemu_put_be64(f, RAM_SAVE_FLAG_EOS); + + return 0; +} + +static int ram_save_iterate(QEMUFile *f, void *opaque) +{ + int ret; + int i; + int64_t t0; + int pages_sent = 0; + + rcu_read_lock(); + if (ram_list.version != last_version) { + reset_ram_globals(); + } + + /* Read version before ram_list.blocks */ + smp_rmb(); + + ram_control_before_iterate(f, RAM_CONTROL_ROUND); + + t0 = qemu_clock_get_ns(QEMU_CLOCK_REALTIME); + i = 0; + while ((ret = qemu_file_rate_limit(f)) == 0) { + int pages; + + pages = ram_find_and_save_block(f, false, &bytes_transferred); + /* no more pages to sent */ + if (pages == 0) { + break; + } + pages_sent += pages; + acct_info.iterations++; + check_guest_throttling(); + /* we want to check in the 1st loop, just in case it was the 1st time + and we had to sync the dirty bitmap. + qemu_get_clock_ns() is a bit expensive, so we only check each some + iterations + */ + if ((i & 63) == 0) { + uint64_t t1 = (qemu_clock_get_ns(QEMU_CLOCK_REALTIME) - t0) / 1000000; + if (t1 > MAX_WAIT) { + DPRINTF("big wait: %" PRIu64 " milliseconds, %d iterations\n", + t1, i); + break; + } + } + i++; + } + flush_compressed_data(f); + rcu_read_unlock(); + + /* + * Must occur before EOS (or any QEMUFile operation) + * because of RDMA protocol. + */ + ram_control_after_iterate(f, RAM_CONTROL_ROUND); + + qemu_put_be64(f, RAM_SAVE_FLAG_EOS); + bytes_transferred += 8; + + ret = qemu_file_get_error(f); + if (ret < 0) { + return ret; + } + + return pages_sent; +} + +/* Called with iothread lock */ +static int ram_save_complete(QEMUFile *f, void *opaque) +{ + rcu_read_lock(); + + migration_bitmap_sync(); + + ram_control_before_iterate(f, RAM_CONTROL_FINISH); + + /* try transferring iterative blocks of memory */ + + /* flush all remaining blocks regardless of rate limiting */ + while (true) { + int pages; + + pages = ram_find_and_save_block(f, true, &bytes_transferred); + /* no more blocks to sent */ + if (pages == 0) { + break; + } + } + + flush_compressed_data(f); + ram_control_after_iterate(f, RAM_CONTROL_FINISH); + migration_end(); + + rcu_read_unlock(); + qemu_put_be64(f, RAM_SAVE_FLAG_EOS); + + return 0; +} + +static uint64_t ram_save_pending(QEMUFile *f, void *opaque, uint64_t max_size) +{ + uint64_t remaining_size; + + remaining_size = ram_save_remaining() * TARGET_PAGE_SIZE; + + if (remaining_size < max_size) { + qemu_mutex_lock_iothread(); + rcu_read_lock(); + migration_bitmap_sync(); + rcu_read_unlock(); + qemu_mutex_unlock_iothread(); + remaining_size = ram_save_remaining() * TARGET_PAGE_SIZE; + } + return remaining_size; +} + +static int load_xbzrle(QEMUFile *f, ram_addr_t addr, void *host) +{ + unsigned int xh_len; + int xh_flags; + + if (!xbzrle_decoded_buf) { + xbzrle_decoded_buf = g_malloc(TARGET_PAGE_SIZE); + } + + /* extract RLE header */ + xh_flags = qemu_get_byte(f); + xh_len = qemu_get_be16(f); + + if (xh_flags != ENCODING_FLAG_XBZRLE) { + error_report("Failed to load XBZRLE page - wrong compression!"); + return -1; + } + + if (xh_len > TARGET_PAGE_SIZE) { + error_report("Failed to load XBZRLE page - len overflow!"); + return -1; + } + /* load data and decode */ + qemu_get_buffer(f, xbzrle_decoded_buf, xh_len); + + /* decode RLE */ + if (xbzrle_decode_buffer(xbzrle_decoded_buf, xh_len, host, + TARGET_PAGE_SIZE) == -1) { + error_report("Failed to load XBZRLE page - decode error!"); + return -1; + } + + return 0; +} + +/* Must be called from within a rcu critical section. + * Returns a pointer from within the RCU-protected ram_list. + */ +static inline void *host_from_stream_offset(QEMUFile *f, + ram_addr_t offset, + int flags) +{ + static RAMBlock *block = NULL; + char id[256]; + uint8_t len; + + if (flags & RAM_SAVE_FLAG_CONTINUE) { + if (!block || block->max_length <= offset) { + error_report("Ack, bad migration stream!"); + return NULL; + } + + return memory_region_get_ram_ptr(block->mr) + offset; + } + + len = qemu_get_byte(f); + qemu_get_buffer(f, (uint8_t *)id, len); + id[len] = 0; + + QLIST_FOREACH_RCU(block, &ram_list.blocks, next) { + if (!strncmp(id, block->idstr, sizeof(id)) && + block->max_length > offset) { + return memory_region_get_ram_ptr(block->mr) + offset; + } + } + + error_report("Can't find block %s!", id); + return NULL; +} + +/* + * If a page (or a whole RDMA chunk) has been + * determined to be zero, then zap it. + */ +void ram_handle_compressed(void *host, uint8_t ch, uint64_t size) +{ + if (ch != 0 || !is_zero_range(host, size)) { + memset(host, ch, size); + } +} + +static void *do_data_decompress(void *opaque) +{ + DecompressParam *param = opaque; + unsigned long pagesize; + + while (!quit_decomp_thread) { + qemu_mutex_lock(¶m->mutex); + while (!param->start && !quit_decomp_thread) { + qemu_cond_wait(¶m->cond, ¶m->mutex); + pagesize = TARGET_PAGE_SIZE; + if (!quit_decomp_thread) { + /* uncompress() will return failed in some case, especially + * when the page is dirted when doing the compression, it's + * not a problem because the dirty page will be retransferred + * and uncompress() won't break the data in other pages. + */ + uncompress((Bytef *)param->des, &pagesize, + (const Bytef *)param->compbuf, param->len); + } + param->start = false; + } + qemu_mutex_unlock(¶m->mutex); + } + + return NULL; +} + +void migrate_decompress_threads_create(void) +{ + int i, thread_count; + + thread_count = migrate_decompress_threads(); + decompress_threads = g_new0(QemuThread, thread_count); + decomp_param = g_new0(DecompressParam, thread_count); + compressed_data_buf = g_malloc0(compressBound(TARGET_PAGE_SIZE)); + quit_decomp_thread = false; + for (i = 0; i < thread_count; i++) { + qemu_mutex_init(&decomp_param[i].mutex); + qemu_cond_init(&decomp_param[i].cond); + decomp_param[i].compbuf = g_malloc0(compressBound(TARGET_PAGE_SIZE)); + qemu_thread_create(decompress_threads + i, "decompress", + do_data_decompress, decomp_param + i, + QEMU_THREAD_JOINABLE); + } +} + +void migrate_decompress_threads_join(void) +{ + int i, thread_count; + + quit_decomp_thread = true; + thread_count = migrate_decompress_threads(); + for (i = 0; i < thread_count; i++) { + qemu_mutex_lock(&decomp_param[i].mutex); + qemu_cond_signal(&decomp_param[i].cond); + qemu_mutex_unlock(&decomp_param[i].mutex); + } + for (i = 0; i < thread_count; i++) { + qemu_thread_join(decompress_threads + i); + qemu_mutex_destroy(&decomp_param[i].mutex); + qemu_cond_destroy(&decomp_param[i].cond); + g_free(decomp_param[i].compbuf); + } + g_free(decompress_threads); + g_free(decomp_param); + g_free(compressed_data_buf); + decompress_threads = NULL; + decomp_param = NULL; + compressed_data_buf = NULL; +} + +static void decompress_data_with_multi_threads(uint8_t *compbuf, + void *host, int len) +{ + int idx, thread_count; + + thread_count = migrate_decompress_threads(); + while (true) { + for (idx = 0; idx < thread_count; idx++) { + if (!decomp_param[idx].start) { + memcpy(decomp_param[idx].compbuf, compbuf, len); + decomp_param[idx].des = host; + decomp_param[idx].len = len; + start_decompression(&decomp_param[idx]); + break; + } + } + if (idx < thread_count) { + break; + } + } +} + +static int ram_load(QEMUFile *f, void *opaque, int version_id) +{ + int flags = 0, ret = 0; + static uint64_t seq_iter; + int len = 0; + + seq_iter++; + + if (version_id != 4) { + ret = -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(); + while (!ret && !(flags & RAM_SAVE_FLAG_EOS)) { + ram_addr_t addr, total_ram_bytes; + void *host; + uint8_t ch; + + addr = qemu_get_be64(f); + flags = addr & ~TARGET_PAGE_MASK; + addr &= TARGET_PAGE_MASK; + + switch (flags & ~RAM_SAVE_FLAG_CONTINUE) { + case RAM_SAVE_FLAG_MEM_SIZE: + /* Synchronize RAM block list */ + total_ram_bytes = addr; + while (!ret && total_ram_bytes) { + RAMBlock *block; + char id[256]; + ram_addr_t length; + + len = qemu_get_byte(f); + qemu_get_buffer(f, (uint8_t *)id, len); + id[len] = 0; + length = qemu_get_be64(f); + + QLIST_FOREACH_RCU(block, &ram_list.blocks, next) { + if (!strncmp(id, block->idstr, sizeof(id))) { + if (length != block->used_length) { + Error *local_err = NULL; + + ret = qemu_ram_resize(block->offset, length, &local_err); + if (local_err) { + error_report_err(local_err); + } + } + break; + } + } + + if (!block) { + error_report("Unknown ramblock \"%s\", cannot " + "accept migration", id); + ret = -EINVAL; + } + + total_ram_bytes -= length; + } + break; + case RAM_SAVE_FLAG_COMPRESS: + host = host_from_stream_offset(f, addr, flags); + if (!host) { + error_report("Illegal RAM offset " RAM_ADDR_FMT, addr); + ret = -EINVAL; + break; + } + ch = qemu_get_byte(f); + ram_handle_compressed(host, ch, TARGET_PAGE_SIZE); + break; + case RAM_SAVE_FLAG_PAGE: + host = host_from_stream_offset(f, addr, flags); + if (!host) { + error_report("Illegal RAM offset " RAM_ADDR_FMT, addr); + ret = -EINVAL; + break; + } + qemu_get_buffer(f, host, TARGET_PAGE_SIZE); + break; + case RAM_SAVE_FLAG_COMPRESS_PAGE: + host = host_from_stream_offset(f, addr, flags); + if (!host) { + error_report("Invalid RAM offset " RAM_ADDR_FMT, addr); + ret = -EINVAL; + break; + } + + len = qemu_get_be32(f); + if (len < 0 || len > compressBound(TARGET_PAGE_SIZE)) { + error_report("Invalid compressed data length: %d", len); + ret = -EINVAL; + break; + } + qemu_get_buffer(f, compressed_data_buf, len); + decompress_data_with_multi_threads(compressed_data_buf, host, len); + break; + case RAM_SAVE_FLAG_XBZRLE: + host = host_from_stream_offset(f, addr, flags); + if (!host) { + error_report("Illegal RAM offset " RAM_ADDR_FMT, addr); + ret = -EINVAL; + break; + } + if (load_xbzrle(f, addr, host) < 0) { + error_report("Failed to decompress XBZRLE page at " + RAM_ADDR_FMT, addr); + ret = -EINVAL; + break; + } + break; + case RAM_SAVE_FLAG_EOS: + /* normal exit */ + break; + default: + if (flags & RAM_SAVE_FLAG_HOOK) { + ram_control_load_hook(f, flags); + } else { + error_report("Unknown combination of migration flags: %#x", + flags); + ret = -EINVAL; + } + } + if (!ret) { + ret = qemu_file_get_error(f); + } + } + + rcu_read_unlock(); + DPRINTF("Completed load of VM with exit code %d seq iteration " + "%" PRIu64 "\n", ret, seq_iter); + return ret; +} + +static SaveVMHandlers savevm_ram_handlers = { + .save_live_setup = ram_save_setup, + .save_live_iterate = ram_save_iterate, + .save_live_complete = ram_save_complete, + .save_live_pending = ram_save_pending, + .load_state = ram_load, + .cancel = ram_migration_cancel, +}; + +void ram_mig_init(void) +{ + qemu_mutex_init(&XBZRLE.lock); + register_savevm_live(NULL, "ram", 0, 4, &savevm_ram_handlers, NULL); +} +/* Stub function that's gets run on the vcpu when its brought out of the + VM to run inside qemu via async_run_on_cpu()*/ + +static void mig_sleep_cpu(void *opq) +{ + qemu_mutex_unlock_iothread(); + g_usleep(30*1000); + qemu_mutex_lock_iothread(); +} + +/* To reduce the dirty rate explicitly disallow the VCPUs from spending + much time in the VM. The migration thread will try to catchup. + Workload will experience a performance drop. +*/ +static void mig_throttle_guest_down(void) +{ + CPUState *cpu; + + qemu_mutex_lock_iothread(); + CPU_FOREACH(cpu) { + async_run_on_cpu(cpu, mig_sleep_cpu, NULL); + } + qemu_mutex_unlock_iothread(); +} + +static void check_guest_throttling(void) +{ + static int64_t t0; + int64_t t1; + + if (!mig_throttle_on) { + return; + } + + if (!t0) { + t0 = qemu_clock_get_ns(QEMU_CLOCK_REALTIME); + return; + } + + t1 = qemu_clock_get_ns(QEMU_CLOCK_REALTIME); + + /* If it has been more than 40 ms since the last time the guest + * was throttled then do it again. + */ + if (40 < (t1-t0)/1000000) { + mig_throttle_guest_down(); + t0 = t1; + } +} diff --git a/migration/rdma.c b/migration/rdma.c index 171c23fc3c..cf5de7e2ae 100644 --- a/migration/rdma.c +++ b/migration/rdma.c @@ -236,13 +236,13 @@ typedef struct RDMALocalBlock { * corresponding RDMALocalBlock with * the information needed to perform the actual RDMA. */ -typedef struct QEMU_PACKED RDMARemoteBlock { +typedef struct QEMU_PACKED RDMADestBlock { uint64_t remote_host_addr; uint64_t offset; uint64_t length; uint32_t remote_rkey; uint32_t padding; -} RDMARemoteBlock; +} RDMADestBlock; static uint64_t htonll(uint64_t v) { @@ -258,20 +258,20 @@ static uint64_t ntohll(uint64_t v) { return ((uint64_t)ntohl(u.lv[0]) << 32) | (uint64_t) ntohl(u.lv[1]); } -static void remote_block_to_network(RDMARemoteBlock *rb) +static void dest_block_to_network(RDMADestBlock *db) { - rb->remote_host_addr = htonll(rb->remote_host_addr); - rb->offset = htonll(rb->offset); - rb->length = htonll(rb->length); - rb->remote_rkey = htonl(rb->remote_rkey); + db->remote_host_addr = htonll(db->remote_host_addr); + db->offset = htonll(db->offset); + db->length = htonll(db->length); + db->remote_rkey = htonl(db->remote_rkey); } -static void network_to_remote_block(RDMARemoteBlock *rb) +static void network_to_dest_block(RDMADestBlock *db) { - rb->remote_host_addr = ntohll(rb->remote_host_addr); - rb->offset = ntohll(rb->offset); - rb->length = ntohll(rb->length); - rb->remote_rkey = ntohl(rb->remote_rkey); + db->remote_host_addr = ntohll(db->remote_host_addr); + db->offset = ntohll(db->offset); + db->length = ntohll(db->length); + db->remote_rkey = ntohl(db->remote_rkey); } /* @@ -350,7 +350,7 @@ typedef struct RDMAContext { * Description of ram blocks used throughout the code. */ RDMALocalBlocks local_ram_blocks; - RDMARemoteBlock *block; + RDMADestBlock *dest_blocks; /* * Migration on *destination* started. @@ -570,10 +570,10 @@ static int rdma_add_block(RDMAContext *rdma, void *host_addr, * in advanced before the migration starts. This tells us where the RAM blocks * are so that we can register them individually. */ -static void qemu_rdma_init_one_block(void *host_addr, +static int qemu_rdma_init_one_block(const char *block_name, void *host_addr, ram_addr_t block_offset, ram_addr_t length, void *opaque) { - rdma_add_block(opaque, host_addr, block_offset, length); + return rdma_add_block(opaque, host_addr, block_offset, length); } /* @@ -590,7 +590,7 @@ static int qemu_rdma_init_ram_blocks(RDMAContext *rdma) memset(local, 0, sizeof *local); qemu_ram_foreach_block(qemu_rdma_init_one_block, rdma); trace_qemu_rdma_init_ram_blocks(local->nb_blocks); - rdma->block = (RDMARemoteBlock *) g_malloc0(sizeof(RDMARemoteBlock) * + rdma->dest_blocks = (RDMADestBlock *) g_malloc0(sizeof(RDMADestBlock) * rdma->local_ram_blocks.nb_blocks); local->init = true; return 0; @@ -790,6 +790,13 @@ static int qemu_rdma_broken_ipv6_kernel(Error **errp, struct ibv_context *verbs) for (x = 0; x < num_devices; x++) { verbs = ibv_open_device(dev_list[x]); + if (!verbs) { + if (errno == EPERM) { + continue; + } else { + return -EINVAL; + } + } if (ibv_query_port(verbs, 1, &port_attr)) { ibv_close_device(verbs); @@ -2177,8 +2184,8 @@ static void qemu_rdma_cleanup(RDMAContext *rdma) rdma->connected = false; } - g_free(rdma->block); - rdma->block = NULL; + g_free(rdma->dest_blocks); + rdma->dest_blocks = NULL; for (idx = 0; idx < RDMA_WRID_MAX; idx++) { if (rdma->wr_data[idx].control_mr) { @@ -2445,7 +2452,6 @@ static void *qemu_rdma_data_init(const char *host_port, Error **errp) if (host_port) { rdma = g_malloc0(sizeof(RDMAContext)); - memset(rdma, 0, sizeof(RDMAContext)); rdma->current_index = -1; rdma->current_chunk = -1; @@ -2967,25 +2973,25 @@ static int qemu_rdma_registration_handle(QEMUFile *f, void *opaque, * their "local" descriptions with what was sent. */ for (i = 0; i < local->nb_blocks; i++) { - rdma->block[i].remote_host_addr = + rdma->dest_blocks[i].remote_host_addr = (uintptr_t)(local->block[i].local_host_addr); if (rdma->pin_all) { - rdma->block[i].remote_rkey = local->block[i].mr->rkey; + rdma->dest_blocks[i].remote_rkey = local->block[i].mr->rkey; } - rdma->block[i].offset = local->block[i].offset; - rdma->block[i].length = local->block[i].length; + rdma->dest_blocks[i].offset = local->block[i].offset; + rdma->dest_blocks[i].length = local->block[i].length; - remote_block_to_network(&rdma->block[i]); + dest_block_to_network(&rdma->dest_blocks[i]); } blocks.len = rdma->local_ram_blocks.nb_blocks - * sizeof(RDMARemoteBlock); + * sizeof(RDMADestBlock); ret = qemu_rdma_post_send_control(rdma, - (uint8_t *) rdma->block, &blocks); + (uint8_t *) rdma->dest_blocks, &blocks); if (ret < 0) { error_report("rdma migration: error sending remote info"); @@ -3141,7 +3147,7 @@ static int qemu_rdma_registration_stop(QEMUFile *f, void *opaque, if (flags == RAM_CONTROL_SETUP) { RDMAControlHeader resp = {.type = RDMA_CONTROL_RAM_BLOCKS_RESULT }; RDMALocalBlocks *local = &rdma->local_ram_blocks; - int reg_result_idx, i, j, nb_remote_blocks; + int reg_result_idx, i, j, nb_dest_blocks; head.type = RDMA_CONTROL_RAM_BLOCKS_REQUEST; trace_qemu_rdma_registration_stop_ram(); @@ -3162,7 +3168,7 @@ static int qemu_rdma_registration_stop(QEMUFile *f, void *opaque, return ret; } - nb_remote_blocks = resp.len / sizeof(RDMARemoteBlock); + nb_dest_blocks = resp.len / sizeof(RDMADestBlock); /* * The protocol uses two different sets of rkeys (mutually exclusive): @@ -3176,7 +3182,7 @@ static int qemu_rdma_registration_stop(QEMUFile *f, void *opaque, * and then propagates the remote ram block descriptions to his local copy. */ - if (local->nb_blocks != nb_remote_blocks) { + if (local->nb_blocks != nb_dest_blocks) { ERROR(errp, "ram blocks mismatch #1! " "Your QEMU command line parameters are probably " "not identical on both the source and destination."); @@ -3184,26 +3190,26 @@ static int qemu_rdma_registration_stop(QEMUFile *f, void *opaque, } qemu_rdma_move_header(rdma, reg_result_idx, &resp); - memcpy(rdma->block, + memcpy(rdma->dest_blocks, rdma->wr_data[reg_result_idx].control_curr, resp.len); - for (i = 0; i < nb_remote_blocks; i++) { - network_to_remote_block(&rdma->block[i]); + for (i = 0; i < nb_dest_blocks; i++) { + network_to_dest_block(&rdma->dest_blocks[i]); /* search local ram blocks */ for (j = 0; j < local->nb_blocks; j++) { - if (rdma->block[i].offset != local->block[j].offset) { + if (rdma->dest_blocks[i].offset != local->block[j].offset) { continue; } - if (rdma->block[i].length != local->block[j].length) { + if (rdma->dest_blocks[i].length != local->block[j].length) { ERROR(errp, "ram blocks mismatch #2! " "Your QEMU command line parameters are probably " "not identical on both the source and destination."); return -EINVAL; } local->block[j].remote_host_addr = - rdma->block[i].remote_host_addr; - local->block[j].remote_rkey = rdma->block[i].remote_rkey; + rdma->dest_blocks[i].remote_host_addr; + local->block[j].remote_rkey = rdma->dest_blocks[i].remote_rkey; break; } diff --git a/migration/savevm.c b/migration/savevm.c new file mode 100644 index 0000000000..2091882196 --- /dev/null +++ b/migration/savevm.c @@ -0,0 +1,1508 @@ +/* + * QEMU System Emulator + * + * Copyright (c) 2003-2008 Fabrice Bellard + * Copyright (c) 2009-2015 Red Hat Inc + * + * Authors: + * Juan Quintela <quintela@redhat.com> + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL + * THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + */ + +#include "config-host.h" +#include "qemu-common.h" +#include "hw/boards.h" +#include "hw/hw.h" +#include "hw/qdev.h" +#include "net/net.h" +#include "monitor/monitor.h" +#include "sysemu/sysemu.h" +#include "qemu/timer.h" +#include "audio/audio.h" +#include "migration/migration.h" +#include "qemu/sockets.h" +#include "qemu/queue.h" +#include "sysemu/cpus.h" +#include "exec/memory.h" +#include "qmp-commands.h" +#include "trace.h" +#include "qemu/iov.h" +#include "block/snapshot.h" +#include "block/qapi.h" + + +#ifndef ETH_P_RARP +#define ETH_P_RARP 0x8035 +#endif +#define ARP_HTYPE_ETH 0x0001 +#define ARP_PTYPE_IP 0x0800 +#define ARP_OP_REQUEST_REV 0x3 + +static bool skip_section_footers; + +static int announce_self_create(uint8_t *buf, + uint8_t *mac_addr) +{ + /* Ethernet header. */ + memset(buf, 0xff, 6); /* destination MAC addr */ + memcpy(buf + 6, mac_addr, 6); /* source MAC addr */ + *(uint16_t *)(buf + 12) = htons(ETH_P_RARP); /* ethertype */ + + /* RARP header. */ + *(uint16_t *)(buf + 14) = htons(ARP_HTYPE_ETH); /* hardware addr space */ + *(uint16_t *)(buf + 16) = htons(ARP_PTYPE_IP); /* protocol addr space */ + *(buf + 18) = 6; /* hardware addr length (ethernet) */ + *(buf + 19) = 4; /* protocol addr length (IPv4) */ + *(uint16_t *)(buf + 20) = htons(ARP_OP_REQUEST_REV); /* opcode */ + memcpy(buf + 22, mac_addr, 6); /* source hw addr */ + memset(buf + 28, 0x00, 4); /* source protocol addr */ + memcpy(buf + 32, mac_addr, 6); /* target hw addr */ + memset(buf + 38, 0x00, 4); /* target protocol addr */ + + /* Padding to get up to 60 bytes (ethernet min packet size, minus FCS). */ + memset(buf + 42, 0x00, 18); + + return 60; /* len (FCS will be added by hardware) */ +} + +static void qemu_announce_self_iter(NICState *nic, void *opaque) +{ + uint8_t buf[60]; + int len; + + trace_qemu_announce_self_iter(qemu_ether_ntoa(&nic->conf->macaddr)); + len = announce_self_create(buf, nic->conf->macaddr.a); + + qemu_send_packet_raw(qemu_get_queue(nic), buf, len); +} + + +static void qemu_announce_self_once(void *opaque) +{ + static int count = SELF_ANNOUNCE_ROUNDS; + QEMUTimer *timer = *(QEMUTimer **)opaque; + + qemu_foreach_nic(qemu_announce_self_iter, NULL); + + if (--count) { + /* delay 50ms, 150ms, 250ms, ... */ + timer_mod(timer, qemu_clock_get_ms(QEMU_CLOCK_REALTIME) + + self_announce_delay(count)); + } else { + timer_del(timer); + timer_free(timer); + } +} + +void qemu_announce_self(void) +{ + static QEMUTimer *timer; + timer = timer_new_ms(QEMU_CLOCK_REALTIME, qemu_announce_self_once, &timer); + qemu_announce_self_once(&timer); +} + +/***********************************************************/ +/* savevm/loadvm support */ + +static ssize_t block_writev_buffer(void *opaque, struct iovec *iov, int iovcnt, + int64_t pos) +{ + int ret; + QEMUIOVector qiov; + + qemu_iovec_init_external(&qiov, iov, iovcnt); + ret = bdrv_writev_vmstate(opaque, &qiov, pos); + if (ret < 0) { + return ret; + } + + return qiov.size; +} + +static int block_put_buffer(void *opaque, const uint8_t *buf, + int64_t pos, int size) +{ + bdrv_save_vmstate(opaque, buf, pos, size); + return size; +} + +static int block_get_buffer(void *opaque, uint8_t *buf, int64_t pos, int size) +{ + return bdrv_load_vmstate(opaque, buf, pos, size); +} + +static int bdrv_fclose(void *opaque) +{ + return bdrv_flush(opaque); +} + +static const QEMUFileOps bdrv_read_ops = { + .get_buffer = block_get_buffer, + .close = bdrv_fclose +}; + +static const QEMUFileOps bdrv_write_ops = { + .put_buffer = block_put_buffer, + .writev_buffer = block_writev_buffer, + .close = bdrv_fclose +}; + +static QEMUFile *qemu_fopen_bdrv(BlockDriverState *bs, int is_writable) +{ + if (is_writable) { + return qemu_fopen_ops(bs, &bdrv_write_ops); + } + return qemu_fopen_ops(bs, &bdrv_read_ops); +} + + +/* QEMUFile timer support. + * Not in qemu-file.c to not add qemu-timer.c as dependency to qemu-file.c + */ + +void timer_put(QEMUFile *f, QEMUTimer *ts) +{ + uint64_t expire_time; + + expire_time = timer_expire_time_ns(ts); + qemu_put_be64(f, expire_time); +} + +void timer_get(QEMUFile *f, QEMUTimer *ts) +{ + uint64_t expire_time; + + expire_time = qemu_get_be64(f); + if (expire_time != -1) { + timer_mod_ns(ts, expire_time); + } else { + timer_del(ts); + } +} + + +/* VMState timer support. + * Not in vmstate.c to not add qemu-timer.c as dependency to vmstate.c + */ + +static int get_timer(QEMUFile *f, void *pv, size_t size) +{ + QEMUTimer *v = pv; + timer_get(f, v); + return 0; +} + +static void put_timer(QEMUFile *f, void *pv, size_t size) +{ + QEMUTimer *v = pv; + timer_put(f, v); +} + +const VMStateInfo vmstate_info_timer = { + .name = "timer", + .get = get_timer, + .put = put_timer, +}; + + +typedef struct CompatEntry { + char idstr[256]; + int instance_id; +} CompatEntry; + +typedef struct SaveStateEntry { + QTAILQ_ENTRY(SaveStateEntry) entry; + char idstr[256]; + int instance_id; + int alias_id; + int version_id; + int section_id; + SaveVMHandlers *ops; + const VMStateDescription *vmsd; + void *opaque; + CompatEntry *compat; + int is_ram; +} SaveStateEntry; + +typedef struct SaveState { + QTAILQ_HEAD(, SaveStateEntry) handlers; + int global_section_id; +} SaveState; + +static SaveState savevm_state = { + .handlers = QTAILQ_HEAD_INITIALIZER(savevm_state.handlers), + .global_section_id = 0, +}; + +static void dump_vmstate_vmsd(FILE *out_file, + const VMStateDescription *vmsd, int indent, + bool is_subsection); + +static void dump_vmstate_vmsf(FILE *out_file, const VMStateField *field, + int indent) +{ + fprintf(out_file, "%*s{\n", indent, ""); + indent += 2; + fprintf(out_file, "%*s\"field\": \"%s\",\n", indent, "", field->name); + fprintf(out_file, "%*s\"version_id\": %d,\n", indent, "", + field->version_id); + fprintf(out_file, "%*s\"field_exists\": %s,\n", indent, "", + field->field_exists ? "true" : "false"); + fprintf(out_file, "%*s\"size\": %zu", indent, "", field->size); + if (field->vmsd != NULL) { + fprintf(out_file, ",\n"); + dump_vmstate_vmsd(out_file, field->vmsd, indent, false); + } + fprintf(out_file, "\n%*s}", indent - 2, ""); +} + +static void dump_vmstate_vmss(FILE *out_file, + const VMStateDescription **subsection, + int indent) +{ + if (*subsection != NULL) { + dump_vmstate_vmsd(out_file, *subsection, indent, true); + } +} + +static void dump_vmstate_vmsd(FILE *out_file, + const VMStateDescription *vmsd, int indent, + bool is_subsection) +{ + if (is_subsection) { + fprintf(out_file, "%*s{\n", indent, ""); + } else { + fprintf(out_file, "%*s\"%s\": {\n", indent, "", "Description"); + } + indent += 2; + fprintf(out_file, "%*s\"name\": \"%s\",\n", indent, "", vmsd->name); + fprintf(out_file, "%*s\"version_id\": %d,\n", indent, "", + vmsd->version_id); + fprintf(out_file, "%*s\"minimum_version_id\": %d", indent, "", + vmsd->minimum_version_id); + if (vmsd->fields != NULL) { + const VMStateField *field = vmsd->fields; + bool first; + + fprintf(out_file, ",\n%*s\"Fields\": [\n", indent, ""); + first = true; + while (field->name != NULL) { + if (field->flags & VMS_MUST_EXIST) { + /* Ignore VMSTATE_VALIDATE bits; these don't get migrated */ + field++; + continue; + } + if (!first) { + fprintf(out_file, ",\n"); + } + dump_vmstate_vmsf(out_file, field, indent + 2); + field++; + first = false; + } + fprintf(out_file, "\n%*s]", indent, ""); + } + if (vmsd->subsections != NULL) { + const VMStateDescription **subsection = vmsd->subsections; + bool first; + + fprintf(out_file, ",\n%*s\"Subsections\": [\n", indent, ""); + first = true; + while (*subsection != NULL) { + if (!first) { + fprintf(out_file, ",\n"); + } + dump_vmstate_vmss(out_file, subsection, indent + 2); + subsection++; + first = false; + } + fprintf(out_file, "\n%*s]", indent, ""); + } + fprintf(out_file, "\n%*s}", indent - 2, ""); +} + +static void dump_machine_type(FILE *out_file) +{ + MachineClass *mc; + + mc = MACHINE_GET_CLASS(current_machine); + + fprintf(out_file, " \"vmschkmachine\": {\n"); + fprintf(out_file, " \"Name\": \"%s\"\n", mc->name); + fprintf(out_file, " },\n"); +} + +void dump_vmstate_json_to_file(FILE *out_file) +{ + GSList *list, *elt; + bool first; + + fprintf(out_file, "{\n"); + dump_machine_type(out_file); + + first = true; + list = object_class_get_list(TYPE_DEVICE, true); + for (elt = list; elt; elt = elt->next) { + DeviceClass *dc = OBJECT_CLASS_CHECK(DeviceClass, elt->data, + TYPE_DEVICE); + const char *name; + int indent = 2; + + if (!dc->vmsd) { + continue; + } + + if (!first) { + fprintf(out_file, ",\n"); + } + name = object_class_get_name(OBJECT_CLASS(dc)); + fprintf(out_file, "%*s\"%s\": {\n", indent, "", name); + indent += 2; + fprintf(out_file, "%*s\"Name\": \"%s\",\n", indent, "", name); + fprintf(out_file, "%*s\"version_id\": %d,\n", indent, "", + dc->vmsd->version_id); + fprintf(out_file, "%*s\"minimum_version_id\": %d,\n", indent, "", + dc->vmsd->minimum_version_id); + + dump_vmstate_vmsd(out_file, dc->vmsd, indent, false); + + fprintf(out_file, "\n%*s}", indent - 2, ""); + first = false; + } + fprintf(out_file, "\n}\n"); + fclose(out_file); +} + +static int calculate_new_instance_id(const char *idstr) +{ + SaveStateEntry *se; + int instance_id = 0; + + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (strcmp(idstr, se->idstr) == 0 + && instance_id <= se->instance_id) { + instance_id = se->instance_id + 1; + } + } + return instance_id; +} + +static int calculate_compat_instance_id(const char *idstr) +{ + SaveStateEntry *se; + int instance_id = 0; + + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (!se->compat) { + continue; + } + + if (strcmp(idstr, se->compat->idstr) == 0 + && instance_id <= se->compat->instance_id) { + instance_id = se->compat->instance_id + 1; + } + } + return instance_id; +} + +/* TODO: Individual devices generally have very little idea about the rest + of the system, so instance_id should be removed/replaced. + Meanwhile pass -1 as instance_id if you do not already have a clearly + distinguishing id for all instances of your device class. */ +int register_savevm_live(DeviceState *dev, + const char *idstr, + int instance_id, + int version_id, + SaveVMHandlers *ops, + void *opaque) +{ + SaveStateEntry *se; + + se = g_malloc0(sizeof(SaveStateEntry)); + se->version_id = version_id; + se->section_id = savevm_state.global_section_id++; + se->ops = ops; + se->opaque = opaque; + se->vmsd = NULL; + /* if this is a live_savem then set is_ram */ + if (ops->save_live_setup != NULL) { + se->is_ram = 1; + } + + if (dev) { + char *id = qdev_get_dev_path(dev); + if (id) { + pstrcpy(se->idstr, sizeof(se->idstr), id); + pstrcat(se->idstr, sizeof(se->idstr), "/"); + g_free(id); + + se->compat = g_malloc0(sizeof(CompatEntry)); + pstrcpy(se->compat->idstr, sizeof(se->compat->idstr), idstr); + se->compat->instance_id = instance_id == -1 ? + calculate_compat_instance_id(idstr) : instance_id; + instance_id = -1; + } + } + pstrcat(se->idstr, sizeof(se->idstr), idstr); + + if (instance_id == -1) { + se->instance_id = calculate_new_instance_id(se->idstr); + } else { + se->instance_id = instance_id; + } + assert(!se->compat || se->instance_id == 0); + /* add at the end of list */ + QTAILQ_INSERT_TAIL(&savevm_state.handlers, se, entry); + return 0; +} + +int register_savevm(DeviceState *dev, + const char *idstr, + int instance_id, + int version_id, + SaveStateHandler *save_state, + LoadStateHandler *load_state, + void *opaque) +{ + SaveVMHandlers *ops = g_malloc0(sizeof(SaveVMHandlers)); + ops->save_state = save_state; + ops->load_state = load_state; + return register_savevm_live(dev, idstr, instance_id, version_id, + ops, opaque); +} + +void unregister_savevm(DeviceState *dev, const char *idstr, void *opaque) +{ + SaveStateEntry *se, *new_se; + char id[256] = ""; + + if (dev) { + char *path = qdev_get_dev_path(dev); + if (path) { + pstrcpy(id, sizeof(id), path); + pstrcat(id, sizeof(id), "/"); + g_free(path); + } + } + pstrcat(id, sizeof(id), idstr); + + QTAILQ_FOREACH_SAFE(se, &savevm_state.handlers, entry, new_se) { + if (strcmp(se->idstr, id) == 0 && se->opaque == opaque) { + QTAILQ_REMOVE(&savevm_state.handlers, se, entry); + if (se->compat) { + g_free(se->compat); + } + g_free(se->ops); + g_free(se); + } + } +} + +int vmstate_register_with_alias_id(DeviceState *dev, int instance_id, + const VMStateDescription *vmsd, + void *opaque, int alias_id, + int required_for_version) +{ + SaveStateEntry *se; + + /* If this triggers, alias support can be dropped for the vmsd. */ + assert(alias_id == -1 || required_for_version >= vmsd->minimum_version_id); + + se = g_malloc0(sizeof(SaveStateEntry)); + se->version_id = vmsd->version_id; + se->section_id = savevm_state.global_section_id++; + se->opaque = opaque; + se->vmsd = vmsd; + se->alias_id = alias_id; + + if (dev) { + char *id = qdev_get_dev_path(dev); + if (id) { + pstrcpy(se->idstr, sizeof(se->idstr), id); + pstrcat(se->idstr, sizeof(se->idstr), "/"); + g_free(id); + + se->compat = g_malloc0(sizeof(CompatEntry)); + pstrcpy(se->compat->idstr, sizeof(se->compat->idstr), vmsd->name); + se->compat->instance_id = instance_id == -1 ? + calculate_compat_instance_id(vmsd->name) : instance_id; + instance_id = -1; + } + } + pstrcat(se->idstr, sizeof(se->idstr), vmsd->name); + + if (instance_id == -1) { + se->instance_id = calculate_new_instance_id(se->idstr); + } else { + se->instance_id = instance_id; + } + assert(!se->compat || se->instance_id == 0); + /* add at the end of list */ + QTAILQ_INSERT_TAIL(&savevm_state.handlers, se, entry); + return 0; +} + +void vmstate_unregister(DeviceState *dev, const VMStateDescription *vmsd, + void *opaque) +{ + SaveStateEntry *se, *new_se; + + QTAILQ_FOREACH_SAFE(se, &savevm_state.handlers, entry, new_se) { + if (se->vmsd == vmsd && se->opaque == opaque) { + QTAILQ_REMOVE(&savevm_state.handlers, se, entry); + if (se->compat) { + g_free(se->compat); + } + g_free(se); + } + } +} + +static int vmstate_load(QEMUFile *f, SaveStateEntry *se, int version_id) +{ + trace_vmstate_load(se->idstr, se->vmsd ? se->vmsd->name : "(old)"); + if (!se->vmsd) { /* Old style */ + return se->ops->load_state(f, se->opaque, version_id); + } + return vmstate_load_state(f, se->vmsd, se->opaque, version_id); +} + +static void vmstate_save_old_style(QEMUFile *f, SaveStateEntry *se, QJSON *vmdesc) +{ + int64_t old_offset, size; + + old_offset = qemu_ftell_fast(f); + se->ops->save_state(f, se->opaque); + size = qemu_ftell_fast(f) - old_offset; + + if (vmdesc) { + json_prop_int(vmdesc, "size", size); + json_start_array(vmdesc, "fields"); + json_start_object(vmdesc, NULL); + json_prop_str(vmdesc, "name", "data"); + json_prop_int(vmdesc, "size", size); + json_prop_str(vmdesc, "type", "buffer"); + json_end_object(vmdesc); + json_end_array(vmdesc); + } +} + +static void vmstate_save(QEMUFile *f, SaveStateEntry *se, QJSON *vmdesc) +{ + trace_vmstate_save(se->idstr, se->vmsd ? se->vmsd->name : "(old)"); + if (!se->vmsd) { + vmstate_save_old_style(f, se, vmdesc); + return; + } + vmstate_save_state(f, se->vmsd, se->opaque, vmdesc); +} + +void savevm_skip_section_footers(void) +{ + skip_section_footers = true; +} + +/* + * Write the header for device section (QEMU_VM_SECTION START/END/PART/FULL) + */ +static void save_section_header(QEMUFile *f, SaveStateEntry *se, + uint8_t section_type) +{ + qemu_put_byte(f, section_type); + qemu_put_be32(f, se->section_id); + + if (section_type == QEMU_VM_SECTION_FULL || + section_type == QEMU_VM_SECTION_START) { + /* ID string */ + size_t len = strlen(se->idstr); + qemu_put_byte(f, len); + qemu_put_buffer(f, (uint8_t *)se->idstr, len); + + qemu_put_be32(f, se->instance_id); + qemu_put_be32(f, se->version_id); + } +} + +/* + * Write a footer onto device sections that catches cases misformatted device + * sections. + */ +static void save_section_footer(QEMUFile *f, SaveStateEntry *se) +{ + if (!skip_section_footers) { + qemu_put_byte(f, QEMU_VM_SECTION_FOOTER); + qemu_put_be32(f, se->section_id); + } +} + +/* + * Read a footer off the wire and check that it matches the expected section + * + * Returns: true if the footer was good + * false if there is a problem (and calls error_report to say why) + */ +static bool check_section_footer(QEMUFile *f, SaveStateEntry *se) +{ + uint8_t read_mark; + uint32_t read_section_id; + + if (skip_section_footers) { + /* No footer to check */ + return true; + } + + read_mark = qemu_get_byte(f); + + if (read_mark != QEMU_VM_SECTION_FOOTER) { + error_report("Missing section footer for %s", se->idstr); + return false; + } + + read_section_id = qemu_get_be32(f); + if (read_section_id != se->section_id) { + error_report("Mismatched section id in footer for %s -" + " read 0x%x expected 0x%x", + se->idstr, read_section_id, se->section_id); + return false; + } + + /* All good */ + return true; +} + +bool qemu_savevm_state_blocked(Error **errp) +{ + SaveStateEntry *se; + + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (se->vmsd && se->vmsd->unmigratable) { + error_setg(errp, "State blocked by non-migratable device '%s'", + se->idstr); + return true; + } + } + return false; +} + +void qemu_savevm_state_header(QEMUFile *f) +{ + trace_savevm_state_header(); + qemu_put_be32(f, QEMU_VM_FILE_MAGIC); + qemu_put_be32(f, QEMU_VM_FILE_VERSION); +} + +void qemu_savevm_state_begin(QEMUFile *f, + const MigrationParams *params) +{ + SaveStateEntry *se; + int ret; + + trace_savevm_state_begin(); + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (!se->ops || !se->ops->set_params) { + continue; + } + se->ops->set_params(params, se->opaque); + } + + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (!se->ops || !se->ops->save_live_setup) { + continue; + } + if (se->ops && se->ops->is_active) { + if (!se->ops->is_active(se->opaque)) { + continue; + } + } + save_section_header(f, se, QEMU_VM_SECTION_START); + + ret = se->ops->save_live_setup(f, se->opaque); + save_section_footer(f, se); + if (ret < 0) { + qemu_file_set_error(f, ret); + break; + } + } +} + +/* + * this function has three return values: + * negative: there was one error, and we have -errno. + * 0 : We haven't finished, caller have to go again + * 1 : We have finished, we can go to complete phase + */ +int qemu_savevm_state_iterate(QEMUFile *f) +{ + SaveStateEntry *se; + int ret = 1; + + trace_savevm_state_iterate(); + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (!se->ops || !se->ops->save_live_iterate) { + continue; + } + if (se->ops && se->ops->is_active) { + if (!se->ops->is_active(se->opaque)) { + continue; + } + } + if (qemu_file_rate_limit(f)) { + return 0; + } + trace_savevm_section_start(se->idstr, se->section_id); + + save_section_header(f, se, QEMU_VM_SECTION_PART); + + ret = se->ops->save_live_iterate(f, se->opaque); + trace_savevm_section_end(se->idstr, se->section_id, ret); + save_section_footer(f, se); + + if (ret < 0) { + qemu_file_set_error(f, ret); + } + if (ret <= 0) { + /* Do not proceed to the next vmstate before this one reported + completion of the current stage. This serializes the migration + and reduces the probability that a faster changing state is + synchronized over and over again. */ + break; + } + } + return ret; +} + +static bool should_send_vmdesc(void) +{ + MachineState *machine = MACHINE(qdev_get_machine()); + return !machine->suppress_vmdesc; +} + +void qemu_savevm_state_complete(QEMUFile *f) +{ + QJSON *vmdesc; + int vmdesc_len; + SaveStateEntry *se; + int ret; + + trace_savevm_state_complete(); + + cpu_synchronize_all_states(); + + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (!se->ops || !se->ops->save_live_complete) { + continue; + } + if (se->ops && se->ops->is_active) { + if (!se->ops->is_active(se->opaque)) { + continue; + } + } + trace_savevm_section_start(se->idstr, se->section_id); + + save_section_header(f, se, QEMU_VM_SECTION_END); + + ret = se->ops->save_live_complete(f, se->opaque); + trace_savevm_section_end(se->idstr, se->section_id, ret); + save_section_footer(f, se); + if (ret < 0) { + qemu_file_set_error(f, ret); + return; + } + } + + vmdesc = qjson_new(); + json_prop_int(vmdesc, "page_size", TARGET_PAGE_SIZE); + json_start_array(vmdesc, "devices"); + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + + if ((!se->ops || !se->ops->save_state) && !se->vmsd) { + continue; + } + trace_savevm_section_start(se->idstr, se->section_id); + + json_start_object(vmdesc, NULL); + json_prop_str(vmdesc, "name", se->idstr); + json_prop_int(vmdesc, "instance_id", se->instance_id); + + save_section_header(f, se, QEMU_VM_SECTION_FULL); + + vmstate_save(f, se, vmdesc); + + json_end_object(vmdesc); + trace_savevm_section_end(se->idstr, se->section_id, 0); + save_section_footer(f, se); + } + + qemu_put_byte(f, QEMU_VM_EOF); + + json_end_array(vmdesc); + qjson_finish(vmdesc); + vmdesc_len = strlen(qjson_get_str(vmdesc)); + + if (should_send_vmdesc()) { + qemu_put_byte(f, QEMU_VM_VMDESCRIPTION); + qemu_put_be32(f, vmdesc_len); + qemu_put_buffer(f, (uint8_t *)qjson_get_str(vmdesc), vmdesc_len); + } + object_unref(OBJECT(vmdesc)); + + qemu_fflush(f); +} + +uint64_t qemu_savevm_state_pending(QEMUFile *f, uint64_t max_size) +{ + SaveStateEntry *se; + uint64_t ret = 0; + + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (!se->ops || !se->ops->save_live_pending) { + continue; + } + if (se->ops && se->ops->is_active) { + if (!se->ops->is_active(se->opaque)) { + continue; + } + } + ret += se->ops->save_live_pending(f, se->opaque, max_size); + } + return ret; +} + +void qemu_savevm_state_cancel(void) +{ + SaveStateEntry *se; + + trace_savevm_state_cancel(); + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (se->ops && se->ops->cancel) { + se->ops->cancel(se->opaque); + } + } +} + +static int qemu_savevm_state(QEMUFile *f, Error **errp) +{ + int ret; + MigrationParams params = { + .blk = 0, + .shared = 0 + }; + + if (qemu_savevm_state_blocked(errp)) { + return -EINVAL; + } + + qemu_mutex_unlock_iothread(); + qemu_savevm_state_header(f); + qemu_savevm_state_begin(f, ¶ms); + qemu_mutex_lock_iothread(); + + while (qemu_file_get_error(f) == 0) { + if (qemu_savevm_state_iterate(f) > 0) { + break; + } + } + + ret = qemu_file_get_error(f); + if (ret == 0) { + qemu_savevm_state_complete(f); + ret = qemu_file_get_error(f); + } + if (ret != 0) { + qemu_savevm_state_cancel(); + error_setg_errno(errp, -ret, "Error while writing VM state"); + } + return ret; +} + +static int qemu_save_device_state(QEMUFile *f) +{ + SaveStateEntry *se; + + qemu_put_be32(f, QEMU_VM_FILE_MAGIC); + qemu_put_be32(f, QEMU_VM_FILE_VERSION); + + cpu_synchronize_all_states(); + + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (se->is_ram) { + continue; + } + if ((!se->ops || !se->ops->save_state) && !se->vmsd) { + continue; + } + + save_section_header(f, se, QEMU_VM_SECTION_FULL); + + vmstate_save(f, se, NULL); + + save_section_footer(f, se); + } + + qemu_put_byte(f, QEMU_VM_EOF); + + return qemu_file_get_error(f); +} + +static SaveStateEntry *find_se(const char *idstr, int instance_id) +{ + SaveStateEntry *se; + + QTAILQ_FOREACH(se, &savevm_state.handlers, entry) { + if (!strcmp(se->idstr, idstr) && + (instance_id == se->instance_id || + instance_id == se->alias_id)) + return se; + /* Migrating from an older version? */ + if (strstr(se->idstr, idstr) && se->compat) { + if (!strcmp(se->compat->idstr, idstr) && + (instance_id == se->compat->instance_id || + instance_id == se->alias_id)) + return se; + } + } + return NULL; +} + +struct LoadStateEntry { + QLIST_ENTRY(LoadStateEntry) entry; + SaveStateEntry *se; + int section_id; + int version_id; +}; + +void loadvm_free_handlers(MigrationIncomingState *mis) +{ + LoadStateEntry *le, *new_le; + + QLIST_FOREACH_SAFE(le, &mis->loadvm_handlers, entry, new_le) { + QLIST_REMOVE(le, entry); + g_free(le); + } +} + +int qemu_loadvm_state(QEMUFile *f) +{ + MigrationIncomingState *mis = migration_incoming_get_current(); + Error *local_err = NULL; + uint8_t section_type; + unsigned int v; + int ret; + int file_error_after_eof = -1; + + if (qemu_savevm_state_blocked(&local_err)) { + error_report_err(local_err); + return -EINVAL; + } + + v = qemu_get_be32(f); + if (v != QEMU_VM_FILE_MAGIC) { + error_report("Not a migration stream"); + return -EINVAL; + } + + v = qemu_get_be32(f); + if (v == QEMU_VM_FILE_VERSION_COMPAT) { + error_report("SaveVM v2 format is obsolete and don't work anymore"); + return -ENOTSUP; + } + if (v != QEMU_VM_FILE_VERSION) { + error_report("Unsupported migration stream version"); + return -ENOTSUP; + } + + while ((section_type = qemu_get_byte(f)) != QEMU_VM_EOF) { + uint32_t instance_id, version_id, section_id; + SaveStateEntry *se; + LoadStateEntry *le; + char idstr[256]; + + trace_qemu_loadvm_state_section(section_type); + switch (section_type) { + case QEMU_VM_SECTION_START: + case QEMU_VM_SECTION_FULL: + /* Read section start */ + section_id = qemu_get_be32(f); + if (!qemu_get_counted_string(f, idstr)) { + error_report("Unable to read ID string for section %u", + section_id); + return -EINVAL; + } + instance_id = qemu_get_be32(f); + version_id = qemu_get_be32(f); + + trace_qemu_loadvm_state_section_startfull(section_id, idstr, + instance_id, version_id); + /* Find savevm section */ + se = find_se(idstr, instance_id); + if (se == NULL) { + error_report("Unknown savevm section or instance '%s' %d", + idstr, instance_id); + ret = -EINVAL; + goto out; + } + + /* Validate version */ + if (version_id > se->version_id) { + error_report("savevm: unsupported version %d for '%s' v%d", + version_id, idstr, se->version_id); + ret = -EINVAL; + goto out; + } + + /* Add entry */ + le = g_malloc0(sizeof(*le)); + + le->se = se; + le->section_id = section_id; + le->version_id = version_id; + QLIST_INSERT_HEAD(&mis->loadvm_handlers, le, entry); + + ret = vmstate_load(f, le->se, le->version_id); + if (ret < 0) { + error_report("error while loading state for instance 0x%x of" + " device '%s'", instance_id, idstr); + goto out; + } + if (!check_section_footer(f, le->se)) { + ret = -EINVAL; + goto out; + } + break; + case QEMU_VM_SECTION_PART: + case QEMU_VM_SECTION_END: + section_id = qemu_get_be32(f); + + trace_qemu_loadvm_state_section_partend(section_id); + QLIST_FOREACH(le, &mis->loadvm_handlers, entry) { + if (le->section_id == section_id) { + break; + } + } + if (le == NULL) { + error_report("Unknown savevm section %d", section_id); + ret = -EINVAL; + goto out; + } + + ret = vmstate_load(f, le->se, le->version_id); + if (ret < 0) { + error_report("error while loading state section id %d(%s)", + section_id, le->se->idstr); + goto out; + } + if (!check_section_footer(f, le->se)) { + ret = -EINVAL; + goto out; + } + break; + default: + error_report("Unknown savevm section type %d", section_type); + ret = -EINVAL; + goto out; + } + } + + file_error_after_eof = qemu_file_get_error(f); + + /* + * Try to read in the VMDESC section as well, so that dumping tools that + * intercept our migration stream have the chance to see it. + */ + if (qemu_get_byte(f) == QEMU_VM_VMDESCRIPTION) { + uint32_t size = qemu_get_be32(f); + uint8_t *buf = g_malloc(0x1000); + + while (size > 0) { + uint32_t read_chunk = MIN(size, 0x1000); + qemu_get_buffer(f, buf, read_chunk); + size -= read_chunk; + } + g_free(buf); + } + + cpu_synchronize_all_post_init(); + + ret = 0; + +out: + if (ret == 0) { + /* We may not have a VMDESC section, so ignore relative errors */ + ret = file_error_after_eof; + } + + return ret; +} + +static BlockDriverState *find_vmstate_bs(void) +{ + BlockDriverState *bs = NULL; + while ((bs = bdrv_next(bs))) { + if (bdrv_can_snapshot(bs)) { + return bs; + } + } + return NULL; +} + +/* + * Deletes snapshots of a given name in all opened images. + */ +static int del_existing_snapshots(Monitor *mon, const char *name) +{ + BlockDriverState *bs; + QEMUSnapshotInfo sn1, *snapshot = &sn1; + Error *err = NULL; + + bs = NULL; + while ((bs = bdrv_next(bs))) { + if (bdrv_can_snapshot(bs) && + bdrv_snapshot_find(bs, snapshot, name) >= 0) { + bdrv_snapshot_delete_by_id_or_name(bs, name, &err); + if (err) { + monitor_printf(mon, + "Error while deleting snapshot on device '%s':" + " %s\n", + bdrv_get_device_name(bs), + error_get_pretty(err)); + error_free(err); + return -1; + } + } + } + + return 0; +} + +void hmp_savevm(Monitor *mon, const QDict *qdict) +{ + BlockDriverState *bs, *bs1; + QEMUSnapshotInfo sn1, *sn = &sn1, old_sn1, *old_sn = &old_sn1; + int ret; + QEMUFile *f; + int saved_vm_running; + uint64_t vm_state_size; + qemu_timeval tv; + struct tm tm; + const char *name = qdict_get_try_str(qdict, "name"); + Error *local_err = NULL; + + /* Verify if there is a device that doesn't support snapshots and is writable */ + bs = NULL; + while ((bs = bdrv_next(bs))) { + + if (!bdrv_is_inserted(bs) || bdrv_is_read_only(bs)) { + continue; + } + + if (!bdrv_can_snapshot(bs)) { + monitor_printf(mon, "Device '%s' is writable but does not support snapshots.\n", + bdrv_get_device_name(bs)); + return; + } + } + + bs = find_vmstate_bs(); + if (!bs) { + monitor_printf(mon, "No block device can accept snapshots\n"); + return; + } + + saved_vm_running = runstate_is_running(); + vm_stop(RUN_STATE_SAVE_VM); + + memset(sn, 0, sizeof(*sn)); + + /* fill auxiliary fields */ + qemu_gettimeofday(&tv); + sn->date_sec = tv.tv_sec; + sn->date_nsec = tv.tv_usec * 1000; + sn->vm_clock_nsec = qemu_clock_get_ns(QEMU_CLOCK_VIRTUAL); + + if (name) { + ret = bdrv_snapshot_find(bs, old_sn, name); + if (ret >= 0) { + pstrcpy(sn->name, sizeof(sn->name), old_sn->name); + pstrcpy(sn->id_str, sizeof(sn->id_str), old_sn->id_str); + } else { + pstrcpy(sn->name, sizeof(sn->name), name); + } + } else { + /* cast below needed for OpenBSD where tv_sec is still 'long' */ + localtime_r((const time_t *)&tv.tv_sec, &tm); + strftime(sn->name, sizeof(sn->name), "vm-%Y%m%d%H%M%S", &tm); + } + + /* Delete old snapshots of the same name */ + if (name && del_existing_snapshots(mon, name) < 0) { + goto the_end; + } + + /* save the VM state */ + f = qemu_fopen_bdrv(bs, 1); + if (!f) { + monitor_printf(mon, "Could not open VM state file\n"); + goto the_end; + } + ret = qemu_savevm_state(f, &local_err); + vm_state_size = qemu_ftell(f); + qemu_fclose(f); + if (ret < 0) { + monitor_printf(mon, "%s\n", error_get_pretty(local_err)); + error_free(local_err); + goto the_end; + } + + /* create the snapshots */ + + bs1 = NULL; + while ((bs1 = bdrv_next(bs1))) { + if (bdrv_can_snapshot(bs1)) { + /* Write VM state size only to the image that contains the state */ + sn->vm_state_size = (bs == bs1 ? vm_state_size : 0); + ret = bdrv_snapshot_create(bs1, sn); + if (ret < 0) { + monitor_printf(mon, "Error while creating snapshot on '%s'\n", + bdrv_get_device_name(bs1)); + } + } + } + + the_end: + if (saved_vm_running) { + vm_start(); + } +} + +void qmp_xen_save_devices_state(const char *filename, Error **errp) +{ + QEMUFile *f; + int saved_vm_running; + int ret; + + saved_vm_running = runstate_is_running(); + vm_stop(RUN_STATE_SAVE_VM); + + f = qemu_fopen(filename, "wb"); + if (!f) { + error_setg_file_open(errp, errno, filename); + goto the_end; + } + ret = qemu_save_device_state(f); + qemu_fclose(f); + if (ret < 0) { + error_set(errp, QERR_IO_ERROR); + } + + the_end: + if (saved_vm_running) { + vm_start(); + } +} + +int load_vmstate(const char *name) +{ + BlockDriverState *bs, *bs_vm_state; + QEMUSnapshotInfo sn; + QEMUFile *f; + int ret; + + bs_vm_state = find_vmstate_bs(); + if (!bs_vm_state) { + error_report("No block device supports snapshots"); + return -ENOTSUP; + } + + /* Don't even try to load empty VM states */ + ret = bdrv_snapshot_find(bs_vm_state, &sn, name); + if (ret < 0) { + return ret; + } else if (sn.vm_state_size == 0) { + error_report("This is a disk-only snapshot. Revert to it offline " + "using qemu-img."); + return -EINVAL; + } + + /* Verify if there is any device that doesn't support snapshots and is + writable and check if the requested snapshot is available too. */ + bs = NULL; + while ((bs = bdrv_next(bs))) { + + if (!bdrv_is_inserted(bs) || bdrv_is_read_only(bs)) { + continue; + } + + if (!bdrv_can_snapshot(bs)) { + error_report("Device '%s' is writable but does not support snapshots.", + bdrv_get_device_name(bs)); + return -ENOTSUP; + } + + ret = bdrv_snapshot_find(bs, &sn, name); + if (ret < 0) { + error_report("Device '%s' does not have the requested snapshot '%s'", + bdrv_get_device_name(bs), name); + return ret; + } + } + + /* Flush all IO requests so they don't interfere with the new state. */ + bdrv_drain_all(); + + bs = NULL; + while ((bs = bdrv_next(bs))) { + if (bdrv_can_snapshot(bs)) { + ret = bdrv_snapshot_goto(bs, name); + if (ret < 0) { + error_report("Error %d while activating snapshot '%s' on '%s'", + ret, name, bdrv_get_device_name(bs)); + return ret; + } + } + } + + /* restore the VM state */ + f = qemu_fopen_bdrv(bs_vm_state, 0); + if (!f) { + error_report("Could not open VM state file"); + return -EINVAL; + } + + qemu_system_reset(VMRESET_SILENT); + migration_incoming_state_new(f); + ret = qemu_loadvm_state(f); + + qemu_fclose(f); + migration_incoming_state_destroy(); + if (ret < 0) { + error_report("Error %d while loading VM state", ret); + return ret; + } + + return 0; +} + +void hmp_delvm(Monitor *mon, const QDict *qdict) +{ + BlockDriverState *bs; + Error *err; + const char *name = qdict_get_str(qdict, "name"); + + if (!find_vmstate_bs()) { + monitor_printf(mon, "No block device supports snapshots\n"); + return; + } + + bs = NULL; + while ((bs = bdrv_next(bs))) { + if (bdrv_can_snapshot(bs)) { + err = NULL; + bdrv_snapshot_delete_by_id_or_name(bs, name, &err); + if (err) { + monitor_printf(mon, + "Error while deleting snapshot on device '%s':" + " %s\n", + bdrv_get_device_name(bs), + error_get_pretty(err)); + error_free(err); + } + } + } +} + +void hmp_info_snapshots(Monitor *mon, const QDict *qdict) +{ + BlockDriverState *bs, *bs1; + QEMUSnapshotInfo *sn_tab, *sn, s, *sn_info = &s; + int nb_sns, i, ret, available; + int total; + int *available_snapshots; + + bs = find_vmstate_bs(); + if (!bs) { + monitor_printf(mon, "No available block device supports snapshots\n"); + return; + } + + nb_sns = bdrv_snapshot_list(bs, &sn_tab); + if (nb_sns < 0) { + monitor_printf(mon, "bdrv_snapshot_list: error %d\n", nb_sns); + return; + } + + if (nb_sns == 0) { + monitor_printf(mon, "There is no snapshot available.\n"); + return; + } + + available_snapshots = g_malloc0(sizeof(int) * nb_sns); + total = 0; + for (i = 0; i < nb_sns; i++) { + sn = &sn_tab[i]; + available = 1; + bs1 = NULL; + + while ((bs1 = bdrv_next(bs1))) { + if (bdrv_can_snapshot(bs1) && bs1 != bs) { + ret = bdrv_snapshot_find(bs1, sn_info, sn->id_str); + if (ret < 0) { + available = 0; + break; + } + } + } + + if (available) { + available_snapshots[total] = i; + total++; + } + } + + if (total > 0) { + bdrv_snapshot_dump((fprintf_function)monitor_printf, mon, NULL); + monitor_printf(mon, "\n"); + for (i = 0; i < total; i++) { + sn = &sn_tab[available_snapshots[i]]; + bdrv_snapshot_dump((fprintf_function)monitor_printf, mon, sn); + monitor_printf(mon, "\n"); + } + } else { + monitor_printf(mon, "There is no suitable snapshot available\n"); + } + + g_free(sn_tab); + g_free(available_snapshots); + +} + +void vmstate_register_ram(MemoryRegion *mr, DeviceState *dev) +{ + qemu_ram_set_idstr(memory_region_get_ram_addr(mr) & TARGET_PAGE_MASK, + memory_region_name(mr), dev); +} + +void vmstate_unregister_ram(MemoryRegion *mr, DeviceState *dev) +{ + qemu_ram_unset_idstr(memory_region_get_ram_addr(mr) & TARGET_PAGE_MASK); +} + +void vmstate_register_ram_global(MemoryRegion *mr) +{ + vmstate_register_ram(mr, NULL); +} diff --git a/migration/vmstate.c b/migration/vmstate.c index e5388f0596..6138d1acb7 100644 --- a/migration/vmstate.c +++ b/migration/vmstate.c @@ -341,11 +341,11 @@ void vmstate_save_state(QEMUFile *f, const VMStateDescription *vmsd, } static const VMStateDescription * - vmstate_get_subsection(const VMStateSubsection *sub, char *idstr) +vmstate_get_subsection(const VMStateDescription **sub, char *idstr) { - while (sub && sub->needed) { - if (strcmp(idstr, sub->vmsd->name) == 0) { - return sub->vmsd; + while (sub && *sub && (*sub)->needed) { + if (strcmp(idstr, (*sub)->name) == 0) { + return *sub; } sub++; } @@ -358,7 +358,7 @@ static int vmstate_subsection_load(QEMUFile *f, const VMStateDescription *vmsd, trace_vmstate_subsection_load(vmsd->name); while (qemu_peek_byte(f, 0) == QEMU_VM_SUBSECTION) { - char idstr[256]; + char idstr[256], *idstr_ret; int ret; uint8_t version_id, len, size; const VMStateDescription *sub_vmsd; @@ -369,11 +369,12 @@ static int vmstate_subsection_load(QEMUFile *f, const VMStateDescription *vmsd, trace_vmstate_subsection_load_bad(vmsd->name, "(short)"); return 0; } - size = qemu_peek_buffer(f, (uint8_t *)idstr, len, 2); + size = qemu_peek_buffer(f, (uint8_t **)&idstr_ret, len, 2); if (size != len) { trace_vmstate_subsection_load_bad(vmsd->name, "(peek fail)"); return 0; } + memcpy(idstr, idstr_ret, size); idstr[size] = 0; if (strncmp(vmsd->name, idstr, strlen(vmsd->name)) != 0) { @@ -405,12 +406,12 @@ static int vmstate_subsection_load(QEMUFile *f, const VMStateDescription *vmsd, static void vmstate_subsection_save(QEMUFile *f, const VMStateDescription *vmsd, void *opaque, QJSON *vmdesc) { - const VMStateSubsection *sub = vmsd->subsections; + const VMStateDescription **sub = vmsd->subsections; bool subsection_found = false; - while (sub && sub->needed) { - if (sub->needed(opaque)) { - const VMStateDescription *vmsd = sub->vmsd; + while (sub && *sub && (*sub)->needed) { + if ((*sub)->needed(opaque)) { + const VMStateDescription *vmsd = *sub; uint8_t len; if (vmdesc) { |