All of lore.kernel.org
 help / color / mirror / Atom feed
* [KVM PATCH v3 0/3] irqfd/eventfd fixes
@ 2009-06-22 16:05 Gregory Haskins
  2009-06-22 16:05 ` [KVM PATCH v3 1/3] kvm: prepare irqfd for having interrupts disabled during eventfd->release Gregory Haskins
                   ` (2 more replies)
  0 siblings, 3 replies; 14+ messages in thread
From: Gregory Haskins @ 2009-06-22 16:05 UTC (permalink / raw)
  To: kvm; +Cc: linux-kernel, mst, avi, paulmck, davidel, mingo, rusty

(Applies to kvm.git/master:4631e094)

The following is the latest attempt to fix the remaining races in
irqfd/eventfd.  For more details, please read the patch headers.

This series has been tested against the kvm-eventfd unit test, and
appears to be functioning properly.  You can download this test here:

ftp://ftp.novell.com/dev/ghaskins/kvm-eventfd.tar.bz2

(Note: I believe either the eventfd solution in 2/3, or the patch that
Davide has proposed should work equally well.  Its purely a question of
which direction we want to go in w.r.t. the wakeup registration).

---

Gregory Haskins (3):
      KVM: Fix races in irqfd using new eventfd_kref_get interface
      eventfd: add internal reference counting to fix notifier race conditions
      kvm: prepare irqfd for having interrupts disabled during eventfd->release


 fs/eventfd.c            |   43 ++++++++++--
 include/linux/eventfd.h |    7 ++
 virt/kvm/eventfd.c      |  173 +++++++++++++++++++++++++++++------------------
 3 files changed, 150 insertions(+), 73 deletions(-)

-- 
Signature

^ permalink raw reply	[flat|nested] 14+ messages in thread

* [KVM PATCH v3 1/3] kvm: prepare irqfd for having interrupts disabled during eventfd->release
  2009-06-22 16:05 [KVM PATCH v3 0/3] irqfd/eventfd fixes Gregory Haskins
@ 2009-06-22 16:05 ` Gregory Haskins
  2009-06-22 16:05 ` [KVM PATCH v3 2/3] eventfd: add internal reference counting to fix notifier race conditions Gregory Haskins
  2009-06-22 16:05 ` [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface Gregory Haskins
  2 siblings, 0 replies; 14+ messages in thread
From: Gregory Haskins @ 2009-06-22 16:05 UTC (permalink / raw)
  To: kvm; +Cc: linux-kernel, mst, avi, paulmck, davidel, mingo, rusty

We need to plug some race conditions on eventfd shutdown.  In order to
do this, we need to change the context in which the release notification
is delivered so that the wqh lock is now held.  However, there is currently
code in the release callback that assumes it can sleep.

We have a slight chicken and egg problem where we cant fix the race
without adding the lock, and we can't add the lock without breaking
the sleepy code.  Normally we could deal with this by making both
changes in an atomic changeset.  However, we want to keep the eventfd
and kvm specific changes isolated to ease the reviewer burden on upstream
eventfd (at the specific request of upstream).  Therefore, we have this
intermediate patch.

This intermediate patch allows the release() method to work in an atomic
context, at the expense of correctness w.r.t. memory-leaks.  Today we have
a race condition.  With this patch applied we leak.  Both issues will be
resolved later in the series.  It is the author's opinion that a leak is
better for bisectability than the hang would be should we leave the sleepy
code in place after the locking changeover.

Signed-off-by: Gregory Haskins <ghaskins@novell.com>
---

 virt/kvm/eventfd.c |   89 ++++++++++++++++------------------------------------
 1 files changed, 27 insertions(+), 62 deletions(-)

diff --git a/virt/kvm/eventfd.c b/virt/kvm/eventfd.c
index a9e7de7..9656027 100644
--- a/virt/kvm/eventfd.c
+++ b/virt/kvm/eventfd.c
@@ -28,7 +28,6 @@
 #include <linux/file.h>
 #include <linux/list.h>
 #include <linux/eventfd.h>
-#include <linux/srcu.h>
 
 /*
  * --------------------------------------------------------------------
@@ -38,8 +37,6 @@
  * --------------------------------------------------------------------
  */
 struct _irqfd {
-	struct mutex              lock;
-	struct srcu_struct        srcu;
 	struct kvm               *kvm;
 	int                       gsi;
 	struct list_head          list;
@@ -53,48 +50,12 @@ static void
 irqfd_inject(struct work_struct *work)
 {
 	struct _irqfd *irqfd = container_of(work, struct _irqfd, inject);
-	struct kvm *kvm;
-	int idx;
+	struct kvm *kvm = irqfd->kvm;
 
-	idx = srcu_read_lock(&irqfd->srcu);
-
-	kvm = rcu_dereference(irqfd->kvm);
-	if (kvm) {
-		mutex_lock(&kvm->irq_lock);
-		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
-		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
-		mutex_unlock(&kvm->irq_lock);
-	}
-
-	srcu_read_unlock(&irqfd->srcu, idx);
-}
-
-static void
-irqfd_disconnect(struct _irqfd *irqfd)
-{
-	struct kvm *kvm;
-
-	mutex_lock(&irqfd->lock);
-
-	kvm = rcu_dereference(irqfd->kvm);
-	rcu_assign_pointer(irqfd->kvm, NULL);
-
-	mutex_unlock(&irqfd->lock);
-
-	if (!kvm)
-		return;
-
-	mutex_lock(&kvm->lock);
-	list_del(&irqfd->list);
-	mutex_unlock(&kvm->lock);
-
-	/*
-	 * It is important to not drop the kvm reference until the next grace
-	 * period because there might be lockless references in flight up
-	 * until then
-	 */
-	synchronize_srcu(&irqfd->srcu);
-	kvm_put_kvm(kvm);
+	mutex_lock(&kvm->irq_lock);
+	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
+	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
+	mutex_unlock(&kvm->irq_lock);
 }
 
 static int
@@ -103,26 +64,24 @@ irqfd_wakeup(wait_queue_t *wait, unsigned mode, int sync, void *key)
 	struct _irqfd *irqfd = container_of(wait, struct _irqfd, wait);
 	unsigned long flags = (unsigned long)key;
 
+	/*
+	 * Assume we will be called with interrupts disabled
+	 */
 	if (flags & POLLIN)
 		/*
-		 * The POLLIN wake_up is called with interrupts disabled.
-		 * Therefore we need to defer the IRQ injection until later
-		 * since we need to acquire the kvm->lock to do so.
+		 * Defer the IRQ injection until later since we need to
+		 * acquire the kvm->lock to do so.
 		 */
 		schedule_work(&irqfd->inject);
 
 	if (flags & POLLHUP) {
 		/*
-		 * The POLLHUP is called unlocked, so it theoretically should
-		 * be safe to remove ourselves from the wqh using the locked
-		 * variant of remove_wait_queue()
+		 * for now, just remove ourselves from the list and let
+		 * the rest dangle.  We will fix this up later once
+		 * the races in eventfd are fixed
 		 */
-		remove_wait_queue(irqfd->wqh, &irqfd->wait);
-		flush_work(&irqfd->inject);
-		irqfd_disconnect(irqfd);
-
-		cleanup_srcu_struct(&irqfd->srcu);
-		kfree(irqfd);
+		__remove_wait_queue(irqfd->wqh, &irqfd->wait);
+		irqfd->wqh = NULL;
 	}
 
 	return 0;
@@ -150,8 +109,6 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
 	if (!irqfd)
 		return -ENOMEM;
 
-	mutex_init(&irqfd->lock);
-	init_srcu_struct(&irqfd->srcu);
 	irqfd->kvm = kvm;
 	irqfd->gsi = gsi;
 	INIT_LIST_HEAD(&irqfd->list);
@@ -172,8 +129,6 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
 
 	events = file->f_op->poll(file, &irqfd->pt);
 
-	kvm_get_kvm(kvm);
-
 	mutex_lock(&kvm->lock);
 	list_add_tail(&irqfd->list, &kvm->irqfds);
 	mutex_unlock(&kvm->lock);
@@ -211,6 +166,16 @@ kvm_irqfd_release(struct kvm *kvm)
 {
 	struct _irqfd *irqfd, *tmp;
 
-	list_for_each_entry_safe(irqfd, tmp, &kvm->irqfds, list)
-		irqfd_disconnect(irqfd);
+	list_for_each_entry_safe(irqfd, tmp, &kvm->irqfds, list) {
+		if (irqfd->wqh)
+			remove_wait_queue(irqfd->wqh, &irqfd->wait);
+
+		flush_work(&irqfd->inject);
+
+		mutex_lock(&kvm->lock);
+		list_del(&irqfd->list);
+		mutex_unlock(&kvm->lock);
+
+		kfree(irqfd);
+	}
 }


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [KVM PATCH v3 2/3] eventfd: add internal reference counting to fix notifier race conditions
  2009-06-22 16:05 [KVM PATCH v3 0/3] irqfd/eventfd fixes Gregory Haskins
  2009-06-22 16:05 ` [KVM PATCH v3 1/3] kvm: prepare irqfd for having interrupts disabled during eventfd->release Gregory Haskins
@ 2009-06-22 16:05 ` Gregory Haskins
  2009-06-22 16:05 ` [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface Gregory Haskins
  2 siblings, 0 replies; 14+ messages in thread
From: Gregory Haskins @ 2009-06-22 16:05 UTC (permalink / raw)
  To: kvm; +Cc: linux-kernel, mst, avi, paulmck, davidel, mingo, rusty

eventfd currently emits a POLLHUP wakeup on f_ops->release() to generate a
"release" callback.  This lets eventfd clients know if the eventfd is about
to go away and is very useful particularly for in-kernel clients.  However,
as it stands today it is not possible to use this feature of eventfd in a
race-free way.  This patch adds some additional logic to eventfd in order
to rectify this problem.

Background:
-----------------------
Eventfd currently only has one reference count mechanism: fget/fput.  This
in of itself is normally fine.  However, if a client expects to be
notified if the eventfd is closed, it cannot hold a fget() reference
itself or the underlying f_ops->release() callback will never be invoked
by VFS.  Therefore we have this somewhat unusual situation where we may
hold a pointer to an eventfd object (by virtue of having a waiter registered
in its wait-queue), but no reference.  To make matters more complicated,
the release callback is issued in an unlocked state.  This makes it nearly
impossible to design a mutual decoupling algorithm: you cannot unhook one
side from the other (or vice versa) without racing.

-----------------------

In summary, there are two fundamental problems:

1) The POLLHUP wakeup is broadcast lockless
2) There are no references to the wait-queue-head (embedded in eventfd_ctx)

We fix this by using the locked variant of wakeup for POLLHUP, and by
adding/exposing a kref to the underlying eventfd_ctx.  Clients should then
be able to govern their usage of the wait-queue as they do for any other
wait-queue in the kernel.

We propose this more raw solution rather than trying to encapsulate the
poll-callback because there are advantages to decoupling the
remove_wait_queue from the kref_put().  Namely, its nice to unhook the
wait-queue inside the wakeup, but to defer the kref_put() until we can
synchronize with the client.

Between these points, we believe we now have a race-free release
mechanism.

Signed-off-by: Gregory Haskins <ghaskins@novell.com>
CC: Davide Libenzi <davidel@xmailserver.org>
---

 fs/eventfd.c            |   43 ++++++++++++++++++++++++++++++++++++-------
 include/linux/eventfd.h |    7 +++++++
 2 files changed, 43 insertions(+), 7 deletions(-)

diff --git a/fs/eventfd.c b/fs/eventfd.c
index 72f5f8d..4806116 100644
--- a/fs/eventfd.c
+++ b/fs/eventfd.c
@@ -17,8 +17,10 @@
 #include <linux/eventfd.h>
 #include <linux/syscalls.h>
 #include <linux/module.h>
+#include <linux/kref.h>
 
 struct eventfd_ctx {
+	struct kref kref;
 	wait_queue_head_t wqh;
 	/*
 	 * Every time that a write(2) is performed on an eventfd, the
@@ -59,17 +61,24 @@ int eventfd_signal(struct file *file, int n)
 }
 EXPORT_SYMBOL_GPL(eventfd_signal);
 
+static void _eventfd_release(struct kref *kref)
+{
+	struct eventfd_ctx *ctx = container_of(kref, struct eventfd_ctx, kref);
+
+	kfree(ctx);
+}
+
+static void _eventfd_put(struct kref *kref)
+{
+	kref_put(kref, &_eventfd_release);
+}
+
 static int eventfd_release(struct inode *inode, struct file *file)
 {
 	struct eventfd_ctx *ctx = file->private_data;
 
-	/*
-	 * No need to hold the lock here, since we are on the file cleanup
-	 * path and the ones still attached to the wait queue will be
-	 * serialized by wake_up_locked_poll().
-	 */
-	wake_up_locked_poll(&ctx->wqh, POLLHUP);
-	kfree(ctx);
+	wake_up_poll(&ctx->wqh, POLLHUP);
+	_eventfd_put(&ctx->kref);
 	return 0;
 }
 
