linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/4] rwsem: Implement writer lock-stealing
@ 2013-02-09  2:45 Michel Lespinasse
  2013-02-09  2:45 ` [PATCH 1/4] rwsem: make the waiter type an enumeration rather than a bitmask Michel Lespinasse
                   ` (4 more replies)
  0 siblings, 5 replies; 15+ messages in thread
From: Michel Lespinasse @ 2013-02-09  2:45 UTC (permalink / raw)
  To: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	alex.shi, yuanhan.liu, dhowells, akpm, tglx

This proposal implements writer lock stealing in lib/rwsem.c, just as
Alex Shi's earlier proposal did for the simpler lib/rwsem-spinlock.c

Patches 1 and 2 are small cleanups that I thought I should separate
from the bulk of the changes; Patch 3 is the generic support for write
lock stealing; Patch 4 is the x86 support (optional, only required for
higher performance).

I am not in an ideal position to push this as I will be out of town
starting from the middle of next week. However, I hope that this
proposal will at least get the ball rolling. I did run some basic
testing including David Howell's synchro-test module (as found in
Andrew's -mm tree).

Michel Lespinasse (4):
  rwsem: make the waiter type an enumeration rather than a bitmask
  rwsem: shorter spinlocked section in rwsem_down_failed_common()
  rwsem: implement write lock stealing
  x86 rwsem: avoid taking slow path when stealing write lock

 arch/x86/include/asm/rwsem.h |  28 +++--
 include/linux/rwsem.h        |   2 +
 lib/rwsem.c                  | 252 ++++++++++++++++++++-----------------------
 3 files changed, 139 insertions(+), 143 deletions(-)

-- 
1.8.1

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

* [PATCH 1/4] rwsem: make the waiter type an enumeration rather than a bitmask
  2013-02-09  2:45 [PATCH 0/4] rwsem: Implement writer lock-stealing Michel Lespinasse
@ 2013-02-09  2:45 ` Michel Lespinasse
  2013-02-09  2:45 ` [PATCH 2/4] rwsem: shorter spinlocked section in rwsem_down_failed_common() Michel Lespinasse
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 15+ messages in thread
From: Michel Lespinasse @ 2013-02-09  2:45 UTC (permalink / raw)
  To: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	alex.shi, yuanhan.liu, dhowells, akpm, tglx

We are not planning to add some new waiter flags, so we can convert the
waiter type into an enumeration.

Background: David Howells suggested I do this back when I tried adding
a new waiter type for unfair readers. However, I believe the cleanup
applies regardless of that use case.

Signed-off-by: Michel Lespinasse <walken@google.com>

---
 lib/rwsem.c | 19 +++++++++++--------
 1 file changed, 11 insertions(+), 8 deletions(-)

