summary refs log tree commit diff stats
path: root/migration/postcopy-ram.c
diff options
context:
space:
mode:
Diffstat (limited to 'migration/postcopy-ram.c')
-rw-r--r--migration/postcopy-ram.c563
1 files changed, 443 insertions, 120 deletions
diff --git a/migration/postcopy-ram.c b/migration/postcopy-ram.c
index 75fd310fb2..45af9a361e 100644
--- a/migration/postcopy-ram.c
+++ b/migration/postcopy-ram.c
@@ -110,19 +110,104 @@ void postcopy_thread_create(MigrationIncomingState *mis,
 #include <sys/eventfd.h>
 #include <linux/userfaultfd.h>
 
+/*
+ * Here we use 24 buckets, which means the last bucket will cover [2^24 us,
+ * 2^25 us) ~= [16, 32) seconds.  It should be far enough to record even
+ * extreme (perf-wise broken) 1G pages moving over, which can sometimes
+ * take a few seconds due to various reasons.  Anything more than that
+ * might be unsensible to account anymore.
+ */
+#define  BLOCKTIME_LATENCY_BUCKET_N  (24)
+
+/* All the time records are in unit of nanoseconds */
 typedef struct PostcopyBlocktimeContext {
-    /* time when page fault initiated per vCPU */
-    uint32_t *page_fault_vcpu_time;
-    /* page address per vCPU */
-    uintptr_t *vcpu_addr;
-    uint32_t total_blocktime;
     /* blocktime per vCPU */
-    uint32_t *vcpu_blocktime;
+    uint64_t *vcpu_blocktime_total;
+    /* count of faults per vCPU */
+    uint64_t *vcpu_faults_count;
+    /*
+     * count of currently blocked faults per vCPU.
+     *
+     * NOTE: Normally there should only be one fault in-progress per vCPU
+     * thread, so logically it _seems_ vcpu_faults_count[] for any vCPU
+     * should be either zero or one.  However, there can be reasons we see
+     * >1 faults on the same vCPU thread.
+     *
+     * CASE (1): since the process to resolve faults (ioctl(UFFDIO_COPY),
+     * for example) is done before taking the mutex that protects the
+     * blocktime context, it can happen that we read more than one faulted
+     * addresses per vCPU.
+     *
+     * One example when we can see >1 faulted addresses for one vCPU:
+     *
+     *  vcpu1 thread       fault thread         resolve thread
+     *  ============       ============         ==============
+     *
+     *  faulted on addr1
+     *                     read uffd msg (addr1)
+     *                     MUTEX_LOCK
+     *                     add entry (cpu1, addr1)
+     *                     MUTEX_UNLOCK
+     *                     request remote fault (addr1)
+     *                                          resolve fault (addr1)
+     *  addr1 resolved, continue..
+     *  faulted on addr2
+     *                     read uffd msg (addr2)
+     *                     MUTEX_LOCK
+     *                     add entry (cpu1, addr2) <--------------- [A]
+     *                     MUTEX_UNLOCK
+     *                                          MUTEX_LOCK
+     *                                          remove entry (cpu1, addr1)
+     *                                          MUTEX_UNLOCK
+     *
+     * In above case, we may see (cpu1, addr1) and (cpu1, addr2) entries to
+     * appear together at [A], when it gets the lock before the resolve
+     * thread.  Use this counter to maintain such case, and only when it
+     * reaches zero we know the vCPU is not blocked anymore.
+     *
+     * CASE (2): theoretically (the author admit to not have verified
+     * this..), one vCPU thread can also generate more than one userfaultfd
+     * message on the same address. It can happen e.g. for whatever reason
+     * the fault got retried before a resolution arrives. In that extremely
+     * rare case, we could also see two (cpu1, addr1) entries.
+     *
+     * In all cases, be prepared with such re-entrancies with this array.
+     *
+     * Using uint8_t should be far enough for now.  For example, when
+     * there're only one resolve thread (postcopy ram listening thread),
+     * the max (concurrent fault entries) should be two.
+     */
+    uint8_t *vcpu_faults_current;
+    /*
+     * The hash that contains addr1->[(cpu1,ts1),(cpu2,ts2) ...] mappings.
+     * Each of the entry is a tuple of (CPU index, fault timestamp) showing
+     * that a fault was requested.
+     */
+    GHashTable *vcpu_addr_hash;
+    /*
+     * Each bucket stores the count of faults that were resolved within the
+     * bucket window [2^N us, 2^(N+1) us).
+     */
+    uint64_t latency_buckets[BLOCKTIME_LATENCY_BUCKET_N];
+    /* total blocktime when all vCPUs are stopped */
+    uint64_t total_blocktime;
     /* point in time when last page fault was initiated */
-    uint32_t last_begin;
+    uint64_t last_begin;
     /* number of vCPU are suspended */
     int smp_cpus_down;
-    uint64_t start_time;
+
+    /*
+     * Fast path for looking up vcpu_index from tid.  NOTE: this result
+     * only reflects the vcpu setup when postcopy is running.  It may not
+     * always match with the current vcpu setup because vcpus can be hot
+     * attached/detached after migration completes.  However this should be
+     * stable when blocktime is using the structure.
+     */
+    GHashTable *tid_to_vcpu_hash;
+    /* Count of non-vCPU faults.  This is only for debugging purpose. */
+    uint64_t non_vcpu_faults;
+    /* total blocktime when a non-vCPU thread is stopped */
+    uint64_t non_vcpu_blocktime_total;
 
     /*
      * Handler for exit event, necessary for
@@ -131,11 +216,41 @@ typedef struct PostcopyBlocktimeContext {
     Notifier exit_notifier;
 } PostcopyBlocktimeContext;
 
+typedef struct {
+    /* The time the fault was triggered */
+    uint64_t fault_time;
+    /*
+     * The vCPU index that was blocked, when cpu==-1, it means it's a
+     * fault from non-vCPU threads.
+     */
+    int cpu;
+} BlocktimeVCPUEntry;
+
+/* Alloc an entry to record a vCPU fault */
+static BlocktimeVCPUEntry *
+blocktime_vcpu_entry_alloc(int cpu, uint64_t fault_time)
+{
+    BlocktimeVCPUEntry *entry = g_new(BlocktimeVCPUEntry, 1);
+
+    entry->fault_time = fault_time;
+    entry->cpu = cpu;
+
+    return entry;
+}
+
+/* Free a @GList of @BlocktimeVCPUEntry */
+static void blocktime_vcpu_list_free(gpointer data)
+{
+    g_list_free_full(data, g_free);
+}
+
 static void destroy_blocktime_context(struct PostcopyBlocktimeContext *ctx)
 {
-    g_free(ctx->page_fault_vcpu_time);
-    g_free(ctx->vcpu_addr);
-    g_free(ctx->vcpu_blocktime);
+    g_hash_table_destroy(ctx->tid_to_vcpu_hash);
+    g_hash_table_destroy(ctx->vcpu_addr_hash);
+    g_free(ctx->vcpu_blocktime_total);
+    g_free(ctx->vcpu_faults_count);
+    g_free(ctx->vcpu_faults_current);
     g_free(ctx);
 }
 
@@ -146,32 +261,65 @@ static void migration_exit_cb(Notifier *n, void *data)
     destroy_blocktime_context(ctx);
 }
 
