summaryrefslogtreecommitdiff
path: root/virt
diff options
context:
space:
mode:
Diffstat (limited to 'virt')
-rw-r--r--virt/kvm/dirty_ring.c109
-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.c16
-rw-r--r--virt/kvm/vfio.c3
-rw-r--r--virt/lib/irqbypass.c190
7 files changed, 260 insertions, 230 deletions
diff --git a/virt/kvm/dirty_ring.c b/virt/kvm/dirty_ring.c
index d14ffc7513ee..02bc6b00d76c 100644
--- a/virt/kvm/dirty_ring.c
+++ b/virt/kvm/dirty_ring.c
@@ -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;
@@ -105,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))
@@ -130,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
@@ -167,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 eec82775c5bf..6c07dd423458 100644
--- a/virt/kvm/kvm_main.c
+++ b/virt/kvm/kvm_main.c
@@ -2558,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;
@@ -2572,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);
@@ -2580,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);
@@ -4962,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);
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);
diff --git a/virt/lib/irqbypass.c b/virt/lib/irqbypass.c
index 28fda42e471b..62c160200be9 100644
--- a/virt/lib/irqbypass.c
+++ b/virt/lib/irqbypass.c
@@ -22,8 +22,8 @@
MODULE_LICENSE("GPL v2");
MODULE_DESCRIPTION("IRQ bypass manager utility module");
-static LIST_HEAD(producers);
-static LIST_HEAD(consumers);
+static DEFINE_XARRAY(producers);
+static DEFINE_XARRAY(consumers);
static DEFINE_MUTEX(lock);
/* @lock must be held when calling connect */
@@ -51,6 +51,10 @@ static int __connect(struct irq_bypass_producer *prod,
if (prod->start)
prod->start(prod);
+ if (!ret) {
+ prod->consumer = cons;
+ cons->producer = prod;
+ }
return ret;
}
@@ -72,56 +76,49 @@ static void __disconnect(struct irq_bypass_producer *prod,
cons->start(cons);
if (prod->start)
prod->start(prod);
+
+ prod->consumer = NULL;
+ cons->producer = NULL;
}
/**
* irq_bypass_register_producer - register IRQ bypass producer
* @producer: pointer to producer structure
+ * @eventfd: pointer to the eventfd context associated with the producer
+ * @irq: Linux IRQ number of the underlying producer device
*
- * Add the provided IRQ producer to the list of producers and connect
- * with any matching token found on the IRQ consumers list.
+ * Add the provided IRQ producer to the set of producers and connect with the
+ * consumer with a matching eventfd, if one exists.
*/
-int irq_bypass_register_producer(struct irq_bypass_producer *producer)
+int irq_bypass_register_producer(struct irq_bypass_producer *producer,
+ struct eventfd_ctx *eventfd, int irq)
{
- struct irq_bypass_producer *tmp;
+ unsigned long index = (unsigned long)eventfd;
struct irq_bypass_consumer *consumer;
int ret;
- if (!producer->token)
+ if (WARN_ON_ONCE(producer->eventfd))
return -EINVAL;
- might_sleep();
-
- if (!try_module_get(THIS_MODULE))
- return -ENODEV;
+ producer->irq = irq;
- mutex_lock(&lock);
+ guard(mutex)(&lock);
- list_for_each_entry(tmp, &producers, node) {
- if (tmp->token == producer->token) {
- ret = -EBUSY;
- goto out_err;
- }
- }
+ ret = xa_insert(&producers, index, producer, GFP_KERNEL);
+ if (ret)
+ return ret;
- list_for_each_entry(consumer, &consumers, node) {
- if (consumer->token == producer->token) {
- ret = __connect(producer, consumer);
- if (ret)
- goto out_err;
- break;
+ consumer = xa_load(&consumers, index);
+ if (consumer) {
+ ret = __connect(producer, consumer);
+ if (ret) {
+ WARN_ON_ONCE(xa_erase(&producers, index) != producer);
+ return ret;
}
}
- list_add(&producer->node, &producers);
-
- mutex_unlock(&lock);
-
+ producer->eventfd = eventfd;
return 0;
-out_err:
- mutex_unlock(&lock);
- module_put(THIS_MODULE);
- return ret;
}
EXPORT_SYMBOL_GPL(irq_bypass_register_producer);
@@ -129,95 +126,65 @@ EXPORT_SYMBOL_GPL(irq_bypass_register_producer);
* irq_bypass_unregister_producer - unregister IRQ bypass producer
* @producer: pointer to producer structure
*
- * Remove a previously registered IRQ producer from the list of producers
- * and disconnect it from any connected IRQ consumer.
+ * Remove a previously registered IRQ producer (note, it's safe to call this
+ * even if registration was unsuccessful). Disconnect from the associated
+ * consumer, if one exists.
*/
void irq_bypass_unregister_producer(struct irq_bypass_producer *producer)
{
- struct irq_bypass_producer *tmp;
- struct irq_bypass_consumer *consumer;
+ unsigned long index = (unsigned long)producer->eventfd;
- if (!producer->token)
+ if (!producer->eventfd)
return;
- might_sleep();
-
- if (!try_module_get(THIS_MODULE))
- return; /* nothing in the list anyway */
-
- mutex_lock(&lock);
-
- list_for_each_entry(tmp, &producers, node) {
- if (tmp->token != producer->token)
- continue;
-
- list_for_each_entry(consumer, &consumers, node) {
- if (consumer->token == producer->token) {
- __disconnect(producer, consumer);
- break;
- }
- }
-
- list_del(&producer->node);
- module_put(THIS_MODULE);
- break;
- }
+ guard(mutex)(&lock);
- mutex_unlock(&lock);
+ if (producer->consumer)
+ __disconnect(producer, producer->consumer);
- module_put(THIS_MODULE);
+ WARN_ON_ONCE(xa_erase(&producers, index) != producer);
+ producer->eventfd = NULL;
}
EXPORT_SYMBOL_GPL(irq_bypass_unregister_producer);
/**
* irq_bypass_register_consumer - register IRQ bypass consumer
* @consumer: pointer to consumer structure
+ * @eventfd: pointer to the eventfd context associated with the consumer
*
- * Add the provided IRQ consumer to the list of consumers and connect
- * with any matching token found on the IRQ producer list.
+ * Add the provided IRQ consumer to the set of consumers and connect with the
+ * producer with a matching eventfd, if one exists.
*/
-int irq_bypass_register_consumer(struct irq_bypass_consumer *consumer)
+int irq_bypass_register_consumer(struct irq_bypass_consumer *consumer,
+ struct eventfd_ctx *eventfd)
{
- struct irq_bypass_consumer *tmp;
+ unsigned long index = (unsigned long)eventfd;
struct irq_bypass_producer *producer;
int ret;
- if (!consumer->token ||
- !consumer->add_producer || !consumer->del_producer)
+ if (WARN_ON_ONCE(consumer->eventfd))
return -EINVAL;
- might_sleep();
-
- if (!try_module_get(THIS_MODULE))
- return -ENODEV;
+ if (!consumer->add_producer || !consumer->del_producer)
+ return -EINVAL;
- mutex_lock(&lock);
+ guard(mutex)(&lock);
- list_for_each_entry(tmp, &consumers, node) {
- if (tmp->token == consumer->token || tmp == consumer) {
- ret = -EBUSY;
- goto out_err;
- }
- }
+ ret = xa_insert(&consumers, index, consumer, GFP_KERNEL);
+ if (ret)
+ return ret;
- list_for_each_entry(producer, &producers, node) {
- if (producer->token == consumer->token) {
- ret = __connect(producer, consumer);
- if (ret)
- goto out_err;
- break;
+ producer = xa_load(&producers, index);
+ if (producer) {
+ ret = __connect(producer, consumer);
+ if (ret) {
+ WARN_ON_ONCE(xa_erase(&consumers, index) != consumer);
+ return ret;
}
}
- list_add(&consumer->node, &consumers);
-
- mutex_unlock(&lock);
-
+ consumer->eventfd = eventfd;
return 0;
-out_err:
- mutex_unlock(&lock);
- module_put(THIS_MODULE);
- return ret;
}
EXPORT_SYMBOL_GPL(irq_bypass_register_consumer);
@@ -225,42 +192,23 @@ EXPORT_SYMBOL_GPL(irq_bypass_register_consumer);
* irq_bypass_unregister_consumer - unregister IRQ bypass consumer
* @consumer: pointer to consumer structure
*
- * Remove a previously registered IRQ consumer from the list of consumers
- * and disconnect it from any connected IRQ producer.
+ * Remove a previously registered IRQ consumer (note, it's safe to call this
+ * even if registration was unsuccessful). Disconnect from the associated
+ * producer, if one exists.
*/
void irq_bypass_unregister_consumer(struct irq_bypass_consumer *consumer)
{
- struct irq_bypass_consumer *tmp;
- struct irq_bypass_producer *producer;
+ unsigned long index = (unsigned long)consumer->eventfd;
- if (!consumer->token)
+ if (!consumer->eventfd)
return;
- might_sleep();
-
- if (!try_module_get(THIS_MODULE))
- return; /* nothing in the list anyway */
-
- mutex_lock(&lock);
-
- list_for_each_entry(tmp, &consumers, node) {
- if (tmp != consumer)
- continue;
-
- list_for_each_entry(producer, &producers, node) {
- if (producer->token == consumer->token) {
- __disconnect(producer, consumer);
- break;
- }
- }
-
- list_del(&consumer->node);
- module_put(THIS_MODULE);
- break;
- }
+ guard(mutex)(&lock);
- mutex_unlock(&lock);
+ if (consumer->producer)
+ __disconnect(consumer->producer, consumer);
- module_put(THIS_MODULE);
+ WARN_ON_ONCE(xa_erase(&consumers, index) != consumer);
+ consumer->eventfd = NULL;
}
EXPORT_SYMBOL_GPL(irq_bypass_unregister_consumer);