diff --git a/lib/rwsem.c b/lib/rwsem.c
index 8337e1b9bb8d..4a6ff093a433 100644
--- a/lib/rwsem.c
+++ b/lib/rwsem.c
@@ -28,12 +28,15 @@ void __init_rwsem(struct rw_semaphore *sem, const char *name,
 
 EXPORT_SYMBOL(__init_rwsem);
 
+enum rwsem_waiter_type {
+	RWSEM_WAITING_FOR_WRITE,
+	RWSEM_WAITING_FOR_READ
+};
+
 struct rwsem_waiter {
 	struct list_head list;
 	struct task_struct *task;
-	unsigned int flags;
-#define RWSEM_WAITING_FOR_READ	0x00000001
-#define RWSEM_WAITING_FOR_WRITE	0x00000002
+	enum rwsem_waiter_type type;
 };
 
 /* Wake types for __rwsem_do_wake().  Note that RWSEM_WAKE_NO_ACTIVE and
@@ -63,7 +66,7 @@ __rwsem_do_wake(struct rw_semaphore *sem, int wake_type)
 	signed long oldcount, woken, loop, adjustment;
 
 	waiter = list_entry(sem->wait_list.next, struct rwsem_waiter, list);
-	if (!(waiter->flags & RWSEM_WAITING_FOR_WRITE))
+	if (waiter->type != RWSEM_WAITING_FOR_WRITE)
 		goto readers_only;
 
 	if (wake_type == RWSEM_WAKE_READ_OWNED)
@@ -132,10 +135,10 @@ __rwsem_do_wake(struct rw_semaphore *sem, int wake_type)
 		waiter = list_entry(waiter->list.next,
 					struct rwsem_waiter, list);
 
-	} while (waiter->flags & RWSEM_WAITING_FOR_READ);
+	} while (waiter->type != RWSEM_WAITING_FOR_WRITE);
 
 	adjustment = woken * RWSEM_ACTIVE_READ_BIAS;
-	if (waiter->flags & RWSEM_WAITING_FOR_READ)
+	if (waiter->type != RWSEM_WAITING_FOR_WRITE)
 		/* hit end of list above */
 		adjustment -= RWSEM_WAITING_BIAS;
 
@@ -171,7 +174,7 @@ __rwsem_do_wake(struct rw_semaphore *sem, int wake_type)
  */
 static struct rw_semaphore __sched *
 rwsem_down_failed_common(struct rw_semaphore *sem,
-			 unsigned int flags, signed long adjustment)
+			 enum rwsem_waiter_type type, signed long adjustment)
 {
 	struct rwsem_waiter waiter;
 	struct task_struct *tsk = current;
@@ -182,7 +185,7 @@ rwsem_down_failed_common(struct rw_semaphore *sem,
 	/* set up my own style of waitqueue */
 	raw_spin_lock_irq(&sem->wait_lock);
 	waiter.task = tsk;
-	waiter.flags = flags;
+	waiter.type = type;
 	get_task_struct(tsk);
 
 	if (list_empty(&sem->wait_list))
-- 
1.8.1

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

* [PATCH 2/4] rwsem: shorter spinlocked section in rwsem_down_failed_common()
  2013-02-09  2:45 [PATCH 0/4] rwsem: Implement writer lock-stealing Michel Lespinasse
  2013-02-09  2:45 ` [PATCH 1/4] rwsem: make the waiter type an enumeration rather than a bitmask Michel Lespinasse
@ 2013-02-09  2:45 ` Michel Lespinasse
  2013-02-09  2:45 ` [PATCH 3/4] rwsem: implement write lock stealing Michel Lespinasse
                   ` (2 subsequent siblings)
  4 siblings, 0 replies; 15+ messages in thread
From: Michel Lespinasse @ 2013-02-09  2:45 UTC (permalink / raw)
  To: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	alex.shi, yuanhan.liu, dhowells, akpm, tglx

This change reduces the size of the spinlocked and TASK_UNINTERRUPTIBLE
sections in rwsem_down_failed_common():

- We only need the sem->wait_lock to insert ourselves on the wait_list;
  the waiter node can be prepared outside of the wait_lock.

- The task state only needs to be set to TASK_UNINTERRUPTIBLE immediately
  before testing waiter.task to see if someone woke us; it doesn't need to
  protect the entire function.

Signed-off-by: Michel Lespinasse <walken@google.com>

---
 lib/rwsem.c | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)

diff --git a/lib/rwsem.c b/lib/rwsem.c
index 4a6ff093a433..cd2d803cbbe4 100644
--- a/lib/rwsem.c
+++ b/lib/rwsem.c
@@ -180,14 +180,12 @@ rwsem_down_failed_common(struct rw_semaphore *sem,
 	struct task_struct *tsk = current;
 	signed long count;
 
-	set_task_state(tsk, TASK_UNINTERRUPTIBLE);
-
 	/* set up my own style of waitqueue */
-	raw_spin_lock_irq(&sem->wait_lock);
 	waiter.task = tsk;
 	waiter.type = type;
 	get_task_struct(tsk);
 
+	raw_spin_lock_irq(&sem->wait_lock);
 	if (list_empty(&sem->wait_list))
 		adjustment += RWSEM_WAITING_BIAS;
 	list_add_tail(&waiter.list, &sem->wait_list);
@@ -210,11 +208,11 @@ rwsem_down_failed_common(struct rw_semaphore *sem,
 	raw_spin_unlock_irq(&sem->wait_lock);
 
 	/* wait to be given the lock */
-	for (;;) {
+	while (true) {
+		set_task_state(tsk, TASK_UNINTERRUPTIBLE);
 		if (!waiter.task)
 			break;
 		schedule();
-		set_task_state(tsk, TASK_UNINTERRUPTIBLE);
 	}
 
 	tsk->state = TASK_RUNNING;
-- 
1.8.1

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

* [PATCH 3/4] rwsem: implement write lock stealing
  2013-02-09  2:45 [PATCH 0/4] rwsem: Implement writer lock-stealing Michel Lespinasse
  2013-02-09  2:45 ` [PATCH 1/4] rwsem: make the waiter type an enumeration rather than a bitmask Michel Lespinasse
  2013-02-09  2:45 ` [PATCH 2/4] rwsem: shorter spinlocked section in rwsem_down_failed_common() Michel Lespinasse
@ 2013-02-09  2:45 ` Michel Lespinasse
  2013-02-09  7:30   ` Hillf Danton
  2013-02-09  2:45 ` [PATCH 4/4] x86 rwsem: avoid taking slow path when stealing write lock Michel Lespinasse
  2013-02-13 13:55 ` [PATCH 0/4] rwsem: Implement writer lock-stealing Alex Shi
  4 siblings, 1 reply; 15+ messages in thread
From: Michel Lespinasse @ 2013-02-09  2:45 UTC (permalink / raw)
  To: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	alex.shi, yuanhan.liu, dhowells, akpm, tglx

Add support for write lock stealing in the generic rwsem code.

rwsem_down_write_failed() is similar to rwsem_down_read_failed(), with
a few significant differences:

- The waiter doesn't try waking blocked threads if there are no active
  lockers - it just steals the lock in that case.

- The waiter doesn't get removed from the wait_list upon wakeup, and
  doesn't get a lock grant either. Therefore, it has to attempt
  stealing the lock before sleeping (if it fails, then it is safe to
  sleep as the active locker will be responsible of waking it up) and
  it has to dequeue itself after successfully acquiring the lock. For
  simplicity, I moved the attempt at stealing the lock within the sem
  wait_lock, so that we can know at that time if we are the last waiter.

- There is no need to get a reference on the task structure, since the
  task is responsible for removing itself from the wait_list. There is
  no risk, like in the rwsem_down_read_failed() case, that a task
  would wake up and exit (thus destroying its task structure) while
  __rwsem_do_wake() is still running - wait_lock protects against that.

The __rwsem_do_wake() funciton is also largely rewritten to account
for the new requirements:

- The code to wake writers is simplified, as it doesn't need to grant them
  the lock (and deal with the possible case where another active locker
  might have appeared)

- The code to wake readers can't just check that there are no writers
  at the start of the operation and assume that this will still be
  true later, since writers might steal the lock until the first
  reader lock is granted. One option could be to grant the first
  reader lock, then count additional readers at the head of the queue,
  grant them extra readers locks if any, and finally wake all these
  readers. However, this gets a bit complex. Instead, I implemented a
  simpler alternative where we maintain a count of queued readers.
  This allows us to grant as many reader locks as necessary at the
  start of the operation, and then wake all these readers (regardless
  of their position in the queue).

Signed-off-by: Michel Lespinasse <walken@google.com>

---
 include/linux/rwsem.h |   2 +
 lib/rwsem.c           | 235 +++++++++++++++++++++++---------------------------
 2 files changed, 109 insertions(+), 128 deletions(-)

diff --git a/include/linux/rwsem.h b/include/linux/rwsem.h
index 8da67d625e13..be6f9649512d 100644
--- a/include/linux/rwsem.h
+++ b/include/linux/rwsem.h
@@ -25,6 +25,7 @@ struct rw_semaphore;
 struct rw_semaphore {
 	long			count;
 	raw_spinlock_t		wait_lock;
+	unsigned int		wait_readers;
 	struct list_head	wait_list;
 #ifdef CONFIG_DEBUG_LOCK_ALLOC
 	struct lockdep_map	dep_map;
@@ -58,6 +59,7 @@ static inline int rwsem_is_locked(struct rw_semaphore *sem)
 #define __RWSEM_INITIALIZER(name)			\
 	{ RWSEM_UNLOCKED_VALUE,				\
 	  __RAW_SPIN_LOCK_UNLOCKED(name.wait_lock),	\
+	  0,						\
 	  LIST_HEAD_INIT((name).wait_list)		\
 	  __RWSEM_DEP_MAP_INIT(name) }
 
diff --git a/lib/rwsem.c b/lib/rwsem.c
index cd2d803cbbe4..5f3cabb9eaa7 100644
--- a/lib/rwsem.c
+++ b/lib/rwsem.c
@@ -39,13 +39,9 @@ struct rwsem_waiter {
 	enum rwsem_waiter_type type;
 };
 
-/* Wake types for __rwsem_do_wake().  Note that RWSEM_WAKE_NO_ACTIVE and
- * RWSEM_WAKE_READ_OWNED imply that the spinlock must have been kept held
- * since the rwsem value was observed.
- */
+/* Wake types for __rwsem_do_wake(). */
 #define RWSEM_WAKE_ANY        0 /* Wake whatever's at head of wait list */
-#define RWSEM_WAKE_NO_ACTIVE  1 /* rwsem was observed with no active thread */
-#define RWSEM_WAKE_READ_OWNED 2 /* rwsem was observed to be read owned */
+#define RWSEM_WAKE_READERS    1 /* Wake readers only */
 
 /*
  * handle the lock release when processes blocked on it that can now run
@@ -63,129 +59,79 @@ __rwsem_do_wake(struct rw_semaphore *sem, int wake_type)
 	struct rwsem_waiter *waiter;
 	struct task_struct *tsk;
 	struct list_head *next;
-	signed long oldcount, woken, loop, adjustment;
+	signed long oldcount, readers, adjustment;
 
 	waiter = list_entry(sem->wait_list.next, struct rwsem_waiter, list);
-	if (waiter->type != RWSEM_WAITING_FOR_WRITE)
-		goto readers_only;
-
-	if (wake_type == RWSEM_WAKE_READ_OWNED)
-		/* Another active reader was observed, so wakeup is not
-		 * likely to succeed. Save the atomic op.
-		 */
+	if (waiter->type == RWSEM_WAITING_FOR_WRITE) {
+		if (wake_type == RWSEM_WAKE_ANY)
+			/* Wake writer at the front of the queue, but do not
+			 * grant it the lock yet as we want other writers
+			 * to be able to steal it.  Readers, on the other hand,
+			 * will block as they will notice the queued writer.
+			 */
+			wake_up_process(waiter->task);
 		goto out;
+	}
 
-	/* There's a writer at the front of the queue - try to grant it the
-	 * write lock.  However, we only wake this writer if we can transition
-	 * the active part of the count from 0 -> 1
+	/* Writers might steal the lock before we grant it to the next reader.
+	 * We grant the lock to readers ASAP so we can bail out early if a
+	 * writer stole the lock.
 	 */
-	adjustment = RWSEM_ACTIVE_WRITE_BIAS;
-	if (waiter->list.next == &sem->wait_list)
-		adjustment -= RWSEM_WAITING_BIAS;
-
- try_again_write:
+	readers = sem->wait_readers;
+	adjustment = readers * RWSEM_ACTIVE_READ_BIAS - RWSEM_WAITING_BIAS;
+ try_reader_grants:
 	oldcount = rwsem_atomic_update(adjustment, sem) - adjustment;
-	if (oldcount & RWSEM_ACTIVE_MASK)
-		/* Someone grabbed the sem already */
-		goto undo_write;
-
-	/* We must be careful not to touch 'waiter' after we set ->task = NULL.
-	 * It is an allocated on the waiter's stack and may become invalid at
-	 * any time after that point (due to a wakeup from another source).
-	 */
-	list_del(&waiter->list);
-	tsk = waiter->task;
-	smp_mb();
-	waiter->task = NULL;
-	wake_up_process(tsk);
-	put_task_struct(tsk);
-	goto out;
-
- readers_only:
-	/* If we come here from up_xxxx(), another thread might have reached
-	 * rwsem_down_failed_common() before we acquired the spinlock and
-	 * woken up a waiter, making it now active.  We prefer to check for
-	 * this first in order to not spend too much time with the spinlock
-	 * held if we're not going to be able to wake up readers in the end.
-	 *
-	 * Note that we do not need to update the rwsem count: any writer
-	 * trying to acquire rwsem will run rwsem_down_write_failed() due
-	 * to the waiting threads and block trying to acquire the spinlock.
-	 *
-	 * We use a dummy atomic update in order to acquire the cache line
-	 * exclusively since we expect to succeed and run the final rwsem
-	 * count adjustment pretty soon.
-	 */
-	if (wake_type == RWSEM_WAKE_ANY &&
-	    rwsem_atomic_update(0, sem) < RWSEM_WAITING_BIAS)
-		/* Someone grabbed the sem for write already */
-		goto out;
-
-	/* Grant an infinite number of read locks to the readers at the front
-	 * of the queue.  Note we increment the 'active part' of the count by
-	 * the number of readers before waking any processes up.
-	 */
-	woken = 0;
-	do {
-		woken++;
-
-		if (waiter->list.next == &sem->wait_list)
-			break;
-
-		waiter = list_entry(waiter->list.next,
-					struct rwsem_waiter, list);
-
-	} while (waiter->type != RWSEM_WAITING_FOR_WRITE);
-
-	adjustment = woken * RWSEM_ACTIVE_READ_BIAS;
-	if (waiter->type != RWSEM_WAITING_FOR_WRITE)
-		/* hit end of list above */
-		adjustment -= RWSEM_WAITING_BIAS;
-
-	rwsem_atomic_add(adjustment, sem);
+	if (unlikely(oldcount < RWSEM_WAITING_BIAS)) {
+		/* A writer stole the lock.  Undo our reader grants. */
+		if (rwsem_atomic_update(-adjustment, sem) < RWSEM_WAITING_BIAS)
+			goto out;
+		/* The writer left.  Retry waking readers. */
+		goto try_reader_grants;
+	}
 
 	next = sem->wait_list.next;
-	for (loop = woken; loop > 0; loop--) {
+	do {
 		waiter = list_entry(next, struct rwsem_waiter, list);
 		next = waiter->list.next;
-		tsk = waiter->task;
-		smp_mb();
-		waiter->task = NULL;
-		wake_up_process(tsk);
-		put_task_struct(tsk);
-	}
-
-	sem->wait_list.next = next;
-	next->prev = &sem->wait_list;
+		if (waiter->type != RWSEM_WAITING_FOR_WRITE) {
+			list_del(&waiter->list);
+
+			/* Set RWSEM_WAITING_BIAS before waking the last reader
+			 * so we know there will be a remaining active locker.
+			 */
+			if (!(--readers) && !list_empty(&sem->wait_list))
+				rwsem_atomic_add(RWSEM_WAITING_BIAS, sem);
+
+			tsk = waiter->task;
+			smp_mb();
+			waiter->task = NULL;
+			wake_up_process(tsk);
+			put_task_struct(tsk);
+		}
+	} while (readers);
+	sem->wait_readers = 0;
 
  out:
 	return sem;
-
-	/* undo the change to the active count, but check for a transition
-	 * 1->0 */
- undo_write:
-	if (rwsem_atomic_update(-adjustment, sem) & RWSEM_ACTIVE_MASK)
-		goto out;
-	goto try_again_write;
 }
 
 /*
- * wait for a lock to be granted
+ * wait for the read lock to be granted
  */
-static struct rw_semaphore __sched *
-rwsem_down_failed_common(struct rw_semaphore *sem,
-			 enum rwsem_waiter_type type, signed long adjustment)
+struct rw_semaphore __sched *rwsem_down_read_failed(struct rw_semaphore *sem)
 {
+	signed long adjustment = -RWSEM_ACTIVE_READ_BIAS;
 	struct rwsem_waiter waiter;
 	struct task_struct *tsk = current;
 	signed long count;
 
 	/* set up my own style of waitqueue */
 	waiter.task = tsk;
-	waiter.type = type;
+	waiter.type = RWSEM_WAITING_FOR_READ;
 	get_task_struct(tsk);
 
 	raw_spin_lock_irq(&sem->wait_lock);
+	sem->wait_readers++;
 	if (list_empty(&sem->wait_list))
 		adjustment += RWSEM_WAITING_BIAS;
 	list_add_tail(&waiter.list, &sem->wait_list);
@@ -193,17 +139,9 @@ rwsem_down_failed_common(struct rw_semaphore *sem,
 	/* we're now waiting on the lock, but no longer actively locking */
 	count = rwsem_atomic_update(adjustment, sem);
 
-	/* If there are no active locks, wake the front queued process(es) up.
-	 *
-	 * Alternatively, if we're called from a failed down_write(), there
-	 * were already threads queued before us and there are no active
-	 * writers, the lock must be read owned; so we try to wake any read
-	 * locks that were queued ahead of us. */
-	if (count == RWSEM_WAITING_BIAS)
-		sem = __rwsem_do_wake(sem, RWSEM_WAKE_NO_ACTIVE);
-	else if (count > RWSEM_WAITING_BIAS &&
-		 adjustment == -RWSEM_ACTIVE_WRITE_BIAS)
-		sem = __rwsem_do_wake(sem, RWSEM_WAKE_READ_OWNED);
+	/* If there are no active locks, wake the front queued process(es). */
+	if (!(count & RWSEM_ACTIVE_MASK))
+		sem = __rwsem_do_wake(sem, RWSEM_WAKE_ANY);
 
 	raw_spin_unlock_irq(&sem->wait_lock);
 
@@ -220,22 +158,63 @@ rwsem_down_failed_common(struct rw_semaphore *sem,
 	return sem;
 }
 
-/*
- * wait for the read lock to be granted
- */
-struct rw_semaphore __sched *rwsem_down_read_failed(struct rw_semaphore *sem)
-{
-	return rwsem_down_failed_common(sem, RWSEM_WAITING_FOR_READ,
-					-RWSEM_ACTIVE_READ_BIAS);
-}
-
-/*
- * wait for the write lock to be granted
- */
+/* wait until we successfully acquire the write lock */
 struct rw_semaphore __sched *rwsem_down_write_failed(struct rw_semaphore *sem)
 {
-	return rwsem_down_failed_common(sem, RWSEM_WAITING_FOR_WRITE,
-					-RWSEM_ACTIVE_WRITE_BIAS);
+	signed long adjustment = -RWSEM_ACTIVE_WRITE_BIAS;
+	struct rwsem_waiter waiter;
+	struct task_struct *tsk = current;
+	signed long count;
+
+	/* set up my own style of waitqueue */
+	waiter.task = tsk;
+	waiter.type = RWSEM_WAITING_FOR_WRITE;
+
+	raw_spin_lock_irq(&sem->wait_lock);
+	if (list_empty(&sem->wait_list))
+		adjustment += RWSEM_WAITING_BIAS;
+	list_add_tail(&waiter.list, &sem->wait_list);
+
+	/* Adjust sem->count to indicate our intention to get queued
+	 * on the lock, but no longer activeley locking.
+	 */
+	count = rwsem_atomic_update(adjustment, sem);
+
+	/* If there were already threads queued before us and there are no
+	 * active writers, the lock must be read owned; so we try to wake
+	 * any read locks that were queued ahead of us.
+	 */
+	if (count > RWSEM_WAITING_BIAS &&
+	    adjustment == -RWSEM_ACTIVE_WRITE_BIAS)
+		sem = __rwsem_do_wake(sem, RWSEM_WAKE_READERS);
+
+	set_task_state(tsk, TASK_UNINTERRUPTIBLE);
+	if (!(count & RWSEM_ACTIVE_MASK)) {
+ trylock:
+
+		/* Try acquiring the write lock. */
+		count = RWSEM_ACTIVE_WRITE_BIAS;
+		if (!list_is_singular(&sem->wait_list))
+			count += RWSEM_WAITING_BIAS;
+		if (cmpxchg(&sem->count, RWSEM_WAITING_BIAS, count) ==
+							RWSEM_WAITING_BIAS) {
+			list_del(&waiter.list);
+			raw_spin_unlock_irq(&sem->wait_lock);
+			tsk->state = TASK_RUNNING;
+			return sem;
+		}
+	}
+
+	raw_spin_unlock_irq(&sem->wait_lock);
+
+	/* Block until there are no active lockers. */
+	do {
+		schedule();
+		set_task_state(tsk, TASK_UNINTERRUPTIBLE);
+	} while (sem->count & RWSEM_ACTIVE_MASK);
+
+	raw_spin_lock_irq(&sem->wait_lock);
+	goto trylock;
 }
 
 /*
@@ -270,7 +249,7 @@ struct rw_semaphore *rwsem_downgrade_wake(struct rw_semaphore *sem)
 
 	/* do nothing if list empty */
 	if (!list_empty(&sem->wait_list))
-		sem = __rwsem_do_wake(sem, RWSEM_WAKE_READ_OWNED);
+		sem = __rwsem_do_wake(sem, RWSEM_WAKE_READERS);
 
 	raw_spin_unlock_irqrestore(&sem->wait_lock, flags);
 
-- 
1.8.1

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

* [PATCH 4/4] x86 rwsem: avoid taking slow path when stealing write lock
  2013-02-09  2:45 [PATCH 0/4] rwsem: Implement writer lock-stealing Michel Lespinasse
                   ` (2 preceding siblings ...)
  2013-02-09  2:45 ` [PATCH 3/4] rwsem: implement write lock stealing Michel Lespinasse
@ 2013-02-09  2:45 ` Michel Lespinasse
  2013-02-13 13:55 ` [PATCH 0/4] rwsem: Implement writer lock-stealing Alex Shi
  4 siblings, 0 replies; 15+ messages in thread
From: Michel Lespinasse @ 2013-02-09  2:45 UTC (permalink / raw)
  To: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	alex.shi, yuanhan.liu, dhowells, akpm, tglx

modify __down_write[_nested] and __down_write_trylock to grab the write
lock whenever the active count is 0, even if there are queued waiters
(they must be writers pending wakeup, since the active count is 0).

Note that this is an optimization only; architectures without this
optimization will still work fine:

- __down_write() would take the slow path which would take the wait_lock
  and then try stealing the lock (as in the spinlocked rwsem implementation)

- __down_write_trylock() would fail, but callers must be ready to deal
  with that - since there are some writers pending wakeup, they could
  have raced with us and obtained the lock before we steal it.

Signed-off-by: Michel Lespinasse <walken@google.com>

---
 arch/x86/include/asm/rwsem.h | 28 +++++++++++++++++++++-------
 1 file changed, 21 insertions(+), 7 deletions(-)

diff --git a/arch/x86/include/asm/rwsem.h b/arch/x86/include/asm/rwsem.h
index 2dbe4a721ce5..cad82c9c2fde 100644
--- a/arch/x86/include/asm/rwsem.h
+++ b/arch/x86/include/asm/rwsem.h
@@ -105,8 +105,8 @@ static inline void __down_write_nested(struct rw_semaphore *sem, int subclass)
 	asm volatile("# beginning down_write\n\t"
 		     LOCK_PREFIX "  xadd      %1,(%2)\n\t"
 		     /* adds 0xffff0001, returns the old value */
-		     "  test      %1,%1\n\t"
-		     /* was the count 0 before? */
+		     "  test " __ASM_SEL(%w1,%k1) "," __ASM_SEL(%w1,%k1) "\n\t"
+		     /* was the active mask 0 before? */
 		     "  jz        1f\n"
 		     "  call call_rwsem_down_write_failed\n"
 		     "1:\n"
@@ -126,11 +126,25 @@ static inline void __down_write(struct rw_semaphore *sem)
  */
 static inline int __down_write_trylock(struct rw_semaphore *sem)
 {
-	long ret = cmpxchg(&sem->count, RWSEM_UNLOCKED_VALUE,
-			   RWSEM_ACTIVE_WRITE_BIAS);
-	if (ret == RWSEM_UNLOCKED_VALUE)
-		return 1;
-	return 0;
+	long result, tmp;
+	asm volatile("# beginning __down_write_trylock\n\t"
+		     "  mov          %0,%1\n\t"
+		     "1:\n\t"
+		     "  test " __ASM_SEL(%w1,%k1) "," __ASM_SEL(%w1,%k1) "\n\t"
+		     /* was the active mask 0 before? */
+		     "  jnz          2f\n\t"
+		     "  mov          %1,%2\n\t"
+		     "  add          %3,%2\n\t"
+		     LOCK_PREFIX "  cmpxchg  %2,%0\n\t"
+		     "  jnz	     1b\n\t"
+		     "2:\n\t"
+		     "  sete         %b1\n\t"
+		     "  movzbl       %b1, %k1\n\t"
+		     "# ending __down_write_trylock\n\t"
+		     : "+m" (sem->count), "=&a" (result), "=&r" (tmp)
+		     : "er" (RWSEM_ACTIVE_WRITE_BIAS)
+		     : "memory", "cc");
+	return result;
 }
 
 /*
-- 
1.8.1

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

* Re: [PATCH 3/4] rwsem: implement write lock stealing
  2013-02-09  2:45 ` [PATCH 3/4] rwsem: implement write lock stealing Michel Lespinasse
@ 2013-02-09  7:30   ` Hillf Danton
  2013-02-09  7:42     ` Michel Lespinasse
  0 siblings, 1 reply; 15+ messages in thread
From: Hillf Danton @ 2013-02-09  7:30 UTC (permalink / raw)
  To: Michel Lespinasse
  Cc: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	alex.shi, yuanhan.liu, dhowells, akpm

On Sat, Feb 9, 2013 at 10:45 AM, Michel Lespinasse <walken@google.com> wrote:
> +               if (waiter->type != RWSEM_WAITING_FOR_WRITE) {
> +                       list_del(&waiter->list);
> +
> +                       /* Set RWSEM_WAITING_BIAS before waking the last reader
> +                        * so we know there will be a remaining active locker.
> +                        */
> +                       if (!(--readers) && !list_empty(&sem->wait_list))
> +                               rwsem_atomic_add(RWSEM_WAITING_BIAS, sem);
> +
> +                       tsk = waiter->task;
> +                       smp_mb();

For what with mb?

> +                       waiter->task = NULL;
> +                       wake_up_process(tsk);
> +                       put_task_struct(tsk);
> +               }
> +       } while (readers);
> +       sem->wait_readers = 0;


> +       /* If there are no active locks, wake the front queued process(es). */

s/locks/lockers/

> +       if (!(count & RWSEM_ACTIVE_MASK))
> +               sem = __rwsem_do_wake(sem, RWSEM_WAKE_ANY);

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

* Re: [PATCH 3/4] rwsem: implement write lock stealing
  2013-02-09  7:30   ` Hillf Danton
@ 2013-02-09  7:42     ` Michel Lespinasse
  2013-02-09 13:11       ` Hillf Danton
  0 siblings, 1 reply; 15+ messages in thread
From: Michel Lespinasse @ 2013-02-09  7:42 UTC (permalink / raw)
  To: Hillf Danton
  Cc: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	alex.shi, yuanhan.liu, dhowells, akpm

On Fri, Feb 8, 2013 at 11:30 PM, Hillf Danton <dhillf@gmail.com> wrote:
> On Sat, Feb 9, 2013 at 10:45 AM, Michel Lespinasse <walken@google.com> wrote:
>> +               if (waiter->type != RWSEM_WAITING_FOR_WRITE) {
>> +                       list_del(&waiter->list);
>> +
>> +                       /* Set RWSEM_WAITING_BIAS before waking the last reader
>> +                        * so we know there will be a remaining active locker.
>> +                        */
>> +                       if (!(--readers) && !list_empty(&sem->wait_list))
>> +                               rwsem_atomic_add(RWSEM_WAITING_BIAS, sem);
>> +
>> +                       tsk = waiter->task;
>> +                       smp_mb();
>
> For what with mb?

This is the same mb that used to be before the readers_only label
before this change.
The intention is that the write to waiter->task must be the last
access to waiter - after that, the reader is granted the lock so it
could do anything, including deallocating the waiter structure as it
gets out of rwsem_down_read_failed() and possibly deallocating the
task structure if it exits

-- 
Michel "Walken" Lespinasse
A program is never fully debugged until the last user dies.

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

* Re: [PATCH 3/4] rwsem: implement write lock stealing
  2013-02-09  7:42     ` Michel Lespinasse
@ 2013-02-09 13:11       ` Hillf Danton
  0 siblings, 0 replies; 15+ messages in thread
From: Hillf Danton @ 2013-02-09 13:11 UTC (permalink / raw)
  To: Michel Lespinasse
  Cc: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	alex.shi, yuanhan.liu, dhowells, akpm

On Sat, Feb 9, 2013 at 3:42 PM, Michel Lespinasse <walken@google.com> wrote:
> On Fri, Feb 8, 2013 at 11:30 PM, Hillf Danton <dhillf@gmail.com> wrote:
>> On Sat, Feb 9, 2013 at 10:45 AM, Michel Lespinasse <walken@google.com> wrote:
>>> +               if (waiter->type != RWSEM_WAITING_FOR_WRITE) {
>>> +                       list_del(&waiter->list);
>>> +
>>> +                       /* Set RWSEM_WAITING_BIAS before waking the last reader
>>> +                        * so we know there will be a remaining active locker.
>>> +                        */
>>> +                       if (!(--readers) && !list_empty(&sem->wait_list))
>>> +                               rwsem_atomic_add(RWSEM_WAITING_BIAS, sem);
>>> +
>>> +                       tsk = waiter->task;
>>> +                       smp_mb();
>>
>> For what with mb?
>
> This is the same mb that used to be before the readers_only label
> before this change.
> The intention is that the write to waiter->task must be the last
> access to waiter - after that, the reader is granted the lock so it
> could do anything, including deallocating the waiter structure as it
> gets out of rwsem_down_read_failed() and possibly deallocating the
> task structure if it exits
>
Perhaps the waiting process needs to see if no more wait needed,
	/* wait to be given the lock */
	for (;;) {
		if (!waiter.task)
			break;
		schedule();
		set_task_state(tsk, TASK_UNINTERRUPTIBLE);
	}
and another mb do exist at the beginning of try_to_wake_up().

And I want to change
+       /* Block until there are no active lockers. */
+       do {
+               schedule();
+               set_task_state(tsk, TASK_UNINTERRUPTIBLE);
+       } while (sem->count & RWSEM_ACTIVE_MASK);

to
	/* Block until there are no active lockers. */
	while (sem->count & RWSEM_ACTIVE_MASK) {
		schedule();
		set_task_state(tsk, TASK_UNINTERRUPTIBLE);
	}
for sure that we have to wait.

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

* Re: [PATCH 0/4] rwsem: Implement writer lock-stealing
  2013-02-09  2:45 [PATCH 0/4] rwsem: Implement writer lock-stealing Michel Lespinasse
                   ` (3 preceding siblings ...)
  2013-02-09  2:45 ` [PATCH 4/4] x86 rwsem: avoid taking slow path when stealing write lock Michel Lespinasse
@ 2013-02-13 13:55 ` Alex Shi
  2013-02-13 14:49   ` Ingo Molnar
  4 siblings, 1 reply; 15+ messages in thread
From: Alex Shi @ 2013-02-13 13:55 UTC (permalink / raw)
  To: Michel Lespinasse
  Cc: linux-kernel, anton, hpa, mingo, arjan, a.p.zijlstra, torvalds,
	yuanhan.liu, dhowells, akpm, tglx

On 02/09/2013 10:45 AM, Michel Lespinasse wrote:
> This proposal implements writer lock stealing in lib/rwsem.c, just as
> Alex Shi's earlier proposal did for the simpler lib/rwsem-spinlock.c

Ops, my patch in tip/urgent is for rwsem. Yuanhan's patch is for
rwsem-spinlock.

Thanks
    Alex

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

* Re: [PATCH 0/4] rwsem: Implement writer lock-stealing
  2013-02-13 13:55 ` [PATCH 0/4] rwsem: Implement writer lock-stealing Alex Shi
@ 2013-02-13 14:49   ` Ingo Molnar
  2013-02-14  1:31     ` Michel Lespinasse
  0 siblings, 1 reply; 15+ messages in thread
From: Ingo Molnar @ 2013-02-13 14:49 UTC (permalink / raw)
  To: Alex Shi
  Cc: Michel Lespinasse, linux-kernel, anton, hpa, arjan, a.p.zijlstra,
	torvalds, yuanhan.liu, dhowells, akpm, tglx


* Alex Shi <alex.shi@intel.com> wrote:

> On 02/09/2013 10:45 AM, Michel Lespinasse wrote:
> > This proposal implements writer lock stealing in lib/rwsem.c, just as
> > Alex Shi's earlier proposal did for the simpler lib/rwsem-spinlock.c
> 
> Ops, my patch in tip/urgent is for rwsem. Yuanhan's patch is 
> for rwsem-spinlock.

Your rwsem patch is queued up for v3.9, in the tip:core/locking 
tree:

  3a15e0e0cdda rwsem: Implement writer lock-stealing for better scalability

Michel, mind having a look at that and possibly generate a delta 
patch, if your patch has changes we should apply?

Thanks,

	Ingo

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

* Re: [PATCH 0/4] rwsem: Implement writer lock-stealing
  2013-02-13 14:49   ` Ingo Molnar
@ 2013-02-14  1:31     ` Michel Lespinasse
  2013-02-14 12:08       ` Alex Shi
  2013-02-20  8:50       ` Alex Shi
  0 siblings, 2 replies; 15+ messages in thread
From: Michel Lespinasse @ 2013-02-14  1:31 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Alex Shi, linux-kernel, anton, hpa, arjan, a.p.zijlstra,
	torvalds, yuanhan.liu, dhowells, akpm, tglx

On Wed, Feb 13, 2013 at 6:49 AM, Ingo Molnar <mingo@kernel.org> wrote:
>
> * Alex Shi <alex.shi@intel.com> wrote:
>
>> On 02/09/2013 10:45 AM, Michel Lespinasse wrote:
>> > This proposal implements writer lock stealing in lib/rwsem.c, just as
>> > Alex Shi's earlier proposal did for the simpler lib/rwsem-spinlock.c
>>
>> Ops, my patch in tip/urgent is for rwsem. Yuanhan's patch is
>> for rwsem-spinlock.

Sorry, my bad. Looks like an email snafu on my side - when I wanted to
save your patch to look through it with more context, I ended up
saving Yuanhan's instead.

> Your rwsem patch is queued up for v3.9, in the tip:core/locking
> tree:
>
>   3a15e0e0cdda rwsem: Implement writer lock-stealing for better scalability
>
> Michel, mind having a look at that and possibly generate a delta
> patch, if your patch has changes we should apply?

The main difference I can see is that my approach makes it possible to
steal the rwsem in a fast path, without going through the lib/rwsem.c
slow path. A few things fall out from that; most notably I had to
change the readers_only side of __rwsem_do_wake() to account with the
possibility of write lock stealing on the fast path, while Alex
doesn't since he forces write lock stealing to use the slow path.
Overall, my changes are more extensive but they also reduce the total
line count, while Alex's proposal goes the other way.

For these reasons, I think I still prefer my approach. However as
mentioned, this is not the best time for me to push it as I'll be away
for a little while. I should be able to get back to this in a couple
weeks, though.

Alex, could you go through my patch and see if there is anything you
find objectionable ? (if not about the details, at least about the
general approach of enabling writer lock stealing on the fast path)

Thanks,

-- 
Michel "Walken" Lespinasse
A program is never fully debugged until the last user dies.

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

* Re: [PATCH 0/4] rwsem: Implement writer lock-stealing
  2013-02-14  1:31     ` Michel Lespinasse
@ 2013-02-14 12:08       ` Alex Shi
  2013-02-20  8:50       ` Alex Shi
  1 sibling, 0 replies; 15+ messages in thread
From: Alex Shi @ 2013-02-14 12:08 UTC (permalink / raw)
  To: Michel Lespinasse
  Cc: Ingo Molnar, linux-kernel, anton, hpa, arjan, a.p.zijlstra,
	torvalds, yuanhan.liu, dhowells, akpm, tglx

On 02/14/2013 09:31 AM, Michel Lespinasse wrote:
> Alex, could you go through my patch and see if there is anything you
> find objectionable ? (if not about the details, at least about the
> general approach of enabling writer lock stealing on the fast path)

I am still in my lunar new year vacation. I may do a aim7 test for you
after back to office.
But as to fast path stealing, I didn't see extra benefit on fast path.

-- 
Thanks
    Alex

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

* Re: [PATCH 0/4] rwsem: Implement writer lock-stealing
  2013-02-14  1:31     ` Michel Lespinasse
  2013-02-14 12:08       ` Alex Shi
@ 2013-02-20  8:50       ` Alex Shi
  2013-02-20 10:40         ` Ingo Molnar
  2013-02-20 16:50         ` Michel Lespinasse
  1 sibling, 2 replies; 15+ messages in thread
From: Alex Shi @ 2013-02-20  8:50 UTC (permalink / raw)
  To: Michel Lespinasse
  Cc: Ingo Molnar, linux-kernel, anton, hpa, arjan, a.p.zijlstra,
	torvalds, yuanhan.liu, dhowells, akpm, tglx


> 
> Alex, could you go through my patch and see if there is anything you
> find objectionable ? (if not about the details, at least about the
> general approach of enabling writer lock stealing on the fast path)
> 

I did a quick review on the patchset and tested the patches 1~3, and 1~3
plus 4th, my patch plus 4th.

The patch looks much complicated, and also goes writing slow path to
steal locking. My patch looks quite straight and simple.

This 1~3 patch has very very similar performance effect with mine.

The highlight patch is the 4th, seems it can provide about ~2% aim7
performance gain(base on both of mine or patches 1~3) on my 4S NHM EX
machine.


-- 
Thanks Alex

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

* Re: [PATCH 0/4] rwsem: Implement writer lock-stealing
  2013-02-20  8:50       ` Alex Shi
@ 2013-02-20 10:40         ` Ingo Molnar
  2013-02-20 16:50         ` Michel Lespinasse
  1 sibling, 0 replies; 15+ messages in thread
From: Ingo Molnar @ 2013-02-20 10:40 UTC (permalink / raw)
  To: Alex Shi
  Cc: Michel Lespinasse, linux-kernel, anton, hpa, arjan, a.p.zijlstra,
	torvalds, yuanhan.liu, dhowells, akpm, tglx


* Alex Shi <alex.shi@intel.com> wrote:

> > Alex, could you go through my patch and see if there is 
> > anything you find objectionable ? (if not about the details, 
> > at least about the general approach of enabling writer lock 
> > stealing on the fast path)
> 
> I did a quick review on the patchset and tested the patches 
> 1~3, and 1~3 plus 4th, my patch plus 4th.
> 
> The patch looks much complicated, and also goes writing slow 
> path to steal locking. My patch looks quite straight and 
> simple.
> 
> This 1~3 patch has very very similar performance effect with 
> mine.
> 
> The highlight patch is the 4th, seems it can provide about ~2% 
> aim7 performance gain(base on both of mine or patches 1~3) on 
> my 4S NHM EX machine.

That speedup would be nice to have.

Thanks,

	Ingo

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

* Re: [PATCH 0/4] rwsem: Implement writer lock-stealing
  2013-02-20  8:50       ` Alex Shi
  2013-02-20 10:40         ` Ingo Molnar
@ 2013-02-20 16:50         ` Michel Lespinasse
  1 sibling, 0 replies; 15+ messages in thread
From: Michel Lespinasse @ 2013-02-20 16:50 UTC (permalink / raw)
  To: Alex Shi
  Cc: Ingo Molnar, linux-kernel, anton, hpa, arjan, a.p.zijlstra,
	torvalds, yuanhan.liu, dhowells, akpm, tglx

On Wed, Feb 20, 2013 at 4:50 PM, Alex Shi <alex.shi@intel.com> wrote:
> I did a quick review on the patchset and tested the patches 1~3, and 1~3
> plus 4th, my patch plus 4th.
>
> The patch looks much complicated, and also goes writing slow path to
> steal locking. My patch looks quite straight and simple.
>
> This 1~3 patch has very very similar performance effect with mine.
>
> The highlight patch is the 4th, seems it can provide about ~2% aim7
> performance gain(base on both of mine or patches 1~3) on my 4S NHM EX
> machine.

Thanks for doing these measurements.

I think we can keep patches 1-2 out of this discussion as they don't
have much complexity and don't have much to do with write stealing
either.

Patch 4 is where fast path write stealing is implemented; however
there are some parts of patch 3 that I think are required - the one I
have in mind right now is when waking readers, the existing code
assumes that writers can't get the lock while it counts how many
reader locks need to be granted. There are several solutions to that -
for example you could grant the first reader lock before counting how
many more you need, but that would result in longer code than in my
proposal. Either way, I believe if you go into fixing lib/rwsem.c to
account for the possibility of fast path lock stealing you'll end up
with something very similar to my proposed patch 3...

(This is kinda how I proceeded with this series, I wrote the fastpath
lock stealing first, then I thought about what's required to support
it in the slow path)

-- 
Michel "Walken" Lespinasse
A program is never fully debugged until the last user dies.

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

end of thread, other threads:[~2013-02-20 16:50 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2013-02-09  2:45 [PATCH 0/4] rwsem: Implement writer lock-stealing Michel Lespinasse
2013-02-09  2:45 ` [PATCH 1/4] rwsem: make the waiter type an enumeration rather than a bitmask Michel Lespinasse
2013-02-09  2:45 ` [PATCH 2/4] rwsem: shorter spinlocked section in rwsem_down_failed_common() Michel Lespinasse
2013-02-09  2:45 ` [PATCH 3/4] rwsem: implement write lock stealing Michel Lespinasse
2013-02-09  7:30   ` Hillf Danton
2013-02-09  7:42     ` Michel Lespinasse
2013-02-09 13:11       ` Hillf Danton
2013-02-09  2:45 ` [PATCH 4/4] x86 rwsem: avoid taking slow path when stealing write lock Michel Lespinasse
2013-02-13 13:55 ` [PATCH 0/4] rwsem: Implement writer lock-stealing Alex Shi
2013-02-13 14:49   ` Ingo Molnar
2013-02-14  1:31     ` Michel Lespinasse
2013-02-14 12:08       ` Alex Shi
2013-02-20  8:50       ` Alex Shi
2013-02-20 10:40         ` Ingo Molnar
2013-02-20 16:50         ` Michel Lespinasse

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).