@@ -209,6 +218,26 @@ struct file *eventfd_fget(int fd)
 }
 EXPORT_SYMBOL_GPL(eventfd_fget);
 
+struct kref *eventfd_kref_get(struct file *file)
+{
+	struct eventfd_ctx *ctx;
+
+	if (file->f_op != &eventfd_fops)
+		return ERR_PTR(-EINVAL);
+
+	ctx = file->private_data;
+	kref_get(&ctx->kref);
+
+	return &ctx->kref;
+}
+EXPORT_SYMBOL_GPL(eventfd_kref_get);
+
+void eventfd_kref_put(struct kref *kref)
+{
+	_eventfd_put(kref);
+}
+EXPORT_SYMBOL_GPL(eventfd_kref_put);
+
 SYSCALL_DEFINE2(eventfd2, unsigned int, count, int, flags)
 {
 	int fd;
diff --git a/include/linux/eventfd.h b/include/linux/eventfd.h
index f45a8ae..c0396b3 100644
--- a/include/linux/eventfd.h
+++ b/include/linux/eventfd.h
@@ -8,6 +8,8 @@
 #ifndef _LINUX_EVENTFD_H
 #define _LINUX_EVENTFD_H
 
+#include <linux/kref.h>
+
 #ifdef CONFIG_EVENTFD
 
 /* For O_CLOEXEC and O_NONBLOCK */
@@ -28,11 +30,16 @@
 #define EFD_FLAGS_SET (EFD_SHARED_FCNTL_FLAGS | EFD_SEMAPHORE)
 
 struct file *eventfd_fget(int fd);
+struct kref *eventfd_kref_get(struct file *file);
+void eventfd_kref_put(struct kref *kref);
 int eventfd_signal(struct file *file, int n);
 
 #else /* CONFIG_EVENTFD */
 
 #define eventfd_fget(fd) ERR_PTR(-ENOSYS)
+#define eventfd_kref_get(file) ERR_PTR(-ENOSYS);
+static inline void eventfd_kref_put(struct kref *kref)
+{ }
 static inline int eventfd_signal(struct file *file, int n)
 { return 0; }
 


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 16:05 [KVM PATCH v3 0/3] irqfd/eventfd fixes Gregory Haskins
  2009-06-22 16:05 ` [KVM PATCH v3 1/3] kvm: prepare irqfd for having interrupts disabled during eventfd->release Gregory Haskins
  2009-06-22 16:05 ` [KVM PATCH v3 2/3] eventfd: add internal reference counting to fix notifier race conditions Gregory Haskins
@ 2009-06-22 16:05 ` Gregory Haskins
  2009-06-22 16:57   ` Michael S. Tsirkin
  2 siblings, 1 reply; 14+ messages in thread
From: Gregory Haskins @ 2009-06-22 16:05 UTC (permalink / raw)
  To: kvm; +Cc: linux-kernel, mst, avi, paulmck, davidel, mingo, rusty

This patch fixes all known races in irqfd, and paves the way to restore
DEASSIGN support.  For details of the eventfd races, please see the patch
presumably commited immediately prior to this one.

In a nutshell, we use eventfd_kref_get/put() to properly manage the
lifetime of the underlying eventfd.  We also use careful coordination
with our workqueue to ensure that all irqfd objects have terminated
before we allow kvm to shutdown.  The logic used for shutdown walks
all open irqfds and releases them.  This logic can be generalized in
the future to allow a subset of irqfds to be released, thus allowing
DEASSIGN support.

Signed-off-by: Gregory Haskins <ghaskins@novell.com>
---

 virt/kvm/eventfd.c |  144 ++++++++++++++++++++++++++++++++++++++++------------
 1 files changed, 110 insertions(+), 34 deletions(-)

diff --git a/virt/kvm/eventfd.c b/virt/kvm/eventfd.c
index 9656027..67985cd 100644
--- a/virt/kvm/eventfd.c
+++ b/virt/kvm/eventfd.c
@@ -28,6 +28,7 @@
 #include <linux/file.h>
 #include <linux/list.h>
 #include <linux/eventfd.h>
+#include <linux/kref.h>
 
 /*
  * --------------------------------------------------------------------
@@ -36,26 +37,68 @@
  * Credit goes to Avi Kivity for the original idea.
  * --------------------------------------------------------------------
  */
+
+enum {
+	irqfd_flags_shutdown,
+};
+
 struct _irqfd {
 	struct kvm               *kvm;
+	struct kref              *eventfd;
 	int                       gsi;
 	struct list_head          list;
 	poll_table                pt;
 	wait_queue_head_t        *wqh;
 	wait_queue_t              wait;
-	struct work_struct        inject;
+	struct work_struct        work;
+	unsigned long             flags;
 };
 
 static void
-irqfd_inject(struct work_struct *work)
+irqfd_release(struct _irqfd *irqfd)
+{
+	eventfd_kref_put(irqfd->eventfd);
+	kfree(irqfd);
+}
+
+static void
+irqfd_work(struct work_struct *work)
 {
-	struct _irqfd *irqfd = container_of(work, struct _irqfd, inject);
+	struct _irqfd *irqfd = container_of(work, struct _irqfd, work);
 	struct kvm *kvm = irqfd->kvm;
 
-	mutex_lock(&kvm->irq_lock);
-	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
-	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
-	mutex_unlock(&kvm->irq_lock);
+	if (!test_bit(irqfd_flags_shutdown, &irqfd->flags)) {
+		/* Inject an interrupt */
+		mutex_lock(&kvm->irq_lock);
+		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
+		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
+		mutex_unlock(&kvm->irq_lock);
+	} else {
+		/* shutdown the irqfd */
+		struct _irqfd *_irqfd = NULL;
+
+		mutex_lock(&kvm->lock);
+
+		if (!list_empty(&irqfd->list))
+			_irqfd = irqfd;
+
+		if (_irqfd)
+			list_del(&_irqfd->list);
+
+		mutex_unlock(&kvm->lock);
+
+		/*
+		 * If the item is not currently on the irqfds list, we know
+		 * we are running concurrently with the KVM side trying to
+		 * remove this item as well.  Since the KVM side should be
+		 * holding the reference now, and will block behind a
+		 * flush_work(), lets just let them do the release() for us
+		 */
+		if (!_irqfd)
+			return;
+
+		irqfd_release(_irqfd);
+	}
 }
 
 static int
@@ -65,25 +108,20 @@ irqfd_wakeup(wait_queue_t *wait, unsigned mode, int sync, void *key)
 	unsigned long flags = (unsigned long)key;
 
 	/*
-	 * Assume we will be called with interrupts disabled
+	 * called with interrupts disabled
 	 */
-	if (flags & POLLIN)
-		/*
-		 * Defer the IRQ injection until later since we need to
-		 * acquire the kvm->lock to do so.
-		 */
-		schedule_work(&irqfd->inject);
-
 	if (flags & POLLHUP) {
 		/*
-		 * for now, just remove ourselves from the list and let
-		 * the rest dangle.  We will fix this up later once
-		 * the races in eventfd are fixed
+		 * ordering is important: shutdown flag must be visible
+		 * before we schedule
 		 */
 		__remove_wait_queue(irqfd->wqh, &irqfd->wait);
-		irqfd->wqh = NULL;
+		set_bit(irqfd_flags_shutdown, &irqfd->flags);
 	}
 
+	if (flags & (POLLHUP | POLLIN))
+		schedule_work(&irqfd->work);
+
 	return 0;
 }
 