+static GHashTable *blocktime_init_tid_to_vcpu_hash(void)
+{
+    /*
+     * TID as an unsigned int can be directly used as the key.  However,
+     * CPU index can NOT be directly used as value, because CPU index can
+     * be 0, which means NULL.  Then when lookup we can never know whether
+     * it's 0 or "not found".  Hence use an indirection for CPU index.
+     */
+    GHashTable *table = g_hash_table_new_full(g_direct_hash, g_direct_equal,
+                                              NULL, g_free);
+    CPUState *cpu;
+
+    /*
+     * Initialize the tid->cpu_id mapping for lookups.  The caller needs to
+     * make sure when reaching here the CPU topology is frozen and will be
+     * stable for the whole blocktime trapping period.
+     */
+    CPU_FOREACH(cpu) {
+        int *value = g_new(int, 1);
+
+        *value = cpu->cpu_index;
+        g_hash_table_insert(table,
+                            GUINT_TO_POINTER((uint32_t)cpu->thread_id),
+                            value);
+        trace_postcopy_blocktime_tid_cpu_map(cpu->cpu_index, cpu->thread_id);
+    }
+
+    return table;
+}
+
 static struct PostcopyBlocktimeContext *blocktime_context_new(void)
 {
     MachineState *ms = MACHINE(qdev_get_machine());
     unsigned int smp_cpus = ms->smp.cpus;
     PostcopyBlocktimeContext *ctx = g_new0(PostcopyBlocktimeContext, 1);
-    ctx->page_fault_vcpu_time = g_new0(uint32_t, smp_cpus);
-    ctx->vcpu_addr = g_new0(uintptr_t, smp_cpus);
-    ctx->vcpu_blocktime = g_new0(uint32_t, smp_cpus);
 
-    ctx->exit_notifier.notify = migration_exit_cb;
-    ctx->start_time = qemu_clock_get_ms(QEMU_CLOCK_REALTIME);
-    qemu_add_exit_notifier(&ctx->exit_notifier);
-    return ctx;
-}
+    /* Initialize all counters to be zeros */
+    memset(ctx->latency_buckets, 0, sizeof(ctx->latency_buckets));
 
