summaryrefslogtreecommitdiff
path: root/virt/kvm
diff options
context:
space:
mode:
Diffstat (limited to 'virt/kvm')
-rw-r--r--virt/kvm/dirty_ring.c120
-rw-r--r--virt/kvm/eventfd.c159
-rw-r--r--virt/kvm/guest_memfd.c11
-rw-r--r--virt/kvm/irqchip.c2
-rw-r--r--virt/kvm/kvm_main.c123
-rw-r--r--virt/kvm/vfio.c3
6 files changed, 280 insertions, 138 deletions
diff --git a/virt/kvm/dirty_ring.c b/virt/kvm/dirty_ring.c
index 7bc74969a819..02bc6b00d76c 100644
--- a/virt/kvm/dirty_ring.c
+++ b/virt/kvm/dirty_ring.c
@@ -11,14 +11,14 @@
#include <trace/events/kvm.h>
#include "kvm_mm.h"
-int __weak kvm_cpu_dirty_log_size(void)
+int __weak kvm_cpu_dirty_log_size(struct kvm *kvm)
{
return 0;
}
-u32 kvm_dirty_ring_get_rsvd_entries(void)
+u32 kvm_dirty_ring_get_rsvd_entries(struct kvm *kvm)
{
- return KVM_DIRTY_RING_RSVD_ENTRIES + kvm_cpu_dirty_log_size();
+ return KVM_DIRTY_RING_RSVD_ENTRIES + kvm_cpu_dirty_log_size(kvm);
}
bool kvm_use_dirty_bitmap(struct kvm *kvm)
@@ -55,9 +55,6 @@ static void kvm_reset_dirty_gfn(struct kvm *kvm, u32 slot, u64 offset, u64 mask)
struct kvm_memory_slot *memslot;
int as_id, id;
- if (!mask)
- return;
-
as_id = slot >> 16;
id = (u16)slot;
@@ -74,14 +71,15 @@ static void kvm_reset_dirty_gfn(struct kvm *kvm, u32 slot, u64 offset, u64 mask)
KVM_MMU_UNLOCK(kvm);
}
-int kvm_dirty_ring_alloc(struct kvm_dirty_ring *ring, int index, u32 size)
+int kvm_dirty_ring_alloc(struct kvm *kvm, struct kvm_dirty_ring *ring,
+ int index, u32 size)
{
ring->dirty_gfns = vzalloc(size);
if (!ring->dirty_gfns)
return -ENOMEM;
ring->size = size / sizeof(struct kvm_dirty_gfn);
- ring->soft_limit = ring->size - kvm_dirty_ring_get_rsvd_entries();
+ ring->soft_limit = ring->size - kvm_dirty_ring_get_rsvd_entries(kvm);
ring->dirty_index = 0;
ring->reset_index = 0;
ring->index = index;
@@ -104,19 +102,38 @@ static inline bool kvm_dirty_gfn_harvested(struct kvm_dirty_gfn *gfn)
return smp_load_acquire(&gfn->flags) & KVM_DIRTY_GFN_F_RESET;
}
-int kvm_dirty_ring_reset(struct kvm *kvm, struct kvm_dirty_ring *ring)
+int kvm_dirty_ring_reset(struct kvm *kvm, struct kvm_dirty_ring *ring,
+ int *nr_entries_reset)
{
+ /*
+ * To minimize mmu_lock contention, batch resets for harvested entries
+ * whose gfns are in the same slot, and are within N frame numbers of
+ * each other, where N is the number of bits in an unsigned long. For
+ * simplicity, process the current set of entries when the next entry
+ * can't be included in the batch.
+ *
+ * Track the current batch slot, the gfn offset into the slot for the
+ * batch, and the bitmask of gfns that need to be reset (relative to
+ * offset). Note, the offset may be adjusted backwards, e.g. so that
+ * a sequence of gfns X, X-1, ... X-N-1 can be batched.
+ */
u32 cur_slot, next_slot;
u64 cur_offset, next_offset;
- unsigned long mask;
- int count = 0;
+ unsigned long mask = 0;
struct kvm_dirty_gfn *entry;
- bool first_round = true;
- /* This is only needed to make compilers happy */
- cur_slot = cur_offset = mask = 0;
+ /*
+ * Ensure concurrent calls to KVM_RESET_DIRTY_RINGS are serialized,
+ * e.g. so that KVM fully resets all entries processed by a given call
+ * before returning to userspace. Holding slots_lock also protects
+ * the various memslot accesses.
+ */
+ lockdep_assert_held(&kvm->slots_lock);
+
+ while (likely((*nr_entries_reset) < INT_MAX)) {
+ if (signal_pending(current))
+ return -EINTR;
- while (true) {
entry = &ring->dirty_gfns[ring->reset_index & (ring->size - 1)];
if (!kvm_dirty_gfn_harvested(entry))
@@ -129,35 +146,64 @@ int kvm_dirty_ring_reset(struct kvm *kvm, struct kvm_dirty_ring *ring)
kvm_dirty_gfn_set_invalid(entry);
ring->reset_index++;
- count++;
- /*
- * Try to coalesce the reset operations when the guest is
- * scanning pages in the same slot.
- */
- if (!first_round && next_slot == cur_slot) {
- s64 delta = next_offset - cur_offset;
-
- if (delta >= 0 && delta < BITS_PER_LONG) {
- mask |= 1ull << delta;
- continue;
+ (*nr_entries_reset)++;
+
+ if (mask) {
+ /*
+ * While the size of each ring is fixed, it's possible
+ * for the ring to be constantly re-dirtied/harvested
+ * while the reset is in-progress (the hard limit exists
+ * only to guard against the count becoming negative).
+ */
+ cond_resched();
+
+ /*
+ * Try to coalesce the reset operations when the guest
+ * is scanning pages in the same slot.
+ */
+ if (next_slot == cur_slot) {
+ s64 delta = next_offset - cur_offset;
+
+ if (delta >= 0 && delta < BITS_PER_LONG) {
+ mask |= 1ull << delta;
+ continue;
+ }
+
+ /* Backwards visit, careful about overflows! */
+ if (delta > -BITS_PER_LONG && delta < 0 &&
+ (mask << -delta >> -delta) == mask) {
+ cur_offset = next_offset;
+ mask = (mask << -delta) | 1;
+ continue;
+ }
}
- /* Backwards visit, careful about overflows! */
- if (delta > -BITS_PER_LONG && delta < 0 &&
- (mask << -delta >> -delta) == mask) {
- cur_offset = next_offset;
- mask = (mask << -delta) | 1;
- continue;
- }
+ /*
+ * Reset the slot for all the harvested entries that
+ * have been gathered, but not yet fully processed.
+ */
+ kvm_reset_dirty_gfn(kvm, cur_slot, cur_offset, mask);
}
- kvm_reset_dirty_gfn(kvm, cur_slot, cur_offset, mask);
+
+ /*
+ * The current slot was reset or this is the first harvested
+ * entry, (re)initialize the metadata.
+ */
cur_slot = next_slot;
cur_offset = next_offset;
mask = 1;
- first_round = false;
}
- kvm_reset_dirty_gfn(kvm, cur_slot, cur_offset, mask);
+ /*
+ * Perform a final reset if there are harvested entries that haven't
+ * been processed, which is guaranteed if at least one harvested was
+ * found. The loop only performs a reset when the "next" entry can't
+ * be batched with the "current" entry(s), and that reset processes the
+ * _current_ entry(s); i.e. the last harvested entry, a.k.a. next, will
+ * always be left pending.
+ */
+ if (mask)
+ kvm_reset_dirty_gfn(kvm, cur_slot, cur_offset, mask);
/*
* The request KVM_REQ_DIRTY_RING_SOFT_FULL will be cleared
@@ -166,7 +212,7 @@ int kvm_dirty_ring_reset(struct kvm *kvm, struct kvm_dirty_ring *ring)
trace_kvm_dirty_ring_reset(ring);
- return count;
+ return 0;
}
void kvm_dirty_ring_push(struct kvm_vcpu *vcpu, u32 slot, u64 offset)
diff --git a/virt/kvm/eventfd.c b/virt/kvm/eventfd.c
index 11e5d1e3f12e..6b1133a6617f 100644
--- a/virt/kvm/eventfd.c
+++ b/virt/kvm/eventfd.c
@@ -204,6 +204,11 @@ irqfd_wakeup(wait_queue_entry_t *wait, unsigned mode, int sync, void *key)
int ret = 0;
if (flags & EPOLLIN) {
+ /*
+ * WARNING: Do NOT take irqfds.lock in any path except EPOLLHUP,
+ * as KVM holds irqfds.lock when registering the irqfd with the
+ * eventfd.
+ */
u64 cnt;
eventfd_ctx_do_read(irqfd->eventfd, &cnt);
@@ -225,6 +230,11 @@ irqfd_wakeup(wait_queue_entry_t *wait, unsigned mode, int sync, void *key)
/* The eventfd is closing, detach from KVM */
unsigned long iflags;
+ /*
+ * Taking irqfds.lock is safe here, as KVM holds a reference to
+ * the eventfd when registering the irqfd, i.e. this path can't
+ * be reached while kvm_irqfd_add() is running.
+ */
spin_lock_irqsave(&kvm->irqfds.lock, iflags);
/*
@@ -245,22 +255,14 @@ irqfd_wakeup(wait_queue_entry_t *wait, unsigned mode, int sync, void *key)
return ret;
}
-static void
-irqfd_ptable_queue_proc(struct file *file, wait_queue_head_t *wqh,
- poll_table *pt)
-{
- struct kvm_kernel_irqfd *irqfd =
- container_of(pt, struct kvm_kernel_irqfd, pt);
- add_wait_queue_priority(wqh, &irqfd->wait);
-}
-
-/* Must be called under irqfds.lock */
static void irqfd_update(struct kvm *kvm, struct kvm_kernel_irqfd *irqfd)
{
struct kvm_kernel_irq_routing_entry *e;
struct kvm_kernel_irq_routing_entry entries[KVM_NR_IRQCHIPS];
int n_entries;
+ lockdep_assert_held(&kvm->irqfds.lock);
+
n_entries = kvm_irq_map_gsi(kvm, entries, irqfd->gsi);
write_seqcount_begin(&irqfd->irq_entry_sc);
@@ -274,6 +276,63 @@ static void irqfd_update(struct kvm *kvm, struct kvm_kernel_irqfd *irqfd)
write_seqcount_end(&irqfd->irq_entry_sc);
}
+struct kvm_irqfd_pt {
+ struct kvm_kernel_irqfd *irqfd;
+ struct kvm *kvm;
+ poll_table pt;
+ int ret;
+};
+
+static void kvm_irqfd_register(struct file *file, wait_queue_head_t *wqh,
+ poll_table *pt)
+{
+ struct kvm_irqfd_pt *p = container_of(pt, struct kvm_irqfd_pt, pt);
+ struct kvm_kernel_irqfd *irqfd = p->irqfd;
+ struct kvm *kvm = p->kvm;
+
+ /*
+ * Note, irqfds.lock protects the irqfd's irq_entry, i.e. its routing,
+ * and irqfds.items. It does NOT protect registering with the eventfd.
+ */
+ spin_lock_irq(&kvm->irqfds.lock);
+
+ /*
+ * Initialize the routing information prior to adding the irqfd to the
+ * eventfd's waitqueue, as irqfd_wakeup() can be invoked as soon as the
+ * irqfd is registered.
+ */
+ irqfd_update(kvm, irqfd);
+
+ /*
+ * Add the irqfd as a priority waiter on the eventfd, with a custom
+ * wake-up handler, so that KVM *and only KVM* is notified whenever the
+ * underlying eventfd is signaled.
+ */
+ init_waitqueue_func_entry(&irqfd->wait, irqfd_wakeup);
+
+ /*
+ * Temporarily lie to lockdep about holding irqfds.lock to avoid a
+ * false positive regarding potential deadlock with irqfd_wakeup()
+ * (see irqfd_wakeup() for details).
+ *
+ * Adding to the wait queue will fail if there is already a priority
+ * waiter, i.e. if the eventfd is associated with another irqfd (in any
+ * VM). Note, kvm_irqfd_deassign() waits for all in-flight shutdown
+ * jobs to complete, i.e. ensures the irqfd has been removed from the
+ * eventfd's waitqueue before returning to userspace.
+ */
+ spin_release(&kvm->irqfds.lock.dep_map, _RET_IP_);
+ p->ret = add_wait_queue_priority_exclusive(wqh, &irqfd->wait);
+ spin_acquire(&kvm->irqfds.lock.dep_map, 0, 0, _RET_IP_);
+ if (p->ret)
+ goto out;
+
+ list_add_tail(&irqfd->list, &kvm->irqfds.items);
+
+out:
+ spin_unlock_irq(&kvm->irqfds.lock);
+}
+
#if IS_ENABLED(CONFIG_HAVE_KVM_IRQ_BYPASS)
void __attribute__((weak)) kvm_arch_irq_bypass_stop(
struct irq_bypass_consumer *cons)
@@ -285,26 +344,20 @@ void __attribute__((weak)) kvm_arch_irq_bypass_start(
{
}
-int __attribute__((weak)) kvm_arch_update_irqfd_routing(
- struct kvm *kvm, unsigned int host_irq,
- uint32_t guest_irq, bool set)
+void __weak kvm_arch_update_irqfd_routing(struct kvm_kernel_irqfd *irqfd,
+ struct kvm_kernel_irq_routing_entry *old,
+ struct kvm_kernel_irq_routing_entry *new)
{
- return 0;
-}
-bool __attribute__((weak)) kvm_arch_irqfd_route_changed(
- struct kvm_kernel_irq_routing_entry *old,
- struct kvm_kernel_irq_routing_entry *new)
-{
- return true;
}
#endif
static int
kvm_irqfd_assign(struct kvm *kvm, struct kvm_irqfd *args)
{
- struct kvm_kernel_irqfd *irqfd, *tmp;
+ struct kvm_kernel_irqfd *irqfd;
struct eventfd_ctx *eventfd = NULL, *resamplefd = NULL;
+ struct kvm_irqfd_pt irqfd_pt;
int ret;
__poll_t events;
int idx;
@@ -390,57 +443,54 @@ kvm_irqfd_assign(struct kvm *kvm, struct kvm_irqfd *args)
}
/*
- * Install our own custom wake-up handling so we are notified via
- * a callback whenever someone signals the underlying eventfd
+ * Set the irqfd routing and add it to KVM's list before registering
+ * the irqfd with the eventfd, so that the routing information is valid
+ * and stays valid, e.g. if there are GSI routing changes, prior to
+ * making the irqfd visible, i.e. before it might be signaled.
+ *
+ * Note, holding SRCU ensures a stable read of routing information, and
+ * also prevents irqfd_shutdown() from freeing the irqfd before it's
+ * fully initialized.
*/
- init_waitqueue_func_entry(&irqfd->wait, irqfd_wakeup);
- init_poll_funcptr(&irqfd->pt, irqfd_ptable_queue_proc);
-
- spin_lock_irq(&kvm->irqfds.lock);
-
- ret = 0;
- list_for_each_entry(tmp, &kvm->irqfds.items, list) {
- if (irqfd->eventfd != tmp->eventfd)
- continue;
- /* This fd is used for another irq already. */
- ret = -EBUSY;
- spin_unlock_irq(&kvm->irqfds.lock);
- goto fail;
- }
-
idx = srcu_read_lock(&kvm->irq_srcu);
- irqfd_update(kvm, irqfd);
-
- list_add_tail(&irqfd->list, &kvm->irqfds.items);
-
- spin_unlock_irq(&kvm->irqfds.lock);
/*
- * Check if there was an event already pending on the eventfd
- * before we registered, and trigger it as if we didn't miss it.
+ * Register the irqfd with the eventfd by polling on the eventfd, and
+ * simultaneously and the irqfd to KVM's list. If there was en event
+ * pending on the eventfd prior to registering, manually trigger IRQ
+ * injection.
*/
- events = vfs_poll(fd_file(f), &irqfd->pt);
+ irqfd_pt.irqfd = irqfd;
+ irqfd_pt.kvm = kvm;
+ init_poll_funcptr(&irqfd_pt.pt, kvm_irqfd_register);
+
+ events = vfs_poll(fd_file(f), &irqfd_pt.pt);
+
+ ret = irqfd_pt.ret;
+ if (ret)
+ goto fail_poll;
if (events & EPOLLIN)
schedule_work(&irqfd->inject);
#if IS_ENABLED(CONFIG_HAVE_KVM_IRQ_BYPASS)
if (kvm_arch_has_irq_bypass()) {
- irqfd->consumer.token = (void *)irqfd->eventfd;
irqfd->consumer.add_producer = kvm_arch_irq_bypass_add_producer;
irqfd->consumer.del_producer = kvm_arch_irq_bypass_del_producer;
irqfd->consumer.stop = kvm_arch_irq_bypass_stop;
irqfd->consumer.start = kvm_arch_irq_bypass_start;
- ret = irq_bypass_register_consumer(&irqfd->consumer);
+ ret = irq_bypass_register_consumer(&irqfd->consumer, irqfd->eventfd);
if (ret)
- pr_info("irq bypass consumer (token %p) registration fails: %d\n",
- irqfd->consumer.token, ret);
+ pr_info("irq bypass consumer (eventfd %p) registration fails: %d\n",
+ irqfd->eventfd, ret);
}
#endif
srcu_read_unlock(&kvm->irq_srcu, idx);
return 0;
+fail_poll:
+ srcu_read_unlock(&kvm->irq_srcu, idx);
fail:
if (irqfd->resampler)
irqfd_resampler_shutdown(irqfd);
@@ -617,13 +667,8 @@ void kvm_irq_routing_update(struct kvm *kvm)
irqfd_update(kvm, irqfd);
#if IS_ENABLED(CONFIG_HAVE_KVM_IRQ_BYPASS)
- if (irqfd->producer &&
- kvm_arch_irqfd_route_changed(&old, &irqfd->irq_entry)) {
- int ret = kvm_arch_update_irqfd_routing(
- irqfd->kvm, irqfd->producer->irq,
- irqfd->gsi, 1);
- WARN_ON(ret);
- }
+ if (irqfd->producer)
+ kvm_arch_update_irqfd_routing(irqfd, &old, &irqfd->irq_entry);
#endif
}
diff --git a/virt/kvm/guest_memfd.c b/virt/kvm/guest_memfd.c
index b2aa6bf24d3a..7d85cc33c0bb 100644
--- a/virt/kvm/guest_memfd.c
+++ b/virt/kvm/guest_memfd.c
@@ -382,23 +382,12 @@ static const struct address_space_operations kvm_gmem_aops = {
#endif
};
-static int kvm_gmem_getattr(struct mnt_idmap *idmap, const struct path *path,
- struct kstat *stat, u32 request_mask,
- unsigned int query_flags)
-{
- struct inode *inode = path->dentry->d_inode;
-
- generic_fillattr(idmap, request_mask, inode, stat);
- return 0;
-}
-
static int kvm_gmem_setattr(struct mnt_idmap *idmap, struct dentry *dentry,
struct iattr *attr)
{
return -EINVAL;
}
static const struct inode_operations kvm_gmem_iops = {
- .getattr = kvm_gmem_getattr,
.setattr = kvm_gmem_setattr,
};
diff --git a/virt/kvm/irqchip.c b/virt/kvm/irqchip.c
index 162d8ed889f2..6ccabfd32287 100644
--- a/virt/kvm/irqchip.c
+++ b/virt/kvm/irqchip.c
@@ -222,8 +222,6 @@ int kvm_set_irq_routing(struct kvm *kvm,
kvm_arch_irq_routing_update(kvm);
mutex_unlock(&kvm->irq_lock);
- kvm_arch_post_irq_routing_update(kvm);
-
synchronize_srcu_expedited(&kvm->irq_srcu);
new = old;
diff --git a/virt/kvm/kvm_main.c b/virt/kvm/kvm_main.c
index e85b33a92624..6c07dd423458 100644
--- a/virt/kvm/kvm_main.c
+++ b/virt/kvm/kvm_main.c
@@ -143,8 +143,6 @@ static int kvm_no_compat_open(struct inode *inode, struct file *file)
#define KVM_COMPAT(c) .compat_ioctl = kvm_no_compat_ioctl, \
.open = kvm_no_compat_open
#endif
-static int kvm_enable_virtualization(void);
-static void kvm_disable_virtualization(void);
static void kvm_io_bus_destroy(struct kvm_io_bus *bus);
@@ -1370,6 +1368,65 @@ static int kvm_vm_release(struct inode *inode, struct file *filp)
return 0;
}
+int kvm_trylock_all_vcpus(struct kvm *kvm)
+{
+ struct kvm_vcpu *vcpu;
+ unsigned long i, j;
+
+ lockdep_assert_held(&kvm->lock);
+
+ kvm_for_each_vcpu(i, vcpu, kvm)
+ if (!mutex_trylock_nest_lock(&vcpu->mutex, &kvm->lock))
+ goto out_unlock;
+ return 0;
+
+out_unlock:
+ kvm_for_each_vcpu(j, vcpu, kvm) {
+ if (i == j)
+ break;
+ mutex_unlock(&vcpu->mutex);
+ }
+ return -EINTR;
+}
+EXPORT_SYMBOL_GPL(kvm_trylock_all_vcpus);
+
+int kvm_lock_all_vcpus(struct kvm *kvm)
+{
+ struct kvm_vcpu *vcpu;
+ unsigned long i, j;
+ int r;
+
+ lockdep_assert_held(&kvm->lock);
+
+ kvm_for_each_vcpu(i, vcpu, kvm) {
+ r = mutex_lock_killable_nest_lock(&vcpu->mutex, &kvm->lock);
+ if (r)
+ goto out_unlock;
+ }
+ return 0;
+
+out_unlock:
+ kvm_for_each_vcpu(j, vcpu, kvm) {
+ if (i == j)
+ break;
+ mutex_unlock(&vcpu->mutex);
+ }
+ return r;
+}
+EXPORT_SYMBOL_GPL(kvm_lock_all_vcpus);
+
+void kvm_unlock_all_vcpus(struct kvm *kvm)
+{
+ struct kvm_vcpu *vcpu;
+ unsigned long i;
+
+ lockdep_assert_held(&kvm->lock);
+
+ kvm_for_each_vcpu(i, vcpu, kvm)
+ mutex_unlock(&vcpu->mutex);
+}
+EXPORT_SYMBOL_GPL(kvm_unlock_all_vcpus);
+
/*
* Allocation size is twice as large as the actual dirty bitmap size.
* See kvm_vm_ioctl_get_dirty_log() why this is needed.
@@ -2501,9 +2558,11 @@ static int kvm_vm_set_mem_attributes(struct kvm *kvm, gfn_t start, gfn_t end,
entry = attributes ? xa_mk_value(attributes) : NULL;
+ trace_kvm_vm_set_mem_attributes(start, end, attributes);
+
mutex_lock(&kvm->slots_lock);
- /* Nothing to do if the entire range as the desired attributes. */
+ /* Nothing to do if the entire range has the desired attributes. */
if (kvm_range_has_memory_attributes(kvm, start, end, ~0, attributes))
goto out_unlock;
@@ -2515,6 +2574,8 @@ static int kvm_vm_set_mem_attributes(struct kvm *kvm, gfn_t start, gfn_t end,
r = xa_reserve(&kvm->mem_attr_array, i, GFP_KERNEL_ACCOUNT);
if (r)
goto out_unlock;
+
+ cond_resched();
}
kvm_handle_gfn_range(kvm, &pre_set_range);
@@ -2523,6 +2584,7 @@ static int kvm_vm_set_mem_attributes(struct kvm *kvm, gfn_t start, gfn_t end,
r = xa_err(xa_store(&kvm->mem_attr_array, i, entry,
GFP_KERNEL_ACCOUNT));
KVM_BUG_ON(r, kvm);
+ cond_resched();
}
kvm_handle_gfn_range(kvm, &post_set_range);
@@ -3741,7 +3803,7 @@ EXPORT_SYMBOL_GPL(kvm_vcpu_wake_up);
/*
* Kick a sleeping VCPU, or a guest VCPU in guest mode, into host kernel mode.
*/
-void kvm_vcpu_kick(struct kvm_vcpu *vcpu)
+void __kvm_vcpu_kick(struct kvm_vcpu *vcpu, bool wait)
{
int me, cpu;
@@ -3770,13 +3832,24 @@ void kvm_vcpu_kick(struct kvm_vcpu *vcpu)
*/
if (kvm_arch_vcpu_should_kick(vcpu)) {
cpu = READ_ONCE(vcpu->cpu);
- if (cpu != me && (unsigned)cpu < nr_cpu_ids && cpu_online(cpu))
- smp_send_reschedule(cpu);
+ if (cpu != me && (unsigned int)cpu < nr_cpu_ids && cpu_online(cpu)) {
+ /*
+ * Use a reschedule IPI to kick the vCPU if the caller
+ * doesn't need to wait for a response, as KVM allows
+ * kicking vCPUs while IRQs are disabled, but using the
+ * SMP function call framework with IRQs disabled can
+ * deadlock due to taking cross-CPU locks.
+ */
+ if (wait)
+ smp_call_function_single(cpu, ack_kick, NULL, wait);
+ else
+ smp_send_reschedule(cpu);
+ }
}
out:
put_cpu();
}
-EXPORT_SYMBOL_GPL(kvm_vcpu_kick);
+EXPORT_SYMBOL_GPL(__kvm_vcpu_kick);
#endif /* !CONFIG_S390 */
int kvm_vcpu_yield_to(struct kvm_vcpu *target)
@@ -4126,7 +4199,7 @@ static int kvm_vm_ioctl_create_vcpu(struct kvm *kvm, unsigned long id)
goto vcpu_free_run_page;
if (kvm->dirty_ring_size) {
- r = kvm_dirty_ring_alloc(&vcpu->dirty_ring,
+ r = kvm_dirty_ring_alloc(kvm, &vcpu->dirty_ring,
id, kvm->dirty_ring_size);
if (r)
goto arch_vcpu_destroy;
@@ -4864,7 +4937,7 @@ static int kvm_vm_ioctl_enable_dirty_log_ring(struct kvm *kvm, u32 size)
return -EINVAL;
/* Should be bigger to keep the reserved entries, or a page */
- if (size < kvm_dirty_ring_get_rsvd_entries() *
+ if (size < kvm_dirty_ring_get_rsvd_entries(kvm) *
sizeof(struct kvm_dirty_gfn) || size < PAGE_SIZE)
return -EINVAL;
@@ -4894,15 +4967,18 @@ static int kvm_vm_ioctl_reset_dirty_pages(struct kvm *kvm)
{
unsigned long i;
struct kvm_vcpu *vcpu;
- int cleared = 0;
+ int cleared = 0, r;
if (!kvm->dirty_ring_size)
return -EINVAL;
mutex_lock(&kvm->slots_lock);
- kvm_for_each_vcpu(i, vcpu, kvm)
- cleared += kvm_dirty_ring_reset(vcpu->kvm, &vcpu->dirty_ring);
+ kvm_for_each_vcpu(i, vcpu, kvm) {
+ r = kvm_dirty_ring_reset(vcpu->kvm, &vcpu->dirty_ring, &cleared);
+ if (r)
+ break;
+ }
mutex_unlock(&kvm->slots_lock);
@@ -5479,8 +5555,9 @@ static struct miscdevice kvm_dev = {
};
#ifdef CONFIG_KVM_GENERIC_HARDWARE_ENABLING
-static bool enable_virt_at_load = true;
+bool enable_virt_at_load = true;
module_param(enable_virt_at_load, bool, 0444);
+EXPORT_SYMBOL_GPL(enable_virt_at_load);
__visible bool kvm_rebooting;
EXPORT_SYMBOL_GPL(kvm_rebooting);
@@ -5589,7 +5666,7 @@ static struct syscore_ops kvm_syscore_ops = {
.shutdown = kvm_shutdown,
};
-static int kvm_enable_virtualization(void)
+int kvm_enable_virtualization(void)
{
int r;
@@ -5634,8 +5711,9 @@ err_cpuhp:
--kvm_usage_count;
return r;
}
+EXPORT_SYMBOL_GPL(kvm_enable_virtualization);
-static void kvm_disable_virtualization(void)
+void kvm_disable_virtualization(void)
{
guard(mutex)(&kvm_usage_lock);
@@ -5646,6 +5724,7 @@ static void kvm_disable_virtualization(void)
cpuhp_remove_state(CPUHP_AP_KVM_ONLINE);
kvm_arch_disable_virtualization();
}
+EXPORT_SYMBOL_GPL(kvm_disable_virtualization);
static int kvm_init_virtualization(void)
{
@@ -5661,21 +5740,11 @@ static void kvm_uninit_virtualization(void)
kvm_disable_virtualization();
}
#else /* CONFIG_KVM_GENERIC_HARDWARE_ENABLING */
-static int kvm_enable_virtualization(void)
-{
- return 0;
-}
-
static int kvm_init_virtualization(void)
{
return 0;
}
-static void kvm_disable_virtualization(void)
-{
-
-}
-
static void kvm_uninit_virtualization(void)
{
@@ -5774,7 +5843,6 @@ static int __kvm_io_bus_write(struct kvm_vcpu *vcpu, struct kvm_io_bus *bus,
return -EOPNOTSUPP;
}
-/* kvm_io_bus_write - called under kvm->slots_lock */
int kvm_io_bus_write(struct kvm_vcpu *vcpu, enum kvm_bus bus_idx, gpa_t addr,
int len, const void *val)
{
@@ -5795,7 +5863,6 @@ int kvm_io_bus_write(struct kvm_vcpu *vcpu, enum kvm_bus bus_idx, gpa_t addr,
}
EXPORT_SYMBOL_GPL(kvm_io_bus_write);
-/* kvm_io_bus_write_cookie - called under kvm->slots_lock */
int kvm_io_bus_write_cookie(struct kvm_vcpu *vcpu, enum kvm_bus bus_idx,
gpa_t addr, int len, const void *val, long cookie)
{
@@ -5845,7 +5912,6 @@ static int __kvm_io_bus_read(struct kvm_vcpu *vcpu, struct kvm_io_bus *bus,
return -EOPNOTSUPP;
}
-/* kvm_io_bus_read - called under kvm->slots_lock */
int kvm_io_bus_read(struct kvm_vcpu *vcpu, enum kvm_bus bus_idx, gpa_t addr,
int len, void *val)
{
@@ -5864,6 +5930,7 @@ int kvm_io_bus_read(struct kvm_vcpu *vcpu, enum kvm_bus bus_idx, gpa_t addr,
r = __kvm_io_bus_read(vcpu, bus, &range, val);
return r < 0 ? r : 0;
}
+EXPORT_SYMBOL_GPL(kvm_io_bus_read);
int kvm_io_bus_register_dev(struct kvm *kvm, enum kvm_bus bus_idx, gpa_t addr,
int len, struct kvm_io_device *dev)
diff --git a/virt/kvm/vfio.c b/virt/kvm/vfio.c
index 196a102e34fb..be50514bbd11 100644
--- a/virt/kvm/vfio.c
+++ b/virt/kvm/vfio.c
@@ -175,7 +175,6 @@ static int kvm_vfio_file_add(struct kvm_device *dev, unsigned int fd)
kvf->file = get_file(filp);
list_add_tail(&kvf->node, &kv->file_list);
- kvm_arch_start_assignment(dev->kvm);
kvm_vfio_file_set_kvm(kvf->file, dev->kvm);
kvm_vfio_update_coherency(dev);
@@ -205,7 +204,6 @@ static int kvm_vfio_file_del(struct kvm_device *dev, unsigned int fd)
continue;
list_del(&kvf->node);
- kvm_arch_end_assignment(dev->kvm);
#ifdef CONFIG_SPAPR_TCE_IOMMU
kvm_spapr_tce_release_vfio_group(dev->kvm, kvf);
#endif
@@ -336,7 +334,6 @@ static void kvm_vfio_release(struct kvm_device *dev)
fput(kvf->file);
list_del(&kvf->node);
kfree(kvf);
- kvm_arch_end_assignment(dev->kvm);
}
kvm_vfio_update_coherency(dev);