@@ -102,6 +140,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
 {
 	struct _irqfd *irqfd;
 	struct file *file = NULL;
+	struct kref *kref = NULL;
 	int ret;
 	unsigned int events;
 
@@ -112,7 +151,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
 	irqfd->kvm = kvm;
 	irqfd->gsi = gsi;
 	INIT_LIST_HEAD(&irqfd->list);
-	INIT_WORK(&irqfd->inject, irqfd_inject);
+	INIT_WORK(&irqfd->work, irqfd_work);
 
 	file = eventfd_fget(fd);
 	if (IS_ERR(file)) {
@@ -133,11 +172,13 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
 	list_add_tail(&irqfd->list, &kvm->irqfds);
 	mutex_unlock(&kvm->lock);
 
-	/*
-	 * Check if there was an event already queued
-	 */
-	if (events & POLLIN)
-		schedule_work(&irqfd->inject);
+	kref = eventfd_kref_get(file);
+	if (IS_ERR(file)) {
+		ret = PTR_ERR(file);
+		goto fail;
+	}
+
+	irqfd->eventfd = kref;
 
 	/*
 	 * do not drop the file until the irqfd is fully initialized, otherwise
@@ -145,9 +186,18 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
 	 */
 	fput(file);
 
+	/*
+	 * Check if there was an event already queued
+	 */
+	if (events & POLLIN)
+		schedule_work(&irqfd->work);
+
 	return 0;
 
 fail:
+	if (kref && !IS_ERR(kref))
+		eventfd_kref_put(kref);
+
 	if (file && !IS_ERR(file))
 		fput(file);
 
@@ -161,21 +211,47 @@ kvm_irqfd_init(struct kvm *kvm)
 	INIT_LIST_HEAD(&kvm->irqfds);
 }
 
+static struct _irqfd *
+irqfd_pop(struct kvm *kvm)
+{
+	struct _irqfd *irqfd = NULL;
+
+	mutex_lock(&kvm->lock);
+
+	if (!list_empty(&kvm->irqfds)) {
+		irqfd = list_first_entry(&kvm->irqfds, struct _irqfd, list);
+		list_del(&irqfd->list);
+	}
+
+	mutex_unlock(&kvm->lock);
+
+	return irqfd;
+}
+
 void
 kvm_irqfd_release(struct kvm *kvm)
 {
-	struct _irqfd *irqfd, *tmp;
+	struct _irqfd *irqfd;
 
-	list_for_each_entry_safe(irqfd, tmp, &kvm->irqfds, list) {
-		if (irqfd->wqh)
-			remove_wait_queue(irqfd->wqh, &irqfd->wait);
+	while ((irqfd = irqfd_pop(kvm))) {
 
-		flush_work(&irqfd->inject);
+		remove_wait_queue(irqfd->wqh, &irqfd->wait);
 
-		mutex_lock(&kvm->lock);
-		list_del(&irqfd->list);
-		mutex_unlock(&kvm->lock);
+		/*
+		 * We guarantee there will be no more notifications after
+		 * the remove_wait_queue returns.  Now lets make sure we
+		 * synchronize behind any outstanding work items before
+		 * releasing the resources
+		 */
+		flush_work(&irqfd->work);
 
-		kfree(irqfd);
+		irqfd_release(irqfd);
 	}
+
+	/*
+	 * We need to wait in case there are any outstanding work-items
+	 * in flight that had already removed themselves from the list
+	 * prior to entry to this function
+	 */
+	flush_scheduled_work();
 }


^ permalink raw reply related	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 16:05 ` [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface Gregory Haskins
@ 2009-06-22 16:57   ` Michael S. Tsirkin
  2009-06-22 17:31     ` Gregory Haskins
  0 siblings, 1 reply; 14+ messages in thread
From: Michael S. Tsirkin @ 2009-06-22 16:57 UTC (permalink / raw)
  To: Gregory Haskins; +Cc: kvm, linux-kernel, avi, paulmck, davidel, mingo, rusty

On Mon, Jun 22, 2009 at 12:05:57PM -0400, Gregory Haskins wrote:
> This patch fixes all known races in irqfd, and paves the way to restore
> DEASSIGN support.  For details of the eventfd races, please see the patch
> presumably commited immediately prior to this one.
> 
> In a nutshell, we use eventfd_kref_get/put() to properly manage the
> lifetime of the underlying eventfd.  We also use careful coordination
> with our workqueue to ensure that all irqfd objects have terminated
> before we allow kvm to shutdown.  The logic used for shutdown walks
> all open irqfds and releases them.  This logic can be generalized in
> the future to allow a subset of irqfds to be released, thus allowing
> DEASSIGN support.
> 
> Signed-off-by: Gregory Haskins <ghaskins@novell.com>

I think this patch is a shade too tricky. Some explanation why below.

But I think irqfd_pop is a good idea.
Here's an alternative design sketch: add a list of irqfds to be shutdown
in kvm, and create a single-threaded workqueue. To kill an irqfd, move
it from list of live irqfds to list of dead irqfds, then schedule work
on a workqueue that walks this list and kills irqfds.

> ---
> 
>  virt/kvm/eventfd.c |  144 ++++++++++++++++++++++++++++++++++++++++------------
>  1 files changed, 110 insertions(+), 34 deletions(-)
> 
> diff --git a/virt/kvm/eventfd.c b/virt/kvm/eventfd.c
> index 9656027..67985cd 100644
> --- a/virt/kvm/eventfd.c
> +++ b/virt/kvm/eventfd.c
> @@ -28,6 +28,7 @@
>  #include <linux/file.h>
>  #include <linux/list.h>
>  #include <linux/eventfd.h>
> +#include <linux/kref.h>
>  
>  /*
>   * --------------------------------------------------------------------
> @@ -36,26 +37,68 @@
>   * Credit goes to Avi Kivity for the original idea.
>   * --------------------------------------------------------------------
>   */
> +
> +enum {
> +	irqfd_flags_shutdown,
> +};
> +
>  struct _irqfd {
>  	struct kvm               *kvm;
> +	struct kref              *eventfd;


Yay, kref.

>  	int                       gsi;
>  	struct list_head          list;
>  	poll_table                pt;
>  	wait_queue_head_t        *wqh;
>  	wait_queue_t              wait;
> -	struct work_struct        inject;
> +	struct work_struct        work;
> +	unsigned long             flags;

Just make it "int shutdown"?

>  };
>  
>  static void
> -irqfd_inject(struct work_struct *work)
> +irqfd_release(struct _irqfd *irqfd)
> +{
> +	eventfd_kref_put(irqfd->eventfd);
> +	kfree(irqfd);
> +}
> +
> +static void
> +irqfd_work(struct work_struct *work)
>  {
> -	struct _irqfd *irqfd = container_of(work, struct _irqfd, inject);
> +	struct _irqfd *irqfd = container_of(work, struct _irqfd, work);
>  	struct kvm *kvm = irqfd->kvm;
>  
> -	mutex_lock(&kvm->irq_lock);
> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
> -	mutex_unlock(&kvm->irq_lock);
> +	if (!test_bit(irqfd_flags_shutdown, &irqfd->flags)) {

Why is it safe to test this bit outside of any lock?

> +		/* Inject an interrupt */
> +		mutex_lock(&kvm->irq_lock);
> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
> +		mutex_unlock(&kvm->irq_lock);
> +	} else {


Not much shared code here - create a separate showdown work struct?
They are cheap ...

> +		/* shutdown the irqfd */
> +		struct _irqfd *_irqfd = NULL;
> +
> +		mutex_lock(&kvm->lock);
> +
> +		if (!list_empty(&irqfd->list))
> +			_irqfd = irqfd;
> +
> +		if (_irqfd)
> +			list_del(&_irqfd->list);
> +
> +		mutex_unlock(&kvm->lock);
> +
> +		/*
> +		 * If the item is not currently on the irqfds list, we know
> +		 * we are running concurrently with the KVM side trying to
> +		 * remove this item as well.

We do? How? As far as I can see list is only empty after it has been
created.  Generally, it would be better to either use a flag or use
list_empty as an indication of going down, but not both.

>  Since the KVM side should be
> +		 * holding the reference now, and will block behind a
> +		 * flush_work(), lets just let them do the release() for us
> +		 */
> +		if (!_irqfd)
> +			return;
> +
> +		irqfd_release(_irqfd);
> +	}
>  }
>  
>  static int
> @@ -65,25 +108,20 @@ irqfd_wakeup(wait_queue_t *wait, unsigned mode, int sync, void *key)
>  	unsigned long flags = (unsigned long)key;
>  
>  	/*
> -	 * Assume we will be called with interrupts disabled
> +	 * called with interrupts disabled
>  	 */
> -	if (flags & POLLIN)
> -		/*
> -		 * Defer the IRQ injection until later since we need to
> -		 * acquire the kvm->lock to do so.
> -		 */
> -		schedule_work(&irqfd->inject);
> -
>  	if (flags & POLLHUP) {
>  		/*
> -		 * for now, just remove ourselves from the list and let
> -		 * the rest dangle.  We will fix this up later once
> -		 * the races in eventfd are fixed
> +		 * ordering is important: shutdown flag must be visible
> +		 * before we schedule
>  		 */
>  		__remove_wait_queue(irqfd->wqh, &irqfd->wait);
> -		irqfd->wqh = NULL;
> +		set_bit(irqfd_flags_shutdown, &irqfd->flags);

So what happens if a previously scheduled work runs on irqfd
and sees this flag? And note that multiple works can run on irqfd
in parallel.

>  	}
>  
> +	if (flags & (POLLHUP | POLLIN))
> +		schedule_work(&irqfd->work);
> +
>  	return 0;
>  }
>  
> @@ -102,6 +140,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>  {
>  	struct _irqfd *irqfd;
>  	struct file *file = NULL;
> +	struct kref *kref = NULL;
>  	int ret;
>  	unsigned int events;
>  
> @@ -112,7 +151,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>  	irqfd->kvm = kvm;
>  	irqfd->gsi = gsi;
>  	INIT_LIST_HEAD(&irqfd->list);
> -	INIT_WORK(&irqfd->inject, irqfd_inject);
> +	INIT_WORK(&irqfd->work, irqfd_work);
>  
>  	file = eventfd_fget(fd);
>  	if (IS_ERR(file)) {
> @@ -133,11 +172,13 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>  	list_add_tail(&irqfd->list, &kvm->irqfds);
>  	mutex_unlock(&kvm->lock);
>  
> -	/*
> -	 * Check if there was an event already queued
> -	 */
> -	if (events & POLLIN)
> -		schedule_work(&irqfd->inject);
> +	kref = eventfd_kref_get(file);
> +	if (IS_ERR(file)) {
> +		ret = PTR_ERR(file);
> +		goto fail;
> +	}
> +
> +	irqfd->eventfd = kref;
>  
>  	/*
>  	 * do not drop the file until the irqfd is fully initialized, otherwise
> @@ -145,9 +186,18 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>  	 */
>  	fput(file);
>  
> +	/*
> +	 * Check if there was an event already queued
> +	 */

This comment seems to confuse more that it clarifies:
queued where? eventfd only counts... Just kill the comment?

> +	if (events & POLLIN)
> +		schedule_work(&irqfd->work);
> +
>  	return 0;
>  
>  fail:
> +	if (kref && !IS_ERR(kref))
> +		eventfd_kref_put(kref);
> +
>  	if (file && !IS_ERR(file))
>  		fput(file);

let's add a couple more labels and avoid the kref/file check
and the initialization above?

>  
> @@ -161,21 +211,47 @@ kvm_irqfd_init(struct kvm *kvm)
>  	INIT_LIST_HEAD(&kvm->irqfds);
>  }
>  
> +static struct _irqfd *
> +irqfd_pop(struct kvm *kvm)
> +{
> +	struct _irqfd *irqfd = NULL;
> +
> +	mutex_lock(&kvm->lock);
> +
> +	if (!list_empty(&kvm->irqfds)) {
> +		irqfd = list_first_entry(&kvm->irqfds, struct _irqfd, list);
> +		list_del(&irqfd->list);
> +	}
> +
> +	mutex_unlock(&kvm->lock);
> +
> +	return irqfd;
> +}
> +
>  void
>  kvm_irqfd_release(struct kvm *kvm)
>  {
> -	struct _irqfd *irqfd, *tmp;
> +	struct _irqfd *irqfd;
>  
> -	list_for_each_entry_safe(irqfd, tmp, &kvm->irqfds, list) {
> -		if (irqfd->wqh)
> -			remove_wait_queue(irqfd->wqh, &irqfd->wait);
> +	while ((irqfd = irqfd_pop(kvm))) {
>  
> -		flush_work(&irqfd->inject);
> +		remove_wait_queue(irqfd->wqh, &irqfd->wait);
>  
> -		mutex_lock(&kvm->lock);
> -		list_del(&irqfd->list);
> -		mutex_unlock(&kvm->lock);
> +		/*
> +		 * We guarantee there will be no more notifications after
> +		 * the remove_wait_queue returns.  Now lets make sure we
> +		 * synchronize behind any outstanding work items before
> +		 * releasing the resources
> +		 */
> +		flush_work(&irqfd->work);
>  
> -		kfree(irqfd);
> +		irqfd_release(irqfd);
>  	}
> +
> +	/*
> +	 * We need to wait in case there are any outstanding work-items
> +	 * in flight that had already removed themselves from the list
> +	 * prior to entry to this function
> +	 */

Looks scary. Why doesn't the flush above cover all cases?

> +	flush_scheduled_work();
>  }

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 16:57   ` Michael S. Tsirkin
@ 2009-06-22 17:31     ` Gregory Haskins
  2009-06-22 17:45       ` Michael S. Tsirkin
  2009-06-23 14:55       ` Gregory Haskins
  0 siblings, 2 replies; 14+ messages in thread
From: Gregory Haskins @ 2009-06-22 17:31 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Gregory Haskins, kvm, linux-kernel, avi, paulmck, davidel, mingo, rusty

[-- Attachment #1: Type: text/plain, Size: 12351 bytes --]

Michael S. Tsirkin wrote:
> On Mon, Jun 22, 2009 at 12:05:57PM -0400, Gregory Haskins wrote:
>   
>> This patch fixes all known races in irqfd, and paves the way to restore
>> DEASSIGN support.  For details of the eventfd races, please see the patch
>> presumably commited immediately prior to this one.
>>
>> In a nutshell, we use eventfd_kref_get/put() to properly manage the
>> lifetime of the underlying eventfd.  We also use careful coordination
>> with our workqueue to ensure that all irqfd objects have terminated
>> before we allow kvm to shutdown.  The logic used for shutdown walks
>> all open irqfds and releases them.  This logic can be generalized in
>> the future to allow a subset of irqfds to be released, thus allowing
>> DEASSIGN support.
>>
>> Signed-off-by: Gregory Haskins <ghaskins@novell.com>
>>     
>
> I think this patch is a shade too tricky. Some explanation why below.
>
> But I think irqfd_pop is a good idea.
>   

Yeah, next we can add something like "irqfd_remove(gsi)" in a similar
way to do DEASSIGN.

> Here's an alternative design sketch: add a list of irqfds to be shutdown
> in kvm, and create a single-threaded workqueue. To kill an irqfd, move
> it from list of live irqfds to list of dead irqfds, then schedule work
> on a workqueue that walks this list and kills irqfds.
>   

Yeah, I actually thought of that too, and I think that will work.  But
then I realized flush_schedule_work does the same thing and its much
less code.  Perhaps it is also much less clear, too ;)  At the very
least, you have made me realize I need to comment better.
>   
>> ---
>>
>>  virt/kvm/eventfd.c |  144 ++++++++++++++++++++++++++++++++++++++++------------
>>  1 files changed, 110 insertions(+), 34 deletions(-)
>>
>> diff --git a/virt/kvm/eventfd.c b/virt/kvm/eventfd.c
>> index 9656027..67985cd 100644
>> --- a/virt/kvm/eventfd.c
>> +++ b/virt/kvm/eventfd.c
>> @@ -28,6 +28,7 @@
>>  #include <linux/file.h>
>>  #include <linux/list.h>
>>  #include <linux/eventfd.h>
>> +#include <linux/kref.h>
>>  
>>  /*
>>   * --------------------------------------------------------------------
>> @@ -36,26 +37,68 @@
>>   * Credit goes to Avi Kivity for the original idea.
>>   * --------------------------------------------------------------------
>>   */
>> +
>> +enum {
>> +	irqfd_flags_shutdown,
>> +};
>> +
>>  struct _irqfd {
>>  	struct kvm               *kvm;
>> +	struct kref              *eventfd;
>>     
>
>
> Yay, kref.
>
>   
>>  	int                       gsi;
>>  	struct list_head          list;
>>  	poll_table                pt;
>>  	wait_queue_head_t        *wqh;
>>  	wait_queue_t              wait;
>> -	struct work_struct        inject;
>> +	struct work_struct        work;
>> +	unsigned long             flags;
>>     
>
> Just make it "int shutdown"?
>   

Yep, that is probably fine but we will have to use an explicit wmb in
lieu of a set_bit operation.  NBD.

>   
>>  };
>>  
>>  static void
>> -irqfd_inject(struct work_struct *work)
>> +irqfd_release(struct _irqfd *irqfd)
>> +{
>> +	eventfd_kref_put(irqfd->eventfd);
>> +	kfree(irqfd);
>> +}
>> +
>> +static void
>> +irqfd_work(struct work_struct *work)
>>  {
>> -	struct _irqfd *irqfd = container_of(work, struct _irqfd, inject);
>> +	struct _irqfd *irqfd = container_of(work, struct _irqfd, work);
>>  	struct kvm *kvm = irqfd->kvm;
>>  
>> -	mutex_lock(&kvm->irq_lock);
>> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
>> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
>> -	mutex_unlock(&kvm->irq_lock);
>> +	if (!test_bit(irqfd_flags_shutdown, &irqfd->flags)) {
>>     
>
> Why is it safe to test this bit outside of any lock?
>   
Because the ordering is guaranteed to set_bit(), schedule_work().  All
we need to do is make sure that the work-queue runs at least one more
time after the flag has been set.  (Of course, I could have screwed up
too, but that was my rationale).

>   
>> +		/* Inject an interrupt */
>> +		mutex_lock(&kvm->irq_lock);
>> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
>> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
>> +		mutex_unlock(&kvm->irq_lock);
>> +	} else {
>>     
>
>
> Not much shared code here - create a separate showdown work struct?
> They are cheap ...
>   

We can't because we need to ensure that all inject-jobs complete before
release-jobs.  Reading the work-queue code, it would be a deadlock for
the release-job to do a flush_work(inject-job).  Therefore, both
workloads are encapsulated into a single job, and we ensure that the job
is launched at least one more time after the flag has been set.

Of course, now that I wrote that,  I realize it was clear-as-mud in the
code and needs some commenting ;)

>   
>> +		/* shutdown the irqfd */
>> +		struct _irqfd *_irqfd = NULL;
>> +
>> +		mutex_lock(&kvm->lock);
>> +
>> +		if (!list_empty(&irqfd->list))
>> +			_irqfd = irqfd;
>> +
>> +		if (_irqfd)
>> +			list_del(&_irqfd->list);
>> +
>> +		mutex_unlock(&kvm->lock);
>> +
>> +		/*
>> +		 * If the item is not currently on the irqfds list, we know
>> +		 * we are running concurrently with the KVM side trying to
>> +		 * remove this item as well.
>>     
>
> We do? How? As far as I can see list is only empty after it has been
> created.  Generally, it would be better to either use a flag or use
> list_empty as an indication of going down, but not both.
>   

I think you are mis-reading that.  list_empty(&irqfd->list) is the
individual irqfd list-item, not the kvm->irqfds list itself.  This
conditional is telling us whether the irqfd in question is on or off the
list (its effectively an irqfd-specific flag), not whether the global
list is empty.  Again, poor commenting on my part.

>   
>>  Since the KVM side should be
>> +		 * holding the reference now, and will block behind a
>> +		 * flush_work(), lets just let them do the release() for us
>> +		 */
>> +		if (!_irqfd)
>> +			return;
>> +
>> +		irqfd_release(_irqfd);
>> +	}
>>  }
>>  
>>  static int
>> @@ -65,25 +108,20 @@ irqfd_wakeup(wait_queue_t *wait, unsigned mode, int sync, void *key)
>>  	unsigned long flags = (unsigned long)key;
>>  
>>  	/*
>> -	 * Assume we will be called with interrupts disabled
>> +	 * called with interrupts disabled
>>  	 */
>> -	if (flags & POLLIN)
>> -		/*
>> -		 * Defer the IRQ injection until later since we need to
>> -		 * acquire the kvm->lock to do so.
>> -		 */
>> -		schedule_work(&irqfd->inject);
>> -
>>  	if (flags & POLLHUP) {
>>  		/*
>> -		 * for now, just remove ourselves from the list and let
>> -		 * the rest dangle.  We will fix this up later once
>> -		 * the races in eventfd are fixed
>> +		 * ordering is important: shutdown flag must be visible
>> +		 * before we schedule
>>  		 */
>>  		__remove_wait_queue(irqfd->wqh, &irqfd->wait);
>> -		irqfd->wqh = NULL;
>> +		set_bit(irqfd_flags_shutdown, &irqfd->flags);
>>     
>
> So what happens if a previously scheduled work runs on irqfd
> and sees this flag?
My original thought was "thats ok", but now that you mention it I am not
so sure.  Ill give it some more thought because maybe you are on to
something.

>  And note that multiple works can run on irqfd
> in parallel.
>   

They can?  I thought work-queue items were guaranteed to only schedule
once?  If what you say is true, its broken, I agree, and Ill need to
revisit.  Let me get back to you.
>   
>>  	}
>>  
>> +	if (flags & (POLLHUP | POLLIN))
>> +		schedule_work(&irqfd->work);
>> +
>>  	return 0;
>>  }
>>  
>> @@ -102,6 +140,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>>  {
>>  	struct _irqfd *irqfd;
>>  	struct file *file = NULL;
>> +	struct kref *kref = NULL;
>>  	int ret;
>>  	unsigned int events;
>>  
>> @@ -112,7 +151,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>>  	irqfd->kvm = kvm;
>>  	irqfd->gsi = gsi;
>>  	INIT_LIST_HEAD(&irqfd->list);
>> -	INIT_WORK(&irqfd->inject, irqfd_inject);
>> +	INIT_WORK(&irqfd->work, irqfd_work);
>>  
>>  	file = eventfd_fget(fd);
>>  	if (IS_ERR(file)) {
>> @@ -133,11 +172,13 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>>  	list_add_tail(&irqfd->list, &kvm->irqfds);
>>  	mutex_unlock(&kvm->lock);
>>  
>> -	/*
>> -	 * Check if there was an event already queued
>> -	 */
>> -	if (events & POLLIN)
>> -		schedule_work(&irqfd->inject);
>> +	kref = eventfd_kref_get(file);
>> +	if (IS_ERR(file)) {
>> +		ret = PTR_ERR(file);
>> +		goto fail;
>> +	}
>> +
>> +	irqfd->eventfd = kref;
>>  
>>  	/*
>>  	 * do not drop the file until the irqfd is fully initialized, otherwise
>> @@ -145,9 +186,18 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>>  	 */
>>  	fput(file);
>>  
>> +	/*
>> +	 * Check if there was an event already queued
>> +	 */
>>     
>
> This comment seems to confuse more that it clarifies:
> queued where? eventfd only counts... Just kill the comment?
>
>   
non-zero values in eventfd are "queued" as a signal.  This test just
checks if an interrupt was already injected before we registered.

>> +	if (events & POLLIN)
>> +		schedule_work(&irqfd->work);
>> +
>>  	return 0;
>>  
>>  fail:
>> +	if (kref && !IS_ERR(kref))
>> +		eventfd_kref_put(kref);
>> +
>>  	if (file && !IS_ERR(file))
>>  		fput(file);
>>     
>
> let's add a couple more labels and avoid the kref/file check
> and the initialization above?
>   

I think that just makes it more confusing, personally.  But I will give
it some thought.

>   
>>  
>> @@ -161,21 +211,47 @@ kvm_irqfd_init(struct kvm *kvm)
>>  	INIT_LIST_HEAD(&kvm->irqfds);
>>  }
>>  
>> +static struct _irqfd *
>> +irqfd_pop(struct kvm *kvm)
>> +{
>> +	struct _irqfd *irqfd = NULL;
>> +
>> +	mutex_lock(&kvm->lock);
>> +
>> +	if (!list_empty(&kvm->irqfds)) {
>> +		irqfd = list_first_entry(&kvm->irqfds, struct _irqfd, list);
>> +		list_del(&irqfd->list);
>> +	}
>> +
>> +	mutex_unlock(&kvm->lock);
>> +
>> +	return irqfd;
>> +}
>> +
>>  void
>>  kvm_irqfd_release(struct kvm *kvm)
>>  {
>> -	struct _irqfd *irqfd, *tmp;
>> +	struct _irqfd *irqfd;
>>  
>> -	list_for_each_entry_safe(irqfd, tmp, &kvm->irqfds, list) {
>> -		if (irqfd->wqh)
>> -			remove_wait_queue(irqfd->wqh, &irqfd->wait);
>> +	while ((irqfd = irqfd_pop(kvm))) {
>>  
>> -		flush_work(&irqfd->inject);
>> +		remove_wait_queue(irqfd->wqh, &irqfd->wait);
>>  
>> -		mutex_lock(&kvm->lock);
>> -		list_del(&irqfd->list);
>> -		mutex_unlock(&kvm->lock);
>> +		/*
>> +		 * We guarantee there will be no more notifications after
>> +		 * the remove_wait_queue returns.  Now lets make sure we
>> +		 * synchronize behind any outstanding work items before
>> +		 * releasing the resources
>> +		 */
>> +		flush_work(&irqfd->work);
>>  
>> -		kfree(irqfd);
>> +		irqfd_release(irqfd);
>>  	}
>> +
>> +	/*
>> +	 * We need to wait in case there are any outstanding work-items
>> +	 * in flight that had already removed themselves from the list
>> +	 * prior to entry to this function
>> +	 */
>>     
>
> Looks scary. Why doesn't the flush above cover all cases?
>   

The path inside the while() is for when KVM wins the race and finds the
item in the list.  It atomically removes it, and is responsible for
freeing it in a coordinated way.  In this case, we must block with the
flush_work() before we can irqfd_release() so that we do not yank the
memory out from under a running work-item.

The flush_scheduled_work() is for when eventfd wins the race and has
already removed itself from the list in the "shutdown" path in the
work-item.  We want to make sure that kvm_irqfd_release() cannot return
until all work-items have exited to prevent something like the kvm.ko
module unloading while the work-item is still in flight.

Thanks Michael,
-Greg
>   
>> +	flush_scheduled_work();
>>  }
>>     
> --
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at  http://www.tux.org/lkml/
>   



[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 266 bytes --]

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 17:31     ` Gregory Haskins
@ 2009-06-22 17:45       ` Michael S. Tsirkin
  2009-06-22 18:03         ` Gregory Haskins
  2009-06-22 18:11         ` Davide Libenzi
  2009-06-23 14:55       ` Gregory Haskins
  1 sibling, 2 replies; 14+ messages in thread
From: Michael S. Tsirkin @ 2009-06-22 17:45 UTC (permalink / raw)
  To: Gregory Haskins
  Cc: Gregory Haskins, kvm, linux-kernel, avi, paulmck, davidel, mingo, rusty

On Mon, Jun 22, 2009 at 01:31:29PM -0400, Gregory Haskins wrote:
> Michael S. Tsirkin wrote:
> > On Mon, Jun 22, 2009 at 12:05:57PM -0400, Gregory Haskins wrote:
> >   
> >> This patch fixes all known races in irqfd, and paves the way to restore
> >> DEASSIGN support.  For details of the eventfd races, please see the patch
> >> presumably commited immediately prior to this one.
> >>
> >> In a nutshell, we use eventfd_kref_get/put() to properly manage the
> >> lifetime of the underlying eventfd.  We also use careful coordination
> >> with our workqueue to ensure that all irqfd objects have terminated
> >> before we allow kvm to shutdown.  The logic used for shutdown walks
> >> all open irqfds and releases them.  This logic can be generalized in
> >> the future to allow a subset of irqfds to be released, thus allowing
> >> DEASSIGN support.
> >>
> >> Signed-off-by: Gregory Haskins <ghaskins@novell.com>
> >>     
> >
> > I think this patch is a shade too tricky. Some explanation why below.
> >
> > But I think irqfd_pop is a good idea.
> >   
> 
> Yeah, next we can add something like "irqfd_remove(gsi)" in a similar
> way to do DEASSIGN.
> 
> > Here's an alternative design sketch: add a list of irqfds to be shutdown
> > in kvm, and create a single-threaded workqueue. To kill an irqfd, move
> > it from list of live irqfds to list of dead irqfds, then schedule work
> > on a workqueue that walks this list and kills irqfds.
> >   
> 
> Yeah, I actually thought of that too, and I think that will work.  But
> then I realized flush_schedule_work does the same thing and its much
> less code.  Perhaps it is also much less clear, too ;)  At the very
> least, you have made me realize I need to comment better.

Not really, it's impossible to document all races one have thought
about and avoided.

> >   
> >> ---
> >>
> >>  virt/kvm/eventfd.c |  144 ++++++++++++++++++++++++++++++++++++++++------------
> >>  1 files changed, 110 insertions(+), 34 deletions(-)
> >>
> >> diff --git a/virt/kvm/eventfd.c b/virt/kvm/eventfd.c
> >> index 9656027..67985cd 100644
> >> --- a/virt/kvm/eventfd.c
> >> +++ b/virt/kvm/eventfd.c
> >> @@ -28,6 +28,7 @@
> >>  #include <linux/file.h>
> >>  #include <linux/list.h>
> >>  #include <linux/eventfd.h>
> >> +#include <linux/kref.h>
> >>  
> >>  /*
> >>   * --------------------------------------------------------------------
> >> @@ -36,26 +37,68 @@
> >>   * Credit goes to Avi Kivity for the original idea.
> >>   * --------------------------------------------------------------------
> >>   */
> >> +
> >> +enum {
> >> +	irqfd_flags_shutdown,
> >> +};
> >> +
> >>  struct _irqfd {
> >>  	struct kvm               *kvm;
> >> +	struct kref              *eventfd;
> >>     
> >
> >
> > Yay, kref.
> >
> >   
> >>  	int                       gsi;
> >>  	struct list_head          list;
> >>  	poll_table                pt;
> >>  	wait_queue_head_t        *wqh;
> >>  	wait_queue_t              wait;
> >> -	struct work_struct        inject;
> >> +	struct work_struct        work;
> >> +	unsigned long             flags;
> >>     
> >
> > Just make it "int shutdown"?
> >   
> 
> Yep, that is probably fine but we will have to use an explicit wmb in
> lieu of a set_bit operation.  NBD.
> 
> >   
> >>  };
> >>  
> >>  static void
> >> -irqfd_inject(struct work_struct *work)
> >> +irqfd_release(struct _irqfd *irqfd)
> >> +{
> >> +	eventfd_kref_put(irqfd->eventfd);
> >> +	kfree(irqfd);
> >> +}
> >> +
> >> +static void
> >> +irqfd_work(struct work_struct *work)
> >>  {
> >> -	struct _irqfd *irqfd = container_of(work, struct _irqfd, inject);
> >> +	struct _irqfd *irqfd = container_of(work, struct _irqfd, work);
> >>  	struct kvm *kvm = irqfd->kvm;
> >>  
> >> -	mutex_lock(&kvm->irq_lock);
> >> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
> >> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
> >> -	mutex_unlock(&kvm->irq_lock);
> >> +	if (!test_bit(irqfd_flags_shutdown, &irqfd->flags)) {
> >>     
> >
> > Why is it safe to test this bit outside of any lock?
> >   
> Because the ordering is guaranteed to set_bit(), schedule_work().  All
> we need to do is make sure that the work-queue runs at least one more
> time after the flag has been set.  (Of course, I could have screwed up
> too, but that was my rationale).
> 
> >   
> >> +		/* Inject an interrupt */
> >> +		mutex_lock(&kvm->irq_lock);
> >> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
> >> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
> >> +		mutex_unlock(&kvm->irq_lock);
> >> +	} else {
> >>     
> >
> >
> > Not much shared code here - create a separate showdown work struct?
> > They are cheap ...
> >   
> 
> We can't because we need to ensure that all inject-jobs complete before
> release-jobs.  Reading the work-queue code, it would be a deadlock for
> the release-job to do a flush_work(inject-job).  Therefore, both
> workloads are encapsulated into a single job, and we ensure that the job
> is launched at least one more time after the flag has been set.

AFAIK schedule_work does not give you in-order guarantees - it's
multithreaded. you will have to create a single-threaded workqueue
if you want in order execution.

> Of course, now that I wrote that,  I realize it was clear-as-mud in the
> code and needs some commenting ;)
> 
> >   
> >> +		/* shutdown the irqfd */
> >> +		struct _irqfd *_irqfd = NULL;
> >> +
> >> +		mutex_lock(&kvm->lock);
> >> +
> >> +		if (!list_empty(&irqfd->list))
> >> +			_irqfd = irqfd;
> >> +
> >> +		if (_irqfd)
> >> +			list_del(&_irqfd->list);
> >> +
> >> +		mutex_unlock(&kvm->lock);
> >> +
> >> +		/*
> >> +		 * If the item is not currently on the irqfds list, we know
> >> +		 * we are running concurrently with the KVM side trying to
> >> +		 * remove this item as well.
> >>     
> >
> > We do? How? As far as I can see list is only empty after it has been
> > created.  Generally, it would be better to either use a flag or use
> > list_empty as an indication of going down, but not both.
> >   
> 
> I think you are mis-reading that.  list_empty(&irqfd->list) is the
> individual irqfd list-item, not the kvm->irqfds list itself.  This
> conditional is telling us whether the irqfd in question is on or off the
> list (its effectively an irqfd-specific flag), not whether the global
> list is empty.  Again, poor commenting on my part.

Yes, but you do INIT_LIST_HEAD in a single place. Once you add
irqfd->list to a list, it won't be empty until you init it again.

> >   
> >>  Since the KVM side should be
> >> +		 * holding the reference now, and will block behind a
> >> +		 * flush_work(), lets just let them do the release() for us
> >> +		 */
> >> +		if (!_irqfd)
> >> +			return;
> >> +
> >> +		irqfd_release(_irqfd);
> >> +	}
> >>  }
> >>  
> >>  static int
> >> @@ -65,25 +108,20 @@ irqfd_wakeup(wait_queue_t *wait, unsigned mode, int sync, void *key)
> >>  	unsigned long flags = (unsigned long)key;
> >>  
> >>  	/*
> >> -	 * Assume we will be called with interrupts disabled
> >> +	 * called with interrupts disabled
> >>  	 */
> >> -	if (flags & POLLIN)
> >> -		/*
> >> -		 * Defer the IRQ injection until later since we need to
> >> -		 * acquire the kvm->lock to do so.
> >> -		 */
> >> -		schedule_work(&irqfd->inject);
> >> -
> >>  	if (flags & POLLHUP) {
> >>  		/*
> >> -		 * for now, just remove ourselves from the list and let
> >> -		 * the rest dangle.  We will fix this up later once
> >> -		 * the races in eventfd are fixed
> >> +		 * ordering is important: shutdown flag must be visible
> >> +		 * before we schedule
> >>  		 */
> >>  		__remove_wait_queue(irqfd->wqh, &irqfd->wait);
> >> -		irqfd->wqh = NULL;
> >> +		set_bit(irqfd_flags_shutdown, &irqfd->flags);
> >>     
> >
> > So what happens if a previously scheduled work runs on irqfd
> > and sees this flag?
> My original thought was "thats ok", but now that you mention it I am not
> so sure.  Ill give it some more thought because maybe you are on to
> something.
> 
> >  And note that multiple works can run on irqfd
> > in parallel.
> >   
> 
> They can?  I thought work-queue items were guaranteed to only schedule
> once?  If what you say is true, its broken, I agree, and Ill need to
> revisit.  Let me get back to you.
> >   
> >>  	}
> >>  
> >> +	if (flags & (POLLHUP | POLLIN))
> >> +		schedule_work(&irqfd->work);
> >> +
> >>  	return 0;
> >>  }
> >>  
> >> @@ -102,6 +140,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
> >>  {
> >>  	struct _irqfd *irqfd;
> >>  	struct file *file = NULL;
> >> +	struct kref *kref = NULL;
> >>  	int ret;
> >>  	unsigned int events;
> >>  
> >> @@ -112,7 +151,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
> >>  	irqfd->kvm = kvm;
> >>  	irqfd->gsi = gsi;
> >>  	INIT_LIST_HEAD(&irqfd->list);
> >> -	INIT_WORK(&irqfd->inject, irqfd_inject);
> >> +	INIT_WORK(&irqfd->work, irqfd_work);
> >>  
> >>  	file = eventfd_fget(fd);
> >>  	if (IS_ERR(file)) {
> >> @@ -133,11 +172,13 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
> >>  	list_add_tail(&irqfd->list, &kvm->irqfds);
> >>  	mutex_unlock(&kvm->lock);
> >>  
> >> -	/*
> >> -	 * Check if there was an event already queued
> >> -	 */
> >> -	if (events & POLLIN)
> >> -		schedule_work(&irqfd->inject);
> >> +	kref = eventfd_kref_get(file);
> >> +	if (IS_ERR(file)) {
> >> +		ret = PTR_ERR(file);
> >> +		goto fail;
> >> +	}
> >> +
> >> +	irqfd->eventfd = kref;
> >>  
> >>  	/*
> >>  	 * do not drop the file until the irqfd is fully initialized, otherwise
> >> @@ -145,9 +186,18 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
> >>  	 */
> >>  	fput(file);
> >>  
> >> +	/*
> >> +	 * Check if there was an event already queued
> >> +	 */
> >>     
> >
> > This comment seems to confuse more that it clarifies:
> > queued where? eventfd only counts... Just kill the comment?
> >
> >   
> non-zero values in eventfd are "queued" as a signal.  This test just
> checks if an interrupt was already injected before we registered.

After have understood the code I see what you mean, but the comment
wasn't helpful and is better left out.

> >> +	if (events & POLLIN)
> >> +		schedule_work(&irqfd->work);
> >> +
> >>  	return 0;
> >>  
> >>  fail:
> >> +	if (kref && !IS_ERR(kref))
> >> +		eventfd_kref_put(kref);
> >> +
> >>  	if (file && !IS_ERR(file))
> >>  		fput(file);
> >>     
> >
> > let's add a couple more labels and avoid the kref/file check
> > and the initialization above?
> >   
> 
> I think that just makes it more confusing, personally.  But I will give
> it some thought.
> 
> >   
> >>  
> >> @@ -161,21 +211,47 @@ kvm_irqfd_init(struct kvm *kvm)
> >>  	INIT_LIST_HEAD(&kvm->irqfds);
> >>  }
> >>  
> >> +static struct _irqfd *
> >> +irqfd_pop(struct kvm *kvm)
> >> +{
> >> +	struct _irqfd *irqfd = NULL;
> >> +
> >> +	mutex_lock(&kvm->lock);
> >> +
> >> +	if (!list_empty(&kvm->irqfds)) {
> >> +		irqfd = list_first_entry(&kvm->irqfds, struct _irqfd, list);
> >> +		list_del(&irqfd->list);
> >> +	}
> >> +
> >> +	mutex_unlock(&kvm->lock);
> >> +
> >> +	return irqfd;
> >> +}
> >> +
> >>  void
> >>  kvm_irqfd_release(struct kvm *kvm)
> >>  {
> >> -	struct _irqfd *irqfd, *tmp;
> >> +	struct _irqfd *irqfd;
> >>  
> >> -	list_for_each_entry_safe(irqfd, tmp, &kvm->irqfds, list) {
> >> -		if (irqfd->wqh)
> >> -			remove_wait_queue(irqfd->wqh, &irqfd->wait);
> >> +	while ((irqfd = irqfd_pop(kvm))) {
> >>  
> >> -		flush_work(&irqfd->inject);
> >> +		remove_wait_queue(irqfd->wqh, &irqfd->wait);
> >>  
> >> -		mutex_lock(&kvm->lock);
> >> -		list_del(&irqfd->list);
> >> -		mutex_unlock(&kvm->lock);
> >> +		/*
> >> +		 * We guarantee there will be no more notifications after
> >> +		 * the remove_wait_queue returns.  Now lets make sure we
> >> +		 * synchronize behind any outstanding work items before
> >> +		 * releasing the resources
> >> +		 */
> >> +		flush_work(&irqfd->work);
> >>  
> >> -		kfree(irqfd);
> >> +		irqfd_release(irqfd);
> >>  	}
> >> +
> >> +	/*
> >> +	 * We need to wait in case there are any outstanding work-items
> >> +	 * in flight that had already removed themselves from the list
> >> +	 * prior to entry to this function
> >> +	 */
> >>     
> >
> > Looks scary. Why doesn't the flush above cover all cases?
> >   
> 
> The path inside the while() is for when KVM wins the race and finds the
> item in the list.  It atomically removes it, and is responsible for
> freeing it in a coordinated way.  In this case, we must block with the
> flush_work() before we can irqfd_release() so that we do not yank the
> memory out from under a running work-item.
> 
> The flush_scheduled_work() is for when eventfd wins the race and has
> already removed itself from the list in the "shutdown" path in the
> work-item.  We want to make sure that kvm_irqfd_release() cannot return
> until all work-items have exited to prevent something like the kvm.ko
> module unloading while the work-item is still in flight.
> Thanks Michael,
> -Greg
> >   
> >> +	flush_scheduled_work();
> >>  }
> >>     
> > --
> > To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> > the body of a message to majordomo@vger.kernel.org
> > More majordomo info at  http://vger.kernel.org/majordomo-info.html
> > Please read the FAQ at  http://www.tux.org/lkml/
> >   
> 
> 



^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 17:45       ` Michael S. Tsirkin
@ 2009-06-22 18:03         ` Gregory Haskins
  2009-06-22 18:26           ` Michael S. Tsirkin
  2009-06-22 18:11         ` Davide Libenzi
  1 sibling, 1 reply; 14+ messages in thread
From: Gregory Haskins @ 2009-06-22 18:03 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Gregory Haskins, kvm, linux-kernel, avi, paulmck, davidel, mingo, rusty

[-- Attachment #1: Type: text/plain, Size: 14643 bytes --]

Michael S. Tsirkin wrote:
> On Mon, Jun 22, 2009 at 01:31:29PM -0400, Gregory Haskins wrote:
>   
>> Michael S. Tsirkin wrote:
>>     
>>> On Mon, Jun 22, 2009 at 12:05:57PM -0400, Gregory Haskins wrote:
>>>   
>>>       
>>>> This patch fixes all known races in irqfd, and paves the way to restore
>>>> DEASSIGN support.  For details of the eventfd races, please see the patch
>>>> presumably commited immediately prior to this one.
>>>>
>>>> In a nutshell, we use eventfd_kref_get/put() to properly manage the
>>>> lifetime of the underlying eventfd.  We also use careful coordination
>>>> with our workqueue to ensure that all irqfd objects have terminated
>>>> before we allow kvm to shutdown.  The logic used for shutdown walks
>>>> all open irqfds and releases them.  This logic can be generalized in
>>>> the future to allow a subset of irqfds to be released, thus allowing
>>>> DEASSIGN support.
>>>>
>>>> Signed-off-by: Gregory Haskins <ghaskins@novell.com>
>>>>     
>>>>         
>>> I think this patch is a shade too tricky. Some explanation why below.
>>>
>>> But I think irqfd_pop is a good idea.
>>>   
>>>       
>> Yeah, next we can add something like "irqfd_remove(gsi)" in a similar
>> way to do DEASSIGN.
>>
>>     
>>> Here's an alternative design sketch: add a list of irqfds to be shutdown
>>> in kvm, and create a single-threaded workqueue. To kill an irqfd, move
>>> it from list of live irqfds to list of dead irqfds, then schedule work
>>> on a workqueue that walks this list and kills irqfds.
>>>   
>>>       
>> Yeah, I actually thought of that too, and I think that will work.  But
>> then I realized flush_schedule_work does the same thing and its much
>> less code.  Perhaps it is also much less clear, too ;)  At the very
>> least, you have made me realize I need to comment better.
>>     
>
> Not really, it's impossible to document all races one have thought
> about and avoided.
>   

Heh, that is a very astute observation.

>   
>>>   
>>>       
>>>> ---
>>>>
>>>>  virt/kvm/eventfd.c |  144 ++++++++++++++++++++++++++++++++++++++++------------
>>>>  1 files changed, 110 insertions(+), 34 deletions(-)
>>>>
>>>> diff --git a/virt/kvm/eventfd.c b/virt/kvm/eventfd.c
>>>> index 9656027..67985cd 100644
>>>> --- a/virt/kvm/eventfd.c
>>>> +++ b/virt/kvm/eventfd.c
>>>> @@ -28,6 +28,7 @@
>>>>  #include <linux/file.h>
>>>>  #include <linux/list.h>
>>>>  #include <linux/eventfd.h>
>>>> +#include <linux/kref.h>
>>>>  
>>>>  /*
>>>>   * --------------------------------------------------------------------
>>>> @@ -36,26 +37,68 @@
>>>>   * Credit goes to Avi Kivity for the original idea.
>>>>   * --------------------------------------------------------------------
>>>>   */
>>>> +
>>>> +enum {
>>>> +	irqfd_flags_shutdown,
>>>> +};
>>>> +
>>>>  struct _irqfd {
>>>>  	struct kvm               *kvm;
>>>> +	struct kref              *eventfd;
>>>>     
>>>>         
>>> Yay, kref.
>>>
>>>   
>>>       
>>>>  	int                       gsi;
>>>>  	struct list_head          list;
>>>>  	poll_table                pt;
>>>>  	wait_queue_head_t        *wqh;
>>>>  	wait_queue_t              wait;
>>>> -	struct work_struct        inject;
>>>> +	struct work_struct        work;
>>>> +	unsigned long             flags;
>>>>     
>>>>         
>>> Just make it "int shutdown"?
>>>   
>>>       
>> Yep, that is probably fine but we will have to use an explicit wmb in
>> lieu of a set_bit operation.  NBD.
>>
>>     
>>>   
>>>       
>>>>  };
>>>>  
>>>>  static void
>>>> -irqfd_inject(struct work_struct *work)
>>>> +irqfd_release(struct _irqfd *irqfd)
>>>> +{
>>>> +	eventfd_kref_put(irqfd->eventfd);
>>>> +	kfree(irqfd);
>>>> +}
>>>> +
>>>> +static void
>>>> +irqfd_work(struct work_struct *work)
>>>>  {
>>>> -	struct _irqfd *irqfd = container_of(work, struct _irqfd, inject);
>>>> +	struct _irqfd *irqfd = container_of(work, struct _irqfd, work);
>>>>  	struct kvm *kvm = irqfd->kvm;
>>>>  
>>>> -	mutex_lock(&kvm->irq_lock);
>>>> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
>>>> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
>>>> -	mutex_unlock(&kvm->irq_lock);
>>>> +	if (!test_bit(irqfd_flags_shutdown, &irqfd->flags)) {
>>>>     
>>>>         
>>> Why is it safe to test this bit outside of any lock?
>>>   
>>>       
>> Because the ordering is guaranteed to set_bit(), schedule_work().  All
>> we need to do is make sure that the work-queue runs at least one more
>> time after the flag has been set.  (Of course, I could have screwed up
>> too, but that was my rationale).
>>
>>     
>>>   
>>>       
>>>> +		/* Inject an interrupt */
>>>> +		mutex_lock(&kvm->irq_lock);
>>>> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
>>>> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
>>>> +		mutex_unlock(&kvm->irq_lock);
>>>> +	} else {
>>>>     
>>>>         
>>> Not much shared code here - create a separate showdown work struct?
>>> They are cheap ...
>>>   
>>>       
>> We can't because we need to ensure that all inject-jobs complete before
>> release-jobs.  Reading the work-queue code, it would be a deadlock for
>> the release-job to do a flush_work(inject-job).  Therefore, both
>> workloads are encapsulated into a single job, and we ensure that the job
>> is launched at least one more time after the flag has been set.
>>     
>
> AFAIK schedule_work does not give you in-order guarantees - it's
> multithreaded. you will have to create a single-threaded workqueue
> if you want in order execution.
>   

Right, that was my understanding as well.  Thats why I do both tasks
from a single work-item ;)

>   
>> Of course, now that I wrote that,  I realize it was clear-as-mud in the
>> code and needs some commenting ;)
>>
>>     
>>>   
>>>       
>>>> +		/* shutdown the irqfd */
>>>> +		struct _irqfd *_irqfd = NULL;
>>>> +
>>>> +		mutex_lock(&kvm->lock);
>>>> +
>>>> +		if (!list_empty(&irqfd->list))
>>>> +			_irqfd = irqfd;
>>>> +
>>>> +		if (_irqfd)
>>>> +			list_del(&_irqfd->list);
>>>> +
>>>> +		mutex_unlock(&kvm->lock);
>>>> +
>>>> +		/*
>>>> +		 * If the item is not currently on the irqfds list, we know
>>>> +		 * we are running concurrently with the KVM side trying to
>>>> +		 * remove this item as well.
>>>>     
>>>>         
>>> We do? How? As far as I can see list is only empty after it has been
>>> created.  Generally, it would be better to either use a flag or use
>>> list_empty as an indication of going down, but not both.
>>>   
>>>       
>> I think you are mis-reading that.  list_empty(&irqfd->list) is the
>> individual irqfd list-item, not the kvm->irqfds list itself.  This
>> conditional is telling us whether the irqfd in question is on or off the
>> list (its effectively an irqfd-specific flag), not whether the global
>> list is empty.  Again, poor commenting on my part.
>>     
>
> Yes, but you do INIT_LIST_HEAD in a single place. Once you add
> irqfd->list to a list, it won't be empty until you init it again.
>   

Good point.  I need list_del_init() and then it would work, right?

>   
>>>   
>>>       
>>>>  Since the KVM side should be
>>>> +		 * holding the reference now, and will block behind a
>>>> +		 * flush_work(), lets just let them do the release() for us
>>>> +		 */
>>>> +		if (!_irqfd)
>>>> +			return;
>>>> +
>>>> +		irqfd_release(_irqfd);
>>>> +	}
>>>>  }
>>>>  
>>>>  static int
>>>> @@ -65,25 +108,20 @@ irqfd_wakeup(wait_queue_t *wait, unsigned mode, int sync, void *key)
>>>>  	unsigned long flags = (unsigned long)key;
>>>>  
>>>>  	/*
>>>> -	 * Assume we will be called with interrupts disabled
>>>> +	 * called with interrupts disabled
>>>>  	 */
>>>> -	if (flags & POLLIN)
>>>> -		/*
>>>> -		 * Defer the IRQ injection until later since we need to
>>>> -		 * acquire the kvm->lock to do so.
>>>> -		 */
>>>> -		schedule_work(&irqfd->inject);
>>>> -
>>>>  	if (flags & POLLHUP) {
>>>>  		/*
>>>> -		 * for now, just remove ourselves from the list and let
>>>> -		 * the rest dangle.  We will fix this up later once
>>>> -		 * the races in eventfd are fixed
>>>> +		 * ordering is important: shutdown flag must be visible
>>>> +		 * before we schedule
>>>>  		 */
>>>>  		__remove_wait_queue(irqfd->wqh, &irqfd->wait);
>>>> -		irqfd->wqh = NULL;
>>>> +		set_bit(irqfd_flags_shutdown, &irqfd->flags);
>>>>     
>>>>         
>>> So what happens if a previously scheduled work runs on irqfd
>>> and sees this flag?
>>>       
>> My original thought was "thats ok", but now that you mention it I am not
>> so sure.  Ill give it some more thought because maybe you are on to
>> something.
>>
>>     
>>>  And note that multiple works can run on irqfd
>>> in parallel.
>>>   
>>>       
>> They can?  I thought work-queue items were guaranteed to only schedule
>> once?  If what you say is true, its broken, I agree, and Ill need to
>> revisit.  Let me get back to you.
>>     
>>>   
>>>       
>>>>  	}
>>>>  
>>>> +	if (flags & (POLLHUP | POLLIN))
>>>> +		schedule_work(&irqfd->work);
>>>> +
>>>>  	return 0;
>>>>  }
>>>>  
>>>> @@ -102,6 +140,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>>>>  {
>>>>  	struct _irqfd *irqfd;
>>>>  	struct file *file = NULL;
>>>> +	struct kref *kref = NULL;
>>>>  	int ret;
>>>>  	unsigned int events;
>>>>  
>>>> @@ -112,7 +151,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>>>>  	irqfd->kvm = kvm;
>>>>  	irqfd->gsi = gsi;
>>>>  	INIT_LIST_HEAD(&irqfd->list);
>>>> -	INIT_WORK(&irqfd->inject, irqfd_inject);
>>>> +	INIT_WORK(&irqfd->work, irqfd_work);
>>>>  
>>>>  	file = eventfd_fget(fd);
>>>>  	if (IS_ERR(file)) {
>>>> @@ -133,11 +172,13 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>>>>  	list_add_tail(&irqfd->list, &kvm->irqfds);
>>>>  	mutex_unlock(&kvm->lock);
>>>>  
>>>> -	/*
>>>> -	 * Check if there was an event already queued
>>>> -	 */
>>>> -	if (events & POLLIN)
>>>> -		schedule_work(&irqfd->inject);
>>>> +	kref = eventfd_kref_get(file);
>>>> +	if (IS_ERR(file)) {
>>>> +		ret = PTR_ERR(file);
>>>> +		goto fail;
>>>> +	}
>>>> +
>>>> +	irqfd->eventfd = kref;
>>>>  
>>>>  	/*
>>>>  	 * do not drop the file until the irqfd is fully initialized, otherwise
>>>> @@ -145,9 +186,18 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
>>>>  	 */
>>>>  	fput(file);
>>>>  
>>>> +	/*
>>>> +	 * Check if there was an event already queued
>>>> +	 */
>>>>     
>>>>         
>>> This comment seems to confuse more that it clarifies:
>>> queued where? eventfd only counts... Just kill the comment?
>>>
>>>   
>>>       
>> non-zero values in eventfd are "queued" as a signal.  This test just
>> checks if an interrupt was already injected before we registered.
>>     
>
> After have understood the code I see what you mean, but the comment
> wasn't helpful and is better left out.
>   

Ok.  What if I say "Check if an interrupt is already pending before we
registered the callback" ;)

>   
>>>> +	if (events & POLLIN)
>>>> +		schedule_work(&irqfd->work);
>>>> +
>>>>  	return 0;
>>>>  
>>>>  fail:
>>>> +	if (kref && !IS_ERR(kref))
>>>> +		eventfd_kref_put(kref);
>>>> +
>>>>  	if (file && !IS_ERR(file))
>>>>  		fput(file);
>>>>     
>>>>         
>>> let's add a couple more labels and avoid the kref/file check
>>> and the initialization above?
>>>   
>>>       
>> I think that just makes it more confusing, personally.  But I will give
>> it some thought.
>>
>>     
>>>   
>>>       
>>>>  
>>>> @@ -161,21 +211,47 @@ kvm_irqfd_init(struct kvm *kvm)
>>>>  	INIT_LIST_HEAD(&kvm->irqfds);
>>>>  }
>>>>  
>>>> +static struct _irqfd *
>>>> +irqfd_pop(struct kvm *kvm)
>>>> +{
>>>> +	struct _irqfd *irqfd = NULL;
>>>> +
>>>> +	mutex_lock(&kvm->lock);
>>>> +
>>>> +	if (!list_empty(&kvm->irqfds)) {
>>>> +		irqfd = list_first_entry(&kvm->irqfds, struct _irqfd, list);
>>>> +		list_del(&irqfd->list);
>>>> +	}
>>>> +
>>>> +	mutex_unlock(&kvm->lock);
>>>> +
>>>> +	return irqfd;
>>>> +}
>>>> +
>>>>  void
>>>>  kvm_irqfd_release(struct kvm *kvm)
>>>>  {
>>>> -	struct _irqfd *irqfd, *tmp;
>>>> +	struct _irqfd *irqfd;
>>>>  
>>>> -	list_for_each_entry_safe(irqfd, tmp, &kvm->irqfds, list) {
>>>> -		if (irqfd->wqh)
>>>> -			remove_wait_queue(irqfd->wqh, &irqfd->wait);
>>>> +	while ((irqfd = irqfd_pop(kvm))) {
>>>>  
>>>> -		flush_work(&irqfd->inject);
>>>> +		remove_wait_queue(irqfd->wqh, &irqfd->wait);
>>>>  
>>>> -		mutex_lock(&kvm->lock);
>>>> -		list_del(&irqfd->list);
>>>> -		mutex_unlock(&kvm->lock);
>>>> +		/*
>>>> +		 * We guarantee there will be no more notifications after
>>>> +		 * the remove_wait_queue returns.  Now lets make sure we
>>>> +		 * synchronize behind any outstanding work items before
>>>> +		 * releasing the resources
>>>> +		 */
>>>> +		flush_work(&irqfd->work);
>>>>  
>>>> -		kfree(irqfd);
>>>> +		irqfd_release(irqfd);
>>>>  	}
>>>> +
>>>> +	/*
>>>> +	 * We need to wait in case there are any outstanding work-items
>>>> +	 * in flight that had already removed themselves from the list
>>>> +	 * prior to entry to this function
>>>> +	 */
>>>>     
>>>>         
>>> Looks scary. Why doesn't the flush above cover all cases?
>>>   
>>>       
>> The path inside the while() is for when KVM wins the race and finds the
>> item in the list.  It atomically removes it, and is responsible for
>> freeing it in a coordinated way.  In this case, we must block with the
>> flush_work() before we can irqfd_release() so that we do not yank the
>> memory out from under a running work-item.
>>
>> The flush_scheduled_work() is for when eventfd wins the race and has
>> already removed itself from the list in the "shutdown" path in the
>> work-item.  We want to make sure that kvm_irqfd_release() cannot return
>> until all work-items have exited to prevent something like the kvm.ko
>> module unloading while the work-item is still in flight.
>> Thanks Michael,
>> -Greg
>>     
>>>   
>>>       
>>>> +	flush_scheduled_work();
>>>>  }
>>>>     
>>>>         
>>> --
>>> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
>>> the body of a message to majordomo@vger.kernel.org
>>> More majordomo info at  http://vger.kernel.org/majordomo-info.html
>>> Please read the FAQ at  http://www.tux.org/lkml/
>>>   
>>>       
>>     
>
>
>   



[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 266 bytes --]

^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 17:45       ` Michael S. Tsirkin
  2009-06-22 18:03         ` Gregory Haskins
@ 2009-06-22 18:11         ` Davide Libenzi
  2009-06-22 18:32           ` Michael S. Tsirkin
  1 sibling, 1 reply; 14+ messages in thread
From: Davide Libenzi @ 2009-06-22 18:11 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Gregory Haskins, Gregory Haskins, kvm, Linux Kernel Mailing List,
	avi, paulmck, Ingo Molnar, Rusty Russell

On Mon, 22 Jun 2009, Michael S. Tsirkin wrote:

> Not really, it's impossible to document all races one have thought
> about and avoided.

Well, when some new code has non-trivial locking/racing logics, you better 
document it as clearly as possible, akpm announced time ago.


- Davide



^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 18:03         ` Gregory Haskins
@ 2009-06-22 18:26           ` Michael S. Tsirkin
  0 siblings, 0 replies; 14+ messages in thread
From: Michael S. Tsirkin @ 2009-06-22 18:26 UTC (permalink / raw)
  To: Gregory Haskins
  Cc: Gregory Haskins, kvm, linux-kernel, avi, paulmck, davidel, mingo, rusty

On Mon, Jun 22, 2009 at 02:03:59PM -0400, Gregory Haskins wrote:
> Michael S. Tsirkin wrote:
> > On Mon, Jun 22, 2009 at 01:31:29PM -0400, Gregory Haskins wrote:
> >   
> >> Michael S. Tsirkin wrote:
> >>     
> >>> On Mon, Jun 22, 2009 at 12:05:57PM -0400, Gregory Haskins wrote:
> >>>   
> >>>       
> >>>> This patch fixes all known races in irqfd, and paves the way to restore
> >>>> DEASSIGN support.  For details of the eventfd races, please see the patch
> >>>> presumably commited immediately prior to this one.
> >>>>
> >>>> In a nutshell, we use eventfd_kref_get/put() to properly manage the
> >>>> lifetime of the underlying eventfd.  We also use careful coordination
> >>>> with our workqueue to ensure that all irqfd objects have terminated
> >>>> before we allow kvm to shutdown.  The logic used for shutdown walks
> >>>> all open irqfds and releases them.  This logic can be generalized in
> >>>> the future to allow a subset of irqfds to be released, thus allowing
> >>>> DEASSIGN support.
> >>>>
> >>>> Signed-off-by: Gregory Haskins <ghaskins@novell.com>
> >>>>     
> >>>>         
> >>> I think this patch is a shade too tricky. Some explanation why below.
> >>>
> >>> But I think irqfd_pop is a good idea.
> >>>   
> >>>       
> >> Yeah, next we can add something like "irqfd_remove(gsi)" in a similar
> >> way to do DEASSIGN.
> >>
> >>     
> >>> Here's an alternative design sketch: add a list of irqfds to be shutdown
> >>> in kvm, and create a single-threaded workqueue. To kill an irqfd, move
> >>> it from list of live irqfds to list of dead irqfds, then schedule work
> >>> on a workqueue that walks this list and kills irqfds.
> >>>   
> >>>       
> >> Yeah, I actually thought of that too, and I think that will work.  But
> >> then I realized flush_schedule_work does the same thing and its much
> >> less code.  Perhaps it is also much less clear, too ;)  At the very
> >> least, you have made me realize I need to comment better.
> >>     
> >
> > Not really, it's impossible to document all races one have thought
> > about and avoided.
> >   
> 
> Heh, that is a very astute observation.
> 
> >   
> >>>   
> >>>       
> >>>> ---
> >>>>
> >>>>  virt/kvm/eventfd.c |  144 ++++++++++++++++++++++++++++++++++++++++------------
> >>>>  1 files changed, 110 insertions(+), 34 deletions(-)
> >>>>
> >>>> diff --git a/virt/kvm/eventfd.c b/virt/kvm/eventfd.c
> >>>> index 9656027..67985cd 100644
> >>>> --- a/virt/kvm/eventfd.c
> >>>> +++ b/virt/kvm/eventfd.c
> >>>> @@ -28,6 +28,7 @@
> >>>>  #include <linux/file.h>
> >>>>  #include <linux/list.h>
> >>>>  #include <linux/eventfd.h>
> >>>> +#include <linux/kref.h>
> >>>>  
> >>>>  /*
> >>>>   * --------------------------------------------------------------------
> >>>> @@ -36,26 +37,68 @@
> >>>>   * Credit goes to Avi Kivity for the original idea.
> >>>>   * --------------------------------------------------------------------
> >>>>   */
> >>>> +
> >>>> +enum {
> >>>> +	irqfd_flags_shutdown,
> >>>> +};
> >>>> +
> >>>>  struct _irqfd {
> >>>>  	struct kvm               *kvm;
> >>>> +	struct kref              *eventfd;
> >>>>     
> >>>>         
> >>> Yay, kref.
> >>>
> >>>   
> >>>       
> >>>>  	int                       gsi;
> >>>>  	struct list_head          list;
> >>>>  	poll_table                pt;
> >>>>  	wait_queue_head_t        *wqh;
> >>>>  	wait_queue_t              wait;
> >>>> -	struct work_struct        inject;
> >>>> +	struct work_struct        work;
> >>>> +	unsigned long             flags;
> >>>>     
> >>>>         
> >>> Just make it "int shutdown"?
> >>>   
> >>>       
> >> Yep, that is probably fine but we will have to use an explicit wmb in
> >> lieu of a set_bit operation.  NBD.
> >>
> >>     
> >>>   
> >>>       
> >>>>  };
> >>>>  
> >>>>  static void
> >>>> -irqfd_inject(struct work_struct *work)
> >>>> +irqfd_release(struct _irqfd *irqfd)
> >>>> +{
> >>>> +	eventfd_kref_put(irqfd->eventfd);
> >>>> +	kfree(irqfd);
> >>>> +}
> >>>> +
> >>>> +static void
> >>>> +irqfd_work(struct work_struct *work)
> >>>>  {
> >>>> -	struct _irqfd *irqfd = container_of(work, struct _irqfd, inject);
> >>>> +	struct _irqfd *irqfd = container_of(work, struct _irqfd, work);
> >>>>  	struct kvm *kvm = irqfd->kvm;
> >>>>  
> >>>> -	mutex_lock(&kvm->irq_lock);
> >>>> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
> >>>> -	kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
> >>>> -	mutex_unlock(&kvm->irq_lock);
> >>>> +	if (!test_bit(irqfd_flags_shutdown, &irqfd->flags)) {
> >>>>     
> >>>>         
> >>> Why is it safe to test this bit outside of any lock?
> >>>   
> >>>       
> >> Because the ordering is guaranteed to set_bit(), schedule_work().  All
> >> we need to do is make sure that the work-queue runs at least one more
> >> time after the flag has been set.  (Of course, I could have screwed up
> >> too, but that was my rationale).
> >>
> >>     
> >>>   
> >>>       
> >>>> +		/* Inject an interrupt */
> >>>> +		mutex_lock(&kvm->irq_lock);
> >>>> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 1);
> >>>> +		kvm_set_irq(kvm, KVM_USERSPACE_IRQ_SOURCE_ID, irqfd->gsi, 0);
> >>>> +		mutex_unlock(&kvm->irq_lock);
> >>>> +	} else {
> >>>>     
> >>>>         
> >>> Not much shared code here - create a separate showdown work struct?
> >>> They are cheap ...
> >>>   
> >>>       
> >> We can't because we need to ensure that all inject-jobs complete before
> >> release-jobs.  Reading the work-queue code, it would be a deadlock for
> >> the release-job to do a flush_work(inject-job).  Therefore, both
> >> workloads are encapsulated into a single job, and we ensure that the job
> >> is launched at least one more time after the flag has been set.
> >>     
> >
> > AFAIK schedule_work does not give you in-order guarantees - it's
> > multithreaded. you will have to create a single-threaded workqueue
> > if you want in order execution.
> >   
> 
> Right, that was my understanding as well.  Thats why I do both tasks
> from a single work-item ;)

I don't think this gives ordering guarantees. If the work is already
running on CPU1 and you do schedule it will start running on CPU2.

> >   
> >> Of course, now that I wrote that,  I realize it was clear-as-mud in the
> >> code and needs some commenting ;)
> >>
> >>     
> >>>   
> >>>       
> >>>> +		/* shutdown the irqfd */
> >>>> +		struct _irqfd *_irqfd = NULL;
> >>>> +
> >>>> +		mutex_lock(&kvm->lock);
> >>>> +
> >>>> +		if (!list_empty(&irqfd->list))
> >>>> +			_irqfd = irqfd;
> >>>> +
> >>>> +		if (_irqfd)
> >>>> +			list_del(&_irqfd->list);
> >>>> +
> >>>> +		mutex_unlock(&kvm->lock);
> >>>> +
> >>>> +		/*
> >>>> +		 * If the item is not currently on the irqfds list, we know
> >>>> +		 * we are running concurrently with the KVM side trying to
> >>>> +		 * remove this item as well.
> >>>>     
> >>>>         
> >>> We do? How? As far as I can see list is only empty after it has been
> >>> created.  Generally, it would be better to either use a flag or use
> >>> list_empty as an indication of going down, but not both.
> >>>   
> >>>       
> >> I think you are mis-reading that.  list_empty(&irqfd->list) is the
> >> individual irqfd list-item, not the kvm->irqfds list itself.  This
> >> conditional is telling us whether the irqfd in question is on or off the
> >> list (its effectively an irqfd-specific flag), not whether the global
> >> list is empty.  Again, poor commenting on my part.
> >>     
> >
> > Yes, but you do INIT_LIST_HEAD in a single place. Once you add
> > irqfd->list to a list, it won't be empty until you init it again.
> >   
> 
> Good point.  I need list_del_init() and then it would work, right?

Hmm, maybe.

> >   
> >>>   
> >>>       
> >>>>  Since the KVM side should be
> >>>> +		 * holding the reference now, and will block behind a
> >>>> +		 * flush_work(), lets just let them do the release() for us
> >>>> +		 */
> >>>> +		if (!_irqfd)
> >>>> +			return;
> >>>> +
> >>>> +		irqfd_release(_irqfd);
> >>>> +	}
> >>>>  }
> >>>>  
> >>>>  static int
> >>>> @@ -65,25 +108,20 @@ irqfd_wakeup(wait_queue_t *wait, unsigned mode, int sync, void *key)
> >>>>  	unsigned long flags = (unsigned long)key;
> >>>>  
> >>>>  	/*
> >>>> -	 * Assume we will be called with interrupts disabled
> >>>> +	 * called with interrupts disabled
> >>>>  	 */
> >>>> -	if (flags & POLLIN)
> >>>> -		/*
> >>>> -		 * Defer the IRQ injection until later since we need to
> >>>> -		 * acquire the kvm->lock to do so.
> >>>> -		 */
> >>>> -		schedule_work(&irqfd->inject);
> >>>> -
> >>>>  	if (flags & POLLHUP) {
> >>>>  		/*
> >>>> -		 * for now, just remove ourselves from the list and let
> >>>> -		 * the rest dangle.  We will fix this up later once
> >>>> -		 * the races in eventfd are fixed
> >>>> +		 * ordering is important: shutdown flag must be visible
> >>>> +		 * before we schedule
> >>>>  		 */
> >>>>  		__remove_wait_queue(irqfd->wqh, &irqfd->wait);
> >>>> -		irqfd->wqh = NULL;
> >>>> +		set_bit(irqfd_flags_shutdown, &irqfd->flags);
> >>>>     
> >>>>         
> >>> So what happens if a previously scheduled work runs on irqfd
> >>> and sees this flag?
> >>>       
> >> My original thought was "thats ok", but now that you mention it I am not
> >> so sure.  Ill give it some more thought because maybe you are on to
> >> something.
> >>
> >>     
> >>>  And note that multiple works can run on irqfd
> >>> in parallel.
> >>>   
> >>>       
> >> They can?  I thought work-queue items were guaranteed to only schedule
> >> once?  If what you say is true, its broken, I agree, and Ill need to
> >> revisit.  Let me get back to you.
> >>     
> >>>   
> >>>       
> >>>>  	}
> >>>>  
> >>>> +	if (flags & (POLLHUP | POLLIN))
> >>>> +		schedule_work(&irqfd->work);
> >>>> +
> >>>>  	return 0;
> >>>>  }
> >>>>  
> >>>> @@ -102,6 +140,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
> >>>>  {
> >>>>  	struct _irqfd *irqfd;
> >>>>  	struct file *file = NULL;
> >>>> +	struct kref *kref = NULL;
> >>>>  	int ret;
> >>>>  	unsigned int events;
> >>>>  
> >>>> @@ -112,7 +151,7 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
> >>>>  	irqfd->kvm = kvm;
> >>>>  	irqfd->gsi = gsi;
> >>>>  	INIT_LIST_HEAD(&irqfd->list);
> >>>> -	INIT_WORK(&irqfd->inject, irqfd_inject);
> >>>> +	INIT_WORK(&irqfd->work, irqfd_work);
> >>>>  
> >>>>  	file = eventfd_fget(fd);
> >>>>  	if (IS_ERR(file)) {
> >>>> @@ -133,11 +172,13 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
> >>>>  	list_add_tail(&irqfd->list, &kvm->irqfds);
> >>>>  	mutex_unlock(&kvm->lock);
> >>>>  
> >>>> -	/*
> >>>> -	 * Check if there was an event already queued
> >>>> -	 */
> >>>> -	if (events & POLLIN)
> >>>> -		schedule_work(&irqfd->inject);
> >>>> +	kref = eventfd_kref_get(file);
> >>>> +	if (IS_ERR(file)) {
> >>>> +		ret = PTR_ERR(file);
> >>>> +		goto fail;
> >>>> +	}
> >>>> +
> >>>> +	irqfd->eventfd = kref;
> >>>>  
> >>>>  	/*
> >>>>  	 * do not drop the file until the irqfd is fully initialized, otherwise
> >>>> @@ -145,9 +186,18 @@ kvm_irqfd(struct kvm *kvm, int fd, int gsi, int flags)
> >>>>  	 */
> >>>>  	fput(file);
> >>>>  
> >>>> +	/*
> >>>> +	 * Check if there was an event already queued
> >>>> +	 */
> >>>>     
> >>>>         
> >>> This comment seems to confuse more that it clarifies:
> >>> queued where? eventfd only counts... Just kill the comment?
> >>>
> >>>   
> >>>       
> >> non-zero values in eventfd are "queued" as a signal.  This test just
> >> checks if an interrupt was already injected before we registered.
> >>     
> >
> > After have understood the code I see what you mean, but the comment
> > wasn't helpful and is better left out.
> >   
> 
> Ok.  What if I say "Check if an interrupt is already pending before we
> registered the callback" ;)

Maybe you can say "check if eventfd was already signalled
before we registered the callback"

> >   
> >>>> +	if (events & POLLIN)
> >>>> +		schedule_work(&irqfd->work);
> >>>> +
> >>>>  	return 0;
> >>>>  
> >>>>  fail:
> >>>> +	if (kref && !IS_ERR(kref))
> >>>> +		eventfd_kref_put(kref);
> >>>> +
> >>>>  	if (file && !IS_ERR(file))
> >>>>  		fput(file);
> >>>>     
> >>>>         
> >>> let's add a couple more labels and avoid the kref/file check
> >>> and the initialization above?
> >>>   
> >>>       
> >> I think that just makes it more confusing, personally.  But I will give
> >> it some thought.
> >>
> >>     
> >>>   
> >>>       
> >>>>  
> >>>> @@ -161,21 +211,47 @@ kvm_irqfd_init(struct kvm *kvm)
> >>>>  	INIT_LIST_HEAD(&kvm->irqfds);
> >>>>  }
> >>>>  
> >>>> +static struct _irqfd *
> >>>> +irqfd_pop(struct kvm *kvm)
> >>>> +{
> >>>> +	struct _irqfd *irqfd = NULL;
> >>>> +
> >>>> +	mutex_lock(&kvm->lock);
> >>>> +
> >>>> +	if (!list_empty(&kvm->irqfds)) {
> >>>> +		irqfd = list_first_entry(&kvm->irqfds, struct _irqfd, list);
> >>>> +		list_del(&irqfd->list);
> >>>> +	}
> >>>> +
> >>>> +	mutex_unlock(&kvm->lock);
> >>>> +
> >>>> +	return irqfd;
> >>>> +}
> >>>> +
> >>>>  void
> >>>>  kvm_irqfd_release(struct kvm *kvm)
> >>>>  {
> >>>> -	struct _irqfd *irqfd, *tmp;
> >>>> +	struct _irqfd *irqfd;
> >>>>  
> >>>> -	list_for_each_entry_safe(irqfd, tmp, &kvm->irqfds, list) {
> >>>> -		if (irqfd->wqh)
> >>>> -			remove_wait_queue(irqfd->wqh, &irqfd->wait);
> >>>> +	while ((irqfd = irqfd_pop(kvm))) {
> >>>>  
> >>>> -		flush_work(&irqfd->inject);
> >>>> +		remove_wait_queue(irqfd->wqh, &irqfd->wait);
> >>>>  
> >>>> -		mutex_lock(&kvm->lock);
> >>>> -		list_del(&irqfd->list);
> >>>> -		mutex_unlock(&kvm->lock);
> >>>> +		/*
> >>>> +		 * We guarantee there will be no more notifications after
> >>>> +		 * the remove_wait_queue returns.  Now lets make sure we
> >>>> +		 * synchronize behind any outstanding work items before
> >>>> +		 * releasing the resources
> >>>> +		 */
> >>>> +		flush_work(&irqfd->work);
> >>>>  
> >>>> -		kfree(irqfd);
> >>>> +		irqfd_release(irqfd);
> >>>>  	}
> >>>> +
> >>>> +	/*
> >>>> +	 * We need to wait in case there are any outstanding work-items
> >>>> +	 * in flight that had already removed themselves from the list
> >>>> +	 * prior to entry to this function
> >>>> +	 */
> >>>>     
> >>>>         
> >>> Looks scary. Why doesn't the flush above cover all cases?
> >>>   
> >>>       
> >> The path inside the while() is for when KVM wins the race and finds the
> >> item in the list.  It atomically removes it, and is responsible for
> >> freeing it in a coordinated way.  In this case, we must block with the
> >> flush_work() before we can irqfd_release() so that we do not yank the
> >> memory out from under a running work-item.
> >>
> >> The flush_scheduled_work() is for when eventfd wins the race and has
> >> already removed itself from the list in the "shutdown" path in the
> >> work-item.  We want to make sure that kvm_irqfd_release() cannot return
> >> until all work-items have exited to prevent something like the kvm.ko
> >> module unloading while the work-item is still in flight.
> >> Thanks Michael,
> >> -Greg
> >>     
> >>>   
> >>>       
> >>>> +	flush_scheduled_work();
> >>>>  }
> >>>>     
> >>>>         
> >>> --
> >>> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> >>> the body of a message to majordomo@vger.kernel.org
> >>> More majordomo info at  http://vger.kernel.org/majordomo-info.html
> >>> Please read the FAQ at  http://www.tux.org/lkml/
> >>>   
> >>>       
> >>     
> >
> >
> >   
> 
> 



^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 18:11         ` Davide Libenzi
@ 2009-06-22 18:32           ` Michael S. Tsirkin
  2009-06-22 18:41             ` Davide Libenzi
  0 siblings, 1 reply; 14+ messages in thread
From: Michael S. Tsirkin @ 2009-06-22 18:32 UTC (permalink / raw)
  To: Davide Libenzi
  Cc: Gregory Haskins, Gregory Haskins, kvm, Linux Kernel Mailing List,
	avi, paulmck, Ingo Molnar, Rusty Russell

On Mon, Jun 22, 2009 at 11:11:12AM -0700, Davide Libenzi wrote:
> On Mon, 22 Jun 2009, Michael S. Tsirkin wrote:
> 
> > Not really, it's impossible to document all races one have thought
> > about and avoided.
> 
> Well, when some new code has non-trivial locking/racing logics, you better 
> document it as clearly as possible, akpm announced time ago.

Absolutely. But IMO documentation in the form of "fields A B C are
protected by lock X" in a single place near field declaration is better
than "we take lock X here to protect against race with user thread using
fields A B C" sprinkled all over the code.



^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 18:32           ` Michael S. Tsirkin
@ 2009-06-22 18:41             ` Davide Libenzi
  2009-06-22 18:52               ` Michael S. Tsirkin
  0 siblings, 1 reply; 14+ messages in thread
From: Davide Libenzi @ 2009-06-22 18:41 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Gregory Haskins, Gregory Haskins, kvm, Linux Kernel Mailing List,
	avi, paulmck, Ingo Molnar, Rusty Russell

On Mon, 22 Jun 2009, Michael S. Tsirkin wrote:

> On Mon, Jun 22, 2009 at 11:11:12AM -0700, Davide Libenzi wrote:
> > On Mon, 22 Jun 2009, Michael S. Tsirkin wrote:
> > 
> > > Not really, it's impossible to document all races one have thought
> > > about and avoided.
> > 
> > Well, when some new code has non-trivial locking/racing logics, you better 
> > document it as clearly as possible, akpm announced time ago.
> 
> Absolutely. But IMO documentation in the form of "fields A B C are
> protected by lock X" in a single place near field declaration is better
> than "we take lock X here to protect against race with user thread using
> fields A B C" sprinkled all over the code.

Exactly what akpm suggested. A fat comment on top of the C file describing 
locking logic and pitfalls.


- Davide



^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 18:41             ` Davide Libenzi
@ 2009-06-22 18:52               ` Michael S. Tsirkin
  0 siblings, 0 replies; 14+ messages in thread
From: Michael S. Tsirkin @ 2009-06-22 18:52 UTC (permalink / raw)
  To: Davide Libenzi
  Cc: Gregory Haskins, Gregory Haskins, kvm, Linux Kernel Mailing List,
	avi, paulmck, Ingo Molnar, Rusty Russell

On Mon, Jun 22, 2009 at 11:41:53AM -0700, Davide Libenzi wrote:
> > Absolutely. But IMO documentation in the form of "fields A B C are
> > protected by lock X" in a single place near field declaration is better
> > than "we take lock X here to protect against race with user thread using
> > fields A B C" sprinkled all over the code.
> 
> Exactly what akpm suggested.

Everything that can be invented has been invented.


^ permalink raw reply	[flat|nested] 14+ messages in thread

* Re: [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface
  2009-06-22 17:31     ` Gregory Haskins
  2009-06-22 17:45       ` Michael S. Tsirkin
@ 2009-06-23 14:55       ` Gregory Haskins
  1 sibling, 0 replies; 14+ messages in thread
From: Gregory Haskins @ 2009-06-23 14:55 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Gregory Haskins, kvm, linux-kernel, avi, paulmck, davidel, mingo, rusty

[-- Attachment #1: Type: text/plain, Size: 461 bytes --]

Gregory Haskins wrote:
> Michael S. Tsirkin wrote:
>   
>
>>  And note that multiple works can run on irqfd
>> in parallel.
>>   
>>     
>
> They can?  I thought work-queue items were guaranteed to only schedule
> once?  If what you say is true, its broken, I agree, and Ill need to
> revisit.  Let me get back to you.
>   

<sigh> Yep, you are right.  This is broken in my code. :(

Will address in the next spin.  Good catch.

-Greg




[-- Attachment #2: OpenPGP digital signature --]
[-- Type: application/pgp-signature, Size: 266 bytes --]

^ permalink raw reply	[flat|nested] 14+ messages in thread

end of thread, other threads:[~2009-06-23 14:56 UTC | newest]

Thread overview: 14+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2009-06-22 16:05 [KVM PATCH v3 0/3] irqfd/eventfd fixes Gregory Haskins
2009-06-22 16:05 ` [KVM PATCH v3 1/3] kvm: prepare irqfd for having interrupts disabled during eventfd->release Gregory Haskins
2009-06-22 16:05 ` [KVM PATCH v3 2/3] eventfd: add internal reference counting to fix notifier race conditions Gregory Haskins
2009-06-22 16:05 ` [KVM PATCH v3 3/3] KVM: Fix races in irqfd using new eventfd_kref_get interface Gregory Haskins
2009-06-22 16:57   ` Michael S. Tsirkin
2009-06-22 17:31     ` Gregory Haskins
2009-06-22 17:45       ` Michael S. Tsirkin
2009-06-22 18:03         ` Gregory Haskins
2009-06-22 18:26           ` Michael S. Tsirkin
2009-06-22 18:11         ` Davide Libenzi
2009-06-22 18:32           ` Michael S. Tsirkin
2009-06-22 18:41             ` Davide Libenzi
2009-06-22 18:52               ` Michael S. Tsirkin
2009-06-23 14:55       ` Gregory Haskins

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.