-static uint32List *get_vcpu_blocktime_list(PostcopyBlocktimeContext *ctx)
-{
-    MachineState *ms = MACHINE(qdev_get_machine());
-    uint32List *list = NULL;
-    int i;
+    ctx->vcpu_blocktime_total = g_new0(uint64_t, smp_cpus);
+    ctx->vcpu_faults_count = g_new0(uint64_t, smp_cpus);
+    ctx->vcpu_faults_current = g_new0(uint8_t, smp_cpus);
+    ctx->tid_to_vcpu_hash = blocktime_init_tid_to_vcpu_hash();
 
-    for (i = ms->smp.cpus - 1; i >= 0; i--) {
-        QAPI_LIST_PREPEND(list, ctx->vcpu_blocktime[i]);
-    }
+    /*
+     * The key (host virtual addresses) will always be gpointer-sized on
+     * either 32bits or 64bits systems, so it'll fit as a direct key.
+     *
+     * The value will be a list of BlocktimeVCPUEntry entries.
+     */
+    ctx->vcpu_addr_hash = g_hash_table_new_full(g_direct_hash,
+                                                g_direct_equal,
+                                                NULL,
+                                                blocktime_vcpu_list_free);
+
+    ctx->exit_notifier.notify = migration_exit_cb;
+    qemu_add_exit_notifier(&ctx->exit_notifier);
 
-    return list;
+    return ctx;
 }
 
 /*
@@ -185,18 +333,64 @@ void fill_destination_postcopy_migration_info(MigrationInfo *info)
 {
     MigrationIncomingState *mis = migration_incoming_get_current();
     PostcopyBlocktimeContext *bc = mis->blocktime_ctx;
+    MachineState *ms = MACHINE(qdev_get_machine());
+    uint64_t latency_total = 0, faults = 0;
+    uint32List *list_blocktime = NULL;
+    uint64List *list_latency = NULL;
+    uint64List *latency_buckets = NULL;
+    int i;
 
     if (!bc) {
         return;
     }
 
+    for (i = ms->smp.cpus - 1; i >= 0; i--) {
+        uint64_t latency, total, count;
+
+        /* Convert ns -> ms */
+        QAPI_LIST_PREPEND(list_blocktime,
+                          (uint32_t)(bc->vcpu_blocktime_total[i] / SCALE_MS));
+
+        /* The rest in nanoseconds */
+        total = bc->vcpu_blocktime_total[i];
+        latency_total += total;
+        count = bc->vcpu_faults_count[i];
+        faults += count;
+
+        if (count) {
+            latency = total / count;
+        } else {
+            /* No fault detected */
+            latency = 0;
+        }
+
+        QAPI_LIST_PREPEND(list_latency, latency);
+    }
+
+    for (i = BLOCKTIME_LATENCY_BUCKET_N - 1; i >= 0; i--) {
+        QAPI_LIST_PREPEND(latency_buckets, bc->latency_buckets[i]);
+    }
+
+    latency_total += bc->non_vcpu_blocktime_total;
+    faults += bc->non_vcpu_faults;
+
+    info->has_postcopy_non_vcpu_latency = true;
+    info->postcopy_non_vcpu_latency = bc->non_vcpu_faults ?
+        (bc->non_vcpu_blocktime_total / bc->non_vcpu_faults) : 0;
     info->has_postcopy_blocktime = true;
-    info->postcopy_blocktime = bc->total_blocktime;
+    /* Convert ns -> ms */
+    info->postcopy_blocktime = (uint32_t)(bc->total_blocktime / SCALE_MS);
     info->has_postcopy_vcpu_blocktime = true;
-    info->postcopy_vcpu_blocktime = get_vcpu_blocktime_list(bc);
+    info->postcopy_vcpu_blocktime = list_blocktime;
+    info->has_postcopy_latency = true;
+    info->postcopy_latency = faults ? (latency_total / faults) : 0;
+    info->has_postcopy_vcpu_latency = true;
+    info->postcopy_vcpu_latency = list_latency;
+    info->has_postcopy_latency_dist = true;
+    info->postcopy_latency_dist = latency_buckets;
 }
 
-static uint32_t get_postcopy_total_blocktime(void)
+static uint64_t get_postcopy_total_blocktime(void)
 {
     MigrationIncomingState *mis = migration_incoming_get_current();
     PostcopyBlocktimeContext *bc = mis->blocktime_ctx;
@@ -300,13 +494,13 @@ static bool ufd_check_and_apply(int ufd, MigrationIncomingState *mis,
     }
 
 #ifdef UFFD_FEATURE_THREAD_ID
+    /*
+     * Postcopy blocktime conditionally needs THREAD_ID feature (introduced
+     * to Linux in 2017). Always try to enable it when QEMU is compiled
+     * with such environment.
+     */
     if (UFFD_FEATURE_THREAD_ID & supported_features) {
         asked_features |= UFFD_FEATURE_THREAD_ID;
-        if (migrate_postcopy_blocktime()) {
-            if (!mis->blocktime_ctx) {
-                mis->blocktime_ctx = blocktime_context_new();
-            }
-        }
     }
 #endif
 
@@ -752,8 +946,12 @@ int postcopy_wake_shared(struct PostCopyFD *pcfd,
                        pagesize);
 }
 
+/*
+ * NOTE: @tid is only used when postcopy-blocktime feature is enabled, and
+ * also optional: when zero is provided, the fault accounting will be ignored.
+ */
 static int postcopy_request_page(MigrationIncomingState *mis, RAMBlock *rb,
-                                 ram_addr_t start, uint64_t haddr)
+                                 ram_addr_t start, uint64_t haddr, uint32_t tid)
 {
     void *aligned = (void *)(uintptr_t)ROUND_DOWN(haddr, qemu_ram_pagesize(rb));
 
@@ -772,7 +970,7 @@ static int postcopy_request_page(MigrationIncomingState *mis, RAMBlock *rb,
         return received ? 0 : postcopy_place_page_zero(mis, aligned, rb);
     }
 
-    return migrate_send_rp_req_pages(mis, rb, start, haddr);
+    return migrate_send_rp_req_pages(mis, rb, start, haddr, tid);
 }
 
 /*
@@ -793,83 +991,204 @@ int postcopy_request_shared_page(struct PostCopyFD *pcfd, RAMBlock *rb,
                                         qemu_ram_get_idstr(rb), rb_offset);
         return postcopy_wake_shared(pcfd, client_addr, rb);
     }
-    postcopy_request_page(mis, rb, aligned_rbo, client_addr);
+    /* TODO: support blocktime tracking */
+    postcopy_request_page(mis, rb, aligned_rbo, client_addr, 0);
     return 0;
 }
 
-static int get_mem_fault_cpu_index(uint32_t pid)
+static int blocktime_get_vcpu(PostcopyBlocktimeContext *ctx, uint32_t tid)
 {
-    CPUState *cpu_iter;
+    int *found;
 
-    CPU_FOREACH(cpu_iter) {
-        if (cpu_iter->thread_id == pid) {
-            trace_get_mem_fault_cpu_index(cpu_iter->cpu_index, pid);
-            return cpu_iter->cpu_index;
-        }
+    found = g_hash_table_lookup(ctx->tid_to_vcpu_hash, GUINT_TO_POINTER(tid));
+    if (!found) {
+        /*
+         * NOTE: this is possible, because QEMU's non-vCPU threads can
+         * also access a missing page.  Or, when KVM async pf is enabled, a
+         * fault can even happen from a kworker..
+         */
+        return -1;
     }
-    trace_get_mem_fault_cpu_index(-1, pid);
-    return -1;
+
+    return *found;
 }
 
-static uint32_t get_low_time_offset(PostcopyBlocktimeContext *dc)
+static uint64_t get_current_ns(void)
 {
-    int64_t start_time_offset = qemu_clock_get_ms(QEMU_CLOCK_REALTIME) -
-                                    dc->start_time;
-    return start_time_offset < 1 ? 1 : start_time_offset & UINT32_MAX;
+    return (uint64_t)qemu_clock_get_ns(QEMU_CLOCK_REALTIME);
+}
+
+/*
+ * Inject an (cpu, fault_time) entry into the database, using addr as key.
+ * When cpu==-1, it means it's a non-vCPU fault.
+ */
+static void blocktime_fault_inject(PostcopyBlocktimeContext *ctx,
+                                   uintptr_t addr, int cpu, uint64_t time)
+{
+    BlocktimeVCPUEntry *entry = blocktime_vcpu_entry_alloc(cpu, time);
+    GHashTable *table = ctx->vcpu_addr_hash;
+    gpointer key = (gpointer)addr;
+    GList *head, *list;
+    gboolean result;
+
+    head = g_hash_table_lookup(table, key);
+    if (head) {
+        /*
+         * If existed, steal the @head for list operation rather than
+         * freeing it, making sure steal succeeded.
+         */
+        result = g_hash_table_steal(table, key);
+        assert(result == TRUE);
+    }
+
+    /*
+     * Now the key is guaranteed to be absent.  Two cases:
+     *
+     * (1) There's no existing entry, list contains the only one. Insert.
+     * (2) There're existing entries, after stealing we own it, prepend the
+     *     result and re-insert.
+     */
+    list = g_list_prepend(head, entry);
+    g_hash_table_insert(table, key, list);
+
+    trace_postcopy_blocktime_begin(addr, time, cpu, !!head);
 }
 
 /*
- * This function is being called when pagefault occurs. It
- * tracks down vCPU blocking time.
+ * This function is being called when pagefault occurs. It tracks down vCPU
+ * blocking time.  It's protected by @page_request_mutex.
  *
  * @addr: faulted host virtual address
  * @ptid: faulted process thread id
  * @rb: ramblock appropriate to addr
  */
-static void mark_postcopy_blocktime_begin(uintptr_t addr, uint32_t ptid,
-                                          RAMBlock *rb)
+void mark_postcopy_blocktime_begin(uintptr_t addr, uint32_t ptid,
+                                   RAMBlock *rb)
 {
-    int cpu, already_received;
+    int cpu;
     MigrationIncomingState *mis = migration_incoming_get_current();
     PostcopyBlocktimeContext *dc = mis->blocktime_ctx;
-    uint32_t low_time_offset;
+    uint64_t current;
 
     if (!dc || ptid == 0) {
         return;
     }
-    cpu = get_mem_fault_cpu_index(ptid);
-    if (cpu < 0) {
-        return;
+
+    /*
+     * The caller should only inject a blocktime entry when the page is
+     * yet missing.
+     */
+    assert(!ramblock_recv_bitmap_test(rb, (void *)addr));
+
+    current = get_current_ns();
+    cpu = blocktime_get_vcpu(dc, ptid);
+
+    if (cpu >= 0) {
+        /* How many faults on this vCPU in total? */
+        dc->vcpu_faults_count[cpu]++;
+
+        /*
+         * Account how many concurrent faults on this vCPU we trapped.  See
+         * comments above vcpu_faults_current[] on why it can be more than one.
+         */
+        if (dc->vcpu_faults_current[cpu]++ == 0) {
+            dc->smp_cpus_down++;
+            /*
+             * We use last_begin to cover (1) the 1st fault on this specific
+             * vCPU, but meanwhile (2) the last vCPU that got blocked.  It's
+             * only used to calculate system-wide blocktime.
+             */
+            dc->last_begin = current;
+        }
+
+        /* Making sure it won't overflow - it really should never! */
+        assert(dc->vcpu_faults_current[cpu] <= 255);
+    } else {
+        /*
+         * For non-vCPU thread faults, we don't care about tid or cpu index
+         * or time the thread is blocked (e.g., a kworker trying to help
+         * KVM when async_pf=on is OK to be blocked and not affect guest
+         * responsiveness), but we care about latency.  Track it with
+         * cpu=-1.
+         *
+         * Note that this will NOT affect blocktime reports on vCPU being
+         * blocked, but only about system-wide latency reports.
+         */
+        dc->non_vcpu_faults++;
     }
 
-    low_time_offset = get_low_time_offset(dc);
-    if (dc->vcpu_addr[cpu] == 0) {
-        qatomic_inc(&dc->smp_cpus_down);
+    blocktime_fault_inject(dc, addr, cpu, current);
+}
+
+static void blocktime_latency_account(PostcopyBlocktimeContext *ctx,
+                                      uint64_t time_us)
+{
+    /*
+     * Convert time (in us) to bucket index it belongs.  Take extra caution
+     * of time_us==0 even if normally rare - when happens put into bucket 0.
+     */
+    int index = time_us ? (63 - clz64(time_us)) : 0;
+
+    assert(index >= 0);
+
+    /* If it's too large, put into top bucket */
+    if (index >= BLOCKTIME_LATENCY_BUCKET_N) {
+        index = BLOCKTIME_LATENCY_BUCKET_N - 1;
     }
 
-    qatomic_xchg(&dc->last_begin, low_time_offset);
-    qatomic_xchg(&dc->page_fault_vcpu_time[cpu], low_time_offset);
-    qatomic_xchg(&dc->vcpu_addr[cpu], addr);
+    ctx->latency_buckets[index]++;
+}
+
+typedef struct {
+    PostcopyBlocktimeContext *ctx;
+    uint64_t current;
+    int affected_cpus;
+    int affected_non_cpus;
+} BlockTimeVCPUIter;
+
+static void blocktime_cpu_list_iter_fn(gpointer data, gpointer user_data)
+{
+    BlockTimeVCPUIter *iter = user_data;
+    PostcopyBlocktimeContext *ctx = iter->ctx;
+    BlocktimeVCPUEntry *entry = data;
+    uint64_t time_passed;
+    int cpu = entry->cpu;
 
     /*
-     * check it here, not at the beginning of the function,
-     * due to, check could occur early than bitmap_set in
-     * qemu_ufd_copy_ioctl
+     * Time should never go back.. so when the fault is resolved it must be
+     * later than when it was faulted.
      */
-    already_received = ramblock_recv_bitmap_test(rb, (void *)addr);
-    if (already_received) {
-        qatomic_xchg(&dc->vcpu_addr[cpu], 0);
-        qatomic_xchg(&dc->page_fault_vcpu_time[cpu], 0);
-        qatomic_dec(&dc->smp_cpus_down);
+    assert(iter->current >= entry->fault_time);
+    time_passed = iter->current - entry->fault_time;
+
+    /* Latency buckets are in microseconds */
+    blocktime_latency_account(ctx, time_passed / SCALE_US);
+
+    if (cpu >= 0) {
+        /*
+         * If we resolved all pending faults on one vCPU due to this page
+         * resolution, take a note.
+         */
+        if (--ctx->vcpu_faults_current[cpu] == 0) {
+            ctx->vcpu_blocktime_total[cpu] += time_passed;
+            iter->affected_cpus += 1;
+        }
+        trace_postcopy_blocktime_end_one(cpu, ctx->vcpu_faults_current[cpu]);
+    } else {
+        iter->affected_non_cpus++;
+        ctx->non_vcpu_blocktime_total += time_passed;
+        /*
+         * We do not maintain how many pending non-vCPU faults because we
+         * do not care about blocktime, only latency.
+         */
+        trace_postcopy_blocktime_end_one(-1, 0);
     }
-    trace_mark_postcopy_blocktime_begin(addr, dc, dc->page_fault_vcpu_time[cpu],
-                                        cpu, already_received);
 }
 
 /*
- *  This function just provide calculated blocktime per cpu and trace it.
- *  Total blocktime is calculated in mark_postcopy_blocktime_end.
- *
+ * This function just provide calculated blocktime per cpu and trace it.
+ * Total blocktime is calculated in mark_postcopy_blocktime_end.  It's
+ * protected by @page_request_mutex.
  *
  * Assume we have 3 CPU
  *
@@ -899,48 +1218,45 @@ static void mark_postcopy_blocktime_end(uintptr_t addr)
     PostcopyBlocktimeContext *dc = mis->blocktime_ctx;
     MachineState *ms = MACHINE(qdev_get_machine());
     unsigned int smp_cpus = ms->smp.cpus;
-    int i, affected_cpu = 0;
-    bool vcpu_total_blocktime = false;
-    uint32_t read_vcpu_time, low_time_offset;
+    BlockTimeVCPUIter iter = {
+        .current = get_current_ns(),
+        .affected_cpus = 0,
+        .affected_non_cpus = 0,
+        .ctx = dc,
+    };
+    gpointer key = (gpointer)addr;
+    GHashTable *table;
+    GList *list;
 
     if (!dc) {
         return;
     }
 
-    low_time_offset = get_low_time_offset(dc);
-    /* lookup cpu, to clear it,
-     * that algorithm looks straightforward, but it's not
-     * optimal, more optimal algorithm is keeping tree or hash
-     * where key is address value is a list of  */
-    for (i = 0; i < smp_cpus; i++) {
-        uint32_t vcpu_blocktime = 0;
-
-        read_vcpu_time = qatomic_fetch_add(&dc->page_fault_vcpu_time[i], 0);
-        if (qatomic_fetch_add(&dc->vcpu_addr[i], 0) != addr ||
-            read_vcpu_time == 0) {
-            continue;
-        }
-        qatomic_xchg(&dc->vcpu_addr[i], 0);
-        vcpu_blocktime = low_time_offset - read_vcpu_time;
-        affected_cpu += 1;
-        /* we need to know is that mark_postcopy_end was due to
-         * faulted page, another possible case it's prefetched
-         * page and in that case we shouldn't be here */
-        if (!vcpu_total_blocktime &&
-            qatomic_fetch_add(&dc->smp_cpus_down, 0) == smp_cpus) {
-            vcpu_total_blocktime = true;
-        }
-        /* continue cycle, due to one page could affect several vCPUs */
-        dc->vcpu_blocktime[i] += vcpu_blocktime;
+    table = dc->vcpu_addr_hash;
+    /* the address wasn't tracked at all? */
+    list = g_hash_table_lookup(table, key);
+    if (!list) {
+        return;
     }
 
-    qatomic_sub(&dc->smp_cpus_down, affected_cpu);
-    if (vcpu_total_blocktime) {
-        dc->total_blocktime += low_time_offset - qatomic_fetch_add(
-                &dc->last_begin, 0);
+    /*
+     * Loop over the set of vCPUs that got blocked on this addr, do the
+     * blocktime accounting.  After that, remove the whole list.
+     */
+    g_list_foreach(list, blocktime_cpu_list_iter_fn, &iter);
+    g_hash_table_remove(table, key);
+
+    /*
+     * If all vCPUs used to be down, and copying this page would free some
+     * vCPUs, then the system-level blocktime ends here.
+     */
+    if (dc->smp_cpus_down == smp_cpus && iter.affected_cpus) {
+        dc->total_blocktime += iter.current - dc->last_begin;
     }
-    trace_mark_postcopy_blocktime_end(addr, dc, dc->total_blocktime,
-                                      affected_cpu);
+    dc->smp_cpus_down -= iter.affected_cpus;
+
+    trace_postcopy_blocktime_end(addr, iter.current, iter.affected_cpus,
+                                 iter.affected_non_cpus);
 }
 
 static void postcopy_pause_fault_thread(MigrationIncomingState *mis)
@@ -1068,17 +1384,14 @@ static void *postcopy_ram_fault_thread(void *opaque)
                                                 qemu_ram_get_idstr(rb),
                                                 rb_offset,
                                                 msg.arg.pagefault.feat.ptid);
-            mark_postcopy_blocktime_begin(
-                    (uintptr_t)(msg.arg.pagefault.address),
-                                msg.arg.pagefault.feat.ptid, rb);
-
 retry:
             /*
              * Send the request to the source - we want to request one
              * of our host page sizes (which is >= TPS)
              */
             ret = postcopy_request_page(mis, rb, rb_offset,
-                                        msg.arg.pagefault.address);
+                                        msg.arg.pagefault.address,
+                                        msg.arg.pagefault.feat.ptid);
             if (ret) {
                 /* May be network failure, try to wait for recovery */
                 postcopy_pause_fault_thread(mis);
@@ -1221,6 +1534,11 @@ int postcopy_ram_incoming_setup(MigrationIncomingState *mis)
         return -1;
     }
 
+    if (migrate_postcopy_blocktime()) {
+        assert(mis->blocktime_ctx == NULL);
+        mis->blocktime_ctx = blocktime_context_new();
+    }
+
     /* Now an eventfd we use to tell the fault-thread to quit */
     mis->userfault_event_fd = eventfd(0, EFD_CLOEXEC);
     if (mis->userfault_event_fd == -1) {
@@ -1299,8 +1617,8 @@ static int qemu_ufd_copy_ioctl(MigrationIncomingState *mis, void *host_addr,
                 qemu_cond_signal(&mis->page_request_cond);
             }
         }
-        qemu_mutex_unlock(&mis->page_request_mutex);
         mark_postcopy_blocktime_end((uintptr_t)host_addr);
+        qemu_mutex_unlock(&mis->page_request_mutex);
     }
     return ret;
 }
@@ -1430,6 +1748,11 @@ int postcopy_wake_shared(struct PostCopyFD *pcfd,
 {
     g_assert_not_reached();
 }
+
+void mark_postcopy_blocktime_begin(uintptr_t addr, uint32_t ptid,
+                                   RAMBlock *rb)
+{
+}
 #endif
 
 /* ------------------------------------------------------------------------- */