All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-16 13:16 ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

Hi Waiman,

As promised; here is the paravirt stuff I did during the trip to BOS last week.

All the !paravirt patches are more or less the same as before (the only real
change is the copyright lines in the first patch).

The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
convoluted and I've no real way to test that but it should be stright fwd to
make work.

I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
it both booted and survived a hackbench run (perf bench sched messaging -g 20
-l 5000).

So while the paravirt code isn't the most optimal code ever conceived it does work.

Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
for the native case, which should greatly reduce the cost of having
CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.

I feel that if someone were to do a Xen patch we can go ahead and merge this
stuff (finally!).

These patches do not implement the paravirt spinlock debug stats currently
implemented (separately) by KVM and Xen, but that should not be too hard to do
on top and in the 'generic' code -- no reason to duplicate all that.

Of course; once this lands people can look at improving the paravirt nonsense.


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

* [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-16 13:16 ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

Hi Waiman,

As promised; here is the paravirt stuff I did during the trip to BOS last week.

All the !paravirt patches are more or less the same as before (the only real
change is the copyright lines in the first patch).

The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
convoluted and I've no real way to test that but it should be stright fwd to
make work.

I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
it both booted and survived a hackbench run (perf bench sched messaging -g 20
-l 5000).

So while the paravirt code isn't the most optimal code ever conceived it does work.

Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
for the native case, which should greatly reduce the cost of having
CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.

I feel that if someone were to do a Xen patch we can go ahead and merge this
stuff (finally!).

These patches do not implement the paravirt spinlock debug stats currently
implemented (separately) by KVM and Xen, but that should not be too hard to do
on top and in the 'generic' code -- no reason to duplicate all that.

Of course; once this lands people can look at improving the paravirt nonsense.

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

* [PATCH 1/9] qspinlock: A simple generic 4-byte queue spinlock
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (2 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: waiman_long-qspinlock-a_simple_generic_4-byte_queue_spinlock.patch --]
[-- Type: text/plain, Size: 16537 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This patch introduces a new generic queue spinlock implementation that
can serve as an alternative to the default ticket spinlock. Compared
with the ticket spinlock, this queue spinlock should be almost as fair
as the ticket spinlock. It has about the same speed in single-thread
and it can be much faster in high contention situations especially when
the spinlock is embedded within the data structure to be protected.

Only in light to moderate contention where the average queue depth
is around 1-3 will this queue spinlock be potentially a bit slower
due to the higher slowpath overhead.

This queue spinlock is especially suit to NUMA machines with a large
number of cores as the chance of spinlock contention is much higher
in those machines. The cost of contention is also higher because of
slower inter-node memory traffic.

Due to the fact that spinlocks are acquired with preemption disabled,
the process will not be migrated to another CPU while it is trying
to get a spinlock. Ignoring interrupt handling, a CPU can only be
contending in one spinlock at any one time. Counting soft IRQ, hard
IRQ and NMI, a CPU can only have a maximum of 4 concurrent lock waiting
activities.  By allocating a set of per-cpu queue nodes and used them
to form a waiting queue, we can encode the queue node address into a
much smaller 24-bit size (including CPU number and queue node index)
leaving one byte for the lock.

Please note that the queue node is only needed when waiting for the
lock. Once the lock is acquired, the queue node can be released to
be used later.

Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-2-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock.h       |  132 +++++++++++++++++++++
 include/asm-generic/qspinlock_types.h |   58 +++++++++
 kernel/Kconfig.locks                  |    7 +
 kernel/locking/Makefile               |    1 
 kernel/locking/mcs_spinlock.h         |    1 
 kernel/locking/qspinlock.c            |  209 ++++++++++++++++++++++++++++++++++
 6 files changed, 408 insertions(+)
 create mode 100644 include/asm-generic/qspinlock.h
 create mode 100644 include/asm-generic/qspinlock_types.h
 create mode 100644 kernel/locking/qspinlock.c

--- /dev/null
+++ b/include/asm-generic/qspinlock.h
@@ -0,0 +1,132 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ */
+#ifndef __ASM_GENERIC_QSPINLOCK_H
+#define __ASM_GENERIC_QSPINLOCK_H
+
+#include <asm-generic/qspinlock_types.h>
+
+/**
+ * queue_spin_is_locked - is the spinlock locked?
+ * @lock: Pointer to queue spinlock structure
+ * Return: 1 if it is locked, 0 otherwise
+ */
+static __always_inline int queue_spin_is_locked(struct qspinlock *lock)
+{
+	return atomic_read(&lock->val);
+}
+
+/**
+ * queue_spin_value_unlocked - is the spinlock structure unlocked?
+ * @lock: queue spinlock structure
+ * Return: 1 if it is unlocked, 0 otherwise
+ *
+ * N.B. Whenever there are tasks waiting for the lock, it is considered
+ *      locked wrt the lockref code to avoid lock stealing by the lockref
+ *      code and change things underneath the lock. This also allows some
+ *      optimizations to be applied without conflict with lockref.
+ */
+static __always_inline int queue_spin_value_unlocked(struct qspinlock lock)
+{
+	return !atomic_read(&lock.val);
+}
+
+/**
+ * queue_spin_is_contended - check if the lock is contended
+ * @lock : Pointer to queue spinlock structure
+ * Return: 1 if lock contended, 0 otherwise
+ */
+static __always_inline int queue_spin_is_contended(struct qspinlock *lock)
+{
+	return atomic_read(&lock->val) & ~_Q_LOCKED_MASK;
+}
+/**
+ * queue_spin_trylock - try to acquire the queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ * Return: 1 if lock acquired, 0 if failed
+ */
+static __always_inline int queue_spin_trylock(struct qspinlock *lock)
+{
+	if (!atomic_read(&lock->val) &&
+	   (atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL) == 0))
+		return 1;
+	return 0;
+}
+
+extern void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+
+/**
+ * queue_spin_lock - acquire a queue spinlock
+ * @lock: Pointer to queue spinlock structure
+ */
+static __always_inline void queue_spin_lock(struct qspinlock *lock)
+{
+	u32 val;
+
+	val = atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL);
+	if (likely(val == 0))
+		return;
+	queue_spin_lock_slowpath(lock, val);
+}
+
+#ifndef queue_spin_unlock
+/**
+ * queue_spin_unlock - release a queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ */
+static __always_inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	/*
+	 * smp_mb__before_atomic() in order to guarantee release semantics
+	 */
+	smp_mb__before_atomic_dec();
+	atomic_sub(_Q_LOCKED_VAL, &lock->val);
+}
+#endif
+
+/**
+ * queue_spin_unlock_wait - wait until current lock holder releases the lock
+ * @lock : Pointer to queue spinlock structure
+ *
+ * There is a very slight possibility of live-lock if the lockers keep coming
+ * and the waiter is just unfortunate enough to not see any unlock state.
+ */
+static inline void queue_spin_unlock_wait(struct qspinlock *lock)
+{
+	while (atomic_read(&lock->val) & _Q_LOCKED_MASK)
+		cpu_relax();
+}
+
+/*
+ * Initializier
+ */
+#define	__ARCH_SPIN_LOCK_UNLOCKED	{ ATOMIC_INIT(0) }
+
+/*
+ * Remapping spinlock architecture specific functions to the corresponding
+ * queue spinlock functions.
+ */
+#define arch_spin_is_locked(l)		queue_spin_is_locked(l)
+#define arch_spin_is_contended(l)	queue_spin_is_contended(l)
+#define arch_spin_value_unlocked(l)	queue_spin_value_unlocked(l)
+#define arch_spin_lock(l)		queue_spin_lock(l)
+#define arch_spin_trylock(l)		queue_spin_trylock(l)
+#define arch_spin_unlock(l)		queue_spin_unlock(l)
+#define arch_spin_lock_flags(l, f)	queue_spin_lock(l)
+#define arch_spin_unlock_wait(l)	queue_spin_unlock_wait(l)
+
+#endif /* __ASM_GENERIC_QSPINLOCK_H */
--- /dev/null
+++ b/include/asm-generic/qspinlock_types.h
@@ -0,0 +1,58 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ */
+#ifndef __ASM_GENERIC_QSPINLOCK_TYPES_H
+#define __ASM_GENERIC_QSPINLOCK_TYPES_H
+
+/*
+ * Including atomic.h with PARAVIRT on will cause compilation errors because
+ * of recursive header file incluson via paravirt_types.h. So don't include
+ * it if PARAVIRT is on.
+ */
+#ifndef CONFIG_PARAVIRT
+#include <linux/types.h>
+#include <linux/atomic.h>
+#endif
+
+typedef struct qspinlock {
+	atomic_t	val;
+} arch_spinlock_t;
+
+/*
+ * Bitfields in the atomic value:
+ *
+ *  0- 7: locked byte
+ *  8- 9: tail index
+ * 10-31: tail cpu (+1)
+ */
+#define	_Q_SET_MASK(type)	(((1U << _Q_ ## type ## _BITS) - 1)\
+				      << _Q_ ## type ## _OFFSET)
+#define _Q_LOCKED_OFFSET	0
+#define _Q_LOCKED_BITS		8
+#define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
+
+#define _Q_TAIL_IDX_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_TAIL_IDX_BITS	2
+#define _Q_TAIL_IDX_MASK	_Q_SET_MASK(TAIL_IDX)
+
+#define _Q_TAIL_CPU_OFFSET	(_Q_TAIL_IDX_OFFSET + _Q_TAIL_IDX_BITS)
+#define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
+#define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
+
+#define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
+
+#endif /* __ASM_GENERIC_QSPINLOCK_TYPES_H */
--- a/kernel/Kconfig.locks
+++ b/kernel/Kconfig.locks
@@ -235,6 +235,13 @@ config LOCK_SPIN_ON_OWNER
        def_bool y
        depends on MUTEX_SPIN_ON_OWNER || RWSEM_SPIN_ON_OWNER
 
+config ARCH_USE_QUEUE_SPINLOCK
+	bool
+
+config QUEUE_SPINLOCK
+	def_bool y if ARCH_USE_QUEUE_SPINLOCK
+	depends on SMP && !PARAVIRT_SPINLOCKS
+
 config ARCH_USE_QUEUE_RWLOCK
 	bool
 
--- a/kernel/locking/Makefile
+++ b/kernel/locking/Makefile
@@ -17,6 +17,7 @@ obj-$(CONFIG_SMP) += spinlock.o
 obj-$(CONFIG_LOCK_SPIN_ON_OWNER) += osq_lock.o
 obj-$(CONFIG_SMP) += lglock.o
 obj-$(CONFIG_PROVE_LOCKING) += spinlock.o
+obj-$(CONFIG_QUEUE_SPINLOCK) += qspinlock.o
 obj-$(CONFIG_RT_MUTEXES) += rtmutex.o
 obj-$(CONFIG_DEBUG_RT_MUTEXES) += rtmutex-debug.o
 obj-$(CONFIG_RT_MUTEX_TESTER) += rtmutex-tester.o
--- a/kernel/locking/mcs_spinlock.h
+++ b/kernel/locking/mcs_spinlock.h
@@ -17,6 +17,7 @@
 struct mcs_spinlock {
 	struct mcs_spinlock *next;
 	int locked; /* 1 if lock acquired */
+	int count;  /* nesting count, see qspinlock.c */
 };
 
 #ifndef arch_mcs_spin_lock_contended
--- /dev/null
+++ b/kernel/locking/qspinlock.c
@@ -0,0 +1,209 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ * (C) Copyright 2013-2014 Red Hat, Inc.
+ * (C) Copyright 2015 Intel Corp.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ *          Peter Zijlstra <peterz@infradead.org>
+ */
+#include <linux/smp.h>
+#include <linux/bug.h>
+#include <linux/cpumask.h>
+#include <linux/percpu.h>
+#include <linux/hardirq.h>
+#include <linux/mutex.h>
+#include <asm/qspinlock.h>
+
+/*
+ * The basic principle of a queue-based spinlock can best be understood
+ * by studying a classic queue-based spinlock implementation called the
+ * MCS lock. The paper below provides a good description for this kind
+ * of lock.
+ *
+ * http://www.cise.ufl.edu/tr/DOC/REP-1992-71.pdf
+ *
+ * This queue spinlock implementation is based on the MCS lock, however to make
+ * it fit the 4 bytes we assume spinlock_t to be, and preserve its existing
+ * API, we must modify it somehow.
+ *
+ * In particular; where the traditional MCS lock consists of a tail pointer
+ * (8 bytes) and needs the next pointer (another 8 bytes) of its own node to
+ * unlock the next pending (next->locked), we compress both these: {tail,
+ * next->locked} into a single u32 value.
+ *
+ * Since a spinlock disables recursion of its own context and there is a limit
+ * to the contexts that can nest; namely: task, softirq, hardirq, nmi. As there
+ * are at most 4 nesting levels, it can be encoded by a 2-bit number. Now
+ * we can encode the tail by combining the 2-bit nesting level with the cpu
+ * number. With one byte for the lock value and 3 bytes for the tail, only a
+ * 32-bit word is now needed. Even though we only need 1 bit for the lock,
+ * we extend it to a full byte to achieve better performance for architectures
+ * that support atomic byte write.
+ *
+ * We also change the first spinner to spin on the lock bit instead of its
+ * node; whereby avoiding the need to carry a node from lock to unlock, and
+ * preserving existing lock API. This also makes the unlock code simpler and
+ * faster.
+ */
+
+#include "mcs_spinlock.h"
+
+/*
+ * Per-CPU queue node structures; we can never have more than 4 nested
+ * contexts: task, softirq, hardirq, nmi.
+ *
+ * Exactly fits one 64-byte cacheline on a 64-bit architecture.
+ */
+static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[4]);
+
+/*
+ * We must be able to distinguish between no-tail and the tail at 0:0,
+ * therefore increment the cpu number by one.
+ */
+
+static inline u32 encode_tail(int cpu, int idx)
+{
+	u32 tail;
+
+#ifdef CONFIG_DEBUG_SPINLOCK
+	BUG_ON(idx > 3);
+#endif
+	tail  = (cpu + 1) << _Q_TAIL_CPU_OFFSET;
+	tail |= idx << _Q_TAIL_IDX_OFFSET; /* assume < 4 */
+
+	return tail;
+}
+
+static inline struct mcs_spinlock *decode_tail(u32 tail)
+{
+	int cpu = (tail >> _Q_TAIL_CPU_OFFSET) - 1;
+	int idx = (tail &  _Q_TAIL_IDX_MASK) >> _Q_TAIL_IDX_OFFSET;
+
+	return per_cpu_ptr(&mcs_nodes[idx], cpu);
+}
+
+/**
+ * queue_spin_lock_slowpath - acquire the queue spinlock
+ * @lock: Pointer to queue spinlock structure
+ * @val: Current value of the queue spinlock 32-bit word
+ *
+ * (queue tail, lock value)
+ *
+ *              fast      :    slow                                  :    unlock
+ *                        :                                          :
+ * uncontended  (0,0)   --:--> (0,1) --------------------------------:--> (*,0)
+ *                        :       | ^--------.                    /  :
+ *                        :       v           \                   |  :
+ * uncontended            :    (n,x) --+--> (n,0)                 |  :
+ *   queue                :       | ^--'                          |  :
+ *                        :       v                               |  :
+ * contended              :    (*,x) --+--> (*,0) -----> (*,1) ---'  :
+ *   queue                :         ^--'                             :
+ *
+ */
+void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	struct mcs_spinlock *prev, *next, *node;
+	u32 new, old, tail;
+	int idx;
+
+	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
+
+	node = this_cpu_ptr(&mcs_nodes[0]);
+	idx = node->count++;
+	tail = encode_tail(smp_processor_id(), idx);
+
+	node += idx;
+	node->locked = 0;
+	node->next = NULL;
+
+	/*
+	 * trylock || xchg(lock, node)
+	 *
+	 * 0,0 -> 0,1 ; no tail, not locked -> no tail, locked.
+	 * p,x -> n,x ; tail was p -> tail is n; preserving locked.
+	 */
+	for (;;) {
+		new = _Q_LOCKED_VAL;
+		if (val)
+			new = tail | (val & _Q_LOCKED_MASK);
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * we won the trylock; forget about queueing.
+	 */
+	if (new == _Q_LOCKED_VAL)
+		goto release;
+
+	/*
+	 * if there was a previous node; link it and wait until reaching the
+	 * head of the waitqueue.
+	 */
+	if (old & ~_Q_LOCKED_MASK) {
+		prev = decode_tail(old);
+		WRITE_ONCE(prev->next, node);
+
+		arch_mcs_spin_lock_contended(&node->locked);
+	}
+
+	/*
+	 * we're at the head of the waitqueue, wait for the owner to go away.
+	 *
+	 * *,x -> *,0
+	 */
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+		cpu_relax();
+
+	/*
+	 * claim the lock:
+	 *
+	 * n,0 -> 0,1 : lock, uncontended
+	 * *,0 -> *,1 : lock, contended
+	 */
+	for (;;) {
+		new = _Q_LOCKED_VAL;
+		if (val != tail)
+			new |= val;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * contended path; wait for next, release.
+	 */
+	if (new != _Q_LOCKED_VAL) {
+		while (!(next = READ_ONCE(node->next)))
+			cpu_relax();
+
+		arch_mcs_spin_unlock_contended(&next->locked);
+	}
+
+release:
+	/*
+	 * release the node
+	 */
+	this_cpu_dec(mcs_nodes[0].count);
+}
+EXPORT_SYMBOL(queue_spin_lock_slowpath);



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

* [PATCH 1/9] qspinlock: A simple generic 4-byte queue spinlock
  2015-03-16 13:16 ` Peter Zijlstra
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: waiman_long-qspinlock-a_simple_generic_4-byte_queue_spinlock.patch --]
[-- Type: text/plain, Size: 16535 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This patch introduces a new generic queue spinlock implementation that
can serve as an alternative to the default ticket spinlock. Compared
with the ticket spinlock, this queue spinlock should be almost as fair
as the ticket spinlock. It has about the same speed in single-thread
and it can be much faster in high contention situations especially when
the spinlock is embedded within the data structure to be protected.

Only in light to moderate contention where the average queue depth
is around 1-3 will this queue spinlock be potentially a bit slower
due to the higher slowpath overhead.

This queue spinlock is especially suit to NUMA machines with a large
number of cores as the chance of spinlock contention is much higher
in those machines. The cost of contention is also higher because of
slower inter-node memory traffic.

Due to the fact that spinlocks are acquired with preemption disabled,
the process will not be migrated to another CPU while it is trying
to get a spinlock. Ignoring interrupt handling, a CPU can only be
contending in one spinlock at any one time. Counting soft IRQ, hard
IRQ and NMI, a CPU can only have a maximum of 4 concurrent lock waiting
activities.  By allocating a set of per-cpu queue nodes and used them
to form a waiting queue, we can encode the queue node address into a
much smaller 24-bit size (including CPU number and queue node index)
leaving one byte for the lock.

Please note that the queue node is only needed when waiting for the
lock. Once the lock is acquired, the queue node can be released to
be used later.

Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-2-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock.h       |  132 +++++++++++++++++++++
 include/asm-generic/qspinlock_types.h |   58 +++++++++
 kernel/Kconfig.locks                  |    7 +
 kernel/locking/Makefile               |    1 
 kernel/locking/mcs_spinlock.h         |    1 
 kernel/locking/qspinlock.c            |  209 ++++++++++++++++++++++++++++++++++
 6 files changed, 408 insertions(+)
 create mode 100644 include/asm-generic/qspinlock.h
 create mode 100644 include/asm-generic/qspinlock_types.h
 create mode 100644 kernel/locking/qspinlock.c

--- /dev/null
+++ b/include/asm-generic/qspinlock.h
@@ -0,0 +1,132 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ */
+#ifndef __ASM_GENERIC_QSPINLOCK_H
+#define __ASM_GENERIC_QSPINLOCK_H
+
+#include <asm-generic/qspinlock_types.h>
+
+/**
+ * queue_spin_is_locked - is the spinlock locked?
+ * @lock: Pointer to queue spinlock structure
+ * Return: 1 if it is locked, 0 otherwise
+ */
+static __always_inline int queue_spin_is_locked(struct qspinlock *lock)
+{
+	return atomic_read(&lock->val);
+}
+
+/**
+ * queue_spin_value_unlocked - is the spinlock structure unlocked?
+ * @lock: queue spinlock structure
+ * Return: 1 if it is unlocked, 0 otherwise
+ *
+ * N.B. Whenever there are tasks waiting for the lock, it is considered
+ *      locked wrt the lockref code to avoid lock stealing by the lockref
+ *      code and change things underneath the lock. This also allows some
+ *      optimizations to be applied without conflict with lockref.
+ */
+static __always_inline int queue_spin_value_unlocked(struct qspinlock lock)
+{
+	return !atomic_read(&lock.val);
+}
+
+/**
+ * queue_spin_is_contended - check if the lock is contended
+ * @lock : Pointer to queue spinlock structure
+ * Return: 1 if lock contended, 0 otherwise
+ */
+static __always_inline int queue_spin_is_contended(struct qspinlock *lock)
+{
+	return atomic_read(&lock->val) & ~_Q_LOCKED_MASK;
+}
+/**
+ * queue_spin_trylock - try to acquire the queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ * Return: 1 if lock acquired, 0 if failed
+ */
+static __always_inline int queue_spin_trylock(struct qspinlock *lock)
+{
+	if (!atomic_read(&lock->val) &&
+	   (atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL) == 0))
+		return 1;
+	return 0;
+}
+
+extern void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+
+/**
+ * queue_spin_lock - acquire a queue spinlock
+ * @lock: Pointer to queue spinlock structure
+ */
+static __always_inline void queue_spin_lock(struct qspinlock *lock)
+{
+	u32 val;
+
+	val = atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL);
+	if (likely(val == 0))
+		return;
+	queue_spin_lock_slowpath(lock, val);
+}
+
+#ifndef queue_spin_unlock
+/**
+ * queue_spin_unlock - release a queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ */
+static __always_inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	/*
+	 * smp_mb__before_atomic() in order to guarantee release semantics
+	 */
+	smp_mb__before_atomic_dec();
+	atomic_sub(_Q_LOCKED_VAL, &lock->val);
+}
+#endif
+
+/**
+ * queue_spin_unlock_wait - wait until current lock holder releases the lock
+ * @lock : Pointer to queue spinlock structure
+ *
+ * There is a very slight possibility of live-lock if the lockers keep coming
+ * and the waiter is just unfortunate enough to not see any unlock state.
+ */
+static inline void queue_spin_unlock_wait(struct qspinlock *lock)
+{
+	while (atomic_read(&lock->val) & _Q_LOCKED_MASK)
+		cpu_relax();
+}
+
+/*
+ * Initializier
+ */
+#define	__ARCH_SPIN_LOCK_UNLOCKED	{ ATOMIC_INIT(0) }
+
+/*
+ * Remapping spinlock architecture specific functions to the corresponding
+ * queue spinlock functions.
+ */
+#define arch_spin_is_locked(l)		queue_spin_is_locked(l)
+#define arch_spin_is_contended(l)	queue_spin_is_contended(l)
+#define arch_spin_value_unlocked(l)	queue_spin_value_unlocked(l)
+#define arch_spin_lock(l)		queue_spin_lock(l)
+#define arch_spin_trylock(l)		queue_spin_trylock(l)
+#define arch_spin_unlock(l)		queue_spin_unlock(l)
+#define arch_spin_lock_flags(l, f)	queue_spin_lock(l)
+#define arch_spin_unlock_wait(l)	queue_spin_unlock_wait(l)
+
+#endif /* __ASM_GENERIC_QSPINLOCK_H */
--- /dev/null
+++ b/include/asm-generic/qspinlock_types.h
@@ -0,0 +1,58 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ */
+#ifndef __ASM_GENERIC_QSPINLOCK_TYPES_H
+#define __ASM_GENERIC_QSPINLOCK_TYPES_H
+
+/*
+ * Including atomic.h with PARAVIRT on will cause compilation errors because
+ * of recursive header file incluson via paravirt_types.h. So don't include
+ * it if PARAVIRT is on.
+ */
+#ifndef CONFIG_PARAVIRT
+#include <linux/types.h>
+#include <linux/atomic.h>
+#endif
+
+typedef struct qspinlock {
+	atomic_t	val;
+} arch_spinlock_t;
+
+/*
+ * Bitfields in the atomic value:
+ *
+ *  0- 7: locked byte
+ *  8- 9: tail index
+ * 10-31: tail cpu (+1)
+ */
+#define	_Q_SET_MASK(type)	(((1U << _Q_ ## type ## _BITS) - 1)\
+				      << _Q_ ## type ## _OFFSET)
+#define _Q_LOCKED_OFFSET	0
+#define _Q_LOCKED_BITS		8
+#define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
+
+#define _Q_TAIL_IDX_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_TAIL_IDX_BITS	2
+#define _Q_TAIL_IDX_MASK	_Q_SET_MASK(TAIL_IDX)
+
+#define _Q_TAIL_CPU_OFFSET	(_Q_TAIL_IDX_OFFSET + _Q_TAIL_IDX_BITS)
+#define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
+#define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
+
+#define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
+
+#endif /* __ASM_GENERIC_QSPINLOCK_TYPES_H */
--- a/kernel/Kconfig.locks
+++ b/kernel/Kconfig.locks
@@ -235,6 +235,13 @@ config LOCK_SPIN_ON_OWNER
        def_bool y
        depends on MUTEX_SPIN_ON_OWNER || RWSEM_SPIN_ON_OWNER
 
+config ARCH_USE_QUEUE_SPINLOCK
+	bool
+
+config QUEUE_SPINLOCK
+	def_bool y if ARCH_USE_QUEUE_SPINLOCK
+	depends on SMP && !PARAVIRT_SPINLOCKS
+
 config ARCH_USE_QUEUE_RWLOCK
 	bool
 
--- a/kernel/locking/Makefile
+++ b/kernel/locking/Makefile
@@ -17,6 +17,7 @@ obj-$(CONFIG_SMP) += spinlock.o
 obj-$(CONFIG_LOCK_SPIN_ON_OWNER) += osq_lock.o
 obj-$(CONFIG_SMP) += lglock.o
 obj-$(CONFIG_PROVE_LOCKING) += spinlock.o
+obj-$(CONFIG_QUEUE_SPINLOCK) += qspinlock.o
 obj-$(CONFIG_RT_MUTEXES) += rtmutex.o
 obj-$(CONFIG_DEBUG_RT_MUTEXES) += rtmutex-debug.o
 obj-$(CONFIG_RT_MUTEX_TESTER) += rtmutex-tester.o
--- a/kernel/locking/mcs_spinlock.h
+++ b/kernel/locking/mcs_spinlock.h
@@ -17,6 +17,7 @@
 struct mcs_spinlock {
 	struct mcs_spinlock *next;
 	int locked; /* 1 if lock acquired */
+	int count;  /* nesting count, see qspinlock.c */
 };
 
 #ifndef arch_mcs_spin_lock_contended
--- /dev/null
+++ b/kernel/locking/qspinlock.c
@@ -0,0 +1,209 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ * (C) Copyright 2013-2014 Red Hat, Inc.
+ * (C) Copyright 2015 Intel Corp.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ *          Peter Zijlstra <peterz@infradead.org>
+ */
+#include <linux/smp.h>
+#include <linux/bug.h>
+#include <linux/cpumask.h>
+#include <linux/percpu.h>
+#include <linux/hardirq.h>
+#include <linux/mutex.h>
+#include <asm/qspinlock.h>
+
+/*
+ * The basic principle of a queue-based spinlock can best be understood
+ * by studying a classic queue-based spinlock implementation called the
+ * MCS lock. The paper below provides a good description for this kind
+ * of lock.
+ *
+ * http://www.cise.ufl.edu/tr/DOC/REP-1992-71.pdf
+ *
+ * This queue spinlock implementation is based on the MCS lock, however to make
+ * it fit the 4 bytes we assume spinlock_t to be, and preserve its existing
+ * API, we must modify it somehow.
+ *
+ * In particular; where the traditional MCS lock consists of a tail pointer
+ * (8 bytes) and needs the next pointer (another 8 bytes) of its own node to
+ * unlock the next pending (next->locked), we compress both these: {tail,
+ * next->locked} into a single u32 value.
+ *
+ * Since a spinlock disables recursion of its own context and there is a limit
+ * to the contexts that can nest; namely: task, softirq, hardirq, nmi. As there
+ * are at most 4 nesting levels, it can be encoded by a 2-bit number. Now
+ * we can encode the tail by combining the 2-bit nesting level with the cpu
+ * number. With one byte for the lock value and 3 bytes for the tail, only a
+ * 32-bit word is now needed. Even though we only need 1 bit for the lock,
+ * we extend it to a full byte to achieve better performance for architectures
+ * that support atomic byte write.
+ *
+ * We also change the first spinner to spin on the lock bit instead of its
+ * node; whereby avoiding the need to carry a node from lock to unlock, and
+ * preserving existing lock API. This also makes the unlock code simpler and
+ * faster.
+ */
+
+#include "mcs_spinlock.h"
+
+/*
+ * Per-CPU queue node structures; we can never have more than 4 nested
+ * contexts: task, softirq, hardirq, nmi.
+ *
+ * Exactly fits one 64-byte cacheline on a 64-bit architecture.
+ */
+static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[4]);
+
+/*
+ * We must be able to distinguish between no-tail and the tail at 0:0,
+ * therefore increment the cpu number by one.
+ */
+
+static inline u32 encode_tail(int cpu, int idx)
+{
+	u32 tail;
+
+#ifdef CONFIG_DEBUG_SPINLOCK
+	BUG_ON(idx > 3);
+#endif
+	tail  = (cpu + 1) << _Q_TAIL_CPU_OFFSET;
+	tail |= idx << _Q_TAIL_IDX_OFFSET; /* assume < 4 */
+
+	return tail;
+}
+
+static inline struct mcs_spinlock *decode_tail(u32 tail)
+{
+	int cpu = (tail >> _Q_TAIL_CPU_OFFSET) - 1;
+	int idx = (tail &  _Q_TAIL_IDX_MASK) >> _Q_TAIL_IDX_OFFSET;
+
+	return per_cpu_ptr(&mcs_nodes[idx], cpu);
+}
+
+/**
+ * queue_spin_lock_slowpath - acquire the queue spinlock
+ * @lock: Pointer to queue spinlock structure
+ * @val: Current value of the queue spinlock 32-bit word
+ *
+ * (queue tail, lock value)
+ *
+ *              fast      :    slow                                  :    unlock
+ *                        :                                          :
+ * uncontended  (0,0)   --:--> (0,1) --------------------------------:--> (*,0)
+ *                        :       | ^--------.                    /  :
+ *                        :       v           \                   |  :
+ * uncontended            :    (n,x) --+--> (n,0)                 |  :
+ *   queue                :       | ^--'                          |  :
+ *                        :       v                               |  :
+ * contended              :    (*,x) --+--> (*,0) -----> (*,1) ---'  :
+ *   queue                :         ^--'                             :
+ *
+ */
+void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	struct mcs_spinlock *prev, *next, *node;
+	u32 new, old, tail;
+	int idx;
+
+	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
+
+	node = this_cpu_ptr(&mcs_nodes[0]);
+	idx = node->count++;
+	tail = encode_tail(smp_processor_id(), idx);
+
+	node += idx;
+	node->locked = 0;
+	node->next = NULL;
+
+	/*
+	 * trylock || xchg(lock, node)
+	 *
+	 * 0,0 -> 0,1 ; no tail, not locked -> no tail, locked.
+	 * p,x -> n,x ; tail was p -> tail is n; preserving locked.
+	 */
+	for (;;) {
+		new = _Q_LOCKED_VAL;
+		if (val)
+			new = tail | (val & _Q_LOCKED_MASK);
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * we won the trylock; forget about queueing.
+	 */
+	if (new == _Q_LOCKED_VAL)
+		goto release;
+
+	/*
+	 * if there was a previous node; link it and wait until reaching the
+	 * head of the waitqueue.
+	 */
+	if (old & ~_Q_LOCKED_MASK) {
+		prev = decode_tail(old);
+		WRITE_ONCE(prev->next, node);
+
+		arch_mcs_spin_lock_contended(&node->locked);
+	}
+
+	/*
+	 * we're at the head of the waitqueue, wait for the owner to go away.
+	 *
+	 * *,x -> *,0
+	 */
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+		cpu_relax();
+
+	/*
+	 * claim the lock:
+	 *
+	 * n,0 -> 0,1 : lock, uncontended
+	 * *,0 -> *,1 : lock, contended
+	 */
+	for (;;) {
+		new = _Q_LOCKED_VAL;
+		if (val != tail)
+			new |= val;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * contended path; wait for next, release.
+	 */
+	if (new != _Q_LOCKED_VAL) {
+		while (!(next = READ_ONCE(node->next)))
+			cpu_relax();
+
+		arch_mcs_spin_unlock_contended(&next->locked);
+	}
+
+release:
+	/*
+	 * release the node
+	 */
+	this_cpu_dec(mcs_nodes[0].count);
+}
+EXPORT_SYMBOL(queue_spin_lock_slowpath);

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

* [PATCH 1/9] qspinlock: A simple generic 4-byte queue spinlock
  2015-03-16 13:16 ` Peter Zijlstra
  (?)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: waiman_long-qspinlock-a_simple_generic_4-byte_queue_spinlock.patch --]
[-- Type: text/plain, Size: 16535 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This patch introduces a new generic queue spinlock implementation that
can serve as an alternative to the default ticket spinlock. Compared
with the ticket spinlock, this queue spinlock should be almost as fair
as the ticket spinlock. It has about the same speed in single-thread
and it can be much faster in high contention situations especially when
the spinlock is embedded within the data structure to be protected.

Only in light to moderate contention where the average queue depth
is around 1-3 will this queue spinlock be potentially a bit slower
due to the higher slowpath overhead.

This queue spinlock is especially suit to NUMA machines with a large
number of cores as the chance of spinlock contention is much higher
in those machines. The cost of contention is also higher because of
slower inter-node memory traffic.

Due to the fact that spinlocks are acquired with preemption disabled,
the process will not be migrated to another CPU while it is trying
to get a spinlock. Ignoring interrupt handling, a CPU can only be
contending in one spinlock at any one time. Counting soft IRQ, hard
IRQ and NMI, a CPU can only have a maximum of 4 concurrent lock waiting
activities.  By allocating a set of per-cpu queue nodes and used them
to form a waiting queue, we can encode the queue node address into a
much smaller 24-bit size (including CPU number and queue node index)
leaving one byte for the lock.

Please note that the queue node is only needed when waiting for the
lock. Once the lock is acquired, the queue node can be released to
be used later.

Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-2-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock.h       |  132 +++++++++++++++++++++
 include/asm-generic/qspinlock_types.h |   58 +++++++++
 kernel/Kconfig.locks                  |    7 +
 kernel/locking/Makefile               |    1 
 kernel/locking/mcs_spinlock.h         |    1 
 kernel/locking/qspinlock.c            |  209 ++++++++++++++++++++++++++++++++++
 6 files changed, 408 insertions(+)
 create mode 100644 include/asm-generic/qspinlock.h
 create mode 100644 include/asm-generic/qspinlock_types.h
 create mode 100644 kernel/locking/qspinlock.c

--- /dev/null
+++ b/include/asm-generic/qspinlock.h
@@ -0,0 +1,132 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ */
+#ifndef __ASM_GENERIC_QSPINLOCK_H
+#define __ASM_GENERIC_QSPINLOCK_H
+
+#include <asm-generic/qspinlock_types.h>
+
+/**
+ * queue_spin_is_locked - is the spinlock locked?
+ * @lock: Pointer to queue spinlock structure
+ * Return: 1 if it is locked, 0 otherwise
+ */
+static __always_inline int queue_spin_is_locked(struct qspinlock *lock)
+{
+	return atomic_read(&lock->val);
+}
+
+/**
+ * queue_spin_value_unlocked - is the spinlock structure unlocked?
+ * @lock: queue spinlock structure
+ * Return: 1 if it is unlocked, 0 otherwise
+ *
+ * N.B. Whenever there are tasks waiting for the lock, it is considered
+ *      locked wrt the lockref code to avoid lock stealing by the lockref
+ *      code and change things underneath the lock. This also allows some
+ *      optimizations to be applied without conflict with lockref.
+ */
+static __always_inline int queue_spin_value_unlocked(struct qspinlock lock)
+{
+	return !atomic_read(&lock.val);
+}
+
+/**
+ * queue_spin_is_contended - check if the lock is contended
+ * @lock : Pointer to queue spinlock structure
+ * Return: 1 if lock contended, 0 otherwise
+ */
+static __always_inline int queue_spin_is_contended(struct qspinlock *lock)
+{
+	return atomic_read(&lock->val) & ~_Q_LOCKED_MASK;
+}
+/**
+ * queue_spin_trylock - try to acquire the queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ * Return: 1 if lock acquired, 0 if failed
+ */
+static __always_inline int queue_spin_trylock(struct qspinlock *lock)
+{
+	if (!atomic_read(&lock->val) &&
+	   (atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL) == 0))
+		return 1;
+	return 0;
+}
+
+extern void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+
+/**
+ * queue_spin_lock - acquire a queue spinlock
+ * @lock: Pointer to queue spinlock structure
+ */
+static __always_inline void queue_spin_lock(struct qspinlock *lock)
+{
+	u32 val;
+
+	val = atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL);
+	if (likely(val == 0))
+		return;
+	queue_spin_lock_slowpath(lock, val);
+}
+
+#ifndef queue_spin_unlock
+/**
+ * queue_spin_unlock - release a queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ */
+static __always_inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	/*
+	 * smp_mb__before_atomic() in order to guarantee release semantics
+	 */
+	smp_mb__before_atomic_dec();
+	atomic_sub(_Q_LOCKED_VAL, &lock->val);
+}
+#endif
+
+/**
+ * queue_spin_unlock_wait - wait until current lock holder releases the lock
+ * @lock : Pointer to queue spinlock structure
+ *
+ * There is a very slight possibility of live-lock if the lockers keep coming
+ * and the waiter is just unfortunate enough to not see any unlock state.
+ */
+static inline void queue_spin_unlock_wait(struct qspinlock *lock)
+{
+	while (atomic_read(&lock->val) & _Q_LOCKED_MASK)
+		cpu_relax();
+}
+
+/*
+ * Initializier
+ */
+#define	__ARCH_SPIN_LOCK_UNLOCKED	{ ATOMIC_INIT(0) }
+
+/*
+ * Remapping spinlock architecture specific functions to the corresponding
+ * queue spinlock functions.
+ */
+#define arch_spin_is_locked(l)		queue_spin_is_locked(l)
+#define arch_spin_is_contended(l)	queue_spin_is_contended(l)
+#define arch_spin_value_unlocked(l)	queue_spin_value_unlocked(l)
+#define arch_spin_lock(l)		queue_spin_lock(l)
+#define arch_spin_trylock(l)		queue_spin_trylock(l)
+#define arch_spin_unlock(l)		queue_spin_unlock(l)
+#define arch_spin_lock_flags(l, f)	queue_spin_lock(l)
+#define arch_spin_unlock_wait(l)	queue_spin_unlock_wait(l)
+
+#endif /* __ASM_GENERIC_QSPINLOCK_H */
--- /dev/null
+++ b/include/asm-generic/qspinlock_types.h
@@ -0,0 +1,58 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ */
+#ifndef __ASM_GENERIC_QSPINLOCK_TYPES_H
+#define __ASM_GENERIC_QSPINLOCK_TYPES_H
+
+/*
+ * Including atomic.h with PARAVIRT on will cause compilation errors because
+ * of recursive header file incluson via paravirt_types.h. So don't include
+ * it if PARAVIRT is on.
+ */
+#ifndef CONFIG_PARAVIRT
+#include <linux/types.h>
+#include <linux/atomic.h>
+#endif
+
+typedef struct qspinlock {
+	atomic_t	val;
+} arch_spinlock_t;
+
+/*
+ * Bitfields in the atomic value:
+ *
+ *  0- 7: locked byte
+ *  8- 9: tail index
+ * 10-31: tail cpu (+1)
+ */
+#define	_Q_SET_MASK(type)	(((1U << _Q_ ## type ## _BITS) - 1)\
+				      << _Q_ ## type ## _OFFSET)
+#define _Q_LOCKED_OFFSET	0
+#define _Q_LOCKED_BITS		8
+#define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
+
+#define _Q_TAIL_IDX_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_TAIL_IDX_BITS	2
+#define _Q_TAIL_IDX_MASK	_Q_SET_MASK(TAIL_IDX)
+
+#define _Q_TAIL_CPU_OFFSET	(_Q_TAIL_IDX_OFFSET + _Q_TAIL_IDX_BITS)
+#define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
+#define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
+
+#define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
+
+#endif /* __ASM_GENERIC_QSPINLOCK_TYPES_H */
--- a/kernel/Kconfig.locks
+++ b/kernel/Kconfig.locks
@@ -235,6 +235,13 @@ config LOCK_SPIN_ON_OWNER
        def_bool y
        depends on MUTEX_SPIN_ON_OWNER || RWSEM_SPIN_ON_OWNER
 
+config ARCH_USE_QUEUE_SPINLOCK
+	bool
+
+config QUEUE_SPINLOCK
+	def_bool y if ARCH_USE_QUEUE_SPINLOCK
+	depends on SMP && !PARAVIRT_SPINLOCKS
+
 config ARCH_USE_QUEUE_RWLOCK
 	bool
 
--- a/kernel/locking/Makefile
+++ b/kernel/locking/Makefile
@@ -17,6 +17,7 @@ obj-$(CONFIG_SMP) += spinlock.o
 obj-$(CONFIG_LOCK_SPIN_ON_OWNER) += osq_lock.o
 obj-$(CONFIG_SMP) += lglock.o
 obj-$(CONFIG_PROVE_LOCKING) += spinlock.o
+obj-$(CONFIG_QUEUE_SPINLOCK) += qspinlock.o
 obj-$(CONFIG_RT_MUTEXES) += rtmutex.o
 obj-$(CONFIG_DEBUG_RT_MUTEXES) += rtmutex-debug.o
 obj-$(CONFIG_RT_MUTEX_TESTER) += rtmutex-tester.o
--- a/kernel/locking/mcs_spinlock.h
+++ b/kernel/locking/mcs_spinlock.h
@@ -17,6 +17,7 @@
 struct mcs_spinlock {
 	struct mcs_spinlock *next;
 	int locked; /* 1 if lock acquired */
+	int count;  /* nesting count, see qspinlock.c */
 };
 
 #ifndef arch_mcs_spin_lock_contended
--- /dev/null
+++ b/kernel/locking/qspinlock.c
@@ -0,0 +1,209 @@
+/*
+ * Queue spinlock
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * (C) Copyright 2013-2014 Hewlett-Packard Development Company, L.P.
+ * (C) Copyright 2013-2014 Red Hat, Inc.
+ * (C) Copyright 2015 Intel Corp.
+ *
+ * Authors: Waiman Long <waiman.long@hp.com>
+ *          Peter Zijlstra <peterz@infradead.org>
+ */
+#include <linux/smp.h>
+#include <linux/bug.h>
+#include <linux/cpumask.h>
+#include <linux/percpu.h>
+#include <linux/hardirq.h>
+#include <linux/mutex.h>
+#include <asm/qspinlock.h>
+
+/*
+ * The basic principle of a queue-based spinlock can best be understood
+ * by studying a classic queue-based spinlock implementation called the
+ * MCS lock. The paper below provides a good description for this kind
+ * of lock.
+ *
+ * http://www.cise.ufl.edu/tr/DOC/REP-1992-71.pdf
+ *
+ * This queue spinlock implementation is based on the MCS lock, however to make
+ * it fit the 4 bytes we assume spinlock_t to be, and preserve its existing
+ * API, we must modify it somehow.
+ *
+ * In particular; where the traditional MCS lock consists of a tail pointer
+ * (8 bytes) and needs the next pointer (another 8 bytes) of its own node to
+ * unlock the next pending (next->locked), we compress both these: {tail,
+ * next->locked} into a single u32 value.
+ *
+ * Since a spinlock disables recursion of its own context and there is a limit
+ * to the contexts that can nest; namely: task, softirq, hardirq, nmi. As there
+ * are at most 4 nesting levels, it can be encoded by a 2-bit number. Now
+ * we can encode the tail by combining the 2-bit nesting level with the cpu
+ * number. With one byte for the lock value and 3 bytes for the tail, only a
+ * 32-bit word is now needed. Even though we only need 1 bit for the lock,
+ * we extend it to a full byte to achieve better performance for architectures
+ * that support atomic byte write.
+ *
+ * We also change the first spinner to spin on the lock bit instead of its
+ * node; whereby avoiding the need to carry a node from lock to unlock, and
+ * preserving existing lock API. This also makes the unlock code simpler and
+ * faster.
+ */
+
+#include "mcs_spinlock.h"
+
+/*
+ * Per-CPU queue node structures; we can never have more than 4 nested
+ * contexts: task, softirq, hardirq, nmi.
+ *
+ * Exactly fits one 64-byte cacheline on a 64-bit architecture.
+ */
+static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[4]);
+
+/*
+ * We must be able to distinguish between no-tail and the tail at 0:0,
+ * therefore increment the cpu number by one.
+ */
+
+static inline u32 encode_tail(int cpu, int idx)
+{
+	u32 tail;
+
+#ifdef CONFIG_DEBUG_SPINLOCK
+	BUG_ON(idx > 3);
+#endif
+	tail  = (cpu + 1) << _Q_TAIL_CPU_OFFSET;
+	tail |= idx << _Q_TAIL_IDX_OFFSET; /* assume < 4 */
+
+	return tail;
+}
+
+static inline struct mcs_spinlock *decode_tail(u32 tail)
+{
+	int cpu = (tail >> _Q_TAIL_CPU_OFFSET) - 1;
+	int idx = (tail &  _Q_TAIL_IDX_MASK) >> _Q_TAIL_IDX_OFFSET;
+
+	return per_cpu_ptr(&mcs_nodes[idx], cpu);
+}
+
+/**
+ * queue_spin_lock_slowpath - acquire the queue spinlock
+ * @lock: Pointer to queue spinlock structure
+ * @val: Current value of the queue spinlock 32-bit word
+ *
+ * (queue tail, lock value)
+ *
+ *              fast      :    slow                                  :    unlock
+ *                        :                                          :
+ * uncontended  (0,0)   --:--> (0,1) --------------------------------:--> (*,0)
+ *                        :       | ^--------.                    /  :
+ *                        :       v           \                   |  :
+ * uncontended            :    (n,x) --+--> (n,0)                 |  :
+ *   queue                :       | ^--'                          |  :
+ *                        :       v                               |  :
+ * contended              :    (*,x) --+--> (*,0) -----> (*,1) ---'  :
+ *   queue                :         ^--'                             :
+ *
+ */
+void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	struct mcs_spinlock *prev, *next, *node;
+	u32 new, old, tail;
+	int idx;
+
+	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
+
+	node = this_cpu_ptr(&mcs_nodes[0]);
+	idx = node->count++;
+	tail = encode_tail(smp_processor_id(), idx);
+
+	node += idx;
+	node->locked = 0;
+	node->next = NULL;
+
+	/*
+	 * trylock || xchg(lock, node)
+	 *
+	 * 0,0 -> 0,1 ; no tail, not locked -> no tail, locked.
+	 * p,x -> n,x ; tail was p -> tail is n; preserving locked.
+	 */
+	for (;;) {
+		new = _Q_LOCKED_VAL;
+		if (val)
+			new = tail | (val & _Q_LOCKED_MASK);
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * we won the trylock; forget about queueing.
+	 */
+	if (new == _Q_LOCKED_VAL)
+		goto release;
+
+	/*
+	 * if there was a previous node; link it and wait until reaching the
+	 * head of the waitqueue.
+	 */
+	if (old & ~_Q_LOCKED_MASK) {
+		prev = decode_tail(old);
+		WRITE_ONCE(prev->next, node);
+
+		arch_mcs_spin_lock_contended(&node->locked);
+	}
+
+	/*
+	 * we're at the head of the waitqueue, wait for the owner to go away.
+	 *
+	 * *,x -> *,0
+	 */
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+		cpu_relax();
+
+	/*
+	 * claim the lock:
+	 *
+	 * n,0 -> 0,1 : lock, uncontended
+	 * *,0 -> *,1 : lock, contended
+	 */
+	for (;;) {
+		new = _Q_LOCKED_VAL;
+		if (val != tail)
+			new |= val;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * contended path; wait for next, release.
+	 */
+	if (new != _Q_LOCKED_VAL) {
+		while (!(next = READ_ONCE(node->next)))
+			cpu_relax();
+
+		arch_mcs_spin_unlock_contended(&next->locked);
+	}
+
+release:
+	/*
+	 * release the node
+	 */
+	this_cpu_dec(mcs_nodes[0].count);
+}
+EXPORT_SYMBOL(queue_spin_lock_slowpath);

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

* [PATCH 2/9] qspinlock, x86: Enable x86-64 to use queue spinlock
  2015-03-16 13:16 ` Peter Zijlstra
@ 2015-03-16 13:16   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: waiman_long-qspinlock_x86-enable_x86-64_to_use_queue_spinlock.patch --]
[-- Type: text/plain, Size: 3855 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This patch makes the necessary changes at the x86 architecture
specific layer to enable the use of queue spinlock for x86-64. As
x86-32 machines are typically not multi-socket. The benefit of queue
spinlock may not be apparent. So queue spinlock is not enabled.

Currently, there is some incompatibilities between the para-virtualized
spinlock code (which hard-codes the use of ticket spinlock) and the
queue spinlock. Therefore, the use of queue spinlock is disabled when
the para-virtualized spinlock is enabled.

The arch/x86/include/asm/qspinlock.h header file includes some x86
specific optimization which will make the queue spinlock code perform
better than the generic implementation.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-3-git-send-email-Waiman.Long@hp.com
---
 arch/x86/Kconfig                      |    1 +
 arch/x86/include/asm/qspinlock.h      |   20 ++++++++++++++++++++
 arch/x86/include/asm/spinlock.h       |    5 +++++
 arch/x86/include/asm/spinlock_types.h |    4 ++++
 4 files changed, 30 insertions(+)
 create mode 100644 arch/x86/include/asm/qspinlock.h

--- a/arch/x86/Kconfig
+++ b/arch/x86/Kconfig
@@ -125,6 +125,7 @@ config X86
 	select MODULES_USE_ELF_RELA if X86_64
 	select CLONE_BACKWARDS if X86_32
 	select ARCH_USE_BUILTIN_BSWAP
+	select ARCH_USE_QUEUE_SPINLOCK
 	select ARCH_USE_QUEUE_RWLOCK
 	select OLD_SIGSUSPEND3 if X86_32 || IA32_EMULATION
 	select OLD_SIGACTION if X86_32
--- /dev/null
+++ b/arch/x86/include/asm/qspinlock.h
@@ -0,0 +1,20 @@
+#ifndef _ASM_X86_QSPINLOCK_H
+#define _ASM_X86_QSPINLOCK_H
+
+#include <asm-generic/qspinlock_types.h>
+
+#define	queue_spin_unlock queue_spin_unlock
+/**
+ * queue_spin_unlock - release a queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ *
+ * An smp_store_release() on the least-significant byte.
+ */
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	smp_store_release((u8 *)lock, 0);
+}
+
+#include <asm-generic/qspinlock.h>
+
+#endif /* _ASM_X86_QSPINLOCK_H */
--- a/arch/x86/include/asm/spinlock.h
+++ b/arch/x86/include/asm/spinlock.h
@@ -42,6 +42,10 @@
 extern struct static_key paravirt_ticketlocks_enabled;
 static __always_inline bool static_key_false(struct static_key *key);
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+#include <asm/qspinlock.h>
+#else
+
 #ifdef CONFIG_PARAVIRT_SPINLOCKS
 
 static inline void __ticket_enter_slowpath(arch_spinlock_t *lock)
@@ -196,6 +200,7 @@ static inline void arch_spin_unlock_wait
 		cpu_relax();
 	}
 }
+#endif /* CONFIG_QUEUE_SPINLOCK */
 
 /*
  * Read-write spinlocks, allowing multiple readers
--- a/arch/x86/include/asm/spinlock_types.h
+++ b/arch/x86/include/asm/spinlock_types.h
@@ -23,6 +23,9 @@ typedef u32 __ticketpair_t;
 
 #define TICKET_SHIFT	(sizeof(__ticket_t) * 8)
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+#include <asm-generic/qspinlock_types.h>
+#else
 typedef struct arch_spinlock {
 	union {
 		__ticketpair_t head_tail;
@@ -33,6 +36,7 @@ typedef struct arch_spinlock {
 } arch_spinlock_t;
 
 #define __ARCH_SPIN_LOCK_UNLOCKED	{ { 0 } }
+#endif /* CONFIG_QUEUE_SPINLOCK */
 
 #include <asm-generic/qrwlock_types.h>
 



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

* [PATCH 2/9] qspinlock, x86: Enable x86-64 to use queue spinlock
@ 2015-03-16 13:16   ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: waiman_long-qspinlock_x86-enable_x86-64_to_use_queue_spinlock.patch --]
[-- Type: text/plain, Size: 3851 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This patch makes the necessary changes at the x86 architecture
specific layer to enable the use of queue spinlock for x86-64. As
x86-32 machines are typically not multi-socket. The benefit of queue
spinlock may not be apparent. So queue spinlock is not enabled.

Currently, there is some incompatibilities between the para-virtualized
spinlock code (which hard-codes the use of ticket spinlock) and the
queue spinlock. Therefore, the use of queue spinlock is disabled when
the para-virtualized spinlock is enabled.

The arch/x86/include/asm/qspinlock.h header file includes some x86
specific optimization which will make the queue spinlock code perform
better than the generic implementation.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-3-git-send-email-Waiman.Long@hp.com
---
 arch/x86/Kconfig                      |    1 +
 arch/x86/include/asm/qspinlock.h      |   20 ++++++++++++++++++++
 arch/x86/include/asm/spinlock.h       |    5 +++++
 arch/x86/include/asm/spinlock_types.h |    4 ++++
 4 files changed, 30 insertions(+)
 create mode 100644 arch/x86/include/asm/qspinlock.h

--- a/arch/x86/Kconfig
+++ b/arch/x86/Kconfig
@@ -125,6 +125,7 @@ config X86
 	select MODULES_USE_ELF_RELA if X86_64
 	select CLONE_BACKWARDS if X86_32
 	select ARCH_USE_BUILTIN_BSWAP
+	select ARCH_USE_QUEUE_SPINLOCK
 	select ARCH_USE_QUEUE_RWLOCK
 	select OLD_SIGSUSPEND3 if X86_32 || IA32_EMULATION
 	select OLD_SIGACTION if X86_32
--- /dev/null
+++ b/arch/x86/include/asm/qspinlock.h
@@ -0,0 +1,20 @@
+#ifndef _ASM_X86_QSPINLOCK_H
+#define _ASM_X86_QSPINLOCK_H
+
+#include <asm-generic/qspinlock_types.h>
+
+#define	queue_spin_unlock queue_spin_unlock
+/**
+ * queue_spin_unlock - release a queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ *
+ * An smp_store_release() on the least-significant byte.
+ */
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	smp_store_release((u8 *)lock, 0);
+}
+
+#include <asm-generic/qspinlock.h>
+
+#endif /* _ASM_X86_QSPINLOCK_H */
--- a/arch/x86/include/asm/spinlock.h
+++ b/arch/x86/include/asm/spinlock.h
@@ -42,6 +42,10 @@
 extern struct static_key paravirt_ticketlocks_enabled;
 static __always_inline bool static_key_false(struct static_key *key);
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+#include <asm/qspinlock.h>
+#else
+
 #ifdef CONFIG_PARAVIRT_SPINLOCKS
 
 static inline void __ticket_enter_slowpath(arch_spinlock_t *lock)
@@ -196,6 +200,7 @@ static inline void arch_spin_unlock_wait
 		cpu_relax();
 	}
 }
+#endif /* CONFIG_QUEUE_SPINLOCK */
 
 /*
  * Read-write spinlocks, allowing multiple readers
--- a/arch/x86/include/asm/spinlock_types.h
+++ b/arch/x86/include/asm/spinlock_types.h
@@ -23,6 +23,9 @@ typedef u32 __ticketpair_t;
 
 #define TICKET_SHIFT	(sizeof(__ticket_t) * 8)
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+#include <asm-generic/qspinlock_types.h>
+#else
 typedef struct arch_spinlock {
 	union {
 		__ticketpair_t head_tail;
@@ -33,6 +36,7 @@ typedef struct arch_spinlock {
 } arch_spinlock_t;
 
 #define __ARCH_SPIN_LOCK_UNLOCKED	{ { 0 } }
+#endif /* CONFIG_QUEUE_SPINLOCK */
 
 #include <asm-generic/qrwlock_types.h>

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

* [PATCH 2/9] qspinlock, x86: Enable x86-64 to use queue spinlock
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (4 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: waiman_long-qspinlock_x86-enable_x86-64_to_use_queue_spinlock.patch --]
[-- Type: text/plain, Size: 3851 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This patch makes the necessary changes at the x86 architecture
specific layer to enable the use of queue spinlock for x86-64. As
x86-32 machines are typically not multi-socket. The benefit of queue
spinlock may not be apparent. So queue spinlock is not enabled.

Currently, there is some incompatibilities between the para-virtualized
spinlock code (which hard-codes the use of ticket spinlock) and the
queue spinlock. Therefore, the use of queue spinlock is disabled when
the para-virtualized spinlock is enabled.

The arch/x86/include/asm/qspinlock.h header file includes some x86
specific optimization which will make the queue spinlock code perform
better than the generic implementation.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-3-git-send-email-Waiman.Long@hp.com
---
 arch/x86/Kconfig                      |    1 +
 arch/x86/include/asm/qspinlock.h      |   20 ++++++++++++++++++++
 arch/x86/include/asm/spinlock.h       |    5 +++++
 arch/x86/include/asm/spinlock_types.h |    4 ++++
 4 files changed, 30 insertions(+)
 create mode 100644 arch/x86/include/asm/qspinlock.h

--- a/arch/x86/Kconfig
+++ b/arch/x86/Kconfig
@@ -125,6 +125,7 @@ config X86
 	select MODULES_USE_ELF_RELA if X86_64
 	select CLONE_BACKWARDS if X86_32
 	select ARCH_USE_BUILTIN_BSWAP
+	select ARCH_USE_QUEUE_SPINLOCK
 	select ARCH_USE_QUEUE_RWLOCK
 	select OLD_SIGSUSPEND3 if X86_32 || IA32_EMULATION
 	select OLD_SIGACTION if X86_32
--- /dev/null
+++ b/arch/x86/include/asm/qspinlock.h
@@ -0,0 +1,20 @@
+#ifndef _ASM_X86_QSPINLOCK_H
+#define _ASM_X86_QSPINLOCK_H
+
+#include <asm-generic/qspinlock_types.h>
+
+#define	queue_spin_unlock queue_spin_unlock
+/**
+ * queue_spin_unlock - release a queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ *
+ * An smp_store_release() on the least-significant byte.
+ */
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	smp_store_release((u8 *)lock, 0);
+}
+
+#include <asm-generic/qspinlock.h>
+
+#endif /* _ASM_X86_QSPINLOCK_H */
--- a/arch/x86/include/asm/spinlock.h
+++ b/arch/x86/include/asm/spinlock.h
@@ -42,6 +42,10 @@
 extern struct static_key paravirt_ticketlocks_enabled;
 static __always_inline bool static_key_false(struct static_key *key);
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+#include <asm/qspinlock.h>
+#else
+
 #ifdef CONFIG_PARAVIRT_SPINLOCKS
 
 static inline void __ticket_enter_slowpath(arch_spinlock_t *lock)
@@ -196,6 +200,7 @@ static inline void arch_spin_unlock_wait
 		cpu_relax();
 	}
 }
+#endif /* CONFIG_QUEUE_SPINLOCK */
 
 /*
  * Read-write spinlocks, allowing multiple readers
--- a/arch/x86/include/asm/spinlock_types.h
+++ b/arch/x86/include/asm/spinlock_types.h
@@ -23,6 +23,9 @@ typedef u32 __ticketpair_t;
 
 #define TICKET_SHIFT	(sizeof(__ticket_t) * 8)
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+#include <asm-generic/qspinlock_types.h>
+#else
 typedef struct arch_spinlock {
 	union {
 		__ticketpair_t head_tail;
@@ -33,6 +36,7 @@ typedef struct arch_spinlock {
 } arch_spinlock_t;
 
 #define __ARCH_SPIN_LOCK_UNLOCKED	{ { 0 } }
+#endif /* CONFIG_QUEUE_SPINLOCK */
 
 #include <asm-generic/qrwlock_types.h>

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

* [PATCH 3/9] qspinlock: Add pending bit
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (5 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: peter_zijlstra-qspinlock-add_pending_bit.patch --]
[-- Type: text/plain, Size: 8000 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

Because the qspinlock needs to touch a second cacheline (the per-cpu
mcs_nodes[]); add a pending bit and allow a single in-word spinner
before we punt to the second cacheline.

It is possible so observe the pending bit without the locked bit when
the last owner has just released but the pending owner has not yet
taken ownership.

In this case we would normally queue -- because the pending bit is
already taken. However, in this case the pending bit is guaranteed
to be released 'soon', therefore wait for it and avoid queueing.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-4-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |   12 ++-
 kernel/locking/qspinlock.c            |  119 ++++++++++++++++++++++++++++------
 2 files changed, 107 insertions(+), 24 deletions(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -36,8 +36,9 @@ typedef struct qspinlock {
  * Bitfields in the atomic value:
  *
  *  0- 7: locked byte
- *  8- 9: tail index
- * 10-31: tail cpu (+1)
+ *     8: pending
+ *  9-10: tail index
+ * 11-31: tail cpu (+1)
  */
 #define	_Q_SET_MASK(type)	(((1U << _Q_ ## type ## _BITS) - 1)\
 				      << _Q_ ## type ## _OFFSET)
@@ -45,7 +46,11 @@ typedef struct qspinlock {
 #define _Q_LOCKED_BITS		8
 #define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
 
-#define _Q_TAIL_IDX_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_PENDING_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_PENDING_BITS		1
+#define _Q_PENDING_MASK		_Q_SET_MASK(PENDING)
+
+#define _Q_TAIL_IDX_OFFSET	(_Q_PENDING_OFFSET + _Q_PENDING_BITS)
 #define _Q_TAIL_IDX_BITS	2
 #define _Q_TAIL_IDX_MASK	_Q_SET_MASK(TAIL_IDX)
 
@@ -54,5 +59,6 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
+#define _Q_PENDING_VAL		(1U << _Q_PENDING_OFFSET)
 
 #endif /* __ASM_GENERIC_QSPINLOCK_TYPES_H */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -94,24 +94,28 @@ static inline struct mcs_spinlock *decod
 	return per_cpu_ptr(&mcs_nodes[idx], cpu);
 }
 
+#define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
+
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
  *
- * (queue tail, lock value)
- *
- *              fast      :    slow                                  :    unlock
- *                        :                                          :
- * uncontended  (0,0)   --:--> (0,1) --------------------------------:--> (*,0)
- *                        :       | ^--------.                    /  :
- *                        :       v           \                   |  :
- * uncontended            :    (n,x) --+--> (n,0)                 |  :
- *   queue                :       | ^--'                          |  :
- *                        :       v                               |  :
- * contended              :    (*,x) --+--> (*,0) -----> (*,1) ---'  :
- *   queue                :         ^--'                             :
+ * (queue tail, pending bit, lock value)
  *
+ *              fast     :    slow                                  :    unlock
+ *                       :                                          :
+ * uncontended  (0,0,0) -:--> (0,0,1) ------------------------------:--> (*,*,0)
+ *                       :       | ^--------.------.             /  :
+ *                       :       v           \      \            |  :
+ * pending               :    (0,1,1) +--> (0,1,0)   \           |  :
+ *                       :       | ^--'              |           |  :
+ *                       :       v                   |           |  :
+ * uncontended           :    (n,x,y) +--> (n,0,0) --'           |  :
+ *   queue               :       | ^--'                          |  :
+ *                       :       v                               |  :
+ * contended             :    (*,x,y) +--> (*,0,0) ---> (*,0,1) -'  :
+ *   queue               :         ^--'                             :
  */
 void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
 {
@@ -121,6 +125,75 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	/*
+	 * wait for in-progress pending->locked hand-overs
+	 *
+	 * 0,1,0 -> 0,0,1
+	 */
+	if (val == _Q_PENDING_VAL) {
+		while ((val = atomic_read(&lock->val)) == _Q_PENDING_VAL)
+			cpu_relax();
+	}
+
+	/*
+	 * trylock || pending
+	 *
+	 * 0,0,0 -> 0,0,1 ; trylock
+	 * 0,0,1 -> 0,1,1 ; pending
+	 */
+	for (;;) {
+		/*
+		 * If we observe any contention; queue.
+		 */
+		if (val & ~_Q_LOCKED_MASK)
+			goto queue;
+
+		new = _Q_LOCKED_VAL;
+		if (val == new)
+			new |= _Q_PENDING_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * we won the trylock
+	 */
+	if (new == _Q_LOCKED_VAL)
+		return;
+
+	/*
+	 * we're pending, wait for the owner to go away.
+	 *
+	 * *,1,1 -> *,1,0
+	 */
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+		cpu_relax();
+
+	/*
+	 * take ownership and clear the pending bit.
+	 *
+	 * *,1,0 -> *,0,1
+	 */
+	for (;;) {
+		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+	return;
+
+	/*
+	 * End of pending bit optimistic spinning and beginning of MCS
+	 * queuing.
+	 */
+queue:
 	node = this_cpu_ptr(&mcs_nodes[0]);
 	idx = node->count++;
 	tail = encode_tail(smp_processor_id(), idx);
@@ -130,15 +203,18 @@ void queue_spin_lock_slowpath(struct qsp
 	node->next = NULL;
 
 	/*
+	 * We have already touched the queueing cacheline; don't bother with
+	 * pending stuff.
+	 *
 	 * trylock || xchg(lock, node)
 	 *
-	 * 0,0 -> 0,1 ; no tail, not locked -> no tail, locked.
-	 * p,x -> n,x ; tail was p -> tail is n; preserving locked.
+	 * 0,0,0 -> 0,0,1 ; no tail, not locked -> no tail, locked.
+	 * p,y,x -> n,y,x ; tail was p -> tail is n; preserving locked.
 	 */
 	for (;;) {
 		new = _Q_LOCKED_VAL;
 		if (val)
-			new = tail | (val & _Q_LOCKED_MASK);
+			new = tail | (val & _Q_LOCKED_PENDING_MASK);
 
 		old = atomic_cmpxchg(&lock->val, val, new);
 		if (old == val)
@@ -157,7 +233,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 * if there was a previous node; link it and wait until reaching the
 	 * head of the waitqueue.
 	 */
-	if (old & ~_Q_LOCKED_MASK) {
+	if (old & ~_Q_LOCKED_PENDING_MASK) {
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);
 
@@ -165,18 +241,19 @@ void queue_spin_lock_slowpath(struct qsp
 	}
 
 	/*
-	 * we're at the head of the waitqueue, wait for the owner to go away.
+	 * we're at the head of the waitqueue, wait for the owner & pending to
+	 * go away.
 	 *
-	 * *,x -> *,0
+	 * *,x,y -> *,0,0
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
 	/*
 	 * claim the lock:
 	 *
-	 * n,0 -> 0,1 : lock, uncontended
-	 * *,0 -> *,1 : lock, contended
+	 * n,0,0 -> 0,0,1 : lock, uncontended
+	 * *,0,0 -> *,0,1 : lock, contended
 	 */
 	for (;;) {
 		new = _Q_LOCKED_VAL;



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

* [PATCH 3/9] qspinlock: Add pending bit
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (6 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: peter_zijlstra-qspinlock-add_pending_bit.patch --]
[-- Type: text/plain, Size: 7998 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

Because the qspinlock needs to touch a second cacheline (the per-cpu
mcs_nodes[]); add a pending bit and allow a single in-word spinner
before we punt to the second cacheline.

It is possible so observe the pending bit without the locked bit when
the last owner has just released but the pending owner has not yet
taken ownership.

In this case we would normally queue -- because the pending bit is
already taken. However, in this case the pending bit is guaranteed
to be released 'soon', therefore wait for it and avoid queueing.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-4-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |   12 ++-
 kernel/locking/qspinlock.c            |  119 ++++++++++++++++++++++++++++------
 2 files changed, 107 insertions(+), 24 deletions(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -36,8 +36,9 @@ typedef struct qspinlock {
  * Bitfields in the atomic value:
  *
  *  0- 7: locked byte
- *  8- 9: tail index
- * 10-31: tail cpu (+1)
+ *     8: pending
+ *  9-10: tail index
+ * 11-31: tail cpu (+1)
  */
 #define	_Q_SET_MASK(type)	(((1U << _Q_ ## type ## _BITS) - 1)\
 				      << _Q_ ## type ## _OFFSET)
@@ -45,7 +46,11 @@ typedef struct qspinlock {
 #define _Q_LOCKED_BITS		8
 #define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
 
-#define _Q_TAIL_IDX_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_PENDING_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_PENDING_BITS		1
+#define _Q_PENDING_MASK		_Q_SET_MASK(PENDING)
+
+#define _Q_TAIL_IDX_OFFSET	(_Q_PENDING_OFFSET + _Q_PENDING_BITS)
 #define _Q_TAIL_IDX_BITS	2
 #define _Q_TAIL_IDX_MASK	_Q_SET_MASK(TAIL_IDX)
 
@@ -54,5 +59,6 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
+#define _Q_PENDING_VAL		(1U << _Q_PENDING_OFFSET)
 
 #endif /* __ASM_GENERIC_QSPINLOCK_TYPES_H */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -94,24 +94,28 @@ static inline struct mcs_spinlock *decod
 	return per_cpu_ptr(&mcs_nodes[idx], cpu);
 }
 
+#define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
+
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
  *
- * (queue tail, lock value)
- *
- *              fast      :    slow                                  :    unlock
- *                        :                                          :
- * uncontended  (0,0)   --:--> (0,1) --------------------------------:--> (*,0)
- *                        :       | ^--------.                    /  :
- *                        :       v           \                   |  :
- * uncontended            :    (n,x) --+--> (n,0)                 |  :
- *   queue                :       | ^--'                          |  :
- *                        :       v                               |  :
- * contended              :    (*,x) --+--> (*,0) -----> (*,1) ---'  :
- *   queue                :         ^--'                             :
+ * (queue tail, pending bit, lock value)
  *
+ *              fast     :    slow                                  :    unlock
+ *                       :                                          :
+ * uncontended  (0,0,0) -:--> (0,0,1) ------------------------------:--> (*,*,0)
+ *                       :       | ^--------.------.             /  :
+ *                       :       v           \      \            |  :
+ * pending               :    (0,1,1) +--> (0,1,0)   \           |  :
+ *                       :       | ^--'              |           |  :
+ *                       :       v                   |           |  :
+ * uncontended           :    (n,x,y) +--> (n,0,0) --'           |  :
+ *   queue               :       | ^--'                          |  :
+ *                       :       v                               |  :
+ * contended             :    (*,x,y) +--> (*,0,0) ---> (*,0,1) -'  :
+ *   queue               :         ^--'                             :
  */
 void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
 {
@@ -121,6 +125,75 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	/*
+	 * wait for in-progress pending->locked hand-overs
+	 *
+	 * 0,1,0 -> 0,0,1
+	 */
+	if (val == _Q_PENDING_VAL) {
+		while ((val = atomic_read(&lock->val)) == _Q_PENDING_VAL)
+			cpu_relax();
+	}
+
+	/*
+	 * trylock || pending
+	 *
+	 * 0,0,0 -> 0,0,1 ; trylock
+	 * 0,0,1 -> 0,1,1 ; pending
+	 */
+	for (;;) {
+		/*
+		 * If we observe any contention; queue.
+		 */
+		if (val & ~_Q_LOCKED_MASK)
+			goto queue;
+
+		new = _Q_LOCKED_VAL;
+		if (val == new)
+			new |= _Q_PENDING_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * we won the trylock
+	 */
+	if (new == _Q_LOCKED_VAL)
+		return;
+
+	/*
+	 * we're pending, wait for the owner to go away.
+	 *
+	 * *,1,1 -> *,1,0
+	 */
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+		cpu_relax();
+
+	/*
+	 * take ownership and clear the pending bit.
+	 *
+	 * *,1,0 -> *,0,1
+	 */
+	for (;;) {
+		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+	return;
+
+	/*
+	 * End of pending bit optimistic spinning and beginning of MCS
+	 * queuing.
+	 */
+queue:
 	node = this_cpu_ptr(&mcs_nodes[0]);
 	idx = node->count++;
 	tail = encode_tail(smp_processor_id(), idx);
@@ -130,15 +203,18 @@ void queue_spin_lock_slowpath(struct qsp
 	node->next = NULL;
 
 	/*
+	 * We have already touched the queueing cacheline; don't bother with
+	 * pending stuff.
+	 *
 	 * trylock || xchg(lock, node)
 	 *
-	 * 0,0 -> 0,1 ; no tail, not locked -> no tail, locked.
-	 * p,x -> n,x ; tail was p -> tail is n; preserving locked.
+	 * 0,0,0 -> 0,0,1 ; no tail, not locked -> no tail, locked.
+	 * p,y,x -> n,y,x ; tail was p -> tail is n; preserving locked.
 	 */
 	for (;;) {
 		new = _Q_LOCKED_VAL;
 		if (val)
-			new = tail | (val & _Q_LOCKED_MASK);
+			new = tail | (val & _Q_LOCKED_PENDING_MASK);
 
 		old = atomic_cmpxchg(&lock->val, val, new);
 		if (old == val)
@@ -157,7 +233,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 * if there was a previous node; link it and wait until reaching the
 	 * head of the waitqueue.
 	 */
-	if (old & ~_Q_LOCKED_MASK) {
+	if (old & ~_Q_LOCKED_PENDING_MASK) {
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);
 
@@ -165,18 +241,19 @@ void queue_spin_lock_slowpath(struct qsp
 	}
 
 	/*
-	 * we're at the head of the waitqueue, wait for the owner to go away.
+	 * we're at the head of the waitqueue, wait for the owner & pending to
+	 * go away.
 	 *
-	 * *,x -> *,0
+	 * *,x,y -> *,0,0
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
 	/*
 	 * claim the lock:
 	 *
-	 * n,0 -> 0,1 : lock, uncontended
-	 * *,0 -> *,1 : lock, contended
+	 * n,0,0 -> 0,0,1 : lock, uncontended
+	 * *,0,0 -> *,0,1 : lock, contended
 	 */
 	for (;;) {
 		new = _Q_LOCKED_VAL;

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

* [PATCH 3/9] qspinlock: Add pending bit
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (7 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: peter_zijlstra-qspinlock-add_pending_bit.patch --]
[-- Type: text/plain, Size: 7998 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

Because the qspinlock needs to touch a second cacheline (the per-cpu
mcs_nodes[]); add a pending bit and allow a single in-word spinner
before we punt to the second cacheline.

It is possible so observe the pending bit without the locked bit when
the last owner has just released but the pending owner has not yet
taken ownership.

In this case we would normally queue -- because the pending bit is
already taken. However, in this case the pending bit is guaranteed
to be released 'soon', therefore wait for it and avoid queueing.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-4-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |   12 ++-
 kernel/locking/qspinlock.c            |  119 ++++++++++++++++++++++++++++------
 2 files changed, 107 insertions(+), 24 deletions(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -36,8 +36,9 @@ typedef struct qspinlock {
  * Bitfields in the atomic value:
  *
  *  0- 7: locked byte
- *  8- 9: tail index
- * 10-31: tail cpu (+1)
+ *     8: pending
+ *  9-10: tail index
+ * 11-31: tail cpu (+1)
  */
 #define	_Q_SET_MASK(type)	(((1U << _Q_ ## type ## _BITS) - 1)\
 				      << _Q_ ## type ## _OFFSET)
@@ -45,7 +46,11 @@ typedef struct qspinlock {
 #define _Q_LOCKED_BITS		8
 #define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
 
-#define _Q_TAIL_IDX_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_PENDING_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#define _Q_PENDING_BITS		1
+#define _Q_PENDING_MASK		_Q_SET_MASK(PENDING)
+
+#define _Q_TAIL_IDX_OFFSET	(_Q_PENDING_OFFSET + _Q_PENDING_BITS)
 #define _Q_TAIL_IDX_BITS	2
 #define _Q_TAIL_IDX_MASK	_Q_SET_MASK(TAIL_IDX)
 
@@ -54,5 +59,6 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
+#define _Q_PENDING_VAL		(1U << _Q_PENDING_OFFSET)
 
 #endif /* __ASM_GENERIC_QSPINLOCK_TYPES_H */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -94,24 +94,28 @@ static inline struct mcs_spinlock *decod
 	return per_cpu_ptr(&mcs_nodes[idx], cpu);
 }
 
+#define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
+
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
  *
- * (queue tail, lock value)
- *
- *              fast      :    slow                                  :    unlock
- *                        :                                          :
- * uncontended  (0,0)   --:--> (0,1) --------------------------------:--> (*,0)
- *                        :       | ^--------.                    /  :
- *                        :       v           \                   |  :
- * uncontended            :    (n,x) --+--> (n,0)                 |  :
- *   queue                :       | ^--'                          |  :
- *                        :       v                               |  :
- * contended              :    (*,x) --+--> (*,0) -----> (*,1) ---'  :
- *   queue                :         ^--'                             :
+ * (queue tail, pending bit, lock value)
  *
+ *              fast     :    slow                                  :    unlock
+ *                       :                                          :
+ * uncontended  (0,0,0) -:--> (0,0,1) ------------------------------:--> (*,*,0)
+ *                       :       | ^--------.------.             /  :
+ *                       :       v           \      \            |  :
+ * pending               :    (0,1,1) +--> (0,1,0)   \           |  :
+ *                       :       | ^--'              |           |  :
+ *                       :       v                   |           |  :
+ * uncontended           :    (n,x,y) +--> (n,0,0) --'           |  :
+ *   queue               :       | ^--'                          |  :
+ *                       :       v                               |  :
+ * contended             :    (*,x,y) +--> (*,0,0) ---> (*,0,1) -'  :
+ *   queue               :         ^--'                             :
  */
 void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
 {
@@ -121,6 +125,75 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	/*
+	 * wait for in-progress pending->locked hand-overs
+	 *
+	 * 0,1,0 -> 0,0,1
+	 */
+	if (val == _Q_PENDING_VAL) {
+		while ((val = atomic_read(&lock->val)) == _Q_PENDING_VAL)
+			cpu_relax();
+	}
+
+	/*
+	 * trylock || pending
+	 *
+	 * 0,0,0 -> 0,0,1 ; trylock
+	 * 0,0,1 -> 0,1,1 ; pending
+	 */
+	for (;;) {
+		/*
+		 * If we observe any contention; queue.
+		 */
+		if (val & ~_Q_LOCKED_MASK)
+			goto queue;
+
+		new = _Q_LOCKED_VAL;
+		if (val == new)
+			new |= _Q_PENDING_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+
+	/*
+	 * we won the trylock
+	 */
+	if (new == _Q_LOCKED_VAL)
+		return;
+
+	/*
+	 * we're pending, wait for the owner to go away.
+	 *
+	 * *,1,1 -> *,1,0
+	 */
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+		cpu_relax();
+
+	/*
+	 * take ownership and clear the pending bit.
+	 *
+	 * *,1,0 -> *,0,1
+	 */
+	for (;;) {
+		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+	return;
+
+	/*
+	 * End of pending bit optimistic spinning and beginning of MCS
+	 * queuing.
+	 */
+queue:
 	node = this_cpu_ptr(&mcs_nodes[0]);
 	idx = node->count++;
 	tail = encode_tail(smp_processor_id(), idx);
@@ -130,15 +203,18 @@ void queue_spin_lock_slowpath(struct qsp
 	node->next = NULL;
 
 	/*
+	 * We have already touched the queueing cacheline; don't bother with
+	 * pending stuff.
+	 *
 	 * trylock || xchg(lock, node)
 	 *
-	 * 0,0 -> 0,1 ; no tail, not locked -> no tail, locked.
-	 * p,x -> n,x ; tail was p -> tail is n; preserving locked.
+	 * 0,0,0 -> 0,0,1 ; no tail, not locked -> no tail, locked.
+	 * p,y,x -> n,y,x ; tail was p -> tail is n; preserving locked.
 	 */
 	for (;;) {
 		new = _Q_LOCKED_VAL;
 		if (val)
-			new = tail | (val & _Q_LOCKED_MASK);
+			new = tail | (val & _Q_LOCKED_PENDING_MASK);
 
 		old = atomic_cmpxchg(&lock->val, val, new);
 		if (old == val)
@@ -157,7 +233,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 * if there was a previous node; link it and wait until reaching the
 	 * head of the waitqueue.
 	 */
-	if (old & ~_Q_LOCKED_MASK) {
+	if (old & ~_Q_LOCKED_PENDING_MASK) {
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);
 
@@ -165,18 +241,19 @@ void queue_spin_lock_slowpath(struct qsp
 	}
 
 	/*
-	 * we're at the head of the waitqueue, wait for the owner to go away.
+	 * we're at the head of the waitqueue, wait for the owner & pending to
+	 * go away.
 	 *
-	 * *,x -> *,0
+	 * *,x,y -> *,0,0
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
 	/*
 	 * claim the lock:
 	 *
-	 * n,0 -> 0,1 : lock, uncontended
-	 * *,0 -> *,1 : lock, contended
+	 * n,0,0 -> 0,0,1 : lock, uncontended
+	 * *,0,0 -> *,0,1 : lock, contended
 	 */
 	for (;;) {
 		new = _Q_LOCKED_VAL;

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

* [PATCH 4/9] qspinlock: Extract out code snippets for the next patch
  2015-03-16 13:16 ` Peter Zijlstra
@ 2015-03-16 13:16   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: waiman_long-qspinlock-extract_out_code_snippets_for_the_next_patch.patch --]
[-- Type: text/plain, Size: 4921 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This is a preparatory patch that extracts out the following 2 code
snippets to prepare for the next performance optimization patch.

 1) the logic for the exchange of new and previous tail code words
    into a new xchg_tail() function.
 2) the logic for clearing the pending bit and setting the locked bit
    into a new clear_pending_set_locked() function.

This patch also simplifies the trylock operation before queuing by
calling queue_spin_trylock() directly.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-5-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |    2 
 kernel/locking/qspinlock.c            |   91 ++++++++++++++++++++++------------
 2 files changed, 62 insertions(+), 31 deletions(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -58,6 +58,8 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
+#define _Q_TAIL_MASK		(_Q_TAIL_IDX_MASK | _Q_TAIL_CPU_MASK)
+
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
 #define _Q_PENDING_VAL		(1U << _Q_PENDING_OFFSET)
 
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -97,6 +97,54 @@ static inline struct mcs_spinlock *decod
 #define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
 
 /**
+ * clear_pending_set_locked - take ownership and clear the pending bit.
+ * @lock: Pointer to queue spinlock structure
+ * @val : Current value of the queue spinlock 32-bit word
+ *
+ * *,1,0 -> *,0,1
+ */
+static __always_inline void
+clear_pending_set_locked(struct qspinlock *lock, u32 val)
+{
+	u32 new, old;
+
+	for (;;) {
+		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+}
+
+/**
+ * xchg_tail - Put in the new queue tail code word & retrieve previous one
+ * @lock : Pointer to queue spinlock structure
+ * @tail : The new queue tail code word
+ * Return: The previous queue tail code word
+ *
+ * xchg(lock, tail)
+ *
+ * p,*,* -> n,*,* ; prev = xchg(lock, node)
+ */
+static __always_inline u32 xchg_tail(struct qspinlock *lock, u32 tail)
+{
+	u32 old, new, val = atomic_read(&lock->val);
+
+	for (;;) {
+		new = (val & _Q_LOCKED_PENDING_MASK) | tail;
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+	return old;
+}
+
+/**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
@@ -178,15 +226,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 *
 	 * *,1,0 -> *,0,1
 	 */
-	for (;;) {
-		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
-			break;
-
-		val = old;
-	}
+	clear_pending_set_locked(lock, val);
 	return;
 
 	/*
@@ -203,37 +243,26 @@ void queue_spin_lock_slowpath(struct qsp
 	node->next = NULL;
 
 	/*
-	 * We have already touched the queueing cacheline; don't bother with
-	 * pending stuff.
-	 *
-	 * trylock || xchg(lock, node)
-	 *
-	 * 0,0,0 -> 0,0,1 ; no tail, not locked -> no tail, locked.
-	 * p,y,x -> n,y,x ; tail was p -> tail is n; preserving locked.
+	 * We touched a (possibly) cold cacheline in the per-cpu queue node;
+	 * attempt the trylock once more in the hope someone let go while we
+	 * weren't watching.
 	 */
-	for (;;) {
-		new = _Q_LOCKED_VAL;
-		if (val)
-			new = tail | (val & _Q_LOCKED_PENDING_MASK);
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
-			break;
-
-		val = old;
-	}
+	if (queue_spin_trylock(lock))
+		goto release;
 
 	/*
-	 * we won the trylock; forget about queueing.
+	 * We have already touched the queueing cacheline; don't bother with
+	 * pending stuff.
+	 *
+	 * p,*,* -> n,*,*
 	 */
-	if (new == _Q_LOCKED_VAL)
-		goto release;
+	old = xchg_tail(lock, tail);
 
 	/*
 	 * if there was a previous node; link it and wait until reaching the
 	 * head of the waitqueue.
 	 */
-	if (old & ~_Q_LOCKED_PENDING_MASK) {
+	if (old & _Q_TAIL_MASK) {
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);
 



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

* [PATCH 4/9] qspinlock: Extract out code snippets for the next patch
@ 2015-03-16 13:16   ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: waiman_long-qspinlock-extract_out_code_snippets_for_the_next_patch.patch --]
[-- Type: text/plain, Size: 4917 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This is a preparatory patch that extracts out the following 2 code
snippets to prepare for the next performance optimization patch.

 1) the logic for the exchange of new and previous tail code words
    into a new xchg_tail() function.
 2) the logic for clearing the pending bit and setting the locked bit
    into a new clear_pending_set_locked() function.

This patch also simplifies the trylock operation before queuing by
calling queue_spin_trylock() directly.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-5-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |    2 
 kernel/locking/qspinlock.c            |   91 ++++++++++++++++++++++------------
 2 files changed, 62 insertions(+), 31 deletions(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -58,6 +58,8 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
+#define _Q_TAIL_MASK		(_Q_TAIL_IDX_MASK | _Q_TAIL_CPU_MASK)
+
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
 #define _Q_PENDING_VAL		(1U << _Q_PENDING_OFFSET)
 
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -97,6 +97,54 @@ static inline struct mcs_spinlock *decod
 #define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
 
 /**
+ * clear_pending_set_locked - take ownership and clear the pending bit.
+ * @lock: Pointer to queue spinlock structure
+ * @val : Current value of the queue spinlock 32-bit word
+ *
+ * *,1,0 -> *,0,1
+ */
+static __always_inline void
+clear_pending_set_locked(struct qspinlock *lock, u32 val)
+{
+	u32 new, old;
+
+	for (;;) {
+		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+}
+
+/**
+ * xchg_tail - Put in the new queue tail code word & retrieve previous one
+ * @lock : Pointer to queue spinlock structure
+ * @tail : The new queue tail code word
+ * Return: The previous queue tail code word
+ *
+ * xchg(lock, tail)
+ *
+ * p,*,* -> n,*,* ; prev = xchg(lock, node)
+ */
+static __always_inline u32 xchg_tail(struct qspinlock *lock, u32 tail)
+{
+	u32 old, new, val = atomic_read(&lock->val);
+
+	for (;;) {
+		new = (val & _Q_LOCKED_PENDING_MASK) | tail;
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+	return old;
+}
+
+/**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
@@ -178,15 +226,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 *
 	 * *,1,0 -> *,0,1
 	 */
-	for (;;) {
-		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
-			break;
-
-		val = old;
-	}
+	clear_pending_set_locked(lock, val);
 	return;
 
 	/*
@@ -203,37 +243,26 @@ void queue_spin_lock_slowpath(struct qsp
 	node->next = NULL;
 
 	/*
-	 * We have already touched the queueing cacheline; don't bother with
-	 * pending stuff.
-	 *
-	 * trylock || xchg(lock, node)
-	 *
-	 * 0,0,0 -> 0,0,1 ; no tail, not locked -> no tail, locked.
-	 * p,y,x -> n,y,x ; tail was p -> tail is n; preserving locked.
+	 * We touched a (possibly) cold cacheline in the per-cpu queue node;
+	 * attempt the trylock once more in the hope someone let go while we
+	 * weren't watching.
 	 */
-	for (;;) {
-		new = _Q_LOCKED_VAL;
-		if (val)
-			new = tail | (val & _Q_LOCKED_PENDING_MASK);
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
-			break;
-
-		val = old;
-	}
+	if (queue_spin_trylock(lock))
+		goto release;
 
 	/*
-	 * we won the trylock; forget about queueing.
+	 * We have already touched the queueing cacheline; don't bother with
+	 * pending stuff.
+	 *
+	 * p,*,* -> n,*,*
 	 */
-	if (new == _Q_LOCKED_VAL)
-		goto release;
+	old = xchg_tail(lock, tail);
 
 	/*
 	 * if there was a previous node; link it and wait until reaching the
 	 * head of the waitqueue.
 	 */
-	if (old & ~_Q_LOCKED_PENDING_MASK) {
+	if (old & _Q_TAIL_MASK) {
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);

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

* [PATCH 4/9] qspinlock: Extract out code snippets for the next patch
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (8 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: waiman_long-qspinlock-extract_out_code_snippets_for_the_next_patch.patch --]
[-- Type: text/plain, Size: 4917 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

This is a preparatory patch that extracts out the following 2 code
snippets to prepare for the next performance optimization patch.

 1) the logic for the exchange of new and previous tail code words
    into a new xchg_tail() function.
 2) the logic for clearing the pending bit and setting the locked bit
    into a new clear_pending_set_locked() function.

This patch also simplifies the trylock operation before queuing by
calling queue_spin_trylock() directly.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-5-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |    2 
 kernel/locking/qspinlock.c            |   91 ++++++++++++++++++++++------------
 2 files changed, 62 insertions(+), 31 deletions(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -58,6 +58,8 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
+#define _Q_TAIL_MASK		(_Q_TAIL_IDX_MASK | _Q_TAIL_CPU_MASK)
+
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
 #define _Q_PENDING_VAL		(1U << _Q_PENDING_OFFSET)
 
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -97,6 +97,54 @@ static inline struct mcs_spinlock *decod
 #define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
 
 /**
+ * clear_pending_set_locked - take ownership and clear the pending bit.
+ * @lock: Pointer to queue spinlock structure
+ * @val : Current value of the queue spinlock 32-bit word
+ *
+ * *,1,0 -> *,0,1
+ */
+static __always_inline void
+clear_pending_set_locked(struct qspinlock *lock, u32 val)
+{
+	u32 new, old;
+
+	for (;;) {
+		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
+
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+}
+
+/**
+ * xchg_tail - Put in the new queue tail code word & retrieve previous one
+ * @lock : Pointer to queue spinlock structure
+ * @tail : The new queue tail code word
+ * Return: The previous queue tail code word
+ *
+ * xchg(lock, tail)
+ *
+ * p,*,* -> n,*,* ; prev = xchg(lock, node)
+ */
+static __always_inline u32 xchg_tail(struct qspinlock *lock, u32 tail)
+{
+	u32 old, new, val = atomic_read(&lock->val);
+
+	for (;;) {
+		new = (val & _Q_LOCKED_PENDING_MASK) | tail;
+		old = atomic_cmpxchg(&lock->val, val, new);
+		if (old == val)
+			break;
+
+		val = old;
+	}
+	return old;
+}
+
+/**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
@@ -178,15 +226,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 *
 	 * *,1,0 -> *,0,1
 	 */
-	for (;;) {
-		new = (val & ~_Q_PENDING_MASK) | _Q_LOCKED_VAL;
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
-			break;
-
-		val = old;
-	}
+	clear_pending_set_locked(lock, val);
 	return;
 
 	/*
@@ -203,37 +243,26 @@ void queue_spin_lock_slowpath(struct qsp
 	node->next = NULL;
 
 	/*
-	 * We have already touched the queueing cacheline; don't bother with
-	 * pending stuff.
-	 *
-	 * trylock || xchg(lock, node)
-	 *
-	 * 0,0,0 -> 0,0,1 ; no tail, not locked -> no tail, locked.
-	 * p,y,x -> n,y,x ; tail was p -> tail is n; preserving locked.
+	 * We touched a (possibly) cold cacheline in the per-cpu queue node;
+	 * attempt the trylock once more in the hope someone let go while we
+	 * weren't watching.
 	 */
-	for (;;) {
-		new = _Q_LOCKED_VAL;
-		if (val)
-			new = tail | (val & _Q_LOCKED_PENDING_MASK);
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
-			break;
-
-		val = old;
-	}
+	if (queue_spin_trylock(lock))
+		goto release;
 
 	/*
-	 * we won the trylock; forget about queueing.
+	 * We have already touched the queueing cacheline; don't bother with
+	 * pending stuff.
+	 *
+	 * p,*,* -> n,*,*
 	 */
-	if (new == _Q_LOCKED_VAL)
-		goto release;
+	old = xchg_tail(lock, tail);
 
 	/*
 	 * if there was a previous node; link it and wait until reaching the
 	 * head of the waitqueue.
 	 */
-	if (old & ~_Q_LOCKED_PENDING_MASK) {
+	if (old & _Q_TAIL_MASK) {
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);

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

* [PATCH 5/9] qspinlock: Optimize for smaller NR_CPUS
  2015-03-16 13:16 ` Peter Zijlstra
@ 2015-03-16 13:16   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: peter_zijlstra-qspinlock-optimize_for_smaller_nr_cpus.patch --]
[-- Type: text/plain, Size: 5852 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

When we allow for a max NR_CPUS < 2^14 we can optimize the pending
wait-acquire and the xchg_tail() operations.

By growing the pending bit to a byte, we reduce the tail to 16bit.
This means we can use xchg16 for the tail part and do away with all
the repeated compxchg() operations.

This in turn allows us to unconditionally acquire; the locked state
as observed by the wait loops cannot change. And because both locked
and pending are now a full byte we can use simple stores for the
state transition, obviating one atomic operation entirely.

This optimization is needed to make the qspinlock achieve performance
parity with ticket spinlock at light load.

All this is horribly broken on Alpha pre EV56 (and any other arch that
cannot do single-copy atomic byte stores).

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-6-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |   13 ++++++
 kernel/locking/qspinlock.c            |   71 +++++++++++++++++++++++++++++++++-
 2 files changed, 83 insertions(+), 1 deletion(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -35,6 +35,14 @@ typedef struct qspinlock {
 /*
  * Bitfields in the atomic value:
  *
+ * When NR_CPUS < 16K
+ *  0- 7: locked byte
+ *     8: pending
+ *  9-15: not used
+ * 16-17: tail index
+ * 18-31: tail cpu (+1)
+ *
+ * When NR_CPUS >= 16K
  *  0- 7: locked byte
  *     8: pending
  *  9-10: tail index
@@ -47,7 +55,11 @@ typedef struct qspinlock {
 #define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
 
 #define _Q_PENDING_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#if CONFIG_NR_CPUS < (1U << 14)
+#define _Q_PENDING_BITS		8
+#else
 #define _Q_PENDING_BITS		1
+#endif
 #define _Q_PENDING_MASK		_Q_SET_MASK(PENDING)
 
 #define _Q_TAIL_IDX_OFFSET	(_Q_PENDING_OFFSET + _Q_PENDING_BITS)
@@ -58,6 +70,7 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
+#define _Q_TAIL_OFFSET		_Q_TAIL_IDX_OFFSET
 #define _Q_TAIL_MASK		(_Q_TAIL_IDX_MASK | _Q_TAIL_CPU_MASK)
 
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -24,6 +24,7 @@
 #include <linux/percpu.h>
 #include <linux/hardirq.h>
 #include <linux/mutex.h>
+#include <asm/byteorder.h>
 #include <asm/qspinlock.h>
 
 /*
@@ -56,6 +57,10 @@
  * node; whereby avoiding the need to carry a node from lock to unlock, and
  * preserving existing lock API. This also makes the unlock code simpler and
  * faster.
+ *
+ * N.B. The current implementation only supports architectures that allow
+ *      atomic operations on smaller 8-bit and 16-bit data types.
+ *
  */
 
 #include "mcs_spinlock.h"
@@ -96,6 +101,64 @@ static inline struct mcs_spinlock *decod
 
 #define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
 
+/*
+ * By using the whole 2nd least significant byte for the pending bit, we
+ * can allow better optimization of the lock acquisition for the pending
+ * bit holder.
+ */
+#if _Q_PENDING_BITS == 8
+
+struct __qspinlock {
+	union {
+		atomic_t val;
+		struct {
+#ifdef __LITTLE_ENDIAN
+			u16	locked_pending;
+			u16	tail;
+#else
+			u16	tail;
+			u16	locked_pending;
+#endif
+		};
+	};
+};
+
+/**
+ * clear_pending_set_locked - take ownership and clear the pending bit.
+ * @lock: Pointer to queue spinlock structure
+ * @val : Current value of the queue spinlock 32-bit word
+ *
+ * *,1,0 -> *,0,1
+ *
+ * Lock stealing is not allowed if this function is used.
+ */
+static __always_inline void
+clear_pending_set_locked(struct qspinlock *lock, u32 val)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	WRITE_ONCE(l->locked_pending, _Q_LOCKED_VAL);
+}
+
+/*
+ * xchg_tail - Put in the new queue tail code word & retrieve previous one
+ * @lock : Pointer to queue spinlock structure
+ * @tail : The new queue tail code word
+ * Return: The previous queue tail code word
+ *
+ * xchg(lock, tail)
+ *
+ * p,*,* -> n,*,* ; prev = xchg(lock, node)
+ */
+static __always_inline u32 xchg_tail(struct qspinlock *lock, u32 tail)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	return (u32)xchg(&l->tail, tail >> _Q_TAIL_OFFSET) << _Q_TAIL_OFFSET;
+}
+
+#else /* _Q_PENDING_BITS == 8 */
+
 /**
  * clear_pending_set_locked - take ownership and clear the pending bit.
  * @lock: Pointer to queue spinlock structure
@@ -143,6 +206,7 @@ static __always_inline u32 xchg_tail(str
 	}
 	return old;
 }
+#endif /* _Q_PENDING_BITS == 8 */
 
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
@@ -217,8 +281,13 @@ void queue_spin_lock_slowpath(struct qsp
 	 * we're pending, wait for the owner to go away.
 	 *
 	 * *,1,1 -> *,1,0
+	 *
+	 * this wait loop must be a load-acquire such that we match the
+	 * store-release that clears the locked bit and create lock
+	 * sequentiality; this is because not all clear_pending_set_locked()
+	 * implementations imply full barriers.
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_MASK)
 		cpu_relax();
 
 	/*



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

* [PATCH 5/9] qspinlock: Optimize for smaller NR_CPUS
@ 2015-03-16 13:16   ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: peter_zijlstra-qspinlock-optimize_for_smaller_nr_cpus.patch --]
[-- Type: text/plain, Size: 5850 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

When we allow for a max NR_CPUS < 2^14 we can optimize the pending
wait-acquire and the xchg_tail() operations.

By growing the pending bit to a byte, we reduce the tail to 16bit.
This means we can use xchg16 for the tail part and do away with all
the repeated compxchg() operations.

This in turn allows us to unconditionally acquire; the locked state
as observed by the wait loops cannot change. And because both locked
and pending are now a full byte we can use simple stores for the
state transition, obviating one atomic operation entirely.

This optimization is needed to make the qspinlock achieve performance
parity with ticket spinlock at light load.

All this is horribly broken on Alpha pre EV56 (and any other arch that
cannot do single-copy atomic byte stores).

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-6-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |   13 ++++++
 kernel/locking/qspinlock.c            |   71 +++++++++++++++++++++++++++++++++-
 2 files changed, 83 insertions(+), 1 deletion(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -35,6 +35,14 @@ typedef struct qspinlock {
 /*
  * Bitfields in the atomic value:
  *
+ * When NR_CPUS < 16K
+ *  0- 7: locked byte
+ *     8: pending
+ *  9-15: not used
+ * 16-17: tail index
+ * 18-31: tail cpu (+1)
+ *
+ * When NR_CPUS >= 16K
  *  0- 7: locked byte
  *     8: pending
  *  9-10: tail index
@@ -47,7 +55,11 @@ typedef struct qspinlock {
 #define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
 
 #define _Q_PENDING_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#if CONFIG_NR_CPUS < (1U << 14)
+#define _Q_PENDING_BITS		8
+#else
 #define _Q_PENDING_BITS		1
+#endif
 #define _Q_PENDING_MASK		_Q_SET_MASK(PENDING)
 
 #define _Q_TAIL_IDX_OFFSET	(_Q_PENDING_OFFSET + _Q_PENDING_BITS)
@@ -58,6 +70,7 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
+#define _Q_TAIL_OFFSET		_Q_TAIL_IDX_OFFSET
 #define _Q_TAIL_MASK		(_Q_TAIL_IDX_MASK | _Q_TAIL_CPU_MASK)
 
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -24,6 +24,7 @@
 #include <linux/percpu.h>
 #include <linux/hardirq.h>
 #include <linux/mutex.h>
+#include <asm/byteorder.h>
 #include <asm/qspinlock.h>
 
 /*
@@ -56,6 +57,10 @@
  * node; whereby avoiding the need to carry a node from lock to unlock, and
  * preserving existing lock API. This also makes the unlock code simpler and
  * faster.
+ *
+ * N.B. The current implementation only supports architectures that allow
+ *      atomic operations on smaller 8-bit and 16-bit data types.
+ *
  */
 
 #include "mcs_spinlock.h"
@@ -96,6 +101,64 @@ static inline struct mcs_spinlock *decod
 
 #define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
 
+/*
+ * By using the whole 2nd least significant byte for the pending bit, we
+ * can allow better optimization of the lock acquisition for the pending
+ * bit holder.
+ */
+#if _Q_PENDING_BITS == 8
+
+struct __qspinlock {
+	union {
+		atomic_t val;
+		struct {
+#ifdef __LITTLE_ENDIAN
+			u16	locked_pending;
+			u16	tail;
+#else
+			u16	tail;
+			u16	locked_pending;
+#endif
+		};
+	};
+};
+
+/**
+ * clear_pending_set_locked - take ownership and clear the pending bit.
+ * @lock: Pointer to queue spinlock structure
+ * @val : Current value of the queue spinlock 32-bit word
+ *
+ * *,1,0 -> *,0,1
+ *
+ * Lock stealing is not allowed if this function is used.
+ */
+static __always_inline void
+clear_pending_set_locked(struct qspinlock *lock, u32 val)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	WRITE_ONCE(l->locked_pending, _Q_LOCKED_VAL);
+}
+
+/*
+ * xchg_tail - Put in the new queue tail code word & retrieve previous one
+ * @lock : Pointer to queue spinlock structure
+ * @tail : The new queue tail code word
+ * Return: The previous queue tail code word
+ *
+ * xchg(lock, tail)
+ *
+ * p,*,* -> n,*,* ; prev = xchg(lock, node)
+ */
+static __always_inline u32 xchg_tail(struct qspinlock *lock, u32 tail)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	return (u32)xchg(&l->tail, tail >> _Q_TAIL_OFFSET) << _Q_TAIL_OFFSET;
+}
+
+#else /* _Q_PENDING_BITS == 8 */
+
 /**
  * clear_pending_set_locked - take ownership and clear the pending bit.
  * @lock: Pointer to queue spinlock structure
@@ -143,6 +206,7 @@ static __always_inline u32 xchg_tail(str
 	}
 	return old;
 }
+#endif /* _Q_PENDING_BITS == 8 */
 
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
@@ -217,8 +281,13 @@ void queue_spin_lock_slowpath(struct qsp
 	 * we're pending, wait for the owner to go away.
 	 *
 	 * *,1,1 -> *,1,0
+	 *
+	 * this wait loop must be a load-acquire such that we match the
+	 * store-release that clears the locked bit and create lock
+	 * sequentiality; this is because not all clear_pending_set_locked()
+	 * implementations imply full barriers.
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_MASK)
 		cpu_relax();
 
 	/*

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

* [PATCH 5/9] qspinlock: Optimize for smaller NR_CPUS
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (11 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: peter_zijlstra-qspinlock-optimize_for_smaller_nr_cpus.patch --]
[-- Type: text/plain, Size: 5850 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

When we allow for a max NR_CPUS < 2^14 we can optimize the pending
wait-acquire and the xchg_tail() operations.

By growing the pending bit to a byte, we reduce the tail to 16bit.
This means we can use xchg16 for the tail part and do away with all
the repeated compxchg() operations.

This in turn allows us to unconditionally acquire; the locked state
as observed by the wait loops cannot change. And because both locked
and pending are now a full byte we can use simple stores for the
state transition, obviating one atomic operation entirely.

This optimization is needed to make the qspinlock achieve performance
parity with ticket spinlock at light load.

All this is horribly broken on Alpha pre EV56 (and any other arch that
cannot do single-copy atomic byte stores).

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-6-git-send-email-Waiman.Long@hp.com
---
 include/asm-generic/qspinlock_types.h |   13 ++++++
 kernel/locking/qspinlock.c            |   71 +++++++++++++++++++++++++++++++++-
 2 files changed, 83 insertions(+), 1 deletion(-)

--- a/include/asm-generic/qspinlock_types.h
+++ b/include/asm-generic/qspinlock_types.h
@@ -35,6 +35,14 @@ typedef struct qspinlock {
 /*
  * Bitfields in the atomic value:
  *
+ * When NR_CPUS < 16K
+ *  0- 7: locked byte
+ *     8: pending
+ *  9-15: not used
+ * 16-17: tail index
+ * 18-31: tail cpu (+1)
+ *
+ * When NR_CPUS >= 16K
  *  0- 7: locked byte
  *     8: pending
  *  9-10: tail index
@@ -47,7 +55,11 @@ typedef struct qspinlock {
 #define _Q_LOCKED_MASK		_Q_SET_MASK(LOCKED)
 
 #define _Q_PENDING_OFFSET	(_Q_LOCKED_OFFSET + _Q_LOCKED_BITS)
+#if CONFIG_NR_CPUS < (1U << 14)
+#define _Q_PENDING_BITS		8
+#else
 #define _Q_PENDING_BITS		1
+#endif
 #define _Q_PENDING_MASK		_Q_SET_MASK(PENDING)
 
 #define _Q_TAIL_IDX_OFFSET	(_Q_PENDING_OFFSET + _Q_PENDING_BITS)
@@ -58,6 +70,7 @@ typedef struct qspinlock {
 #define _Q_TAIL_CPU_BITS	(32 - _Q_TAIL_CPU_OFFSET)
 #define _Q_TAIL_CPU_MASK	_Q_SET_MASK(TAIL_CPU)
 
+#define _Q_TAIL_OFFSET		_Q_TAIL_IDX_OFFSET
 #define _Q_TAIL_MASK		(_Q_TAIL_IDX_MASK | _Q_TAIL_CPU_MASK)
 
 #define _Q_LOCKED_VAL		(1U << _Q_LOCKED_OFFSET)
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -24,6 +24,7 @@
 #include <linux/percpu.h>
 #include <linux/hardirq.h>
 #include <linux/mutex.h>
+#include <asm/byteorder.h>
 #include <asm/qspinlock.h>
 
 /*
@@ -56,6 +57,10 @@
  * node; whereby avoiding the need to carry a node from lock to unlock, and
  * preserving existing lock API. This also makes the unlock code simpler and
  * faster.
+ *
+ * N.B. The current implementation only supports architectures that allow
+ *      atomic operations on smaller 8-bit and 16-bit data types.
+ *
  */
 
 #include "mcs_spinlock.h"
@@ -96,6 +101,64 @@ static inline struct mcs_spinlock *decod
 
 #define _Q_LOCKED_PENDING_MASK (_Q_LOCKED_MASK | _Q_PENDING_MASK)
 
+/*
+ * By using the whole 2nd least significant byte for the pending bit, we
+ * can allow better optimization of the lock acquisition for the pending
+ * bit holder.
+ */
+#if _Q_PENDING_BITS == 8
+
+struct __qspinlock {
+	union {
+		atomic_t val;
+		struct {
+#ifdef __LITTLE_ENDIAN
+			u16	locked_pending;
+			u16	tail;
+#else
+			u16	tail;
+			u16	locked_pending;
+#endif
+		};
+	};
+};
+
+/**
+ * clear_pending_set_locked - take ownership and clear the pending bit.
+ * @lock: Pointer to queue spinlock structure
+ * @val : Current value of the queue spinlock 32-bit word
+ *
+ * *,1,0 -> *,0,1
+ *
+ * Lock stealing is not allowed if this function is used.
+ */
+static __always_inline void
+clear_pending_set_locked(struct qspinlock *lock, u32 val)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	WRITE_ONCE(l->locked_pending, _Q_LOCKED_VAL);
+}
+
+/*
+ * xchg_tail - Put in the new queue tail code word & retrieve previous one
+ * @lock : Pointer to queue spinlock structure
+ * @tail : The new queue tail code word
+ * Return: The previous queue tail code word
+ *
+ * xchg(lock, tail)
+ *
+ * p,*,* -> n,*,* ; prev = xchg(lock, node)
+ */
+static __always_inline u32 xchg_tail(struct qspinlock *lock, u32 tail)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	return (u32)xchg(&l->tail, tail >> _Q_TAIL_OFFSET) << _Q_TAIL_OFFSET;
+}
+
+#else /* _Q_PENDING_BITS == 8 */
+
 /**
  * clear_pending_set_locked - take ownership and clear the pending bit.
  * @lock: Pointer to queue spinlock structure
@@ -143,6 +206,7 @@ static __always_inline u32 xchg_tail(str
 	}
 	return old;
 }
+#endif /* _Q_PENDING_BITS == 8 */
 
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
@@ -217,8 +281,13 @@ void queue_spin_lock_slowpath(struct qsp
 	 * we're pending, wait for the owner to go away.
 	 *
 	 * *,1,1 -> *,1,0
+	 *
+	 * this wait loop must be a load-acquire such that we match the
+	 * store-release that clears the locked bit and create lock
+	 * sequentiality; this is because not all clear_pending_set_locked()
+	 * implementations imply full barriers.
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_MASK)
+	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_MASK)
 		cpu_relax();
 
 	/*

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

* [PATCH 6/9] qspinlock: Use a simple write to grab the lock
  2015-03-16 13:16 ` Peter Zijlstra
@ 2015-03-16 13:16   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: waiman_long-qspinlock-use_a_simple_write_to_grab_the_lock.patch --]
[-- Type: text/plain, Size: 6675 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

Currently, atomic_cmpxchg() is used to get the lock. However, this
is not really necessary if there is more than one task in the queue
and the queue head don't need to reset the tail code. For that case,
a simple write to set the lock bit is enough as the queue head will
be the only one eligible to get the lock as long as it checks that
both the lock and pending bits are not set. The current pending bit
waiting code will ensure that the bit will not be set as soon as the
tail code in the lock is set.

With that change, the are some slight improvement in the performance
of the queue spinlock in the 5M loop micro-benchmark run on a 4-socket
Westere-EX machine as shown in the tables below.

		[Standalone/Embedded - same node]
  # of tasks	Before patch	After patch	%Change
  ----------	-----------	----------	-------
       3	 2324/2321	2248/2265	 -3%/-2%
       4	 2890/2896	2819/2831	 -2%/-2%
       5	 3611/3595	3522/3512	 -2%/-2%
       6	 4281/4276	4173/4160	 -3%/-3%
       7	 5018/5001	4875/4861	 -3%/-3%
       8	 5759/5750	5563/5568	 -3%/-3%

		[Standalone/Embedded - different nodes]
  # of tasks	Before patch	After patch	%Change
  ----------	-----------	----------	-------
       3	12242/12237	12087/12093	 -1%/-1%
       4	10688/10696	10507/10521	 -2%/-2%

It was also found that this change produced a much bigger performance
improvement in the newer IvyBridge-EX chip and was essentially to close
the performance gap between the ticket spinlock and queue spinlock.

The disk workload of the AIM7 benchmark was run on a 4-socket
Westmere-EX machine with both ext4 and xfs RAM disks at 3000 users
on a 3.14 based kernel. The results of the test runs were:

                AIM7 XFS Disk Test
  kernel                 JPM    Real Time   Sys Time    Usr Time
  -----                  ---    ---------   --------    --------
  ticketlock            5678233    3.17       96.61       5.81
  qspinlock             5750799    3.13       94.83       5.97

                AIM7 EXT4 Disk Test
  kernel                 JPM    Real Time   Sys Time    Usr Time
  -----                  ---    ---------   --------    --------
  ticketlock            1114551   16.15      509.72       7.11
  qspinlock             2184466    8.24      232.99       6.01

The ext4 filesystem run had a much higher spinlock contention than
the xfs filesystem run.

The "ebizzy -m" test was also run with the following results:

  kernel               records/s  Real Time   Sys Time    Usr Time
  -----                ---------  ---------   --------    --------
  ticketlock             2075       10.00      216.35       3.49
  qspinlock              3023       10.00      198.20       4.80

Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-7-git-send-email-Waiman.Long@hp.com
---
 kernel/locking/qspinlock.c |   61 +++++++++++++++++++++++++++++++++------------
 1 file changed, 45 insertions(+), 16 deletions(-)

--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -105,24 +105,33 @@ static inline struct mcs_spinlock *decod
  * By using the whole 2nd least significant byte for the pending bit, we
  * can allow better optimization of the lock acquisition for the pending
  * bit holder.
+ *
+ * This internal structure is also used by the set_locked function which
+ * is not restricted to _Q_PENDING_BITS == 8.
  */
-#if _Q_PENDING_BITS == 8
-
 struct __qspinlock {
 	union {
 		atomic_t val;
-		struct {
 #ifdef __LITTLE_ENDIAN
+		u8	 locked;
+		struct {
 			u16	locked_pending;
 			u16	tail;
+		};
 #else
+		struct {
 			u16	tail;
 			u16	locked_pending;
-#endif
 		};
+		struct {
+			u8	reserved[3];
+			u8	locked;
+		};
+#endif
 	};
 };
 
+#if _Q_PENDING_BITS == 8
 /**
  * clear_pending_set_locked - take ownership and clear the pending bit.
  * @lock: Pointer to queue spinlock structure
@@ -209,6 +218,19 @@ static __always_inline u32 xchg_tail(str
 #endif /* _Q_PENDING_BITS == 8 */
 
 /**
+ * set_locked - Set the lock bit and own the lock
+ * @lock: Pointer to queue spinlock structure
+ *
+ * *,*,0 -> *,0,1
+ */
+static __always_inline void set_locked(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	WRITE_ONCE(l->locked, _Q_LOCKED_VAL);
+}
+
+/**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
@@ -343,8 +365,13 @@ void queue_spin_lock_slowpath(struct qsp
 	 * go away.
 	 *
 	 * *,x,y -> *,0,0
+	 *
+	 * this wait loop must use a load-acquire such that we match the
+	 * store-release that clears the locked bit and create lock
+	 * sequentiality; this is because the set_locked() function below
+	 * does not imply a full barrier.
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_PENDING_MASK)
+	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
 	/*
@@ -352,15 +379,19 @@ void queue_spin_lock_slowpath(struct qsp
 	 *
 	 * n,0,0 -> 0,0,1 : lock, uncontended
 	 * *,0,0 -> *,0,1 : lock, contended
+	 *
+	 * If the queue head is the only one in the queue (lock value == tail),
+	 * clear the tail code and grab the lock. Otherwise, we only need
+	 * to grab the lock.
 	 */
 	for (;;) {
-		new = _Q_LOCKED_VAL;
-		if (val != tail)
-			new |= val;
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
+		if (val != tail) {
+			set_locked(lock);
 			break;
+		}
+		old = atomic_cmpxchg(&lock->val, val, _Q_LOCKED_VAL);
+		if (old == val)
+			goto release;	/* No contention */
 
 		val = old;
 	}
@@ -368,12 +399,10 @@ void queue_spin_lock_slowpath(struct qsp
 	/*
 	 * contended path; wait for next, release.
 	 */
-	if (new != _Q_LOCKED_VAL) {
-		while (!(next = READ_ONCE(node->next)))
-			cpu_relax();
+	while (!(next = READ_ONCE(node->next)))
+		cpu_relax();
 
-		arch_mcs_spin_unlock_contended(&next->locked);
-	}
+	arch_mcs_spin_unlock_contended(&next->locked);
 
 release:
 	/*



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

* [PATCH 6/9] qspinlock: Use a simple write to grab the lock
@ 2015-03-16 13:16   ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: waiman_long-qspinlock-use_a_simple_write_to_grab_the_lock.patch --]
[-- Type: text/plain, Size: 6673 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

Currently, atomic_cmpxchg() is used to get the lock. However, this
is not really necessary if there is more than one task in the queue
and the queue head don't need to reset the tail code. For that case,
a simple write to set the lock bit is enough as the queue head will
be the only one eligible to get the lock as long as it checks that
both the lock and pending bits are not set. The current pending bit
waiting code will ensure that the bit will not be set as soon as the
tail code in the lock is set.

With that change, the are some slight improvement in the performance
of the queue spinlock in the 5M loop micro-benchmark run on a 4-socket
Westere-EX machine as shown in the tables below.

		[Standalone/Embedded - same node]
  # of tasks	Before patch	After patch	%Change
  ----------	-----------	----------	-------
       3	 2324/2321	2248/2265	 -3%/-2%
       4	 2890/2896	2819/2831	 -2%/-2%
       5	 3611/3595	3522/3512	 -2%/-2%
       6	 4281/4276	4173/4160	 -3%/-3%
       7	 5018/5001	4875/4861	 -3%/-3%
       8	 5759/5750	5563/5568	 -3%/-3%

		[Standalone/Embedded - different nodes]
  # of tasks	Before patch	After patch	%Change
  ----------	-----------	----------	-------
       3	12242/12237	12087/12093	 -1%/-1%
       4	10688/10696	10507/10521	 -2%/-2%

It was also found that this change produced a much bigger performance
improvement in the newer IvyBridge-EX chip and was essentially to close
the performance gap between the ticket spinlock and queue spinlock.

The disk workload of the AIM7 benchmark was run on a 4-socket
Westmere-EX machine with both ext4 and xfs RAM disks at 3000 users
on a 3.14 based kernel. The results of the test runs were:

                AIM7 XFS Disk Test
  kernel                 JPM    Real Time   Sys Time    Usr Time
  -----                  ---    ---------   --------    --------
  ticketlock            5678233    3.17       96.61       5.81
  qspinlock             5750799    3.13       94.83       5.97

                AIM7 EXT4 Disk Test
  kernel                 JPM    Real Time   Sys Time    Usr Time
  -----                  ---    ---------   --------    --------
  ticketlock            1114551   16.15      509.72       7.11
  qspinlock             2184466    8.24      232.99       6.01

The ext4 filesystem run had a much higher spinlock contention than
the xfs filesystem run.

The "ebizzy -m" test was also run with the following results:

  kernel               records/s  Real Time   Sys Time    Usr Time
  -----                ---------  ---------   --------    --------
  ticketlock             2075       10.00      216.35       3.49
  qspinlock              3023       10.00      198.20       4.80

Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-7-git-send-email-Waiman.Long@hp.com
---
 kernel/locking/qspinlock.c |   61 +++++++++++++++++++++++++++++++++------------
 1 file changed, 45 insertions(+), 16 deletions(-)

--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -105,24 +105,33 @@ static inline struct mcs_spinlock *decod
  * By using the whole 2nd least significant byte for the pending bit, we
  * can allow better optimization of the lock acquisition for the pending
  * bit holder.
+ *
+ * This internal structure is also used by the set_locked function which
+ * is not restricted to _Q_PENDING_BITS == 8.
  */
-#if _Q_PENDING_BITS == 8
-
 struct __qspinlock {
 	union {
 		atomic_t val;
-		struct {
 #ifdef __LITTLE_ENDIAN
+		u8	 locked;
+		struct {
 			u16	locked_pending;
 			u16	tail;
+		};
 #else
+		struct {
 			u16	tail;
 			u16	locked_pending;
-#endif
 		};
+		struct {
+			u8	reserved[3];
+			u8	locked;
+		};
+#endif
 	};
 };
 
+#if _Q_PENDING_BITS == 8
 /**
  * clear_pending_set_locked - take ownership and clear the pending bit.
  * @lock: Pointer to queue spinlock structure
@@ -209,6 +218,19 @@ static __always_inline u32 xchg_tail(str
 #endif /* _Q_PENDING_BITS == 8 */
 
 /**
+ * set_locked - Set the lock bit and own the lock
+ * @lock: Pointer to queue spinlock structure
+ *
+ * *,*,0 -> *,0,1
+ */
+static __always_inline void set_locked(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	WRITE_ONCE(l->locked, _Q_LOCKED_VAL);
+}
+
+/**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
@@ -343,8 +365,13 @@ void queue_spin_lock_slowpath(struct qsp
 	 * go away.
 	 *
 	 * *,x,y -> *,0,0
+	 *
+	 * this wait loop must use a load-acquire such that we match the
+	 * store-release that clears the locked bit and create lock
+	 * sequentiality; this is because the set_locked() function below
+	 * does not imply a full barrier.
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_PENDING_MASK)
+	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
 	/*
@@ -352,15 +379,19 @@ void queue_spin_lock_slowpath(struct qsp
 	 *
 	 * n,0,0 -> 0,0,1 : lock, uncontended
 	 * *,0,0 -> *,0,1 : lock, contended
+	 *
+	 * If the queue head is the only one in the queue (lock value == tail),
+	 * clear the tail code and grab the lock. Otherwise, we only need
+	 * to grab the lock.
 	 */
 	for (;;) {
-		new = _Q_LOCKED_VAL;
-		if (val != tail)
-			new |= val;
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
+		if (val != tail) {
+			set_locked(lock);
 			break;
+		}
+		old = atomic_cmpxchg(&lock->val, val, _Q_LOCKED_VAL);
+		if (old == val)
+			goto release;	/* No contention */
 
 		val = old;
 	}
@@ -368,12 +399,10 @@ void queue_spin_lock_slowpath(struct qsp
 	/*
 	 * contended path; wait for next, release.
 	 */
-	if (new != _Q_LOCKED_VAL) {
-		while (!(next = READ_ONCE(node->next)))
-			cpu_relax();
+	while (!(next = READ_ONCE(node->next)))
+		cpu_relax();
 
-		arch_mcs_spin_unlock_contended(&next->locked);
-	}
+	arch_mcs_spin_unlock_contended(&next->locked);
 
 release:
 	/*

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

* [PATCH 6/9] qspinlock: Use a simple write to grab the lock
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (13 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: waiman_long-qspinlock-use_a_simple_write_to_grab_the_lock.patch --]
[-- Type: text/plain, Size: 6673 bytes --]

From: Waiman Long <Waiman.Long@hp.com>

Currently, atomic_cmpxchg() is used to get the lock. However, this
is not really necessary if there is more than one task in the queue
and the queue head don't need to reset the tail code. For that case,
a simple write to set the lock bit is enough as the queue head will
be the only one eligible to get the lock as long as it checks that
both the lock and pending bits are not set. The current pending bit
waiting code will ensure that the bit will not be set as soon as the
tail code in the lock is set.

With that change, the are some slight improvement in the performance
of the queue spinlock in the 5M loop micro-benchmark run on a 4-socket
Westere-EX machine as shown in the tables below.

		[Standalone/Embedded - same node]
  # of tasks	Before patch	After patch	%Change
  ----------	-----------	----------	-------
       3	 2324/2321	2248/2265	 -3%/-2%
       4	 2890/2896	2819/2831	 -2%/-2%
       5	 3611/3595	3522/3512	 -2%/-2%
       6	 4281/4276	4173/4160	 -3%/-3%
       7	 5018/5001	4875/4861	 -3%/-3%
       8	 5759/5750	5563/5568	 -3%/-3%

		[Standalone/Embedded - different nodes]
  # of tasks	Before patch	After patch	%Change
  ----------	-----------	----------	-------
       3	12242/12237	12087/12093	 -1%/-1%
       4	10688/10696	10507/10521	 -2%/-2%

It was also found that this change produced a much bigger performance
improvement in the newer IvyBridge-EX chip and was essentially to close
the performance gap between the ticket spinlock and queue spinlock.

The disk workload of the AIM7 benchmark was run on a 4-socket
Westmere-EX machine with both ext4 and xfs RAM disks at 3000 users
on a 3.14 based kernel. The results of the test runs were:

                AIM7 XFS Disk Test
  kernel                 JPM    Real Time   Sys Time    Usr Time
  -----                  ---    ---------   --------    --------
  ticketlock            5678233    3.17       96.61       5.81
  qspinlock             5750799    3.13       94.83       5.97

                AIM7 EXT4 Disk Test
  kernel                 JPM    Real Time   Sys Time    Usr Time
  -----                  ---    ---------   --------    --------
  ticketlock            1114551   16.15      509.72       7.11
  qspinlock             2184466    8.24      232.99       6.01

The ext4 filesystem run had a much higher spinlock contention than
the xfs filesystem run.

The "ebizzy -m" test was also run with the following results:

  kernel               records/s  Real Time   Sys Time    Usr Time
  -----                ---------  ---------   --------    --------
  ticketlock             2075       10.00      216.35       3.49
  qspinlock              3023       10.00      198.20       4.80

Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-7-git-send-email-Waiman.Long@hp.com
---
 kernel/locking/qspinlock.c |   61 +++++++++++++++++++++++++++++++++------------
 1 file changed, 45 insertions(+), 16 deletions(-)

--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -105,24 +105,33 @@ static inline struct mcs_spinlock *decod
  * By using the whole 2nd least significant byte for the pending bit, we
  * can allow better optimization of the lock acquisition for the pending
  * bit holder.
+ *
+ * This internal structure is also used by the set_locked function which
+ * is not restricted to _Q_PENDING_BITS == 8.
  */
-#if _Q_PENDING_BITS == 8
-
 struct __qspinlock {
 	union {
 		atomic_t val;
-		struct {
 #ifdef __LITTLE_ENDIAN
+		u8	 locked;
+		struct {
 			u16	locked_pending;
 			u16	tail;
+		};
 #else
+		struct {
 			u16	tail;
 			u16	locked_pending;
-#endif
 		};
+		struct {
+			u8	reserved[3];
+			u8	locked;
+		};
+#endif
 	};
 };
 
+#if _Q_PENDING_BITS == 8
 /**
  * clear_pending_set_locked - take ownership and clear the pending bit.
  * @lock: Pointer to queue spinlock structure
@@ -209,6 +218,19 @@ static __always_inline u32 xchg_tail(str
 #endif /* _Q_PENDING_BITS == 8 */
 
 /**
+ * set_locked - Set the lock bit and own the lock
+ * @lock: Pointer to queue spinlock structure
+ *
+ * *,*,0 -> *,0,1
+ */
+static __always_inline void set_locked(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+
+	WRITE_ONCE(l->locked, _Q_LOCKED_VAL);
+}
+
+/**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
  * @val: Current value of the queue spinlock 32-bit word
@@ -343,8 +365,13 @@ void queue_spin_lock_slowpath(struct qsp
 	 * go away.
 	 *
 	 * *,x,y -> *,0,0
+	 *
+	 * this wait loop must use a load-acquire such that we match the
+	 * store-release that clears the locked bit and create lock
+	 * sequentiality; this is because the set_locked() function below
+	 * does not imply a full barrier.
 	 */
-	while ((val = atomic_read(&lock->val)) & _Q_LOCKED_PENDING_MASK)
+	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
 	/*
@@ -352,15 +379,19 @@ void queue_spin_lock_slowpath(struct qsp
 	 *
 	 * n,0,0 -> 0,0,1 : lock, uncontended
 	 * *,0,0 -> *,0,1 : lock, contended
+	 *
+	 * If the queue head is the only one in the queue (lock value == tail),
+	 * clear the tail code and grab the lock. Otherwise, we only need
+	 * to grab the lock.
 	 */
 	for (;;) {
-		new = _Q_LOCKED_VAL;
-		if (val != tail)
-			new |= val;
-
-		old = atomic_cmpxchg(&lock->val, val, new);
-		if (old == val)
+		if (val != tail) {
+			set_locked(lock);
 			break;
+		}
+		old = atomic_cmpxchg(&lock->val, val, _Q_LOCKED_VAL);
+		if (old == val)
+			goto release;	/* No contention */
 
 		val = old;
 	}
@@ -368,12 +399,10 @@ void queue_spin_lock_slowpath(struct qsp
 	/*
 	 * contended path; wait for next, release.
 	 */
-	if (new != _Q_LOCKED_VAL) {
-		while (!(next = READ_ONCE(node->next)))
-			cpu_relax();
+	while (!(next = READ_ONCE(node->next)))
+		cpu_relax();
 
-		arch_mcs_spin_unlock_contended(&next->locked);
-	}
+	arch_mcs_spin_unlock_contended(&next->locked);
 
 release:
 	/*

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

* [PATCH 7/9] qspinlock: Revert to test-and-set on hypervisors
  2015-03-16 13:16 ` Peter Zijlstra
@ 2015-03-16 13:16   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: peter_zijlstra-qspinlock-revert_to_test-and-set_on_hypervisors.patch --]
[-- Type: text/plain, Size: 2555 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

When we detect a hypervisor (!paravirt, see qspinlock paravirt support
patches), revert to a simple test-and-set lock to avoid the horrors
of queue preemption.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-8-git-send-email-Waiman.Long@hp.com
---
 arch/x86/include/asm/qspinlock.h |   14 ++++++++++++++
 include/asm-generic/qspinlock.h  |    7 +++++++
 kernel/locking/qspinlock.c       |    3 +++
 3 files changed, 24 insertions(+)

--- a/arch/x86/include/asm/qspinlock.h
+++ b/arch/x86/include/asm/qspinlock.h
@@ -1,6 +1,7 @@
 #ifndef _ASM_X86_QSPINLOCK_H
 #define _ASM_X86_QSPINLOCK_H
 
+#include <asm/cpufeature.h>
 #include <asm-generic/qspinlock_types.h>
 
 #define	queue_spin_unlock queue_spin_unlock
@@ -15,6 +16,19 @@ static inline void queue_spin_unlock(str
 	smp_store_release((u8 *)lock, 0);
 }
 
+#define virt_queue_spin_lock virt_queue_spin_lock
+
+static inline bool virt_queue_spin_lock(struct qspinlock *lock)
+{
+	if (!static_cpu_has(X86_FEATURE_HYPERVISOR))
+		return false;
+
+	while (atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL) != 0)
+		cpu_relax();
+
+	return true;
+}
+
 #include <asm-generic/qspinlock.h>
 
 #endif /* _ASM_X86_QSPINLOCK_H */
--- a/include/asm-generic/qspinlock.h
+++ b/include/asm-generic/qspinlock.h
@@ -111,6 +111,13 @@ static inline void queue_spin_unlock_wai
 		cpu_relax();
 }
 
+#ifndef virt_queue_spin_lock
+static __always_inline bool virt_queue_spin_lock(struct qspinlock *lock)
+{
+	return false;
+}
+#endif
+
 /*
  * Initializier
  */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -259,6 +259,9 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	if (virt_queue_spin_lock(lock))
+		return;
+
 	/*
 	 * wait for in-progress pending->locked hand-overs
 	 *



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

* [PATCH 7/9] qspinlock: Revert to test-and-set on hypervisors
@ 2015-03-16 13:16   ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: peter_zijlstra-qspinlock-revert_to_test-and-set_on_hypervisors.patch --]
[-- Type: text/plain, Size: 2553 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

When we detect a hypervisor (!paravirt, see qspinlock paravirt support
patches), revert to a simple test-and-set lock to avoid the horrors
of queue preemption.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-8-git-send-email-Waiman.Long@hp.com
---
 arch/x86/include/asm/qspinlock.h |   14 ++++++++++++++
 include/asm-generic/qspinlock.h  |    7 +++++++
 kernel/locking/qspinlock.c       |    3 +++
 3 files changed, 24 insertions(+)

--- a/arch/x86/include/asm/qspinlock.h
+++ b/arch/x86/include/asm/qspinlock.h
@@ -1,6 +1,7 @@
 #ifndef _ASM_X86_QSPINLOCK_H
 #define _ASM_X86_QSPINLOCK_H
 
+#include <asm/cpufeature.h>
 #include <asm-generic/qspinlock_types.h>
 
 #define	queue_spin_unlock queue_spin_unlock
@@ -15,6 +16,19 @@ static inline void queue_spin_unlock(str
 	smp_store_release((u8 *)lock, 0);
 }
 
+#define virt_queue_spin_lock virt_queue_spin_lock
+
+static inline bool virt_queue_spin_lock(struct qspinlock *lock)
+{
+	if (!static_cpu_has(X86_FEATURE_HYPERVISOR))
+		return false;
+
+	while (atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL) != 0)
+		cpu_relax();
+
+	return true;
+}
+
 #include <asm-generic/qspinlock.h>
 
 #endif /* _ASM_X86_QSPINLOCK_H */
--- a/include/asm-generic/qspinlock.h
+++ b/include/asm-generic/qspinlock.h
@@ -111,6 +111,13 @@ static inline void queue_spin_unlock_wai
 		cpu_relax();
 }
 
+#ifndef virt_queue_spin_lock
+static __always_inline bool virt_queue_spin_lock(struct qspinlock *lock)
+{
+	return false;
+}
+#endif
+
 /*
  * Initializier
  */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -259,6 +259,9 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	if (virt_queue_spin_lock(lock))
+		return;
+
 	/*
 	 * wait for in-progress pending->locked hand-overs
 	 *

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

* [PATCH 7/9] qspinlock: Revert to test-and-set on hypervisors
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (14 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: peter_zijlstra-qspinlock-revert_to_test-and-set_on_hypervisors.patch --]
[-- Type: text/plain, Size: 2553 bytes --]

From: Peter Zijlstra <peterz@infradead.org>

When we detect a hypervisor (!paravirt, see qspinlock paravirt support
patches), revert to a simple test-and-set lock to avoid the horrors
of queue preemption.

Cc: Ingo Molnar <mingo@redhat.com>
Cc: David Vrabel <david.vrabel@citrix.com>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Scott J Norton <scott.norton@hp.com>
Cc: Paolo Bonzini <paolo.bonzini@gmail.com>
Cc: Douglas Hatch <doug.hatch@hp.com>
Cc: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: Rik van Riel <riel@redhat.com>
Cc: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com>
Signed-off-by: Waiman Long <Waiman.Long@hp.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: http://lkml.kernel.org/r/1421784755-21945-8-git-send-email-Waiman.Long@hp.com
---
 arch/x86/include/asm/qspinlock.h |   14 ++++++++++++++
 include/asm-generic/qspinlock.h  |    7 +++++++
 kernel/locking/qspinlock.c       |    3 +++
 3 files changed, 24 insertions(+)

--- a/arch/x86/include/asm/qspinlock.h
+++ b/arch/x86/include/asm/qspinlock.h
@@ -1,6 +1,7 @@
 #ifndef _ASM_X86_QSPINLOCK_H
 #define _ASM_X86_QSPINLOCK_H
 
+#include <asm/cpufeature.h>
 #include <asm-generic/qspinlock_types.h>
 
 #define	queue_spin_unlock queue_spin_unlock
@@ -15,6 +16,19 @@ static inline void queue_spin_unlock(str
 	smp_store_release((u8 *)lock, 0);
 }
 
+#define virt_queue_spin_lock virt_queue_spin_lock
+
+static inline bool virt_queue_spin_lock(struct qspinlock *lock)
+{
+	if (!static_cpu_has(X86_FEATURE_HYPERVISOR))
+		return false;
+
+	while (atomic_cmpxchg(&lock->val, 0, _Q_LOCKED_VAL) != 0)
+		cpu_relax();
+
+	return true;
+}
+
 #include <asm-generic/qspinlock.h>
 
 #endif /* _ASM_X86_QSPINLOCK_H */
--- a/include/asm-generic/qspinlock.h
+++ b/include/asm-generic/qspinlock.h
@@ -111,6 +111,13 @@ static inline void queue_spin_unlock_wai
 		cpu_relax();
 }
 
+#ifndef virt_queue_spin_lock
+static __always_inline bool virt_queue_spin_lock(struct qspinlock *lock)
+{
+	return false;
+}
+#endif
+
 /*
  * Initializier
  */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -259,6 +259,9 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	if (virt_queue_spin_lock(lock))
+		return;
+
 	/*
 	 * wait for in-progress pending->locked hand-overs
 	 *

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

* [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-16 13:16 ` Peter Zijlstra
@ 2015-03-16 13:16   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: peterz-qspinlock-paravirt-crap-again.patch --]
[-- Type: text/plain, Size: 9320 bytes --]

Implement simple paravirt support for the qspinlock.

Provide a separate (second) version of the spin_lock_slowpath for
paravirt along with a special unlock path.

The second slowpath is generated by adding a few pv hooks to the
normal slowpath, but where those will compile away for the native
case, they expand into special wait/wake code for the pv version.

The actual MCS queue can use extra storage in the mcs_nodes[] array to
keep track of state and therefore uses directed wakeups.

The head contender has no such storage available and reverts to the
per-cpu lock entry similar to the current kvm code. We can do a single
enrty because any nesting will wake the vcpu and cause the lower loop
to retry.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 include/asm-generic/qspinlock.h     |    3 
 kernel/locking/qspinlock.c          |   69 +++++++++++++-
 kernel/locking/qspinlock_paravirt.h |  177 ++++++++++++++++++++++++++++++++++++
 3 files changed, 248 insertions(+), 1 deletion(-)

--- a/include/asm-generic/qspinlock.h
+++ b/include/asm-generic/qspinlock.h
@@ -118,6 +118,9 @@ static __always_inline bool virt_queue_s
 }
 #endif
 
+extern void __pv_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+extern void __pv_queue_spin_unlock(struct qspinlock *lock);
+
 /*
  * Initializier
  */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -18,6 +18,9 @@
  * Authors: Waiman Long <waiman.long@hp.com>
  *          Peter Zijlstra <peterz@infradead.org>
  */
+
+#ifndef _GEN_PV_LOCK_SLOWPATH
+
 #include <linux/smp.h>
 #include <linux/bug.h>
 #include <linux/cpumask.h>
@@ -65,13 +68,21 @@
 
 #include "mcs_spinlock.h"
 
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+#define MAX_NODES	8
+#else
+#define MAX_NODES	4
+#endif
+
 /*
  * Per-CPU queue node structures; we can never have more than 4 nested
  * contexts: task, softirq, hardirq, nmi.
  *
  * Exactly fits one 64-byte cacheline on a 64-bit architecture.
+ *
+ * PV doubles the storage and uses the second cacheline for PV state.
  */
-static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[4]);
+static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[MAX_NODES]);
 
 /*
  * We must be able to distinguish between no-tail and the tail at 0:0,
@@ -230,6 +241,32 @@ static __always_inline void set_locked(s
 	WRITE_ONCE(l->locked, _Q_LOCKED_VAL);
 }
 
+
+/*
+ * Generate the native code for queue_spin_unlock_slowpath(); provide NOPs for
+ * all the PV callbacks.
+ */
+
+static __always_inline void __pv_init_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_wait_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_kick_node(struct mcs_spinlock *node) { }
+
+static __always_inline void __pv_wait_head(struct qspinlock *lock) { }
+
+#define pv_enabled()		false
+
+#define pv_init_node		__pv_init_node
+#define pv_wait_node		__pv_wait_node
+#define pv_kick_node		__pv_kick_node
+
+#define pv_wait_head		__pv_wait_head
+
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+#define queue_spin_lock_slowpath	native_queue_spin_lock_slowpath
+#endif
+
+#endif /* _GEN_PV_LOCK_SLOWPATH */
+
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
@@ -259,6 +296,9 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	if (pv_enabled())
+		goto queue;
+
 	if (virt_queue_spin_lock(lock))
 		return;
 
@@ -335,6 +375,7 @@ void queue_spin_lock_slowpath(struct qsp
 	node += idx;
 	node->locked = 0;
 	node->next = NULL;
+	pv_init_node(node);
 
 	/*
 	 * We touched a (possibly) cold cacheline in the per-cpu queue node;
@@ -360,6 +401,7 @@ void queue_spin_lock_slowpath(struct qsp
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);
 
+		pv_wait_node(node);
 		arch_mcs_spin_lock_contended(&node->locked);
 	}
 
@@ -374,6 +416,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 * sequentiality; this is because the set_locked() function below
 	 * does not imply a full barrier.
 	 */
+	pv_wait_head(lock);
 	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
@@ -406,6 +449,7 @@ void queue_spin_lock_slowpath(struct qsp
 		cpu_relax();
 
 	arch_mcs_spin_unlock_contended(&next->locked);
+	pv_kick_node(next);
 
 release:
 	/*
@@ -414,3 +458,26 @@ void queue_spin_lock_slowpath(struct qsp
 	this_cpu_dec(mcs_nodes[0].count);
 }
 EXPORT_SYMBOL(queue_spin_lock_slowpath);
+
+/*
+ * Generate the paravirt code for queue_spin_unlock_slowpath().
+ */
+#if !defined(_GEN_PV_LOCK_SLOWPATH) && defined(CONFIG_PARAVIRT_SPINLOCKS)
+#define _GEN_PV_LOCK_SLOWPATH
+
+#undef pv_enabled
+#define pv_enabled()	true
+
+#undef pv_init_node
+#undef pv_wait_node
+#undef pv_kick_node
+
+#undef pv_wait_head
+
+#undef queue_spin_lock_slowpath
+#define queue_spin_lock_slowpath	__pv_queue_spin_lock_slowpath
+
+#include "qspinlock_paravirt.h"
+#include "qspinlock.c"
+
+#endif
--- /dev/null
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -0,0 +1,177 @@
+#ifndef _GEN_PV_LOCK_SLOWPATH
+#error "do not include this file"
+#endif
+
+/*
+ * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
+ * of spinning them.
+ *
+ * This relies on the architecture to provide two paravirt hypercalls:
+ *
+ *   pv_wait(u8 *ptr, u8 val) -- suspends the vcpu if *ptr == val
+ *   pv_kick(cpu)             -- wakes a suspended vcpu
+ *
+ * Using these we implement __pv_queue_spin_lock_slowpath() and
+ * __pv_queue_spin_unlock() to replace native_queue_spin_lock_slowpath() and
+ * native_queue_spin_unlock().
+ */
+
+#define _Q_SLOW_VAL	(2U << _Q_LOCKED_OFFSET)
+
+enum vcpu_state {
+	vcpu_running = 0,
+	vcpu_halted,
+};
+
+struct pv_node {
+	struct mcs_spinlock	mcs;
+	struct mcs_spinlock	__res[3];
+
+	int			cpu;
+	u8			state;
+};
+
+/*
+ * Initialize the PV part of the mcs_spinlock node.
+ */
+static void pv_init_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+
+	BUILD_BUG_ON(sizeof(struct pv_node) > 5*sizeof(struct mcs_spinlock));
+
+	pn->cpu = smp_processor_id();
+	pn->state = vcpu_running;
+}
+
+/*
+ * Wait for node->locked to become true, halt the vcpu after a short spin.
+ * pv_kick_node() is used to wake the vcpu again.
+ */
+static void pv_wait_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+	int loop;
+
+	for (;;) {
+		for (loop = SPIN_THRESHOLD; loop; loop--) {
+			if (READ_ONCE(node->locked))
+				goto done;
+
+			cpu_relax();
+		}
+
+		/*
+		 * Order pn->state vs pn->locked thusly:
+		 *
+		 * [S] pn->state = vcpu_halted	  [S] next->locked = 1
+		 *     MB			      MB
+		 * [L] pn->locked		[RmW] pn->state = vcpu_running
+		 *
+		 * Matches the xchg() from pv_kick_node().
+		 */
+		(void)xchg(&pn->state, vcpu_halted);
+
+		if (READ_ONCE(node->locked))
+			goto done;
+
+		pv_wait(&pn->state, vcpu_halted);
+	}
+done:
+	pn->state = vcpu_running;
+
+	/*
+	 * By now our node->locked should be 1 and our caller will not actually
+	 * spin-wait for it. We do however rely on our caller to do a
+	 * load-acquire for us.
+	 */
+}
+
+/*
+ * Called after setting next->locked = 1, used to wake those stuck in
+ * pv_wait_node().
+ */
+static void pv_kick_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+
+	/*
+	 * Note that because node->locked is already set, this actual mcs_spinlock
+	 * entry could be re-used already.
+	 *
+	 * This should be fine however, kicking people for no reason is harmless.
+	 *
+	 * See the comment in pv_wait_node().
+	 */
+	if (xchg(&pn->state, vcpu_running) == vcpu_halted)
+		pv_kick(pn->cpu);
+}
+
+static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+
+/*
+ * Wait for l->locked to become clear; halt the vcpu after a short spin.
+ * __pv_queue_spin_unlock() will wake us.
+ */
+static void pv_wait_head(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+	int loop;
+
+	for (;;) {
+		for (loop = SPIN_THRESHOLD; loop; loop--) {
+			if (!READ_ONCE(l->locked))
+				goto done;
+
+			cpu_relax();
+		}
+
+		this_cpu_write(__pv_lock_wait, lock);
+		/*
+		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
+		 *
+		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
+		 *     MB                             MB
+		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+		 *
+		 * Matches the xchg() in pv_queue_spin_unlock().
+		 */
+		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
+			goto done;
+
+		pv_wait(&l->locked, _Q_SLOW_VAL);
+	}
+done:
+	this_cpu_write(__pv_lock_wait, NULL);
+
+	/*
+	 * Lock is unlocked now; the caller will acquire it without waiting.
+	 * As with pv_wait_node() we rely on the caller to do a load-acquire
+	 * for us.
+	 */
+}
+
+/*
+ * To be used in stead of queue_spin_unlock() for paravirt locks. Wakes
+ * pv_wait_head() if appropriate.
+ */
+void __pv_queue_spin_unlock(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+	int cpu;
+
+	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
+		return;
+
+	/*
+	 * At this point the memory pointed at by lock can be freed/reused,
+	 * however we can still use the pointer value to search in our cpu
+	 * array.
+	 *
+	 * XXX: get rid of this loop
+	 */
+	for_each_possible_cpu(cpu) {
+		if (per_cpu(__pv_lock_wait, cpu) == lock)
+			pv_kick(cpu);
+	}
+}



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

* [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-03-16 13:16   ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: peterz-qspinlock-paravirt-crap-again.patch --]
[-- Type: text/plain, Size: 9318 bytes --]

Implement simple paravirt support for the qspinlock.

Provide a separate (second) version of the spin_lock_slowpath for
paravirt along with a special unlock path.

The second slowpath is generated by adding a few pv hooks to the
normal slowpath, but where those will compile away for the native
case, they expand into special wait/wake code for the pv version.

The actual MCS queue can use extra storage in the mcs_nodes[] array to
keep track of state and therefore uses directed wakeups.

The head contender has no such storage available and reverts to the
per-cpu lock entry similar to the current kvm code. We can do a single
enrty because any nesting will wake the vcpu and cause the lower loop
to retry.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 include/asm-generic/qspinlock.h     |    3 
 kernel/locking/qspinlock.c          |   69 +++++++++++++-
 kernel/locking/qspinlock_paravirt.h |  177 ++++++++++++++++++++++++++++++++++++
 3 files changed, 248 insertions(+), 1 deletion(-)

--- a/include/asm-generic/qspinlock.h
+++ b/include/asm-generic/qspinlock.h
@@ -118,6 +118,9 @@ static __always_inline bool virt_queue_s
 }
 #endif
 
+extern void __pv_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+extern void __pv_queue_spin_unlock(struct qspinlock *lock);
+
 /*
  * Initializier
  */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -18,6 +18,9 @@
  * Authors: Waiman Long <waiman.long@hp.com>
  *          Peter Zijlstra <peterz@infradead.org>
  */
+
+#ifndef _GEN_PV_LOCK_SLOWPATH
+
 #include <linux/smp.h>
 #include <linux/bug.h>
 #include <linux/cpumask.h>
@@ -65,13 +68,21 @@
 
 #include "mcs_spinlock.h"
 
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+#define MAX_NODES	8
+#else
+#define MAX_NODES	4
+#endif
+
 /*
  * Per-CPU queue node structures; we can never have more than 4 nested
  * contexts: task, softirq, hardirq, nmi.
  *
  * Exactly fits one 64-byte cacheline on a 64-bit architecture.
+ *
+ * PV doubles the storage and uses the second cacheline for PV state.
  */
-static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[4]);
+static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[MAX_NODES]);
 
 /*
  * We must be able to distinguish between no-tail and the tail at 0:0,
@@ -230,6 +241,32 @@ static __always_inline void set_locked(s
 	WRITE_ONCE(l->locked, _Q_LOCKED_VAL);
 }
 
+
+/*
+ * Generate the native code for queue_spin_unlock_slowpath(); provide NOPs for
+ * all the PV callbacks.
+ */
+
+static __always_inline void __pv_init_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_wait_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_kick_node(struct mcs_spinlock *node) { }
+
+static __always_inline void __pv_wait_head(struct qspinlock *lock) { }
+
+#define pv_enabled()		false
+
+#define pv_init_node		__pv_init_node
+#define pv_wait_node		__pv_wait_node
+#define pv_kick_node		__pv_kick_node
+
+#define pv_wait_head		__pv_wait_head
+
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+#define queue_spin_lock_slowpath	native_queue_spin_lock_slowpath
+#endif
+
+#endif /* _GEN_PV_LOCK_SLOWPATH */
+
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
@@ -259,6 +296,9 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	if (pv_enabled())
+		goto queue;
+
 	if (virt_queue_spin_lock(lock))
 		return;
 
@@ -335,6 +375,7 @@ void queue_spin_lock_slowpath(struct qsp
 	node += idx;
 	node->locked = 0;
 	node->next = NULL;
+	pv_init_node(node);
 
 	/*
 	 * We touched a (possibly) cold cacheline in the per-cpu queue node;
@@ -360,6 +401,7 @@ void queue_spin_lock_slowpath(struct qsp
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);
 
+		pv_wait_node(node);
 		arch_mcs_spin_lock_contended(&node->locked);
 	}
 
@@ -374,6 +416,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 * sequentiality; this is because the set_locked() function below
 	 * does not imply a full barrier.
 	 */
+	pv_wait_head(lock);
 	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
@@ -406,6 +449,7 @@ void queue_spin_lock_slowpath(struct qsp
 		cpu_relax();
 
 	arch_mcs_spin_unlock_contended(&next->locked);
+	pv_kick_node(next);
 
 release:
 	/*
@@ -414,3 +458,26 @@ void queue_spin_lock_slowpath(struct qsp
 	this_cpu_dec(mcs_nodes[0].count);
 }
 EXPORT_SYMBOL(queue_spin_lock_slowpath);
+
+/*
+ * Generate the paravirt code for queue_spin_unlock_slowpath().
+ */
+#if !defined(_GEN_PV_LOCK_SLOWPATH) && defined(CONFIG_PARAVIRT_SPINLOCKS)
+#define _GEN_PV_LOCK_SLOWPATH
+
+#undef pv_enabled
+#define pv_enabled()	true
+
+#undef pv_init_node
+#undef pv_wait_node
+#undef pv_kick_node
+
+#undef pv_wait_head
+
+#undef queue_spin_lock_slowpath
+#define queue_spin_lock_slowpath	__pv_queue_spin_lock_slowpath
+
+#include "qspinlock_paravirt.h"
+#include "qspinlock.c"
+
+#endif
--- /dev/null
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -0,0 +1,177 @@
+#ifndef _GEN_PV_LOCK_SLOWPATH
+#error "do not include this file"
+#endif
+
+/*
+ * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
+ * of spinning them.
+ *
+ * This relies on the architecture to provide two paravirt hypercalls:
+ *
+ *   pv_wait(u8 *ptr, u8 val) -- suspends the vcpu if *ptr == val
+ *   pv_kick(cpu)             -- wakes a suspended vcpu
+ *
+ * Using these we implement __pv_queue_spin_lock_slowpath() and
+ * __pv_queue_spin_unlock() to replace native_queue_spin_lock_slowpath() and
+ * native_queue_spin_unlock().
+ */
+
+#define _Q_SLOW_VAL	(2U << _Q_LOCKED_OFFSET)
+
+enum vcpu_state {
+	vcpu_running = 0,
+	vcpu_halted,
+};
+
+struct pv_node {
+	struct mcs_spinlock	mcs;
+	struct mcs_spinlock	__res[3];
+
+	int			cpu;
+	u8			state;
+};
+
+/*
+ * Initialize the PV part of the mcs_spinlock node.
+ */
+static void pv_init_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+
+	BUILD_BUG_ON(sizeof(struct pv_node) > 5*sizeof(struct mcs_spinlock));
+
+	pn->cpu = smp_processor_id();
+	pn->state = vcpu_running;
+}
+
+/*
+ * Wait for node->locked to become true, halt the vcpu after a short spin.
+ * pv_kick_node() is used to wake the vcpu again.
+ */
+static void pv_wait_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+	int loop;
+
+	for (;;) {
+		for (loop = SPIN_THRESHOLD; loop; loop--) {
+			if (READ_ONCE(node->locked))
+				goto done;
+
+			cpu_relax();
+		}
+
+		/*
+		 * Order pn->state vs pn->locked thusly:
+		 *
+		 * [S] pn->state = vcpu_halted	  [S] next->locked = 1
+		 *     MB			      MB
+		 * [L] pn->locked		[RmW] pn->state = vcpu_running
+		 *
+		 * Matches the xchg() from pv_kick_node().
+		 */
+		(void)xchg(&pn->state, vcpu_halted);
+
+		if (READ_ONCE(node->locked))
+			goto done;
+
+		pv_wait(&pn->state, vcpu_halted);
+	}
+done:
+	pn->state = vcpu_running;
+
+	/*
+	 * By now our node->locked should be 1 and our caller will not actually
+	 * spin-wait for it. We do however rely on our caller to do a
+	 * load-acquire for us.
+	 */
+}
+
+/*
+ * Called after setting next->locked = 1, used to wake those stuck in
+ * pv_wait_node().
+ */
+static void pv_kick_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+
+	/*
+	 * Note that because node->locked is already set, this actual mcs_spinlock
+	 * entry could be re-used already.
+	 *
+	 * This should be fine however, kicking people for no reason is harmless.
+	 *
+	 * See the comment in pv_wait_node().
+	 */
+	if (xchg(&pn->state, vcpu_running) == vcpu_halted)
+		pv_kick(pn->cpu);
+}
+
+static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+
+/*
+ * Wait for l->locked to become clear; halt the vcpu after a short spin.
+ * __pv_queue_spin_unlock() will wake us.
+ */
+static void pv_wait_head(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+	int loop;
+
+	for (;;) {
+		for (loop = SPIN_THRESHOLD; loop; loop--) {
+			if (!READ_ONCE(l->locked))
+				goto done;
+
+			cpu_relax();
+		}
+
+		this_cpu_write(__pv_lock_wait, lock);
+		/*
+		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
+		 *
+		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
+		 *     MB                             MB
+		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+		 *
+		 * Matches the xchg() in pv_queue_spin_unlock().
+		 */
+		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
+			goto done;
+
+		pv_wait(&l->locked, _Q_SLOW_VAL);
+	}
+done:
+	this_cpu_write(__pv_lock_wait, NULL);
+
+	/*
+	 * Lock is unlocked now; the caller will acquire it without waiting.
+	 * As with pv_wait_node() we rely on the caller to do a load-acquire
+	 * for us.
+	 */
+}
+
+/*
+ * To be used in stead of queue_spin_unlock() for paravirt locks. Wakes
+ * pv_wait_head() if appropriate.
+ */
+void __pv_queue_spin_unlock(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+	int cpu;
+
+	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
+		return;
+
+	/*
+	 * At this point the memory pointed at by lock can be freed/reused,
+	 * however we can still use the pointer value to search in our cpu
+	 * array.
+	 *
+	 * XXX: get rid of this loop
+	 */
+	for_each_possible_cpu(cpu) {
+		if (per_cpu(__pv_lock_wait, cpu) == lock)
+			pv_kick(cpu);
+	}
+}

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

* [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (17 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: peterz-qspinlock-paravirt-crap-again.patch --]
[-- Type: text/plain, Size: 9318 bytes --]

Implement simple paravirt support for the qspinlock.

Provide a separate (second) version of the spin_lock_slowpath for
paravirt along with a special unlock path.

The second slowpath is generated by adding a few pv hooks to the
normal slowpath, but where those will compile away for the native
case, they expand into special wait/wake code for the pv version.

The actual MCS queue can use extra storage in the mcs_nodes[] array to
keep track of state and therefore uses directed wakeups.

The head contender has no such storage available and reverts to the
per-cpu lock entry similar to the current kvm code. We can do a single
enrty because any nesting will wake the vcpu and cause the lower loop
to retry.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 include/asm-generic/qspinlock.h     |    3 
 kernel/locking/qspinlock.c          |   69 +++++++++++++-
 kernel/locking/qspinlock_paravirt.h |  177 ++++++++++++++++++++++++++++++++++++
 3 files changed, 248 insertions(+), 1 deletion(-)

--- a/include/asm-generic/qspinlock.h
+++ b/include/asm-generic/qspinlock.h
@@ -118,6 +118,9 @@ static __always_inline bool virt_queue_s
 }
 #endif
 
+extern void __pv_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+extern void __pv_queue_spin_unlock(struct qspinlock *lock);
+
 /*
  * Initializier
  */
--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -18,6 +18,9 @@
  * Authors: Waiman Long <waiman.long@hp.com>
  *          Peter Zijlstra <peterz@infradead.org>
  */
+
+#ifndef _GEN_PV_LOCK_SLOWPATH
+
 #include <linux/smp.h>
 #include <linux/bug.h>
 #include <linux/cpumask.h>
@@ -65,13 +68,21 @@
 
 #include "mcs_spinlock.h"
 
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+#define MAX_NODES	8
+#else
+#define MAX_NODES	4
+#endif
+
 /*
  * Per-CPU queue node structures; we can never have more than 4 nested
  * contexts: task, softirq, hardirq, nmi.
  *
  * Exactly fits one 64-byte cacheline on a 64-bit architecture.
+ *
+ * PV doubles the storage and uses the second cacheline for PV state.
  */
-static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[4]);
+static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[MAX_NODES]);
 
 /*
  * We must be able to distinguish between no-tail and the tail at 0:0,
@@ -230,6 +241,32 @@ static __always_inline void set_locked(s
 	WRITE_ONCE(l->locked, _Q_LOCKED_VAL);
 }
 
+
+/*
+ * Generate the native code for queue_spin_unlock_slowpath(); provide NOPs for
+ * all the PV callbacks.
+ */
+
+static __always_inline void __pv_init_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_wait_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_kick_node(struct mcs_spinlock *node) { }
+
+static __always_inline void __pv_wait_head(struct qspinlock *lock) { }
+
+#define pv_enabled()		false
+
+#define pv_init_node		__pv_init_node
+#define pv_wait_node		__pv_wait_node
+#define pv_kick_node		__pv_kick_node
+
+#define pv_wait_head		__pv_wait_head
+
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+#define queue_spin_lock_slowpath	native_queue_spin_lock_slowpath
+#endif
+
+#endif /* _GEN_PV_LOCK_SLOWPATH */
+
 /**
  * queue_spin_lock_slowpath - acquire the queue spinlock
  * @lock: Pointer to queue spinlock structure
@@ -259,6 +296,9 @@ void queue_spin_lock_slowpath(struct qsp
 
 	BUILD_BUG_ON(CONFIG_NR_CPUS >= (1U << _Q_TAIL_CPU_BITS));
 
+	if (pv_enabled())
+		goto queue;
+
 	if (virt_queue_spin_lock(lock))
 		return;
 
@@ -335,6 +375,7 @@ void queue_spin_lock_slowpath(struct qsp
 	node += idx;
 	node->locked = 0;
 	node->next = NULL;
+	pv_init_node(node);
 
 	/*
 	 * We touched a (possibly) cold cacheline in the per-cpu queue node;
@@ -360,6 +401,7 @@ void queue_spin_lock_slowpath(struct qsp
 		prev = decode_tail(old);
 		WRITE_ONCE(prev->next, node);
 
+		pv_wait_node(node);
 		arch_mcs_spin_lock_contended(&node->locked);
 	}
 
@@ -374,6 +416,7 @@ void queue_spin_lock_slowpath(struct qsp
 	 * sequentiality; this is because the set_locked() function below
 	 * does not imply a full barrier.
 	 */
+	pv_wait_head(lock);
 	while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
 		cpu_relax();
 
@@ -406,6 +449,7 @@ void queue_spin_lock_slowpath(struct qsp
 		cpu_relax();
 
 	arch_mcs_spin_unlock_contended(&next->locked);
+	pv_kick_node(next);
 
 release:
 	/*
@@ -414,3 +458,26 @@ void queue_spin_lock_slowpath(struct qsp
 	this_cpu_dec(mcs_nodes[0].count);
 }
 EXPORT_SYMBOL(queue_spin_lock_slowpath);
+
+/*
+ * Generate the paravirt code for queue_spin_unlock_slowpath().
+ */
+#if !defined(_GEN_PV_LOCK_SLOWPATH) && defined(CONFIG_PARAVIRT_SPINLOCKS)
+#define _GEN_PV_LOCK_SLOWPATH
+
+#undef pv_enabled
+#define pv_enabled()	true
+
+#undef pv_init_node
+#undef pv_wait_node
+#undef pv_kick_node
+
+#undef pv_wait_head
+
+#undef queue_spin_lock_slowpath
+#define queue_spin_lock_slowpath	__pv_queue_spin_lock_slowpath
+
+#include "qspinlock_paravirt.h"
+#include "qspinlock.c"
+
+#endif
--- /dev/null
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -0,0 +1,177 @@
+#ifndef _GEN_PV_LOCK_SLOWPATH
+#error "do not include this file"
+#endif
+
+/*
+ * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
+ * of spinning them.
+ *
+ * This relies on the architecture to provide two paravirt hypercalls:
+ *
+ *   pv_wait(u8 *ptr, u8 val) -- suspends the vcpu if *ptr == val
+ *   pv_kick(cpu)             -- wakes a suspended vcpu
+ *
+ * Using these we implement __pv_queue_spin_lock_slowpath() and
+ * __pv_queue_spin_unlock() to replace native_queue_spin_lock_slowpath() and
+ * native_queue_spin_unlock().
+ */
+
+#define _Q_SLOW_VAL	(2U << _Q_LOCKED_OFFSET)
+
+enum vcpu_state {
+	vcpu_running = 0,
+	vcpu_halted,
+};
+
+struct pv_node {
+	struct mcs_spinlock	mcs;
+	struct mcs_spinlock	__res[3];
+
+	int			cpu;
+	u8			state;
+};
+
+/*
+ * Initialize the PV part of the mcs_spinlock node.
+ */
+static void pv_init_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+
+	BUILD_BUG_ON(sizeof(struct pv_node) > 5*sizeof(struct mcs_spinlock));
+
+	pn->cpu = smp_processor_id();
+	pn->state = vcpu_running;
+}
+
+/*
+ * Wait for node->locked to become true, halt the vcpu after a short spin.
+ * pv_kick_node() is used to wake the vcpu again.
+ */
+static void pv_wait_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+	int loop;
+
+	for (;;) {
+		for (loop = SPIN_THRESHOLD; loop; loop--) {
+			if (READ_ONCE(node->locked))
+				goto done;
+
+			cpu_relax();
+		}
+
+		/*
+		 * Order pn->state vs pn->locked thusly:
+		 *
+		 * [S] pn->state = vcpu_halted	  [S] next->locked = 1
+		 *     MB			      MB
+		 * [L] pn->locked		[RmW] pn->state = vcpu_running
+		 *
+		 * Matches the xchg() from pv_kick_node().
+		 */
+		(void)xchg(&pn->state, vcpu_halted);
+
+		if (READ_ONCE(node->locked))
+			goto done;
+
+		pv_wait(&pn->state, vcpu_halted);
+	}
+done:
+	pn->state = vcpu_running;
+
+	/*
+	 * By now our node->locked should be 1 and our caller will not actually
+	 * spin-wait for it. We do however rely on our caller to do a
+	 * load-acquire for us.
+	 */
+}
+
+/*
+ * Called after setting next->locked = 1, used to wake those stuck in
+ * pv_wait_node().
+ */
+static void pv_kick_node(struct mcs_spinlock *node)
+{
+	struct pv_node *pn = (struct pv_node *)node;
+
+	/*
+	 * Note that because node->locked is already set, this actual mcs_spinlock
+	 * entry could be re-used already.
+	 *
+	 * This should be fine however, kicking people for no reason is harmless.
+	 *
+	 * See the comment in pv_wait_node().
+	 */
+	if (xchg(&pn->state, vcpu_running) == vcpu_halted)
+		pv_kick(pn->cpu);
+}
+
+static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+
+/*
+ * Wait for l->locked to become clear; halt the vcpu after a short spin.
+ * __pv_queue_spin_unlock() will wake us.
+ */
+static void pv_wait_head(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+	int loop;
+
+	for (;;) {
+		for (loop = SPIN_THRESHOLD; loop; loop--) {
+			if (!READ_ONCE(l->locked))
+				goto done;
+
+			cpu_relax();
+		}
+
+		this_cpu_write(__pv_lock_wait, lock);
+		/*
+		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
+		 *
+		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
+		 *     MB                             MB
+		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+		 *
+		 * Matches the xchg() in pv_queue_spin_unlock().
+		 */
+		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
+			goto done;
+
+		pv_wait(&l->locked, _Q_SLOW_VAL);
+	}
+done:
+	this_cpu_write(__pv_lock_wait, NULL);
+
+	/*
+	 * Lock is unlocked now; the caller will acquire it without waiting.
+	 * As with pv_wait_node() we rely on the caller to do a load-acquire
+	 * for us.
+	 */
+}
+
+/*
+ * To be used in stead of queue_spin_unlock() for paravirt locks. Wakes
+ * pv_wait_head() if appropriate.
+ */
+void __pv_queue_spin_unlock(struct qspinlock *lock)
+{
+	struct __qspinlock *l = (void *)lock;
+	int cpu;
+
+	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
+		return;
+
+	/*
+	 * At this point the memory pointed at by lock can be freed/reused,
+	 * however we can still use the pointer value to search in our cpu
+	 * array.
+	 *
+	 * XXX: get rid of this loop
+	 */
+	for_each_possible_cpu(cpu) {
+		if (per_cpu(__pv_lock_wait, cpu) == lock)
+			pv_kick(cpu);
+	}
+}

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

* [PATCH 9/9] qspinlock,x86,kvm: Implement KVM support for paravirt qspinlock
  2015-03-16 13:16 ` Peter Zijlstra
@ 2015-03-16 13:16   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: tglx, mingo, hpa, peterz, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

[-- Attachment #1: peterz-qspinlock-paravirt-x86-crap.patch --]
[-- Type: text/plain, Size: 10535 bytes --]

Implement the paravirt qspinlock for x86-kvm.

We use the regular paravirt call patching to switch between:

  native_queue_spin_lock_slowpath()	__pv_queue_spin_lock_slowpath()
  native_queue_spin_unlock()		__pv_queue_spin_unlock()

We use a callee saved call for the unlock function which reduces the
i-cache footprint and allows 'inlining' of SPIN_UNLOCK functions
again.

We further optimize the unlock path by patching the direct call with a
"movb $0,%arg1" if we are indeed using the native unlock code. This
makes the unlock code almost as fast as the !PARAVIRT case.

This significantly lowers the overhead of having
CONFIG_PARAVIRT_SPINLOCKS enabled, even for native code.


Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 arch/x86/Kconfig                      |    2 -
 arch/x86/include/asm/paravirt.h       |   28 ++++++++++++++++++++-
 arch/x86/include/asm/paravirt_types.h |   10 +++++++
 arch/x86/include/asm/qspinlock.h      |   22 ++++++++++++++++-
 arch/x86/kernel/kvm.c                 |   44 ++++++++++++++++++++++++++++++++++
 arch/x86/kernel/paravirt-spinlocks.c  |   24 +++++++++++++++++-
 arch/x86/kernel/paravirt_patch_32.c   |   22 +++++++++++++----
 arch/x86/kernel/paravirt_patch_64.c   |   22 +++++++++++++----
 kernel/Kconfig.locks                  |    2 -
 9 files changed, 163 insertions(+), 13 deletions(-)

--- a/arch/x86/Kconfig
+++ b/arch/x86/Kconfig
@@ -692,7 +692,7 @@ config PARAVIRT_DEBUG
 config PARAVIRT_SPINLOCKS
 	bool "Paravirtualization layer for spinlocks"
 	depends on PARAVIRT && SMP
-	select UNINLINE_SPIN_UNLOCK
+	select UNINLINE_SPIN_UNLOCK if !QUEUE_SPINLOCK
 	---help---
 	  Paravirtualized spinlocks allow a pvops backend to replace the
 	  spinlock implementation with something virtualization-friendly
--- a/arch/x86/include/asm/paravirt.h
+++ b/arch/x86/include/asm/paravirt.h
@@ -712,6 +712,30 @@ static inline void __set_fixmap(unsigned
 
 #if defined(CONFIG_SMP) && defined(CONFIG_PARAVIRT_SPINLOCKS)
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+static __always_inline void pv_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	PVOP_VCALL2(pv_lock_ops.queue_spin_lock_slowpath, lock, val);
+}
+
+static __always_inline void pv_queue_spin_unlock(struct qspinlock *lock)
+{
+	PVOP_VCALLEE1(pv_lock_ops.queue_spin_unlock, lock);
+}
+
+static __always_inline void pv_wait(u8 *ptr, u8 val)
+{
+	PVOP_VCALL2(pv_lock_ops.wait, ptr, val);
+}
+
+static __always_inline void pv_kick(int cpu)
+{
+	PVOP_VCALL1(pv_lock_ops.kick, cpu);
+}
+
+#else /* !CONFIG_QUEUE_SPINLOCK */
+
 static __always_inline void __ticket_lock_spinning(struct arch_spinlock *lock,
 							__ticket_t ticket)
 {
@@ -724,7 +748,9 @@ static __always_inline void __ticket_unl
 	PVOP_VCALL2(pv_lock_ops.unlock_kick, lock, ticket);
 }
 
-#endif
+#endif /* CONFIG_QUEUE_SPINLOCK */
+
+#endif /* SMP && PARAVIRT_SPINLOCKS */
 
 #ifdef CONFIG_X86_32
 #define PV_SAVE_REGS "pushl %ecx; pushl %edx;"
--- a/arch/x86/include/asm/paravirt_types.h
+++ b/arch/x86/include/asm/paravirt_types.h
@@ -333,9 +333,19 @@ struct arch_spinlock;
 typedef u16 __ticket_t;
 #endif
 
+struct qspinlock;
+
 struct pv_lock_ops {
+#ifdef CONFIG_QUEUE_SPINLOCK
+	void (*queue_spin_lock_slowpath)(struct qspinlock *lock, u32 val);
+	struct paravirt_callee_save queue_spin_unlock;
+
+	void (*wait)(u8 *ptr, u8 val);
+	void (*kick)(int cpu);
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	struct paravirt_callee_save lock_spinning;
 	void (*unlock_kick)(struct arch_spinlock *lock, __ticket_t ticket);
+#endif /* !CONFIG_QUEUE_SPINLOCK */
 };
 
 /* This contains all the paravirt structures: we get a convenient
--- a/arch/x86/include/asm/qspinlock.h
+++ b/arch/x86/include/asm/qspinlock.h
@@ -3,6 +3,7 @@
 
 #include <asm/cpufeature.h>
 #include <asm-generic/qspinlock_types.h>
+#include <asm/paravirt.h>
 
 #define	queue_spin_unlock queue_spin_unlock
 /**
@@ -11,11 +12,30 @@
  *
  * An smp_store_release() on the least-significant byte.
  */
-static inline void queue_spin_unlock(struct qspinlock *lock)
+static inline void native_queue_spin_unlock(struct qspinlock *lock)
 {
 	smp_store_release((u8 *)lock, 0);
 }
 
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+extern void native_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+
+static inline void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	pv_queue_spin_lock_slowpath(lock, val);
+}
+
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	pv_queue_spin_unlock(lock);
+}
+#else
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	native_queue_spin_unlock(lock);
+}
+#endif
+
 #define virt_queue_spin_lock virt_queue_spin_lock
 
 static inline bool virt_queue_spin_lock(struct qspinlock *lock)
--- a/arch/x86/kernel/kvm.c
+++ b/arch/x86/kernel/kvm.c
@@ -584,6 +584,41 @@ static void kvm_kick_cpu(int cpu)
 	kvm_hypercall2(KVM_HC_KICK_CPU, flags, apicid);
 }
 
+
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+#include <asm/qspinlock.h>
+
+PV_CALLEE_SAVE_REGS_THUNK(__pv_queue_spin_unlock);
+
+static void kvm_wait(u8 *ptr, u8 val)
+{
+	unsigned long flags;
+
+	if (in_nmi())
+		return;
+
+	local_irq_save(flags);
+
+	if (READ_ONCE(*ptr) != val)
+		goto out;
+
+	/*
+	 * halt until it's our turn and kicked. Note that we do safe halt
+	 * for irq enabled case to avoid hang when lock info is overwritten
+	 * in irq spinlock slowpath and no spurious interrupt occur to save us.
+	 */
+	if (arch_irqs_disabled_flags(flags))
+		halt();
+	else
+		safe_halt();
+
+out:
+	local_irq_restore(flags);
+}
+
+#else /* !CONFIG_QUEUE_SPINLOCK */
+
 enum kvm_contention_stat {
 	TAKEN_SLOW,
 	TAKEN_SLOW_PICKUP,
@@ -817,6 +852,8 @@ static void kvm_unlock_kick(struct arch_
 	}
 }
 
+#endif /* !CONFIG_QUEUE_SPINLOCK */
+
 /*
  * Setup pv_lock_ops to exploit KVM_FEATURE_PV_UNHALT if present.
  */
@@ -828,8 +865,15 @@ void __init kvm_spinlock_init(void)
 	if (!kvm_para_has_feature(KVM_FEATURE_PV_UNHALT))
 		return;
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+	pv_lock_ops.queue_spin_lock_slowpath = __pv_queue_spin_lock_slowpath;
+	pv_lock_ops.queue_spin_unlock = PV_CALLEE_SAVE(__pv_queue_spin_unlock);
+	pv_lock_ops.wait = kvm_wait;
+	pv_lock_ops.kick = kvm_kick_cpu;
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	pv_lock_ops.lock_spinning = PV_CALLEE_SAVE(kvm_lock_spinning);
 	pv_lock_ops.unlock_kick = kvm_unlock_kick;
+#endif
 }
 
 static __init int kvm_spinlock_init_jump(void)
--- a/arch/x86/kernel/paravirt-spinlocks.c
+++ b/arch/x86/kernel/paravirt-spinlocks.c
@@ -8,11 +8,33 @@
 
 #include <asm/paravirt.h>
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+__visible void __native_queue_spin_unlock(struct qspinlock *lock)
+{
+	native_queue_spin_unlock(lock);
+}
+
+PV_CALLEE_SAVE_REGS_THUNK(__native_queue_spin_unlock);
+
+bool pv_is_native_spin_unlock(void)
+{
+	return pv_lock_ops.queue_spin_unlock.func ==
+		__raw_callee_save___native_queue_spin_unlock;
+}
+#endif
+
 struct pv_lock_ops pv_lock_ops = {
 #ifdef CONFIG_SMP
+#ifdef CONFIG_QUEUE_SPINLOCK
+	.queue_spin_lock_slowpath = native_queue_spin_lock_slowpath,
+	.queue_spin_unlock = PV_CALLEE_SAVE(__native_queue_spin_unlock),
+	.wait = paravirt_nop,
+	.kick = paravirt_nop,
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	.lock_spinning = __PV_IS_CALLEE_SAVE(paravirt_nop),
 	.unlock_kick = paravirt_nop,
-#endif
+#endif /* !CONFIG_QUEUE_SPINLOCK */
+#endif /* SMP */
 };
 EXPORT_SYMBOL(pv_lock_ops);
 
--- a/arch/x86/kernel/paravirt_patch_32.c
+++ b/arch/x86/kernel/paravirt_patch_32.c
@@ -12,6 +12,10 @@ DEF_NATIVE(pv_mmu_ops, read_cr3, "mov %c
 DEF_NATIVE(pv_cpu_ops, clts, "clts");
 DEF_NATIVE(pv_cpu_ops, read_tsc, "rdtsc");
 
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCKS)
+DEF_NATIVE(pv_lock_ops, queue_spin_unlock, "movb $0, (%eax)");
+#endif
+
 unsigned paravirt_patch_ident_32(void *insnbuf, unsigned len)
 {
 	/* arg in %eax, return in %eax */
@@ -24,6 +28,8 @@ unsigned paravirt_patch_ident_64(void *i
 	return 0;
 }
 
+extern bool pv_is_native_spin_unlock(void);
+
 unsigned native_patch(u8 type, u16 clobbers, void *ibuf,
 		      unsigned long addr, unsigned len)
 {
@@ -47,14 +53,22 @@ unsigned native_patch(u8 type, u16 clobb
 		PATCH_SITE(pv_mmu_ops, write_cr3);
 		PATCH_SITE(pv_cpu_ops, clts);
 		PATCH_SITE(pv_cpu_ops, read_tsc);
-
-	patch_site:
-		ret = paravirt_patch_insns(ibuf, len, start, end);
-		break;
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCKS)
+		case PARAVIRT_PATCH(pv_lock_ops.queue_spin_unlock):
+			if (pv_is_native_spin_unlock()) {
+				start = start_pv_lock_ops_queue_spin_unlock;
+				end   = end_pv_lock_ops_queue_spin_unlock;
+				goto patch_site;
+			}
+#endif
 
 	default:
 		ret = paravirt_patch_default(type, clobbers, ibuf, addr, len);
 		break;
+
+	patch_site:
+		ret = paravirt_patch_insns(ibuf, len, start, end);
+		break;
 	}
 #undef PATCH_SITE
 	return ret;
--- a/arch/x86/kernel/paravirt_patch_64.c
+++ b/arch/x86/kernel/paravirt_patch_64.c
@@ -21,6 +21,10 @@ DEF_NATIVE(pv_cpu_ops, swapgs, "swapgs")
 DEF_NATIVE(, mov32, "mov %edi, %eax");
 DEF_NATIVE(, mov64, "mov %rdi, %rax");
 
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCK)
+DEF_NATIVE(pv_lock_ops, queue_spin_unlock, "movb $0, (%rdi)");
+#endif
+
 unsigned paravirt_patch_ident_32(void *insnbuf, unsigned len)
 {
 	return paravirt_patch_insns(insnbuf, len,
@@ -33,6 +37,8 @@ unsigned paravirt_patch_ident_64(void *i
 				    start__mov64, end__mov64);
 }
 
+extern bool pv_is_native_spin_unlock(void);
+
 unsigned native_patch(u8 type, u16 clobbers, void *ibuf,
 		      unsigned long addr, unsigned len)
 {
@@ -59,14 +65,22 @@ unsigned native_patch(u8 type, u16 clobb
 		PATCH_SITE(pv_cpu_ops, clts);
 		PATCH_SITE(pv_mmu_ops, flush_tlb_single);
 		PATCH_SITE(pv_cpu_ops, wbinvd);
-
-	patch_site:
-		ret = paravirt_patch_insns(ibuf, len, start, end);
-		break;
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCK)
+		case PARAVIRT_PATCH(pv_lock_ops.queue_spin_unlock):
+			if (pv_is_native_spin_unlock()) {
+				start = start_pv_lock_ops_queue_spin_unlock;
+				end   = end_pv_lock_ops_queue_spin_unlock;
+				goto patch_site;
+			}
+#endif
 
 	default:
 		ret = paravirt_patch_default(type, clobbers, ibuf, addr, len);
 		break;
+
+	patch_site:
+		ret = paravirt_patch_insns(ibuf, len, start, end);
+		break;
 	}
 #undef PATCH_SITE
 	return ret;
--- a/kernel/Kconfig.locks
+++ b/kernel/Kconfig.locks
@@ -240,7 +240,7 @@ config ARCH_USE_QUEUE_SPINLOCK
 
 config QUEUE_SPINLOCK
 	def_bool y if ARCH_USE_QUEUE_SPINLOCK
-	depends on SMP && !PARAVIRT_SPINLOCKS
+	depends on SMP
 
 config ARCH_USE_QUEUE_RWLOCK
 	bool



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

* [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
@ 2015-03-16 13:16   ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	konrad.wilk, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, david.vrabel, torvalds

[-- Attachment #1: peterz-qspinlock-paravirt-x86-crap.patch --]
[-- Type: text/plain, Size: 10533 bytes --]

Implement the paravirt qspinlock for x86-kvm.

We use the regular paravirt call patching to switch between:

  native_queue_spin_lock_slowpath()	__pv_queue_spin_lock_slowpath()
  native_queue_spin_unlock()		__pv_queue_spin_unlock()

We use a callee saved call for the unlock function which reduces the
i-cache footprint and allows 'inlining' of SPIN_UNLOCK functions
again.

We further optimize the unlock path by patching the direct call with a
"movb $0,%arg1" if we are indeed using the native unlock code. This
makes the unlock code almost as fast as the !PARAVIRT case.

This significantly lowers the overhead of having
CONFIG_PARAVIRT_SPINLOCKS enabled, even for native code.


Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 arch/x86/Kconfig                      |    2 -
 arch/x86/include/asm/paravirt.h       |   28 ++++++++++++++++++++-
 arch/x86/include/asm/paravirt_types.h |   10 +++++++
 arch/x86/include/asm/qspinlock.h      |   22 ++++++++++++++++-
 arch/x86/kernel/kvm.c                 |   44 ++++++++++++++++++++++++++++++++++
 arch/x86/kernel/paravirt-spinlocks.c  |   24 +++++++++++++++++-
 arch/x86/kernel/paravirt_patch_32.c   |   22 +++++++++++++----
 arch/x86/kernel/paravirt_patch_64.c   |   22 +++++++++++++----
 kernel/Kconfig.locks                  |    2 -
 9 files changed, 163 insertions(+), 13 deletions(-)

--- a/arch/x86/Kconfig
+++ b/arch/x86/Kconfig
@@ -692,7 +692,7 @@ config PARAVIRT_DEBUG
 config PARAVIRT_SPINLOCKS
 	bool "Paravirtualization layer for spinlocks"
 	depends on PARAVIRT && SMP
-	select UNINLINE_SPIN_UNLOCK
+	select UNINLINE_SPIN_UNLOCK if !QUEUE_SPINLOCK
 	---help---
 	  Paravirtualized spinlocks allow a pvops backend to replace the
 	  spinlock implementation with something virtualization-friendly
--- a/arch/x86/include/asm/paravirt.h
+++ b/arch/x86/include/asm/paravirt.h
@@ -712,6 +712,30 @@ static inline void __set_fixmap(unsigned
 
 #if defined(CONFIG_SMP) && defined(CONFIG_PARAVIRT_SPINLOCKS)
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+static __always_inline void pv_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	PVOP_VCALL2(pv_lock_ops.queue_spin_lock_slowpath, lock, val);
+}
+
+static __always_inline void pv_queue_spin_unlock(struct qspinlock *lock)
+{
+	PVOP_VCALLEE1(pv_lock_ops.queue_spin_unlock, lock);
+}
+
+static __always_inline void pv_wait(u8 *ptr, u8 val)
+{
+	PVOP_VCALL2(pv_lock_ops.wait, ptr, val);
+}
+
+static __always_inline void pv_kick(int cpu)
+{
+	PVOP_VCALL1(pv_lock_ops.kick, cpu);
+}
+
+#else /* !CONFIG_QUEUE_SPINLOCK */
+
 static __always_inline void __ticket_lock_spinning(struct arch_spinlock *lock,
 							__ticket_t ticket)
 {
@@ -724,7 +748,9 @@ static __always_inline void __ticket_unl
 	PVOP_VCALL2(pv_lock_ops.unlock_kick, lock, ticket);
 }
 
-#endif
+#endif /* CONFIG_QUEUE_SPINLOCK */
+
+#endif /* SMP && PARAVIRT_SPINLOCKS */
 
 #ifdef CONFIG_X86_32
 #define PV_SAVE_REGS "pushl %ecx; pushl %edx;"
--- a/arch/x86/include/asm/paravirt_types.h
+++ b/arch/x86/include/asm/paravirt_types.h
@@ -333,9 +333,19 @@ struct arch_spinlock;
 typedef u16 __ticket_t;
 #endif
 
+struct qspinlock;
+
 struct pv_lock_ops {
+#ifdef CONFIG_QUEUE_SPINLOCK
+	void (*queue_spin_lock_slowpath)(struct qspinlock *lock, u32 val);
+	struct paravirt_callee_save queue_spin_unlock;
+
+	void (*wait)(u8 *ptr, u8 val);
+	void (*kick)(int cpu);
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	struct paravirt_callee_save lock_spinning;
 	void (*unlock_kick)(struct arch_spinlock *lock, __ticket_t ticket);
+#endif /* !CONFIG_QUEUE_SPINLOCK */
 };
 
 /* This contains all the paravirt structures: we get a convenient
--- a/arch/x86/include/asm/qspinlock.h
+++ b/arch/x86/include/asm/qspinlock.h
@@ -3,6 +3,7 @@
 
 #include <asm/cpufeature.h>
 #include <asm-generic/qspinlock_types.h>
+#include <asm/paravirt.h>
 
 #define	queue_spin_unlock queue_spin_unlock
 /**
@@ -11,11 +12,30 @@
  *
  * An smp_store_release() on the least-significant byte.
  */
-static inline void queue_spin_unlock(struct qspinlock *lock)
+static inline void native_queue_spin_unlock(struct qspinlock *lock)
 {
 	smp_store_release((u8 *)lock, 0);
 }
 
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+extern void native_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+
+static inline void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	pv_queue_spin_lock_slowpath(lock, val);
+}
+
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	pv_queue_spin_unlock(lock);
+}
+#else
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	native_queue_spin_unlock(lock);
+}
+#endif
+
 #define virt_queue_spin_lock virt_queue_spin_lock
 
 static inline bool virt_queue_spin_lock(struct qspinlock *lock)
--- a/arch/x86/kernel/kvm.c
+++ b/arch/x86/kernel/kvm.c
@@ -584,6 +584,41 @@ static void kvm_kick_cpu(int cpu)
 	kvm_hypercall2(KVM_HC_KICK_CPU, flags, apicid);
 }
 
+
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+#include <asm/qspinlock.h>
+
+PV_CALLEE_SAVE_REGS_THUNK(__pv_queue_spin_unlock);
+
+static void kvm_wait(u8 *ptr, u8 val)
+{
+	unsigned long flags;
+
+	if (in_nmi())
+		return;
+
+	local_irq_save(flags);
+
+	if (READ_ONCE(*ptr) != val)
+		goto out;
+
+	/*
+	 * halt until it's our turn and kicked. Note that we do safe halt
+	 * for irq enabled case to avoid hang when lock info is overwritten
+	 * in irq spinlock slowpath and no spurious interrupt occur to save us.
+	 */
+	if (arch_irqs_disabled_flags(flags))
+		halt();
+	else
+		safe_halt();
+
+out:
+	local_irq_restore(flags);
+}
+
+#else /* !CONFIG_QUEUE_SPINLOCK */
+
 enum kvm_contention_stat {
 	TAKEN_SLOW,
 	TAKEN_SLOW_PICKUP,
@@ -817,6 +852,8 @@ static void kvm_unlock_kick(struct arch_
 	}
 }
 
+#endif /* !CONFIG_QUEUE_SPINLOCK */
+
 /*
  * Setup pv_lock_ops to exploit KVM_FEATURE_PV_UNHALT if present.
  */
@@ -828,8 +865,15 @@ void __init kvm_spinlock_init(void)
 	if (!kvm_para_has_feature(KVM_FEATURE_PV_UNHALT))
 		return;
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+	pv_lock_ops.queue_spin_lock_slowpath = __pv_queue_spin_lock_slowpath;
+	pv_lock_ops.queue_spin_unlock = PV_CALLEE_SAVE(__pv_queue_spin_unlock);
+	pv_lock_ops.wait = kvm_wait;
+	pv_lock_ops.kick = kvm_kick_cpu;
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	pv_lock_ops.lock_spinning = PV_CALLEE_SAVE(kvm_lock_spinning);
 	pv_lock_ops.unlock_kick = kvm_unlock_kick;
+#endif
 }
 
 static __init int kvm_spinlock_init_jump(void)
--- a/arch/x86/kernel/paravirt-spinlocks.c
+++ b/arch/x86/kernel/paravirt-spinlocks.c
@@ -8,11 +8,33 @@
 
 #include <asm/paravirt.h>
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+__visible void __native_queue_spin_unlock(struct qspinlock *lock)
+{
+	native_queue_spin_unlock(lock);
+}
+
+PV_CALLEE_SAVE_REGS_THUNK(__native_queue_spin_unlock);
+
+bool pv_is_native_spin_unlock(void)
+{
+	return pv_lock_ops.queue_spin_unlock.func ==
+		__raw_callee_save___native_queue_spin_unlock;
+}
+#endif
+
 struct pv_lock_ops pv_lock_ops = {
 #ifdef CONFIG_SMP
+#ifdef CONFIG_QUEUE_SPINLOCK
+	.queue_spin_lock_slowpath = native_queue_spin_lock_slowpath,
+	.queue_spin_unlock = PV_CALLEE_SAVE(__native_queue_spin_unlock),
+	.wait = paravirt_nop,
+	.kick = paravirt_nop,
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	.lock_spinning = __PV_IS_CALLEE_SAVE(paravirt_nop),
 	.unlock_kick = paravirt_nop,
-#endif
+#endif /* !CONFIG_QUEUE_SPINLOCK */
+#endif /* SMP */
 };
 EXPORT_SYMBOL(pv_lock_ops);
 
--- a/arch/x86/kernel/paravirt_patch_32.c
+++ b/arch/x86/kernel/paravirt_patch_32.c
@@ -12,6 +12,10 @@ DEF_NATIVE(pv_mmu_ops, read_cr3, "mov %c
 DEF_NATIVE(pv_cpu_ops, clts, "clts");
 DEF_NATIVE(pv_cpu_ops, read_tsc, "rdtsc");
 
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCKS)
+DEF_NATIVE(pv_lock_ops, queue_spin_unlock, "movb $0, (%eax)");
+#endif
+
 unsigned paravirt_patch_ident_32(void *insnbuf, unsigned len)
 {
 	/* arg in %eax, return in %eax */
@@ -24,6 +28,8 @@ unsigned paravirt_patch_ident_64(void *i
 	return 0;
 }
 
+extern bool pv_is_native_spin_unlock(void);
+
 unsigned native_patch(u8 type, u16 clobbers, void *ibuf,
 		      unsigned long addr, unsigned len)
 {
@@ -47,14 +53,22 @@ unsigned native_patch(u8 type, u16 clobb
 		PATCH_SITE(pv_mmu_ops, write_cr3);
 		PATCH_SITE(pv_cpu_ops, clts);
 		PATCH_SITE(pv_cpu_ops, read_tsc);
-
-	patch_site:
-		ret = paravirt_patch_insns(ibuf, len, start, end);
-		break;
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCKS)
+		case PARAVIRT_PATCH(pv_lock_ops.queue_spin_unlock):
+			if (pv_is_native_spin_unlock()) {
+				start = start_pv_lock_ops_queue_spin_unlock;
+				end   = end_pv_lock_ops_queue_spin_unlock;
+				goto patch_site;
+			}
+#endif
 
 	default:
 		ret = paravirt_patch_default(type, clobbers, ibuf, addr, len);
 		break;
+
+	patch_site:
+		ret = paravirt_patch_insns(ibuf, len, start, end);
+		break;
 	}
 #undef PATCH_SITE
 	return ret;
--- a/arch/x86/kernel/paravirt_patch_64.c
+++ b/arch/x86/kernel/paravirt_patch_64.c
@@ -21,6 +21,10 @@ DEF_NATIVE(pv_cpu_ops, swapgs, "swapgs")
 DEF_NATIVE(, mov32, "mov %edi, %eax");
 DEF_NATIVE(, mov64, "mov %rdi, %rax");
 
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCK)
+DEF_NATIVE(pv_lock_ops, queue_spin_unlock, "movb $0, (%rdi)");
+#endif
+
 unsigned paravirt_patch_ident_32(void *insnbuf, unsigned len)
 {
 	return paravirt_patch_insns(insnbuf, len,
@@ -33,6 +37,8 @@ unsigned paravirt_patch_ident_64(void *i
 				    start__mov64, end__mov64);
 }
 
+extern bool pv_is_native_spin_unlock(void);
+
 unsigned native_patch(u8 type, u16 clobbers, void *ibuf,
 		      unsigned long addr, unsigned len)
 {
@@ -59,14 +65,22 @@ unsigned native_patch(u8 type, u16 clobb
 		PATCH_SITE(pv_cpu_ops, clts);
 		PATCH_SITE(pv_mmu_ops, flush_tlb_single);
 		PATCH_SITE(pv_cpu_ops, wbinvd);
-
-	patch_site:
-		ret = paravirt_patch_insns(ibuf, len, start, end);
-		break;
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCK)
+		case PARAVIRT_PATCH(pv_lock_ops.queue_spin_unlock):
+			if (pv_is_native_spin_unlock()) {
+				start = start_pv_lock_ops_queue_spin_unlock;
+				end   = end_pv_lock_ops_queue_spin_unlock;
+				goto patch_site;
+			}
+#endif
 
 	default:
 		ret = paravirt_patch_default(type, clobbers, ibuf, addr, len);
 		break;
+
+	patch_site:
+		ret = paravirt_patch_insns(ibuf, len, start, end);
+		break;
 	}
 #undef PATCH_SITE
 	return ret;
--- a/kernel/Kconfig.locks
+++ b/kernel/Kconfig.locks
@@ -240,7 +240,7 @@ config ARCH_USE_QUEUE_SPINLOCK
 
 config QUEUE_SPINLOCK
 	def_bool y if ARCH_USE_QUEUE_SPINLOCK
-	depends on SMP && !PARAVIRT_SPINLOCKS
+	depends on SMP
 
 config ARCH_USE_QUEUE_RWLOCK
 	bool

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

* [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (18 preceding siblings ...)
  (?)
@ 2015-03-16 13:16 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

[-- Attachment #1: peterz-qspinlock-paravirt-x86-crap.patch --]
[-- Type: text/plain, Size: 10533 bytes --]

Implement the paravirt qspinlock for x86-kvm.

We use the regular paravirt call patching to switch between:

  native_queue_spin_lock_slowpath()	__pv_queue_spin_lock_slowpath()
  native_queue_spin_unlock()		__pv_queue_spin_unlock()

We use a callee saved call for the unlock function which reduces the
i-cache footprint and allows 'inlining' of SPIN_UNLOCK functions
again.

We further optimize the unlock path by patching the direct call with a
"movb $0,%arg1" if we are indeed using the native unlock code. This
makes the unlock code almost as fast as the !PARAVIRT case.

This significantly lowers the overhead of having
CONFIG_PARAVIRT_SPINLOCKS enabled, even for native code.


Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 arch/x86/Kconfig                      |    2 -
 arch/x86/include/asm/paravirt.h       |   28 ++++++++++++++++++++-
 arch/x86/include/asm/paravirt_types.h |   10 +++++++
 arch/x86/include/asm/qspinlock.h      |   22 ++++++++++++++++-
 arch/x86/kernel/kvm.c                 |   44 ++++++++++++++++++++++++++++++++++
 arch/x86/kernel/paravirt-spinlocks.c  |   24 +++++++++++++++++-
 arch/x86/kernel/paravirt_patch_32.c   |   22 +++++++++++++----
 arch/x86/kernel/paravirt_patch_64.c   |   22 +++++++++++++----
 kernel/Kconfig.locks                  |    2 -
 9 files changed, 163 insertions(+), 13 deletions(-)

--- a/arch/x86/Kconfig
+++ b/arch/x86/Kconfig
@@ -692,7 +692,7 @@ config PARAVIRT_DEBUG
 config PARAVIRT_SPINLOCKS
 	bool "Paravirtualization layer for spinlocks"
 	depends on PARAVIRT && SMP
-	select UNINLINE_SPIN_UNLOCK
+	select UNINLINE_SPIN_UNLOCK if !QUEUE_SPINLOCK
 	---help---
 	  Paravirtualized spinlocks allow a pvops backend to replace the
 	  spinlock implementation with something virtualization-friendly
--- a/arch/x86/include/asm/paravirt.h
+++ b/arch/x86/include/asm/paravirt.h
@@ -712,6 +712,30 @@ static inline void __set_fixmap(unsigned
 
 #if defined(CONFIG_SMP) && defined(CONFIG_PARAVIRT_SPINLOCKS)
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+static __always_inline void pv_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	PVOP_VCALL2(pv_lock_ops.queue_spin_lock_slowpath, lock, val);
+}
+
+static __always_inline void pv_queue_spin_unlock(struct qspinlock *lock)
+{
+	PVOP_VCALLEE1(pv_lock_ops.queue_spin_unlock, lock);
+}
+
+static __always_inline void pv_wait(u8 *ptr, u8 val)
+{
+	PVOP_VCALL2(pv_lock_ops.wait, ptr, val);
+}
+
+static __always_inline void pv_kick(int cpu)
+{
+	PVOP_VCALL1(pv_lock_ops.kick, cpu);
+}
+
+#else /* !CONFIG_QUEUE_SPINLOCK */
+
 static __always_inline void __ticket_lock_spinning(struct arch_spinlock *lock,
 							__ticket_t ticket)
 {
@@ -724,7 +748,9 @@ static __always_inline void __ticket_unl
 	PVOP_VCALL2(pv_lock_ops.unlock_kick, lock, ticket);
 }
 
-#endif
+#endif /* CONFIG_QUEUE_SPINLOCK */
+
+#endif /* SMP && PARAVIRT_SPINLOCKS */
 
 #ifdef CONFIG_X86_32
 #define PV_SAVE_REGS "pushl %ecx; pushl %edx;"
--- a/arch/x86/include/asm/paravirt_types.h
+++ b/arch/x86/include/asm/paravirt_types.h
@@ -333,9 +333,19 @@ struct arch_spinlock;
 typedef u16 __ticket_t;
 #endif
 
+struct qspinlock;
+
 struct pv_lock_ops {
+#ifdef CONFIG_QUEUE_SPINLOCK
+	void (*queue_spin_lock_slowpath)(struct qspinlock *lock, u32 val);
+	struct paravirt_callee_save queue_spin_unlock;
+
+	void (*wait)(u8 *ptr, u8 val);
+	void (*kick)(int cpu);
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	struct paravirt_callee_save lock_spinning;
 	void (*unlock_kick)(struct arch_spinlock *lock, __ticket_t ticket);
+#endif /* !CONFIG_QUEUE_SPINLOCK */
 };
 
 /* This contains all the paravirt structures: we get a convenient
--- a/arch/x86/include/asm/qspinlock.h
+++ b/arch/x86/include/asm/qspinlock.h
@@ -3,6 +3,7 @@
 
 #include <asm/cpufeature.h>
 #include <asm-generic/qspinlock_types.h>
+#include <asm/paravirt.h>
 
 #define	queue_spin_unlock queue_spin_unlock
 /**
@@ -11,11 +12,30 @@
  *
  * An smp_store_release() on the least-significant byte.
  */
-static inline void queue_spin_unlock(struct qspinlock *lock)
+static inline void native_queue_spin_unlock(struct qspinlock *lock)
 {
 	smp_store_release((u8 *)lock, 0);
 }
 
+#ifdef CONFIG_PARAVIRT_SPINLOCKS
+extern void native_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
+
+static inline void queue_spin_lock_slowpath(struct qspinlock *lock, u32 val)
+{
+	pv_queue_spin_lock_slowpath(lock, val);
+}
+
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	pv_queue_spin_unlock(lock);
+}
+#else
+static inline void queue_spin_unlock(struct qspinlock *lock)
+{
+	native_queue_spin_unlock(lock);
+}
+#endif
+
 #define virt_queue_spin_lock virt_queue_spin_lock
 
 static inline bool virt_queue_spin_lock(struct qspinlock *lock)
--- a/arch/x86/kernel/kvm.c
+++ b/arch/x86/kernel/kvm.c
@@ -584,6 +584,41 @@ static void kvm_kick_cpu(int cpu)
 	kvm_hypercall2(KVM_HC_KICK_CPU, flags, apicid);
 }
 
+
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+#include <asm/qspinlock.h>
+
+PV_CALLEE_SAVE_REGS_THUNK(__pv_queue_spin_unlock);
+
+static void kvm_wait(u8 *ptr, u8 val)
+{
+	unsigned long flags;
+
+	if (in_nmi())
+		return;
+
+	local_irq_save(flags);
+
+	if (READ_ONCE(*ptr) != val)
+		goto out;
+
+	/*
+	 * halt until it's our turn and kicked. Note that we do safe halt
+	 * for irq enabled case to avoid hang when lock info is overwritten
+	 * in irq spinlock slowpath and no spurious interrupt occur to save us.
+	 */
+	if (arch_irqs_disabled_flags(flags))
+		halt();
+	else
+		safe_halt();
+
+out:
+	local_irq_restore(flags);
+}
+
+#else /* !CONFIG_QUEUE_SPINLOCK */
+
 enum kvm_contention_stat {
 	TAKEN_SLOW,
 	TAKEN_SLOW_PICKUP,
@@ -817,6 +852,8 @@ static void kvm_unlock_kick(struct arch_
 	}
 }
 
+#endif /* !CONFIG_QUEUE_SPINLOCK */
+
 /*
  * Setup pv_lock_ops to exploit KVM_FEATURE_PV_UNHALT if present.
  */
@@ -828,8 +865,15 @@ void __init kvm_spinlock_init(void)
 	if (!kvm_para_has_feature(KVM_FEATURE_PV_UNHALT))
 		return;
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+	pv_lock_ops.queue_spin_lock_slowpath = __pv_queue_spin_lock_slowpath;
+	pv_lock_ops.queue_spin_unlock = PV_CALLEE_SAVE(__pv_queue_spin_unlock);
+	pv_lock_ops.wait = kvm_wait;
+	pv_lock_ops.kick = kvm_kick_cpu;
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	pv_lock_ops.lock_spinning = PV_CALLEE_SAVE(kvm_lock_spinning);
 	pv_lock_ops.unlock_kick = kvm_unlock_kick;
+#endif
 }
 
 static __init int kvm_spinlock_init_jump(void)
--- a/arch/x86/kernel/paravirt-spinlocks.c
+++ b/arch/x86/kernel/paravirt-spinlocks.c
@@ -8,11 +8,33 @@
 
 #include <asm/paravirt.h>
 
+#ifdef CONFIG_QUEUE_SPINLOCK
+__visible void __native_queue_spin_unlock(struct qspinlock *lock)
+{
+	native_queue_spin_unlock(lock);
+}
+
+PV_CALLEE_SAVE_REGS_THUNK(__native_queue_spin_unlock);
+
+bool pv_is_native_spin_unlock(void)
+{
+	return pv_lock_ops.queue_spin_unlock.func ==
+		__raw_callee_save___native_queue_spin_unlock;
+}
+#endif
+
 struct pv_lock_ops pv_lock_ops = {
 #ifdef CONFIG_SMP
+#ifdef CONFIG_QUEUE_SPINLOCK
+	.queue_spin_lock_slowpath = native_queue_spin_lock_slowpath,
+	.queue_spin_unlock = PV_CALLEE_SAVE(__native_queue_spin_unlock),
+	.wait = paravirt_nop,
+	.kick = paravirt_nop,
+#else /* !CONFIG_QUEUE_SPINLOCK */
 	.lock_spinning = __PV_IS_CALLEE_SAVE(paravirt_nop),
 	.unlock_kick = paravirt_nop,
-#endif
+#endif /* !CONFIG_QUEUE_SPINLOCK */
+#endif /* SMP */
 };
 EXPORT_SYMBOL(pv_lock_ops);
 
--- a/arch/x86/kernel/paravirt_patch_32.c
+++ b/arch/x86/kernel/paravirt_patch_32.c
@@ -12,6 +12,10 @@ DEF_NATIVE(pv_mmu_ops, read_cr3, "mov %c
 DEF_NATIVE(pv_cpu_ops, clts, "clts");
 DEF_NATIVE(pv_cpu_ops, read_tsc, "rdtsc");
 
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCKS)
+DEF_NATIVE(pv_lock_ops, queue_spin_unlock, "movb $0, (%eax)");
+#endif
+
 unsigned paravirt_patch_ident_32(void *insnbuf, unsigned len)
 {
 	/* arg in %eax, return in %eax */
@@ -24,6 +28,8 @@ unsigned paravirt_patch_ident_64(void *i
 	return 0;
 }
 
+extern bool pv_is_native_spin_unlock(void);
+
 unsigned native_patch(u8 type, u16 clobbers, void *ibuf,
 		      unsigned long addr, unsigned len)
 {
@@ -47,14 +53,22 @@ unsigned native_patch(u8 type, u16 clobb
 		PATCH_SITE(pv_mmu_ops, write_cr3);
 		PATCH_SITE(pv_cpu_ops, clts);
 		PATCH_SITE(pv_cpu_ops, read_tsc);
-
-	patch_site:
-		ret = paravirt_patch_insns(ibuf, len, start, end);
-		break;
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCKS)
+		case PARAVIRT_PATCH(pv_lock_ops.queue_spin_unlock):
+			if (pv_is_native_spin_unlock()) {
+				start = start_pv_lock_ops_queue_spin_unlock;
+				end   = end_pv_lock_ops_queue_spin_unlock;
+				goto patch_site;
+			}
+#endif
 
 	default:
 		ret = paravirt_patch_default(type, clobbers, ibuf, addr, len);
 		break;
+
+	patch_site:
+		ret = paravirt_patch_insns(ibuf, len, start, end);
+		break;
 	}
 #undef PATCH_SITE
 	return ret;
--- a/arch/x86/kernel/paravirt_patch_64.c
+++ b/arch/x86/kernel/paravirt_patch_64.c
@@ -21,6 +21,10 @@ DEF_NATIVE(pv_cpu_ops, swapgs, "swapgs")
 DEF_NATIVE(, mov32, "mov %edi, %eax");
 DEF_NATIVE(, mov64, "mov %rdi, %rax");
 
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCK)
+DEF_NATIVE(pv_lock_ops, queue_spin_unlock, "movb $0, (%rdi)");
+#endif
+
 unsigned paravirt_patch_ident_32(void *insnbuf, unsigned len)
 {
 	return paravirt_patch_insns(insnbuf, len,
@@ -33,6 +37,8 @@ unsigned paravirt_patch_ident_64(void *i
 				    start__mov64, end__mov64);
 }
 
+extern bool pv_is_native_spin_unlock(void);
+
 unsigned native_patch(u8 type, u16 clobbers, void *ibuf,
 		      unsigned long addr, unsigned len)
 {
@@ -59,14 +65,22 @@ unsigned native_patch(u8 type, u16 clobb
 		PATCH_SITE(pv_cpu_ops, clts);
 		PATCH_SITE(pv_mmu_ops, flush_tlb_single);
 		PATCH_SITE(pv_cpu_ops, wbinvd);
-
-	patch_site:
-		ret = paravirt_patch_insns(ibuf, len, start, end);
-		break;
+#if defined(CONFIG_PARAVIRT_SPINLOCKS) && defined(CONFIG_QUEUE_SPINLOCK)
+		case PARAVIRT_PATCH(pv_lock_ops.queue_spin_unlock):
+			if (pv_is_native_spin_unlock()) {
+				start = start_pv_lock_ops_queue_spin_unlock;
+				end   = end_pv_lock_ops_queue_spin_unlock;
+				goto patch_site;
+			}
+#endif
 
 	default:
 		ret = paravirt_patch_default(type, clobbers, ibuf, addr, len);
 		break;
+
+	patch_site:
+		ret = paravirt_patch_insns(ibuf, len, start, end);
+		break;
 	}
 #undef PATCH_SITE
 	return ret;
--- a/kernel/Kconfig.locks
+++ b/kernel/Kconfig.locks
@@ -240,7 +240,7 @@ config ARCH_USE_QUEUE_SPINLOCK
 
 config QUEUE_SPINLOCK
 	def_bool y if ARCH_USE_QUEUE_SPINLOCK
-	depends on SMP && !PARAVIRT_SPINLOCKS
+	depends on SMP
 
 config ARCH_USE_QUEUE_RWLOCK
 	bool

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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
  (?)
  (?)
@ 2015-03-16 14:08   ` David Vrabel
  -1 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-16 14:08 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: raghavendra.kt, kvm, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

On 16/03/15 13:16, Peter Zijlstra wrote:
> Hi Waiman,
> 
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
> 
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
> 
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
> 
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
> 
> So while the paravirt code isn't the most optimal code ever conceived it does work.
> 
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

I can look at this.  It looks pretty straight-forward.

> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.

I think this is fine.

David

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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-16 14:08   ` David Vrabel
  0 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-16 14:08 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: linux-arch, riel, x86, kvm, oleg, scott.norton, raghavendra.kt,
	paolo.bonzini, linux-kernel, virtualization, mingo, doug.hatch,
	hpa, luto, xen-devel, boris.ostrovsky, paulmck, torvalds, tglx,
	david.vrabel

On 16/03/15 13:16, Peter Zijlstra wrote:
> Hi Waiman,
> 
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
> 
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
> 
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
> 
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
> 
> So while the paravirt code isn't the most optimal code ever conceived it does work.
> 
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

I can look at this.  It looks pretty straight-forward.

> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.

I think this is fine.

David

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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-16 14:08   ` David Vrabel
  0 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-16 14:08 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: raghavendra.kt, kvm, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

On 16/03/15 13:16, Peter Zijlstra wrote:
> Hi Waiman,
> 
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
> 
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
> 
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
> 
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
> 
> So while the paravirt code isn't the most optimal code ever conceived it does work.
> 
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

I can look at this.  It looks pretty straight-forward.

> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.

I think this is fine.

David

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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-16 14:08   ` David Vrabel
  0 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-16 14:08 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: linux-arch, riel, x86, kvm, oleg, scott.norton, raghavendra.kt,
	paolo.bonzini, linux-kernel, virtualization, mingo, doug.hatch,
	hpa, luto, xen-devel, boris.ostrovsky, paulmck, torvalds, tglx,
	david.vrabel

On 16/03/15 13:16, Peter Zijlstra wrote:
> Hi Waiman,
> 
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
> 
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
> 
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
> 
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
> 
> So while the paravirt code isn't the most optimal code ever conceived it does work.
> 
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

I can look at this.  It looks pretty straight-forward.

> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.

I think this is fine.

David

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (20 preceding siblings ...)
  (?)
@ 2015-03-16 14:08 ` David Vrabel
  -1 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-16 14:08 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: linux-arch, riel, x86, kvm, oleg, scott.norton, raghavendra.kt,
	paolo.bonzini, linux-kernel, virtualization, mingo, doug.hatch,
	hpa, luto, xen-devel, boris.ostrovsky, paulmck, torvalds, tglx,
	david.vrabel

On 16/03/15 13:16, Peter Zijlstra wrote:
> Hi Waiman,
> 
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
> 
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
> 
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
> 
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
> 
> So while the paravirt code isn't the most optimal code ever conceived it does work.
> 
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

I can look at this.  It looks pretty straight-forward.

> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.

I think this is fine.

David

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (24 preceding siblings ...)
  (?)
@ 2015-03-18 20:36 ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-18 20:36 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> Hi Waiman,
>
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
>
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
>
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
>
> So while the paravirt code isn't the most optimal code ever conceived it does work.
>
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
>
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
>
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
>
> Of course; once this lands people can look at improving the paravirt nonsense.
>

Thanks for sending this out. I have no problem with the !paravirt patch. 
I do have some comments on the paravirt one which I will reply individually.

Cheers,
Longman

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (23 preceding siblings ...)
  (?)
@ 2015-03-18 20:36 ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-18 20:36 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> Hi Waiman,
>
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
>
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
>
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
>
> So while the paravirt code isn't the most optimal code ever conceived it does work.
>
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
>
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
>
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
>
> Of course; once this lands people can look at improving the paravirt nonsense.
>

Thanks for sending this out. I have no problem with the !paravirt patch. 
I do have some comments on the paravirt one which I will reply individually.

Cheers,
Longman

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (22 preceding siblings ...)
  (?)
@ 2015-03-18 20:36 ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-18 20:36 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> Hi Waiman,
>
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
>
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
>
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
>
> So while the paravirt code isn't the most optimal code ever conceived it does work.
>
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
>
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
>
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
>
> Of course; once this lands people can look at improving the paravirt nonsense.
>

Thanks for sending this out. I have no problem with the !paravirt patch. 
I do have some comments on the paravirt one which I will reply individually.

Cheers,
Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-16 13:16   ` Peter Zijlstra
  (?)
@ 2015-03-18 20:50   ` Waiman Long
  2015-03-19 10:12     ` Peter Zijlstra
                       ` (2 more replies)
  -1 siblings, 3 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-18 20:50 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch


[-- Attachment #1.1: Type: text/plain, Size: 10903 bytes --]

On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> Implement simple paravirt support for the qspinlock.
>
> Provide a separate (second) version of the spin_lock_slowpath for
> paravirt along with a special unlock path.
>
> The second slowpath is generated by adding a few pv hooks to the
> normal slowpath, but where those will compile away for the native
> case, they expand into special wait/wake code for the pv version.
>
> The actual MCS queue can use extra storage in the mcs_nodes[] array to
> keep track of state and therefore uses directed wakeups.
>
> The head contender has no such storage available and reverts to the
> per-cpu lock entry similar to the current kvm code. We can do a single
> enrty because any nesting will wake the vcpu and cause the lower loop
> to retry.
>
> Signed-off-by: Peter Zijlstra (Intel)<peterz@infradead.org>
> ---
>   include/asm-generic/qspinlock.h     |    3
>   kernel/locking/qspinlock.c          |   69 +++++++++++++-
>   kernel/locking/qspinlock_paravirt.h |  177 ++++++++++++++++++++++++++++++++++++
>   3 files changed, 248 insertions(+), 1 deletion(-)
>
> --- a/include/asm-generic/qspinlock.h
> +++ b/include/asm-generic/qspinlock.h
> @@ -118,6 +118,9 @@ static __always_inline bool virt_queue_s
>   }
>   #endif
>
> +extern void __pv_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
> +extern void __pv_queue_spin_unlock(struct qspinlock *lock);
> +
>   /*
>    * Initializier
>    */
> --- a/kernel/locking/qspinlock.c
> +++ b/kernel/locking/qspinlock.c
> @@ -18,6 +18,9 @@
>    * Authors: Waiman Long<waiman.long@hp.com>
>    *          Peter Zijlstra<peterz@infradead.org>
>    */
> +
> +#ifndef _GEN_PV_LOCK_SLOWPATH
> +
>   #include<linux/smp.h>
>   #include<linux/bug.h>
>   #include<linux/cpumask.h>
> @@ -65,13 +68,21 @@
>
>   #include "mcs_spinlock.h"
>
> +#ifdef CONFIG_PARAVIRT_SPINLOCKS
> +#define MAX_NODES	8
> +#else
> +#define MAX_NODES	4
> +#endif
> +
>   /*
>    * Per-CPU queue node structures; we can never have more than 4 nested
>    * contexts: task, softirq, hardirq, nmi.
>    *
>    * Exactly fits one 64-byte cacheline on a 64-bit architecture.
> + *
> + * PV doubles the storage and uses the second cacheline for PV state.
>    */
> -static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[4]);
> +static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[MAX_NODES]);
>
>   /*
>    * We must be able to distinguish between no-tail and the tail at 0:0,
> @@ -230,6 +241,32 @@ static __always_inline void set_locked(s
>   	WRITE_ONCE(l->locked, _Q_LOCKED_VAL);
>   }
>
> +
> +/*
> + * Generate the native code for queue_spin_unlock_slowpath(); provide NOPs for
> + * all the PV callbacks.
> + */
> +
> +static __always_inline void __pv_init_node(struct mcs_spinlock *node) { }
> +static __always_inline void __pv_wait_node(struct mcs_spinlock *node) { }
> +static __always_inline void __pv_kick_node(struct mcs_spinlock *node) { }
> +
> +static __always_inline void __pv_wait_head(struct qspinlock *lock) { }
> +
> +#define pv_enabled()		false
> +
> +#define pv_init_node		__pv_init_node
> +#define pv_wait_node		__pv_wait_node
> +#define pv_kick_node		__pv_kick_node
> +
> +#define pv_wait_head		__pv_wait_head
> +
> +#ifdef CONFIG_PARAVIRT_SPINLOCKS
> +#define queue_spin_lock_slowpath	native_queue_spin_lock_slowpath
> +#endif
> +
> +#endif /* _GEN_PV_LOCK_SLOWPATH */
> +
>   /**
>    * queue_spin_lock_slowpath - acquire the queue spinlock
>    * @lock: Pointer to queue spinlock structure
> @@ -259,6 +296,9 @@ void queue_spin_lock_slowpath(struct qsp
>
>   	BUILD_BUG_ON(CONFIG_NR_CPUS>= (1U<<  _Q_TAIL_CPU_BITS));
>
> +	if (pv_enabled())
> +		goto queue;
> +
>   	if (virt_queue_spin_lock(lock))
>   		return;
>
> @@ -335,6 +375,7 @@ void queue_spin_lock_slowpath(struct qsp
>   	node += idx;
>   	node->locked = 0;
>   	node->next = NULL;
> +	pv_init_node(node);
>
>   	/*
>   	 * We touched a (possibly) cold cacheline in the per-cpu queue node;
> @@ -360,6 +401,7 @@ void queue_spin_lock_slowpath(struct qsp
>   		prev = decode_tail(old);
>   		WRITE_ONCE(prev->next, node);
>
> +		pv_wait_node(node);
>   		arch_mcs_spin_lock_contended(&node->locked);
>   	}
>
> @@ -374,6 +416,7 @@ void queue_spin_lock_slowpath(struct qsp
>   	 * sequentiality; this is because the set_locked() function below
>   	 * does not imply a full barrier.
>   	 */
> +	pv_wait_head(lock);
>   	while ((val = smp_load_acquire(&lock->val.counter))&  _Q_LOCKED_PENDING_MASK)
>   		cpu_relax();
>
> @@ -406,6 +449,7 @@ void queue_spin_lock_slowpath(struct qsp
>   		cpu_relax();
>
>   	arch_mcs_spin_unlock_contended(&next->locked);
> +	pv_kick_node(next);
>
>   release:
>   	/*
> @@ -414,3 +458,26 @@ void queue_spin_lock_slowpath(struct qsp
>   	this_cpu_dec(mcs_nodes[0].count);
>   }
>   EXPORT_SYMBOL(queue_spin_lock_slowpath);
> +
> +/*
> + * Generate the paravirt code for queue_spin_unlock_slowpath().
> + */
> +#if !defined(_GEN_PV_LOCK_SLOWPATH)&&  defined(CONFIG_PARAVIRT_SPINLOCKS)
> +#define _GEN_PV_LOCK_SLOWPATH
> +
> +#undef pv_enabled
> +#define pv_enabled()	true
> +
> +#undef pv_init_node
> +#undef pv_wait_node
> +#undef pv_kick_node
> +
> +#undef pv_wait_head
> +
> +#undef queue_spin_lock_slowpath
> +#define queue_spin_lock_slowpath	__pv_queue_spin_lock_slowpath
> +
> +#include "qspinlock_paravirt.h"
> +#include "qspinlock.c"
> +
> +#endif
> --- /dev/null
> +++ b/kernel/locking/qspinlock_paravirt.h
> @@ -0,0 +1,177 @@
> +#ifndef _GEN_PV_LOCK_SLOWPATH
> +#error "do not include this file"
> +#endif
> +
> +/*
> + * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
> + * of spinning them.
> + *
> + * This relies on the architecture to provide two paravirt hypercalls:
> + *
> + *   pv_wait(u8 *ptr, u8 val) -- suspends the vcpu if *ptr == val
> + *   pv_kick(cpu)             -- wakes a suspended vcpu
> + *
> + * Using these we implement __pv_queue_spin_lock_slowpath() and
> + * __pv_queue_spin_unlock() to replace native_queue_spin_lock_slowpath() and
> + * native_queue_spin_unlock().
> + */
> +
> +#define _Q_SLOW_VAL	(2U<<  _Q_LOCKED_OFFSET)
> +
> +enum vcpu_state {
> +	vcpu_running = 0,
> +	vcpu_halted,
> +};
> +
> +struct pv_node {
> +	struct mcs_spinlock	mcs;
> +	struct mcs_spinlock	__res[3];
> +
> +	int			cpu;
> +	u8			state;
> +};
> +
> +/*
> + * Initialize the PV part of the mcs_spinlock node.
> + */
> +static void pv_init_node(struct mcs_spinlock *node)
> +{
> +	struct pv_node *pn = (struct pv_node *)node;
> +
> +	BUILD_BUG_ON(sizeof(struct pv_node)>  5*sizeof(struct mcs_spinlock));
> +
> +	pn->cpu = smp_processor_id();
> +	pn->state = vcpu_running;
> +}
> +
> +/*
> + * Wait for node->locked to become true, halt the vcpu after a short spin.
> + * pv_kick_node() is used to wake the vcpu again.
> + */
> +static void pv_wait_node(struct mcs_spinlock *node)
> +{
> +	struct pv_node *pn = (struct pv_node *)node;
> +	int loop;
> +
> +	for (;;) {
> +		for (loop = SPIN_THRESHOLD; loop; loop--) {
> +			if (READ_ONCE(node->locked))
> +				goto done;
> +
> +			cpu_relax();
> +		}
> +
> +		/*
> +		 * Order pn->state vs pn->locked thusly:
> +		 *
> +		 * [S] pn->state = vcpu_halted	  [S] next->locked = 1
> +		 *     MB			      MB
> +		 * [L] pn->locked		[RmW] pn->state = vcpu_running
> +		 *
> +		 * Matches the xchg() from pv_kick_node().
> +		 */
> +		(void)xchg(&pn->state, vcpu_halted);
> +
> +		if (READ_ONCE(node->locked))
> +			goto done;
> +
> +		pv_wait(&pn->state, vcpu_halted);
> +	}
> +done:
> +	pn->state = vcpu_running;
> +
> +	/*
> +	 * By now our node->locked should be 1 and our caller will not actually
> +	 * spin-wait for it. We do however rely on our caller to do a
> +	 * load-acquire for us.
> +	 */
> +}
> +
> +/*
> + * Called after setting next->locked = 1, used to wake those stuck in
> + * pv_wait_node().
> + */
> +static void pv_kick_node(struct mcs_spinlock *node)
> +{
> +	struct pv_node *pn = (struct pv_node *)node;
> +
> +	/*
> +	 * Note that because node->locked is already set, this actual mcs_spinlock
> +	 * entry could be re-used already.
> +	 *
> +	 * This should be fine however, kicking people for no reason is harmless.
> +	 *
> +	 * See the comment in pv_wait_node().
> +	 */
> +	if (xchg(&pn->state, vcpu_running) == vcpu_halted)
> +		pv_kick(pn->cpu);
> +}
> +
> +static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
> +
> +/*
> + * Wait for l->locked to become clear; halt the vcpu after a short spin.
> + * __pv_queue_spin_unlock() will wake us.
> + */
> +static void pv_wait_head(struct qspinlock *lock)
> +{
> +	struct __qspinlock *l = (void *)lock;
> +	int loop;
> +
> +	for (;;) {
> +		for (loop = SPIN_THRESHOLD; loop; loop--) {
> +			if (!READ_ONCE(l->locked))
> +				goto done;
> +
> +			cpu_relax();
> +		}
> +
> +		this_cpu_write(__pv_lock_wait, lock);

We may run into the same problem of needing to have 4 queue nodes per 
CPU. If an interrupt happens just after the write and before the actual 
wait and it goes through the same sequence, it will overwrite the 
__pv_lock_wait[] entry. So we may have lost wakeup. That is why the 
pvticket lock code did that just before the actual wait with interrupt 
disabled. We probably couldn't disable interrupt here. So we may need to 
move the actual write to the KVM and Xen code if we keep the current logic.

> +		/*
> +		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
> +		 *
> +		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
> +		 *     MB                             MB
> +		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
> +		 *
> +		 * Matches the xchg() in pv_queue_spin_unlock().
> +		 */
> +		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
> +			goto done;
> +
> +		pv_wait(&l->locked, _Q_SLOW_VAL);
> +	}
> +done:
> +	this_cpu_write(__pv_lock_wait, NULL);
> +
> +	/*
> +	 * Lock is unlocked now; the caller will acquire it without waiting.
> +	 * As with pv_wait_node() we rely on the caller to do a load-acquire
> +	 * for us.
> +	 */
> +}
> +
> +/*
> + * To be used in stead of queue_spin_unlock() for paravirt locks. Wakes
> + * pv_wait_head() if appropriate.
> + */
> +void __pv_queue_spin_unlock(struct qspinlock *lock)
> +{
> +	struct __qspinlock *l = (void *)lock;
> +	int cpu;
> +
> +	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
> +		return;
> +
> +	/*
> +	 * At this point the memory pointed at by lock can be freed/reused,
> +	 * however we can still use the pointer value to search in our cpu
> +	 * array.
> +	 *
> +	 * XXX: get rid of this loop
> +	 */
> +	for_each_possible_cpu(cpu) {
> +		if (per_cpu(__pv_lock_wait, cpu) == lock)
> +			pv_kick(cpu);
> +	}
> +}

I do want to get rid of this loop too. On average, we have to scan about 
half the number of CPUs available. So it isn't that different 
performance-wise compared with my original idea of following the list 
from tail to head. And how about your idea of propagating the current 
head down the linked list?

-Longman


[-- Attachment #1.2: Type: text/html, Size: 11627 bytes --]

[-- Attachment #2: Type: text/plain, Size: 183 bytes --]

_______________________________________________
Virtualization mailing list
Virtualization@lists.linux-foundation.org
https://lists.linuxfoundation.org/mailman/listinfo/virtualization

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-16 13:16   ` Peter Zijlstra
  (?)
  (?)
@ 2015-03-18 20:50   ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-18 20:50 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch


[-- Attachment #1.1: Type: text/plain, Size: 10903 bytes --]

On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> Implement simple paravirt support for the qspinlock.
>
> Provide a separate (second) version of the spin_lock_slowpath for
> paravirt along with a special unlock path.
>
> The second slowpath is generated by adding a few pv hooks to the
> normal slowpath, but where those will compile away for the native
> case, they expand into special wait/wake code for the pv version.
>
> The actual MCS queue can use extra storage in the mcs_nodes[] array to
> keep track of state and therefore uses directed wakeups.
>
> The head contender has no such storage available and reverts to the
> per-cpu lock entry similar to the current kvm code. We can do a single
> enrty because any nesting will wake the vcpu and cause the lower loop
> to retry.
>
> Signed-off-by: Peter Zijlstra (Intel)<peterz@infradead.org>
> ---
>   include/asm-generic/qspinlock.h     |    3
>   kernel/locking/qspinlock.c          |   69 +++++++++++++-
>   kernel/locking/qspinlock_paravirt.h |  177 ++++++++++++++++++++++++++++++++++++
>   3 files changed, 248 insertions(+), 1 deletion(-)
>
> --- a/include/asm-generic/qspinlock.h
> +++ b/include/asm-generic/qspinlock.h
> @@ -118,6 +118,9 @@ static __always_inline bool virt_queue_s
>   }
>   #endif
>
> +extern void __pv_queue_spin_lock_slowpath(struct qspinlock *lock, u32 val);
> +extern void __pv_queue_spin_unlock(struct qspinlock *lock);
> +
>   /*
>    * Initializier
>    */
> --- a/kernel/locking/qspinlock.c
> +++ b/kernel/locking/qspinlock.c
> @@ -18,6 +18,9 @@
>    * Authors: Waiman Long<waiman.long@hp.com>
>    *          Peter Zijlstra<peterz@infradead.org>
>    */
> +
> +#ifndef _GEN_PV_LOCK_SLOWPATH
> +
>   #include<linux/smp.h>
>   #include<linux/bug.h>
>   #include<linux/cpumask.h>
> @@ -65,13 +68,21 @@
>
>   #include "mcs_spinlock.h"
>
> +#ifdef CONFIG_PARAVIRT_SPINLOCKS
> +#define MAX_NODES	8
> +#else
> +#define MAX_NODES	4
> +#endif
> +
>   /*
>    * Per-CPU queue node structures; we can never have more than 4 nested
>    * contexts: task, softirq, hardirq, nmi.
>    *
>    * Exactly fits one 64-byte cacheline on a 64-bit architecture.
> + *
> + * PV doubles the storage and uses the second cacheline for PV state.
>    */
> -static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[4]);
> +static DEFINE_PER_CPU_ALIGNED(struct mcs_spinlock, mcs_nodes[MAX_NODES]);
>
>   /*
>    * We must be able to distinguish between no-tail and the tail at 0:0,
> @@ -230,6 +241,32 @@ static __always_inline void set_locked(s
>   	WRITE_ONCE(l->locked, _Q_LOCKED_VAL);
>   }
>
> +
> +/*
> + * Generate the native code for queue_spin_unlock_slowpath(); provide NOPs for
> + * all the PV callbacks.
> + */
> +
> +static __always_inline void __pv_init_node(struct mcs_spinlock *node) { }
> +static __always_inline void __pv_wait_node(struct mcs_spinlock *node) { }
> +static __always_inline void __pv_kick_node(struct mcs_spinlock *node) { }
> +
> +static __always_inline void __pv_wait_head(struct qspinlock *lock) { }
> +
> +#define pv_enabled()		false
> +
> +#define pv_init_node		__pv_init_node
> +#define pv_wait_node		__pv_wait_node
> +#define pv_kick_node		__pv_kick_node
> +
> +#define pv_wait_head		__pv_wait_head
> +
> +#ifdef CONFIG_PARAVIRT_SPINLOCKS
> +#define queue_spin_lock_slowpath	native_queue_spin_lock_slowpath
> +#endif
> +
> +#endif /* _GEN_PV_LOCK_SLOWPATH */
> +
>   /**
>    * queue_spin_lock_slowpath - acquire the queue spinlock
>    * @lock: Pointer to queue spinlock structure
> @@ -259,6 +296,9 @@ void queue_spin_lock_slowpath(struct qsp
>
>   	BUILD_BUG_ON(CONFIG_NR_CPUS>= (1U<<  _Q_TAIL_CPU_BITS));
>
> +	if (pv_enabled())
> +		goto queue;
> +
>   	if (virt_queue_spin_lock(lock))
>   		return;
>
> @@ -335,6 +375,7 @@ void queue_spin_lock_slowpath(struct qsp
>   	node += idx;
>   	node->locked = 0;
>   	node->next = NULL;
> +	pv_init_node(node);
>
>   	/*
>   	 * We touched a (possibly) cold cacheline in the per-cpu queue node;
> @@ -360,6 +401,7 @@ void queue_spin_lock_slowpath(struct qsp
>   		prev = decode_tail(old);
>   		WRITE_ONCE(prev->next, node);
>
> +		pv_wait_node(node);
>   		arch_mcs_spin_lock_contended(&node->locked);
>   	}
>
> @@ -374,6 +416,7 @@ void queue_spin_lock_slowpath(struct qsp
>   	 * sequentiality; this is because the set_locked() function below
>   	 * does not imply a full barrier.
>   	 */
> +	pv_wait_head(lock);
>   	while ((val = smp_load_acquire(&lock->val.counter))&  _Q_LOCKED_PENDING_MASK)
>   		cpu_relax();
>
> @@ -406,6 +449,7 @@ void queue_spin_lock_slowpath(struct qsp
>   		cpu_relax();
>
>   	arch_mcs_spin_unlock_contended(&next->locked);
> +	pv_kick_node(next);
>
>   release:
>   	/*
> @@ -414,3 +458,26 @@ void queue_spin_lock_slowpath(struct qsp
>   	this_cpu_dec(mcs_nodes[0].count);
>   }
>   EXPORT_SYMBOL(queue_spin_lock_slowpath);
> +
> +/*
> + * Generate the paravirt code for queue_spin_unlock_slowpath().
> + */
> +#if !defined(_GEN_PV_LOCK_SLOWPATH)&&  defined(CONFIG_PARAVIRT_SPINLOCKS)
> +#define _GEN_PV_LOCK_SLOWPATH
> +
> +#undef pv_enabled
> +#define pv_enabled()	true
> +
> +#undef pv_init_node
> +#undef pv_wait_node
> +#undef pv_kick_node
> +
> +#undef pv_wait_head
> +
> +#undef queue_spin_lock_slowpath
> +#define queue_spin_lock_slowpath	__pv_queue_spin_lock_slowpath
> +
> +#include "qspinlock_paravirt.h"
> +#include "qspinlock.c"
> +
> +#endif
> --- /dev/null
> +++ b/kernel/locking/qspinlock_paravirt.h
> @@ -0,0 +1,177 @@
> +#ifndef _GEN_PV_LOCK_SLOWPATH
> +#error "do not include this file"
> +#endif
> +
> +/*
> + * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
> + * of spinning them.
> + *
> + * This relies on the architecture to provide two paravirt hypercalls:
> + *
> + *   pv_wait(u8 *ptr, u8 val) -- suspends the vcpu if *ptr == val
> + *   pv_kick(cpu)             -- wakes a suspended vcpu
> + *
> + * Using these we implement __pv_queue_spin_lock_slowpath() and
> + * __pv_queue_spin_unlock() to replace native_queue_spin_lock_slowpath() and
> + * native_queue_spin_unlock().
> + */
> +
> +#define _Q_SLOW_VAL	(2U<<  _Q_LOCKED_OFFSET)
> +
> +enum vcpu_state {
> +	vcpu_running = 0,
> +	vcpu_halted,
> +};
> +
> +struct pv_node {
> +	struct mcs_spinlock	mcs;
> +	struct mcs_spinlock	__res[3];
> +
> +	int			cpu;
> +	u8			state;
> +};
> +
> +/*
> + * Initialize the PV part of the mcs_spinlock node.
> + */
> +static void pv_init_node(struct mcs_spinlock *node)
> +{
> +	struct pv_node *pn = (struct pv_node *)node;
> +
> +	BUILD_BUG_ON(sizeof(struct pv_node)>  5*sizeof(struct mcs_spinlock));
> +
> +	pn->cpu = smp_processor_id();
> +	pn->state = vcpu_running;
> +}
> +
> +/*
> + * Wait for node->locked to become true, halt the vcpu after a short spin.
> + * pv_kick_node() is used to wake the vcpu again.
> + */
> +static void pv_wait_node(struct mcs_spinlock *node)
> +{
> +	struct pv_node *pn = (struct pv_node *)node;
> +	int loop;
> +
> +	for (;;) {
> +		for (loop = SPIN_THRESHOLD; loop; loop--) {
> +			if (READ_ONCE(node->locked))
> +				goto done;
> +
> +			cpu_relax();
> +		}
> +
> +		/*
> +		 * Order pn->state vs pn->locked thusly:
> +		 *
> +		 * [S] pn->state = vcpu_halted	  [S] next->locked = 1
> +		 *     MB			      MB
> +		 * [L] pn->locked		[RmW] pn->state = vcpu_running
> +		 *
> +		 * Matches the xchg() from pv_kick_node().
> +		 */
> +		(void)xchg(&pn->state, vcpu_halted);
> +
> +		if (READ_ONCE(node->locked))
> +			goto done;
> +
> +		pv_wait(&pn->state, vcpu_halted);
> +	}
> +done:
> +	pn->state = vcpu_running;
> +
> +	/*
> +	 * By now our node->locked should be 1 and our caller will not actually
> +	 * spin-wait for it. We do however rely on our caller to do a
> +	 * load-acquire for us.
> +	 */
> +}
> +
> +/*
> + * Called after setting next->locked = 1, used to wake those stuck in
> + * pv_wait_node().
> + */
> +static void pv_kick_node(struct mcs_spinlock *node)
> +{
> +	struct pv_node *pn = (struct pv_node *)node;
> +
> +	/*
> +	 * Note that because node->locked is already set, this actual mcs_spinlock
> +	 * entry could be re-used already.
> +	 *
> +	 * This should be fine however, kicking people for no reason is harmless.
> +	 *
> +	 * See the comment in pv_wait_node().
> +	 */
> +	if (xchg(&pn->state, vcpu_running) == vcpu_halted)
> +		pv_kick(pn->cpu);
> +}
> +
> +static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
> +
> +/*
> + * Wait for l->locked to become clear; halt the vcpu after a short spin.
> + * __pv_queue_spin_unlock() will wake us.
> + */
> +static void pv_wait_head(struct qspinlock *lock)
> +{
> +	struct __qspinlock *l = (void *)lock;
> +	int loop;
> +
> +	for (;;) {
> +		for (loop = SPIN_THRESHOLD; loop; loop--) {
> +			if (!READ_ONCE(l->locked))
> +				goto done;
> +
> +			cpu_relax();
> +		}
> +
> +		this_cpu_write(__pv_lock_wait, lock);

We may run into the same problem of needing to have 4 queue nodes per 
CPU. If an interrupt happens just after the write and before the actual 
wait and it goes through the same sequence, it will overwrite the 
__pv_lock_wait[] entry. So we may have lost wakeup. That is why the 
pvticket lock code did that just before the actual wait with interrupt 
disabled. We probably couldn't disable interrupt here. So we may need to 
move the actual write to the KVM and Xen code if we keep the current logic.

> +		/*
> +		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
> +		 *
> +		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
> +		 *     MB                             MB
> +		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
> +		 *
> +		 * Matches the xchg() in pv_queue_spin_unlock().
> +		 */
> +		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
> +			goto done;
> +
> +		pv_wait(&l->locked, _Q_SLOW_VAL);
> +	}
> +done:
> +	this_cpu_write(__pv_lock_wait, NULL);
> +
> +	/*
> +	 * Lock is unlocked now; the caller will acquire it without waiting.
> +	 * As with pv_wait_node() we rely on the caller to do a load-acquire
> +	 * for us.
> +	 */
> +}
> +
> +/*
> + * To be used in stead of queue_spin_unlock() for paravirt locks. Wakes
> + * pv_wait_head() if appropriate.
> + */
> +void __pv_queue_spin_unlock(struct qspinlock *lock)
> +{
> +	struct __qspinlock *l = (void *)lock;
> +	int cpu;
> +
> +	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
> +		return;
> +
> +	/*
> +	 * At this point the memory pointed at by lock can be freed/reused,
> +	 * however we can still use the pointer value to search in our cpu
> +	 * array.
> +	 *
> +	 * XXX: get rid of this loop
> +	 */
> +	for_each_possible_cpu(cpu) {
> +		if (per_cpu(__pv_lock_wait, cpu) == lock)
> +			pv_kick(cpu);
> +	}
> +}

I do want to get rid of this loop too. On average, we have to scan about 
half the number of CPUs available. So it isn't that different 
performance-wise compared with my original idea of following the list 
from tail to head. And how about your idea of propagating the current 
head down the linked list?

-Longman


[-- Attachment #1.2: Type: text/html, Size: 11627 bytes --]

[-- Attachment #2: Type: text/plain, Size: 126 bytes --]

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
  2015-03-16 13:16   ` [PATCH 9/9] qspinlock, x86, kvm: " Peter Zijlstra
  (?)
@ 2015-03-19  2:45   ` Waiman Long
  2015-03-19 10:01     ` Peter Zijlstra
  2015-03-19 10:01       ` Peter Zijlstra
  -1 siblings, 2 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-19  2:45 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch


[-- Attachment #1.1: Type: text/plain, Size: 1947 bytes --]

On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> Implement the paravirt qspinlock for x86-kvm.
>
> We use the regular paravirt call patching to switch between:
>
>    native_queue_spin_lock_slowpath()	__pv_queue_spin_lock_slowpath()
>    native_queue_spin_unlock()		__pv_queue_spin_unlock()
>
> We use a callee saved call for the unlock function which reduces the
> i-cache footprint and allows 'inlining' of SPIN_UNLOCK functions
> again.
>
> We further optimize the unlock path by patching the direct call with a
> "movb $0,%arg1" if we are indeed using the native unlock code. This
> makes the unlock code almost as fast as the !PARAVIRT case.
>
> This significantly lowers the overhead of having
> CONFIG_PARAVIRT_SPINLOCKS enabled, even for native code.
>
>
> Signed-off-by: Peter Zijlstra (Intel)<peterz@infradead.org>

I do have some concern about this call site patching mechanism as the 
modification is not atomic. The spin_unlock() calls are in many places 
in the kernel. There is a possibility that a thread is calling a certain 
spin_unlock call site while it is being patched by another one with the 
alternative() function call.

So far, I don't see any problem with bare metal where 
paravirt_patch_insns() is used to patch it to the move instruction. 
However, in a virtual guest enivornment where paravirt_patch_call() was 
used, there were situations where the system panic because of page fault 
on some invalid memory in the kthread. If you look at the 
paravirt_patch_call(), you will see:

     :
b->opcode = 0xe8; /* call */
b->delta = delta;

If another CPU reads the instruction at the call site at the right 
moment, it will get the modified call instruction, but not the new delta 
value. It will then jump to a random location. I believe that was 
causing the system panic that I saw.

So I think it is kind of risky to use it here unless we can guarantee 
that call site patching is atomic wrt other CPUs.

-Longman


[-- Attachment #1.2: Type: text/html, Size: 2703 bytes --]

[-- Attachment #2: Type: text/plain, Size: 183 bytes --]

_______________________________________________
Virtualization mailing list
Virtualization@lists.linux-foundation.org
https://lists.linuxfoundation.org/mailman/listinfo/virtualization

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

* Re: [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
  2015-03-16 13:16   ` [PATCH 9/9] qspinlock, x86, kvm: " Peter Zijlstra
  (?)
  (?)
@ 2015-03-19  2:45   ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-19  2:45 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch


[-- Attachment #1.1: Type: text/plain, Size: 1947 bytes --]

On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> Implement the paravirt qspinlock for x86-kvm.
>
> We use the regular paravirt call patching to switch between:
>
>    native_queue_spin_lock_slowpath()	__pv_queue_spin_lock_slowpath()
>    native_queue_spin_unlock()		__pv_queue_spin_unlock()
>
> We use a callee saved call for the unlock function which reduces the
> i-cache footprint and allows 'inlining' of SPIN_UNLOCK functions
> again.
>
> We further optimize the unlock path by patching the direct call with a
> "movb $0,%arg1" if we are indeed using the native unlock code. This
> makes the unlock code almost as fast as the !PARAVIRT case.
>
> This significantly lowers the overhead of having
> CONFIG_PARAVIRT_SPINLOCKS enabled, even for native code.
>
>
> Signed-off-by: Peter Zijlstra (Intel)<peterz@infradead.org>

I do have some concern about this call site patching mechanism as the 
modification is not atomic. The spin_unlock() calls are in many places 
in the kernel. There is a possibility that a thread is calling a certain 
spin_unlock call site while it is being patched by another one with the 
alternative() function call.

So far, I don't see any problem with bare metal where 
paravirt_patch_insns() is used to patch it to the move instruction. 
However, in a virtual guest enivornment where paravirt_patch_call() was 
used, there were situations where the system panic because of page fault 
on some invalid memory in the kthread. If you look at the 
paravirt_patch_call(), you will see:

     :
b->opcode = 0xe8; /* call */
b->delta = delta;

If another CPU reads the instruction at the call site at the right 
moment, it will get the modified call instruction, but not the new delta 
value. It will then jump to a random location. I believe that was 
causing the system panic that I saw.

So I think it is kind of risky to use it here unless we can guarantee 
that call site patching is atomic wrt other CPUs.

-Longman


[-- Attachment #1.2: Type: text/html, Size: 2703 bytes --]

[-- Attachment #2: Type: text/plain, Size: 126 bytes --]

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH 9/9] qspinlock,x86,kvm: Implement KVM support for paravirt qspinlock
  2015-03-19  2:45   ` Waiman Long
@ 2015-03-19 10:01       ` Peter Zijlstra
  2015-03-19 10:01       ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 10:01 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Wed, Mar 18, 2015 at 10:45:55PM -0400, Waiman Long wrote:
> On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> I do have some concern about this call site patching mechanism as the
> modification is not atomic. The spin_unlock() calls are in many places in
> the kernel. There is a possibility that a thread is calling a certain
> spin_unlock call site while it is being patched by another one with the
> alternative() function call.
> 
> So far, I don't see any problem with bare metal where paravirt_patch_insns()
> is used to patch it to the move instruction. However, in a virtual guest
> enivornment where paravirt_patch_call() was used, there were situations
> where the system panic because of page fault on some invalid memory in the
> kthread. If you look at the paravirt_patch_call(), you will see:
> 
>     :
> b->opcode = 0xe8; /* call */
> b->delta = delta;
> 
> If another CPU reads the instruction at the call site at the right moment,
> it will get the modified call instruction, but not the new delta value. It
> will then jump to a random location. I believe that was causing the system
> panic that I saw.
> 
> So I think it is kind of risky to use it here unless we can guarantee that
> call site patching is atomic wrt other CPUs.

Just look at where the patching is done:

init/main.c:start_kernel()
  check_bugs()
    alternative_instructions()
      apply_paravirt()

We're UP and not holding any locks, disable IRQs (see text_poke_early())
and have NMIs 'disabled'.

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

* Re: [PATCH 9/9] qspinlock,x86,kvm: Implement KVM support for paravirt qspinlock
@ 2015-03-19 10:01       ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 10:01 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Mar 18, 2015 at 10:45:55PM -0400, Waiman Long wrote:
> On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> I do have some concern about this call site patching mechanism as the
> modification is not atomic. The spin_unlock() calls are in many places in
> the kernel. There is a possibility that a thread is calling a certain
> spin_unlock call site while it is being patched by another one with the
> alternative() function call.
> 
> So far, I don't see any problem with bare metal where paravirt_patch_insns()
> is used to patch it to the move instruction. However, in a virtual guest
> enivornment where paravirt_patch_call() was used, there were situations
> where the system panic because of page fault on some invalid memory in the
> kthread. If you look at the paravirt_patch_call(), you will see:
> 
>     :
> b->opcode = 0xe8; /* call */
> b->delta = delta;
> 
> If another CPU reads the instruction at the call site at the right moment,
> it will get the modified call instruction, but not the new delta value. It
> will then jump to a random location. I believe that was causing the system
> panic that I saw.
> 
> So I think it is kind of risky to use it here unless we can guarantee that
> call site patching is atomic wrt other CPUs.

Just look at where the patching is done:

init/main.c:start_kernel()
  check_bugs()
    alternative_instructions()
      apply_paravirt()

We're UP and not holding any locks, disable IRQs (see text_poke_early())
and have NMIs 'disabled'.

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

* Re: [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
  2015-03-19  2:45   ` Waiman Long
@ 2015-03-19 10:01     ` Peter Zijlstra
  2015-03-19 10:01       ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 10:01 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Wed, Mar 18, 2015 at 10:45:55PM -0400, Waiman Long wrote:
> On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
> I do have some concern about this call site patching mechanism as the
> modification is not atomic. The spin_unlock() calls are in many places in
> the kernel. There is a possibility that a thread is calling a certain
> spin_unlock call site while it is being patched by another one with the
> alternative() function call.
> 
> So far, I don't see any problem with bare metal where paravirt_patch_insns()
> is used to patch it to the move instruction. However, in a virtual guest
> enivornment where paravirt_patch_call() was used, there were situations
> where the system panic because of page fault on some invalid memory in the
> kthread. If you look at the paravirt_patch_call(), you will see:
> 
>     :
> b->opcode = 0xe8; /* call */
> b->delta = delta;
> 
> If another CPU reads the instruction at the call site at the right moment,
> it will get the modified call instruction, but not the new delta value. It
> will then jump to a random location. I believe that was causing the system
> panic that I saw.
> 
> So I think it is kind of risky to use it here unless we can guarantee that
> call site patching is atomic wrt other CPUs.

Just look at where the patching is done:

init/main.c:start_kernel()
  check_bugs()
    alternative_instructions()
      apply_paravirt()

We're UP and not holding any locks, disable IRQs (see text_poke_early())
and have NMIs 'disabled'.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-18 20:50   ` Waiman Long
  2015-03-19 10:12     ` Peter Zijlstra
  2015-03-19 10:12     ` Peter Zijlstra
@ 2015-03-19 10:12     ` Peter Zijlstra
  2015-03-19 12:25         ` Peter Zijlstra
  2015-03-19 12:25       ` Peter Zijlstra
  2 siblings, 2 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 10:12 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Wed, Mar 18, 2015 at 04:50:37PM -0400, Waiman Long wrote:
> >+		this_cpu_write(__pv_lock_wait, lock);
> 
> We may run into the same problem of needing to have 4 queue nodes per CPU.
> If an interrupt happens just after the write and before the actual wait and
> it goes through the same sequence, it will overwrite the __pv_lock_wait[]
> entry. So we may have lost wakeup. That is why the pvticket lock code did
> that just before the actual wait with interrupt disabled. We probably
> couldn't disable interrupt here. So we may need to move the actual write to
> the KVM and Xen code if we keep the current logic.

Hmm indeed. So I didn't actually mean to keep this code, but yes I
missed that.

> >+	/*
> >+	 * At this point the memory pointed at by lock can be freed/reused,
> >+	 * however we can still use the pointer value to search in our cpu
> >+	 * array.
> >+	 *
> >+	 * XXX: get rid of this loop
> >+	 */
> >+	for_each_possible_cpu(cpu) {
> >+		if (per_cpu(__pv_lock_wait, cpu) == lock)
> >+			pv_kick(cpu);
> >+	}
> >+}
> 
> I do want to get rid of this loop too. On average, we have to scan about
> half the number of CPUs available. So it isn't that different
> performance-wise compared with my original idea of following the list from
> tail to head. And how about your idea of propagating the current head down
> the linked list?

Yeah, so I was half done with that patch when I fell asleep on the
flight home. Didn't get around to 'fixing' it. It applies and builds but
doesn't actually work.

_However_ I'm not sure I actually like it much.

The problem I have with it are these wait loops, they can generate the
same problem we're trying to avoid.

So I was now thinking of hashing the lock pointer; let me go and quickly
put something together.

---
 kernel/locking/qspinlock.c          |    8 +-
 kernel/locking/qspinlock_paravirt.h |  124 ++++++++++++++++++++++++++++--------
 2 files changed, 101 insertions(+), 31 deletions(-)

--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -246,10 +246,10 @@ static __always_inline void set_locked(s
  */
 
 static __always_inline void __pv_init_node(struct mcs_spinlock *node) { }
-static __always_inline void __pv_wait_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_wait_node(u32 old, struct mcs_spinlock *node) { }
 static __always_inline void __pv_kick_node(struct mcs_spinlock *node) { }
 
-static __always_inline void __pv_wait_head(struct qspinlock *lock) { }
+static __always_inline void __pv_wait_head(struct qspinlock *lock, struct mcs_spinlock *node) { }
 
 #define pv_enabled()           false
 
@@ -399,7 +399,7 @@ void queue_spin_lock_slowpath(struct qsp
                prev = decode_tail(old);
                WRITE_ONCE(prev->next, node);
 
-               pv_wait_node(node);
+               pv_wait_node(old, node);
                arch_mcs_spin_lock_contended(&node->locked);
        }
 
@@ -414,7 +414,7 @@ void queue_spin_lock_slowpath(struct qsp
         * sequentiality; this is because the set_locked() function below
         * does not imply a full barrier.
         */
-       pv_wait_head(lock);
+       pv_wait_head(lock, node);
        while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
                cpu_relax();
 
--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -29,8 +29,14 @@ struct pv_node {
 
        int                     cpu;
        u8                      state;
+       struct pv_node          *head;
 };
 
+static inline struct pv_node *pv_decode_tail(u32 tail)
+{
+       return (struct pv_node *)decode_tail(tail);
+}
+
 /*
  * Initialize the PV part of the mcs_spinlock node.
  */
@@ -42,17 +48,49 @@ static void pv_init_node(struct mcs_spin
 
        pn->cpu = smp_processor_id();
        pn->state = vcpu_running;
+       pn->head = NULL;
+}
+
+static void pv_propagate_head(struct pv_node *pn, struct pv_node *ppn)
+{
+       /*
+        * When we race against the first waiter or ourselves we have to wait
+        * until the previous link updates its head pointer before we can
+        * propagate it.
+        */
+       while (!READ_ONCE(ppn->head)) {
+               /*
+                * queue_spin_lock_slowpath could have been waiting for the
+                * node->next store above before setting node->locked.
+                */
+               if (ppn->mcs.locked)
+                       return;
+
+               cpu_relax();
+       }
+       /*
+        * If we interleave such that the store from pv_set_head() happens
+        * between our load and store we could have over-written the new head
+        * value.
+        *
+        * Therefore use cmpxchg to only propagate the old head value if our
+        * head value is unmodified.
+        */
+       (void)cmpxchg(&pn->head, NULL, READ_ONCE(ppn->head));
 }
 
 /*
  * Wait for node->locked to become true, halt the vcpu after a short spin.
  * pv_kick_node() is used to wake the vcpu again.
  */
-static void pv_wait_node(struct mcs_spinlock *node)
+static void pv_wait_node(u32 old, struct mcs_spinlock *node)
 {
+       struct pv_node *ppn = pv_decode_tail(old);
        struct pv_node *pn = (struct pv_node *)node;
        int loop;
 
+       pv_propagate_head(pn, ppn);
+
        for (;;) {
                for (loop = SPIN_THRESHOLD; loop; loop--) {
                        if (READ_ONCE(node->locked))
@@ -107,13 +145,33 @@ static void pv_kick_node(struct mcs_spin
                pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+static void pv_set_head(struct qspinlock *lock, struct pv_node *head)
+{
+       struct pv_node *tail, *new_tail;
+
+       new_tail = pv_decode_tail(atomic_read(&lock->val));
+       do {
+               tail = new_tail;
+               while (!READ_ONCE(tail->head))
+                       cpu_relax();
+
+               (void)xchg(&tail->head, head);
+               /*
+                * pv_set_head()                pv_wait_node()
+                *
+                * [S] tail->head, head         [S] lock->tail
+                *     MB                           MB
+                * [L] lock->tail               [L] tail->head
+                */
+               new_tail = pv_decode_tail(atomic_read(&lock->val));
+       } while (tail != new_tail);
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
  * __pv_queue_spin_unlock() will wake us.
  */
-static void pv_wait_head(struct qspinlock *lock)
+static void pv_wait_head(struct qspinlock *lock, struct mcs_spinlock *node)
 {
        struct __qspinlock *l = (void *)lock;
        int loop;
@@ -121,28 +179,24 @@ static void pv_wait_head(struct qspinloc
        for (;;) {
                for (loop = SPIN_THRESHOLD; loop; loop--) {
                        if (!READ_ONCE(l->locked))
-                               goto done;
+                               return;
 
                        cpu_relax();
                }
 
-               this_cpu_write(__pv_lock_wait, lock);
+               pv_set_head(lock, (struct pv_node *)node);
                /*
-                * __pv_lock_wait must be set before setting _Q_SLOW_VAL
-                *
-                * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
-                *     MB                             MB
-                * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+                * The head must be set before we set _Q_SLOW_VAL such that
+                * when pv_queue_spin_unlock() observes _Q_SLOW_VAL it must
+                * also observe the head pointer.
                 *
-                * Matches the xchg() in pv_queue_spin_unlock().
+                * We rely on the implied MB from the below cmpxchg()
                 */
                if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
-                       goto done;
+                       return;
 
                pv_wait(&l->locked, _Q_SLOW_VAL);
        }
-done:
-       this_cpu_write(__pv_lock_wait, NULL);
 
        /*
         * Lock is unlocked now; the caller will acquire it without waiting.
@@ -157,21 +211,37 @@ static void pv_wait_head(struct qspinloc
  */
 void __pv_queue_spin_unlock(struct qspinlock *lock)
 {
-       struct __qspinlock *l = (void *)lock;
-       int cpu;
+       u32 old, val = atomic_read(&lock->val);
+       struct pv_node *pn = NULL;
 
-       if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
-               return;
+       for (;;) {
+               if (val & _Q_SLOW_VAL) {
+                       /*
+                        * If we observe _Q_SLOW_VAL we must also observe
+                        * pn->head; see pv_wait_head();
+                        */
+                       smp_rmb();
+                       pn = pv_decode_tail(val);
+                       while (!READ_ONCE(pn->head))
+                               cpu_relax();
+                       pn = READ_ONCE(pn->head);
+               }
+               /*
+                * The cmpxchg() ensures the above loads are complete before
+                * we release the lock.
+                */
+               old = atomic_cmpxchg(&lock->val, val, val & _Q_TAIL_MASK);
+               if (old == val)
+                       break;
+
+               val = old;
+       }
 
        /*
-        * At this point the memory pointed at by lock can be freed/reused,
-        * however we can still use the pointer value to search in our cpu
-        * array.
-        *
-        * XXX: get rid of this loop
+        * We've freed the lock so the lock storage can be gone; however since
+        * the pv_node structure is static storage we can safely use that
+        * still.
         */
-       for_each_possible_cpu(cpu) {
-               if (per_cpu(__pv_lock_wait, cpu) == lock)
-                       pv_kick(cpu);
-       }
+       if (pn)
+               pv_kick(pn->cpu);
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-18 20:50   ` Waiman Long
@ 2015-03-19 10:12     ` Peter Zijlstra
  2015-03-19 10:12     ` Peter Zijlstra
  2015-03-19 10:12     ` Peter Zijlstra
  2 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 10:12 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Mar 18, 2015 at 04:50:37PM -0400, Waiman Long wrote:
> >+		this_cpu_write(__pv_lock_wait, lock);
> 
> We may run into the same problem of needing to have 4 queue nodes per CPU.
> If an interrupt happens just after the write and before the actual wait and
> it goes through the same sequence, it will overwrite the __pv_lock_wait[]
> entry. So we may have lost wakeup. That is why the pvticket lock code did
> that just before the actual wait with interrupt disabled. We probably
> couldn't disable interrupt here. So we may need to move the actual write to
> the KVM and Xen code if we keep the current logic.

Hmm indeed. So I didn't actually mean to keep this code, but yes I
missed that.

> >+	/*
> >+	 * At this point the memory pointed at by lock can be freed/reused,
> >+	 * however we can still use the pointer value to search in our cpu
> >+	 * array.
> >+	 *
> >+	 * XXX: get rid of this loop
> >+	 */
> >+	for_each_possible_cpu(cpu) {
> >+		if (per_cpu(__pv_lock_wait, cpu) == lock)
> >+			pv_kick(cpu);
> >+	}
> >+}
> 
> I do want to get rid of this loop too. On average, we have to scan about
> half the number of CPUs available. So it isn't that different
> performance-wise compared with my original idea of following the list from
> tail to head. And how about your idea of propagating the current head down
> the linked list?

Yeah, so I was half done with that patch when I fell asleep on the
flight home. Didn't get around to 'fixing' it. It applies and builds but
doesn't actually work.

_However_ I'm not sure I actually like it much.

The problem I have with it are these wait loops, they can generate the
same problem we're trying to avoid.

So I was now thinking of hashing the lock pointer; let me go and quickly
put something together.

---
 kernel/locking/qspinlock.c          |    8 +-
 kernel/locking/qspinlock_paravirt.h |  124 ++++++++++++++++++++++++++++--------
 2 files changed, 101 insertions(+), 31 deletions(-)

--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -246,10 +246,10 @@ static __always_inline void set_locked(s
  */
 
 static __always_inline void __pv_init_node(struct mcs_spinlock *node) { }
-static __always_inline void __pv_wait_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_wait_node(u32 old, struct mcs_spinlock *node) { }
 static __always_inline void __pv_kick_node(struct mcs_spinlock *node) { }
 
-static __always_inline void __pv_wait_head(struct qspinlock *lock) { }
+static __always_inline void __pv_wait_head(struct qspinlock *lock, struct mcs_spinlock *node) { }
 
 #define pv_enabled()           false
 
@@ -399,7 +399,7 @@ void queue_spin_lock_slowpath(struct qsp
                prev = decode_tail(old);
                WRITE_ONCE(prev->next, node);
 
-               pv_wait_node(node);
+               pv_wait_node(old, node);
                arch_mcs_spin_lock_contended(&node->locked);
        }
 
@@ -414,7 +414,7 @@ void queue_spin_lock_slowpath(struct qsp
         * sequentiality; this is because the set_locked() function below
         * does not imply a full barrier.
         */
-       pv_wait_head(lock);
+       pv_wait_head(lock, node);
        while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
                cpu_relax();
 
--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -29,8 +29,14 @@ struct pv_node {
 
        int                     cpu;
        u8                      state;
+       struct pv_node          *head;
 };
 
+static inline struct pv_node *pv_decode_tail(u32 tail)
+{
+       return (struct pv_node *)decode_tail(tail);
+}
+
 /*
  * Initialize the PV part of the mcs_spinlock node.
  */
@@ -42,17 +48,49 @@ static void pv_init_node(struct mcs_spin
 
        pn->cpu = smp_processor_id();
        pn->state = vcpu_running;
+       pn->head = NULL;
+}
+
+static void pv_propagate_head(struct pv_node *pn, struct pv_node *ppn)
+{
+       /*
+        * When we race against the first waiter or ourselves we have to wait
+        * until the previous link updates its head pointer before we can
+        * propagate it.
+        */
+       while (!READ_ONCE(ppn->head)) {
+               /*
+                * queue_spin_lock_slowpath could have been waiting for the
+                * node->next store above before setting node->locked.
+                */
+               if (ppn->mcs.locked)
+                       return;
+
+               cpu_relax();
+       }
+       /*
+        * If we interleave such that the store from pv_set_head() happens
+        * between our load and store we could have over-written the new head
+        * value.
+        *
+        * Therefore use cmpxchg to only propagate the old head value if our
+        * head value is unmodified.
+        */
+       (void)cmpxchg(&pn->head, NULL, READ_ONCE(ppn->head));
 }
 
 /*
  * Wait for node->locked to become true, halt the vcpu after a short spin.
  * pv_kick_node() is used to wake the vcpu again.
  */
-static void pv_wait_node(struct mcs_spinlock *node)
+static void pv_wait_node(u32 old, struct mcs_spinlock *node)
 {
+       struct pv_node *ppn = pv_decode_tail(old);
        struct pv_node *pn = (struct pv_node *)node;
        int loop;
 
+       pv_propagate_head(pn, ppn);
+
        for (;;) {
                for (loop = SPIN_THRESHOLD; loop; loop--) {
                        if (READ_ONCE(node->locked))
@@ -107,13 +145,33 @@ static void pv_kick_node(struct mcs_spin
                pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+static void pv_set_head(struct qspinlock *lock, struct pv_node *head)
+{
+       struct pv_node *tail, *new_tail;
+
+       new_tail = pv_decode_tail(atomic_read(&lock->val));
+       do {
+               tail = new_tail;
+               while (!READ_ONCE(tail->head))
+                       cpu_relax();
+
+               (void)xchg(&tail->head, head);
+               /*
+                * pv_set_head()                pv_wait_node()
+                *
+                * [S] tail->head, head         [S] lock->tail
+                *     MB                           MB
+                * [L] lock->tail               [L] tail->head
+                */
+               new_tail = pv_decode_tail(atomic_read(&lock->val));
+       } while (tail != new_tail);
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
  * __pv_queue_spin_unlock() will wake us.
  */
-static void pv_wait_head(struct qspinlock *lock)
+static void pv_wait_head(struct qspinlock *lock, struct mcs_spinlock *node)
 {
        struct __qspinlock *l = (void *)lock;
        int loop;
@@ -121,28 +179,24 @@ static void pv_wait_head(struct qspinloc
        for (;;) {
                for (loop = SPIN_THRESHOLD; loop; loop--) {
                        if (!READ_ONCE(l->locked))
-                               goto done;
+                               return;
 
                        cpu_relax();
                }
 
-               this_cpu_write(__pv_lock_wait, lock);
+               pv_set_head(lock, (struct pv_node *)node);
                /*
-                * __pv_lock_wait must be set before setting _Q_SLOW_VAL
-                *
-                * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
-                *     MB                             MB
-                * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+                * The head must be set before we set _Q_SLOW_VAL such that
+                * when pv_queue_spin_unlock() observes _Q_SLOW_VAL it must
+                * also observe the head pointer.
                 *
-                * Matches the xchg() in pv_queue_spin_unlock().
+                * We rely on the implied MB from the below cmpxchg()
                 */
                if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
-                       goto done;
+                       return;
 
                pv_wait(&l->locked, _Q_SLOW_VAL);
        }
-done:
-       this_cpu_write(__pv_lock_wait, NULL);
 
        /*
         * Lock is unlocked now; the caller will acquire it without waiting.
@@ -157,21 +211,37 @@ static void pv_wait_head(struct qspinloc
  */
 void __pv_queue_spin_unlock(struct qspinlock *lock)
 {
-       struct __qspinlock *l = (void *)lock;
-       int cpu;
+       u32 old, val = atomic_read(&lock->val);
+       struct pv_node *pn = NULL;
 
-       if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
-               return;
+       for (;;) {
+               if (val & _Q_SLOW_VAL) {
+                       /*
+                        * If we observe _Q_SLOW_VAL we must also observe
+                        * pn->head; see pv_wait_head();
+                        */
+                       smp_rmb();
+                       pn = pv_decode_tail(val);
+                       while (!READ_ONCE(pn->head))
+                               cpu_relax();
+                       pn = READ_ONCE(pn->head);
+               }
+               /*
+                * The cmpxchg() ensures the above loads are complete before
+                * we release the lock.
+                */
+               old = atomic_cmpxchg(&lock->val, val, val & _Q_TAIL_MASK);
+               if (old == val)
+                       break;
+
+               val = old;
+       }
 
        /*
-        * At this point the memory pointed at by lock can be freed/reused,
-        * however we can still use the pointer value to search in our cpu
-        * array.
-        *
-        * XXX: get rid of this loop
+        * We've freed the lock so the lock storage can be gone; however since
+        * the pv_node structure is static storage we can safely use that
+        * still.
         */
-       for_each_possible_cpu(cpu) {
-               if (per_cpu(__pv_lock_wait, cpu) == lock)
-                       pv_kick(cpu);
-       }
+       if (pn)
+               pv_kick(pn->cpu);
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-18 20:50   ` Waiman Long
  2015-03-19 10:12     ` Peter Zijlstra
@ 2015-03-19 10:12     ` Peter Zijlstra
  2015-03-19 10:12     ` Peter Zijlstra
  2 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 10:12 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Wed, Mar 18, 2015 at 04:50:37PM -0400, Waiman Long wrote:
> >+		this_cpu_write(__pv_lock_wait, lock);
> 
> We may run into the same problem of needing to have 4 queue nodes per CPU.
> If an interrupt happens just after the write and before the actual wait and
> it goes through the same sequence, it will overwrite the __pv_lock_wait[]
> entry. So we may have lost wakeup. That is why the pvticket lock code did
> that just before the actual wait with interrupt disabled. We probably
> couldn't disable interrupt here. So we may need to move the actual write to
> the KVM and Xen code if we keep the current logic.

Hmm indeed. So I didn't actually mean to keep this code, but yes I
missed that.

> >+	/*
> >+	 * At this point the memory pointed at by lock can be freed/reused,
> >+	 * however we can still use the pointer value to search in our cpu
> >+	 * array.
> >+	 *
> >+	 * XXX: get rid of this loop
> >+	 */
> >+	for_each_possible_cpu(cpu) {
> >+		if (per_cpu(__pv_lock_wait, cpu) == lock)
> >+			pv_kick(cpu);
> >+	}
> >+}
> 
> I do want to get rid of this loop too. On average, we have to scan about
> half the number of CPUs available. So it isn't that different
> performance-wise compared with my original idea of following the list from
> tail to head. And how about your idea of propagating the current head down
> the linked list?

Yeah, so I was half done with that patch when I fell asleep on the
flight home. Didn't get around to 'fixing' it. It applies and builds but
doesn't actually work.

_However_ I'm not sure I actually like it much.

The problem I have with it are these wait loops, they can generate the
same problem we're trying to avoid.

So I was now thinking of hashing the lock pointer; let me go and quickly
put something together.

---
 kernel/locking/qspinlock.c          |    8 +-
 kernel/locking/qspinlock_paravirt.h |  124 ++++++++++++++++++++++++++++--------
 2 files changed, 101 insertions(+), 31 deletions(-)

--- a/kernel/locking/qspinlock.c
+++ b/kernel/locking/qspinlock.c
@@ -246,10 +246,10 @@ static __always_inline void set_locked(s
  */
 
 static __always_inline void __pv_init_node(struct mcs_spinlock *node) { }
-static __always_inline void __pv_wait_node(struct mcs_spinlock *node) { }
+static __always_inline void __pv_wait_node(u32 old, struct mcs_spinlock *node) { }
 static __always_inline void __pv_kick_node(struct mcs_spinlock *node) { }
 
-static __always_inline void __pv_wait_head(struct qspinlock *lock) { }
+static __always_inline void __pv_wait_head(struct qspinlock *lock, struct mcs_spinlock *node) { }
 
 #define pv_enabled()           false
 
@@ -399,7 +399,7 @@ void queue_spin_lock_slowpath(struct qsp
                prev = decode_tail(old);
                WRITE_ONCE(prev->next, node);
 
-               pv_wait_node(node);
+               pv_wait_node(old, node);
                arch_mcs_spin_lock_contended(&node->locked);
        }
 
@@ -414,7 +414,7 @@ void queue_spin_lock_slowpath(struct qsp
         * sequentiality; this is because the set_locked() function below
         * does not imply a full barrier.
         */
-       pv_wait_head(lock);
+       pv_wait_head(lock, node);
        while ((val = smp_load_acquire(&lock->val.counter)) & _Q_LOCKED_PENDING_MASK)
                cpu_relax();
 
--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -29,8 +29,14 @@ struct pv_node {
 
        int                     cpu;
        u8                      state;
+       struct pv_node          *head;
 };
 
+static inline struct pv_node *pv_decode_tail(u32 tail)
+{
+       return (struct pv_node *)decode_tail(tail);
+}
+
 /*
  * Initialize the PV part of the mcs_spinlock node.
  */
@@ -42,17 +48,49 @@ static void pv_init_node(struct mcs_spin
 
        pn->cpu = smp_processor_id();
        pn->state = vcpu_running;
+       pn->head = NULL;
+}
+
+static void pv_propagate_head(struct pv_node *pn, struct pv_node *ppn)
+{
+       /*
+        * When we race against the first waiter or ourselves we have to wait
+        * until the previous link updates its head pointer before we can
+        * propagate it.
+        */
+       while (!READ_ONCE(ppn->head)) {
+               /*
+                * queue_spin_lock_slowpath could have been waiting for the
+                * node->next store above before setting node->locked.
+                */
+               if (ppn->mcs.locked)
+                       return;
+
+               cpu_relax();
+       }
+       /*
+        * If we interleave such that the store from pv_set_head() happens
+        * between our load and store we could have over-written the new head
+        * value.
+        *
+        * Therefore use cmpxchg to only propagate the old head value if our
+        * head value is unmodified.
+        */
+       (void)cmpxchg(&pn->head, NULL, READ_ONCE(ppn->head));
 }
 
 /*
  * Wait for node->locked to become true, halt the vcpu after a short spin.
  * pv_kick_node() is used to wake the vcpu again.
  */
-static void pv_wait_node(struct mcs_spinlock *node)
+static void pv_wait_node(u32 old, struct mcs_spinlock *node)
 {
+       struct pv_node *ppn = pv_decode_tail(old);
        struct pv_node *pn = (struct pv_node *)node;
        int loop;
 
+       pv_propagate_head(pn, ppn);
+
        for (;;) {
                for (loop = SPIN_THRESHOLD; loop; loop--) {
                        if (READ_ONCE(node->locked))
@@ -107,13 +145,33 @@ static void pv_kick_node(struct mcs_spin
                pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+static void pv_set_head(struct qspinlock *lock, struct pv_node *head)
+{
+       struct pv_node *tail, *new_tail;
+
+       new_tail = pv_decode_tail(atomic_read(&lock->val));
+       do {
+               tail = new_tail;
+               while (!READ_ONCE(tail->head))
+                       cpu_relax();
+
+               (void)xchg(&tail->head, head);
+               /*
+                * pv_set_head()                pv_wait_node()
+                *
+                * [S] tail->head, head         [S] lock->tail
+                *     MB                           MB
+                * [L] lock->tail               [L] tail->head
+                */
+               new_tail = pv_decode_tail(atomic_read(&lock->val));
+       } while (tail != new_tail);
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
  * __pv_queue_spin_unlock() will wake us.
  */
-static void pv_wait_head(struct qspinlock *lock)
+static void pv_wait_head(struct qspinlock *lock, struct mcs_spinlock *node)
 {
        struct __qspinlock *l = (void *)lock;
        int loop;
@@ -121,28 +179,24 @@ static void pv_wait_head(struct qspinloc
        for (;;) {
                for (loop = SPIN_THRESHOLD; loop; loop--) {
                        if (!READ_ONCE(l->locked))
-                               goto done;
+                               return;
 
                        cpu_relax();
                }
 
-               this_cpu_write(__pv_lock_wait, lock);
+               pv_set_head(lock, (struct pv_node *)node);
                /*
-                * __pv_lock_wait must be set before setting _Q_SLOW_VAL
-                *
-                * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
-                *     MB                             MB
-                * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+                * The head must be set before we set _Q_SLOW_VAL such that
+                * when pv_queue_spin_unlock() observes _Q_SLOW_VAL it must
+                * also observe the head pointer.
                 *
-                * Matches the xchg() in pv_queue_spin_unlock().
+                * We rely on the implied MB from the below cmpxchg()
                 */
                if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
-                       goto done;
+                       return;
 
                pv_wait(&l->locked, _Q_SLOW_VAL);
        }
-done:
-       this_cpu_write(__pv_lock_wait, NULL);
 
        /*
         * Lock is unlocked now; the caller will acquire it without waiting.
@@ -157,21 +211,37 @@ static void pv_wait_head(struct qspinloc
  */
 void __pv_queue_spin_unlock(struct qspinlock *lock)
 {
-       struct __qspinlock *l = (void *)lock;
-       int cpu;
+       u32 old, val = atomic_read(&lock->val);
+       struct pv_node *pn = NULL;
 
-       if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
-               return;
+       for (;;) {
+               if (val & _Q_SLOW_VAL) {
+                       /*
+                        * If we observe _Q_SLOW_VAL we must also observe
+                        * pn->head; see pv_wait_head();
+                        */
+                       smp_rmb();
+                       pn = pv_decode_tail(val);
+                       while (!READ_ONCE(pn->head))
+                               cpu_relax();
+                       pn = READ_ONCE(pn->head);
+               }
+               /*
+                * The cmpxchg() ensures the above loads are complete before
+                * we release the lock.
+                */
+               old = atomic_cmpxchg(&lock->val, val, val & _Q_TAIL_MASK);
+               if (old == val)
+                       break;
+
+               val = old;
+       }
 
        /*
-        * At this point the memory pointed at by lock can be freed/reused,
-        * however we can still use the pointer value to search in our cpu
-        * array.
-        *
-        * XXX: get rid of this loop
+        * We've freed the lock so the lock storage can be gone; however since
+        * the pv_node structure is static storage we can safely use that
+        * still.
         */
-       for_each_possible_cpu(cpu) {
-               if (per_cpu(__pv_lock_wait, cpu) == lock)
-                       pv_kick(cpu);
-       }
+       if (pn)
+               pv_kick(pn->cpu);
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 10:12     ` Peter Zijlstra
@ 2015-03-19 12:25         ` Peter Zijlstra
  2015-03-19 12:25       ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 12:25 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
> So I was now thinking of hashing the lock pointer; let me go and quickly
> put something together.

A little something like so; ideally we'd allocate the hashtable since
NR_CPUS is kinda bloated, but it shows the idea I think.

And while this has loops in (the rehashing thing) their fwd progress
does not depend on other CPUs.

And I suspect that for the typical lock contention scenarios its
unlikely we ever really get into long rehashing chains.

---
 include/linux/lfsr.h                |   49 ++++++++++++
 kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
 2 files changed, 178 insertions(+), 14 deletions(-)

--- /dev/null
+++ b/include/linux/lfsr.h
@@ -0,0 +1,49 @@
+#ifndef _LINUX_LFSR_H
+#define _LINUX_LFSR_H
+
+/*
+ * Simple Binary Galois Linear Feedback Shift Register
+ *
+ * http://en.wikipedia.org/wiki/Linear_feedback_shift_register
+ *
+ */
+
+extern void __lfsr_needs_more_taps(void);
+
+static __always_inline u32 lfsr_taps(int bits)
+{
+	if (bits ==  1) return 0x0001;
+	if (bits ==  2) return 0x0001;
+	if (bits ==  3) return 0x0003;
+	if (bits ==  4) return 0x0009;
+	if (bits ==  5) return 0x0012;
+	if (bits ==  6) return 0x0021;
+	if (bits ==  7) return 0x0041;
+	if (bits ==  8) return 0x008E;
+	if (bits ==  9) return 0x0108;
+	if (bits == 10) return 0x0204;
+	if (bits == 11) return 0x0402;
+	if (bits == 12) return 0x0829;
+	if (bits == 13) return 0x100D;
+	if (bits == 14) return 0x2015;
+
+	/*
+	 * For more taps see:
+	 *   http://users.ece.cmu.edu/~koopman/lfsr/index.html
+	 */
+	__lfsr_needs_more_taps();
+
+	return 0;
+}
+
+static inline u32 lfsr(u32 val, int bits)
+{
+	u32 bit = val & 1;
+
+	val >>= 1;
+	if (bit)
+		val ^= lfsr_taps(bits);
+	return val;
+}
+
+#endif /* _LINUX_LFSR_H */
--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -2,6 +2,9 @@
 #error "do not include this file"
 #endif
 
+#include <linux/hash.h>
+#include <linux/lfsr.h>
+
 /*
  * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
  * of spinning them.
@@ -107,7 +110,120 @@ static void pv_kick_node(struct mcs_spin
 		pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+/*
+ * Hash table using open addressing with an LFSR probe sequence.
+ *
+ * Since we should not be holding locks from NMI context (very rare indeed) the
+ * max load factor is 0.75, which is around the point where open addressing
+ * breaks down.
+ *
+ * Instead of probing just the immediate bucket we probe all buckets in the
+ * same cacheline.
+ *
+ * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
+ *
+ */
+
+#define HB_RESERVED	((struct qspinlock *)1)
+
+struct pv_hash_bucket {
+	struct qspinlock *lock;
+	int cpu;
+};
+
+/*
+ * XXX dynamic allocate using nr_cpu_ids instead...
+ */
+#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
+
+#if PV_LOCK_HASH_BITS < 6
+#undef PV_LOCK_HASH_BITS
+#define PB_LOCK_HASH_BITS	6
+#endif
+
+#define PV_LOCK_HASH_SIZE	(1 << PV_LOCK_HASH_BITS)
+
+static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
+
+#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
+
+static inline u32 hash_align(u32 hash)
+{
+	return hash & ~(PV_HB_PER_LINE - 1);
+}
+
+static struct qspinlock **pv_hash(struct qspinlock *lock)
+{
+	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb, *end;
+
+	if (!hash)
+		hash = 1;
+
+	hb = &__pv_lock_hash[hash_align(hash)];
+	for (;;) {
+		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
+			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {
+				WRITE_ONCE(hb->cpu, smp_processor_id());
+				/*
+				 * Since we must read lock first and cpu
+				 * second, we must write cpu first and lock
+				 * second, therefore use HB_RESERVE to mark an
+				 * entry in use before writing the values.
+				 *
+				 * This can cause hb_hash_find() to not find a
+				 * cpu even though _Q_SLOW_VAL, this is not a
+				 * problem since we re-check l->locked before
+				 * going to sleep and the unlock will have
+				 * cleared l->locked already.
+				 */
+				smp_wmb(); /* matches rmb from pv_hash_find */
+				WRITE_ONCE(hb->lock, lock);
+				goto done;
+			}
+		}
+
+		hash = lfsr(hash, PV_LOCK_HASH_BITS);
+		hb = &__pv_lock_hash[hash_align(hash)];
+	}
+
+done:
+	return &hb->lock;
+}
+
+static int pv_hash_find(struct qspinlock *lock)
+{
+	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb, *end;
+	int cpu = -1;
+
+	if (!hash)
+		hash = 1;
+
+	hb = &__pv_lock_hash[hash_align(hash)];
+	for (;;) {
+		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
+			struct qspinlock *l = READ_ONCE(hb->lock);
+
+			/*
+			 * If we hit an unused bucket, there is no match.
+			 */
+			if (!l)
+				goto done;
+
+			if (l == lock) {
+				smp_rmb(); /* matches wmb from pv_hash() */
+				cpu = READ_ONCE(hb->cpu);
+				goto done;
+			}
+		}
+
+		hash = lfsr(hash, PV_LOCK_HASH_BITS);
+		hb = &__pv_lock_hash[hash_align(hash)];
+	}
+done:
+	return cpu;
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
@@ -116,6 +232,7 @@ static DEFINE_PER_CPU(struct qspinlock *
 static void pv_wait_head(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
+	struct qspinlock **lp = NULL;
 	int loop;
 
 	for (;;) {
@@ -126,13 +243,13 @@ static void pv_wait_head(struct qspinloc
 			cpu_relax();
 		}
 
-		this_cpu_write(__pv_lock_wait, lock);
+		lp = pv_hash(lock);
 		/*
-		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
+		 * lp  must be set before setting _Q_SLOW_VAL
 		 *
-		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
+		 * [S] lp = lock                [RmW] l = l->locked = 0
 		 *     MB                             MB
-		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+		 * [S] l->locked = _Q_SLOW_VAL  [L]   lp
 		 *
 		 * Matches the xchg() in pv_queue_spin_unlock().
 		 */
@@ -142,7 +259,8 @@ static void pv_wait_head(struct qspinloc
 		pv_wait(&l->locked, _Q_SLOW_VAL);
 	}
 done:
-	this_cpu_write(__pv_lock_wait, NULL);
+	if (lp)
+		WRITE_ONCE(*lp, NULL);
 
 	/*
 	 * Lock is unlocked now; the caller will acquire it without waiting.
@@ -165,13 +283,10 @@ void __pv_queue_spin_unlock(struct qspin
 
 	/*
 	 * At this point the memory pointed at by lock can be freed/reused,
-	 * however we can still use the pointer value to search in our cpu
-	 * array.
-	 *
-	 * XXX: get rid of this loop
+	 * however we can still use the pointer value to search in our hash
+	 * table.
 	 */
-	for_each_possible_cpu(cpu) {
-		if (per_cpu(__pv_lock_wait, cpu) == lock)
-			pv_kick(cpu);
-	}
+	cpu = pv_hash_find(lock);
+	if (cpu >= 0)
+		pv_kick(cpu);
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-03-19 12:25         ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 12:25 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
> So I was now thinking of hashing the lock pointer; let me go and quickly
> put something together.

A little something like so; ideally we'd allocate the hashtable since
NR_CPUS is kinda bloated, but it shows the idea I think.

And while this has loops in (the rehashing thing) their fwd progress
does not depend on other CPUs.

And I suspect that for the typical lock contention scenarios its
unlikely we ever really get into long rehashing chains.

---
 include/linux/lfsr.h                |   49 ++++++++++++
 kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
 2 files changed, 178 insertions(+), 14 deletions(-)

--- /dev/null
+++ b/include/linux/lfsr.h
@@ -0,0 +1,49 @@
+#ifndef _LINUX_LFSR_H
+#define _LINUX_LFSR_H
+
+/*
+ * Simple Binary Galois Linear Feedback Shift Register
+ *
+ * http://en.wikipedia.org/wiki/Linear_feedback_shift_register
+ *
+ */
+
+extern void __lfsr_needs_more_taps(void);
+
+static __always_inline u32 lfsr_taps(int bits)
+{
+	if (bits ==  1) return 0x0001;
+	if (bits ==  2) return 0x0001;
+	if (bits ==  3) return 0x0003;
+	if (bits ==  4) return 0x0009;
+	if (bits ==  5) return 0x0012;
+	if (bits ==  6) return 0x0021;
+	if (bits ==  7) return 0x0041;
+	if (bits ==  8) return 0x008E;
+	if (bits ==  9) return 0x0108;
+	if (bits == 10) return 0x0204;
+	if (bits == 11) return 0x0402;
+	if (bits == 12) return 0x0829;
+	if (bits == 13) return 0x100D;
+	if (bits == 14) return 0x2015;
+
+	/*
+	 * For more taps see:
+	 *   http://users.ece.cmu.edu/~koopman/lfsr/index.html
+	 */
+	__lfsr_needs_more_taps();
+
+	return 0;
+}
+
+static inline u32 lfsr(u32 val, int bits)
+{
+	u32 bit = val & 1;
+
+	val >>= 1;
+	if (bit)
+		val ^= lfsr_taps(bits);
+	return val;
+}
+
+#endif /* _LINUX_LFSR_H */
--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -2,6 +2,9 @@
 #error "do not include this file"
 #endif
 
+#include <linux/hash.h>
+#include <linux/lfsr.h>
+
 /*
  * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
  * of spinning them.
@@ -107,7 +110,120 @@ static void pv_kick_node(struct mcs_spin
 		pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+/*
+ * Hash table using open addressing with an LFSR probe sequence.
+ *
+ * Since we should not be holding locks from NMI context (very rare indeed) the
+ * max load factor is 0.75, which is around the point where open addressing
+ * breaks down.
+ *
+ * Instead of probing just the immediate bucket we probe all buckets in the
+ * same cacheline.
+ *
+ * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
+ *
+ */
+
+#define HB_RESERVED	((struct qspinlock *)1)
+
+struct pv_hash_bucket {
+	struct qspinlock *lock;
+	int cpu;
+};
+
+/*
+ * XXX dynamic allocate using nr_cpu_ids instead...
+ */
+#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
+
+#if PV_LOCK_HASH_BITS < 6
+#undef PV_LOCK_HASH_BITS
+#define PB_LOCK_HASH_BITS	6
+#endif
+
+#define PV_LOCK_HASH_SIZE	(1 << PV_LOCK_HASH_BITS)
+
+static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
+
+#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
+
+static inline u32 hash_align(u32 hash)
+{
+	return hash & ~(PV_HB_PER_LINE - 1);
+}
+
+static struct qspinlock **pv_hash(struct qspinlock *lock)
+{
+	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb, *end;
+
+	if (!hash)
+		hash = 1;
+
+	hb = &__pv_lock_hash[hash_align(hash)];
+	for (;;) {
+		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
+			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {
+				WRITE_ONCE(hb->cpu, smp_processor_id());
+				/*
+				 * Since we must read lock first and cpu
+				 * second, we must write cpu first and lock
+				 * second, therefore use HB_RESERVE to mark an
+				 * entry in use before writing the values.
+				 *
+				 * This can cause hb_hash_find() to not find a
+				 * cpu even though _Q_SLOW_VAL, this is not a
+				 * problem since we re-check l->locked before
+				 * going to sleep and the unlock will have
+				 * cleared l->locked already.
+				 */
+				smp_wmb(); /* matches rmb from pv_hash_find */
+				WRITE_ONCE(hb->lock, lock);
+				goto done;
+			}
+		}
+
+		hash = lfsr(hash, PV_LOCK_HASH_BITS);
+		hb = &__pv_lock_hash[hash_align(hash)];
+	}
+
+done:
+	return &hb->lock;
+}
+
+static int pv_hash_find(struct qspinlock *lock)
+{
+	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb, *end;
+	int cpu = -1;
+
+	if (!hash)
+		hash = 1;
+
+	hb = &__pv_lock_hash[hash_align(hash)];
+	for (;;) {
+		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
+			struct qspinlock *l = READ_ONCE(hb->lock);
+
+			/*
+			 * If we hit an unused bucket, there is no match.
+			 */
+			if (!l)
+				goto done;
+
+			if (l == lock) {
+				smp_rmb(); /* matches wmb from pv_hash() */
+				cpu = READ_ONCE(hb->cpu);
+				goto done;
+			}
+		}
+
+		hash = lfsr(hash, PV_LOCK_HASH_BITS);
+		hb = &__pv_lock_hash[hash_align(hash)];
+	}
+done:
+	return cpu;
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
@@ -116,6 +232,7 @@ static DEFINE_PER_CPU(struct qspinlock *
 static void pv_wait_head(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
+	struct qspinlock **lp = NULL;
 	int loop;
 
 	for (;;) {
@@ -126,13 +243,13 @@ static void pv_wait_head(struct qspinloc
 			cpu_relax();
 		}
 
-		this_cpu_write(__pv_lock_wait, lock);
+		lp = pv_hash(lock);
 		/*
-		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
+		 * lp  must be set before setting _Q_SLOW_VAL
 		 *
-		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
+		 * [S] lp = lock                [RmW] l = l->locked = 0
 		 *     MB                             MB
-		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+		 * [S] l->locked = _Q_SLOW_VAL  [L]   lp
 		 *
 		 * Matches the xchg() in pv_queue_spin_unlock().
 		 */
@@ -142,7 +259,8 @@ static void pv_wait_head(struct qspinloc
 		pv_wait(&l->locked, _Q_SLOW_VAL);
 	}
 done:
-	this_cpu_write(__pv_lock_wait, NULL);
+	if (lp)
+		WRITE_ONCE(*lp, NULL);
 
 	/*
 	 * Lock is unlocked now; the caller will acquire it without waiting.
@@ -165,13 +283,10 @@ void __pv_queue_spin_unlock(struct qspin
 
 	/*
 	 * At this point the memory pointed at by lock can be freed/reused,
-	 * however we can still use the pointer value to search in our cpu
-	 * array.
-	 *
-	 * XXX: get rid of this loop
+	 * however we can still use the pointer value to search in our hash
+	 * table.
 	 */
-	for_each_possible_cpu(cpu) {
-		if (per_cpu(__pv_lock_wait, cpu) == lock)
-			pv_kick(cpu);
-	}
+	cpu = pv_hash_find(lock);
+	if (cpu >= 0)
+		pv_kick(cpu);
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 10:12     ` Peter Zijlstra
  2015-03-19 12:25         ` Peter Zijlstra
@ 2015-03-19 12:25       ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 12:25 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
> So I was now thinking of hashing the lock pointer; let me go and quickly
> put something together.

A little something like so; ideally we'd allocate the hashtable since
NR_CPUS is kinda bloated, but it shows the idea I think.

And while this has loops in (the rehashing thing) their fwd progress
does not depend on other CPUs.

And I suspect that for the typical lock contention scenarios its
unlikely we ever really get into long rehashing chains.

---
 include/linux/lfsr.h                |   49 ++++++++++++
 kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
 2 files changed, 178 insertions(+), 14 deletions(-)

--- /dev/null
+++ b/include/linux/lfsr.h
@@ -0,0 +1,49 @@
+#ifndef _LINUX_LFSR_H
+#define _LINUX_LFSR_H
+
+/*
+ * Simple Binary Galois Linear Feedback Shift Register
+ *
+ * http://en.wikipedia.org/wiki/Linear_feedback_shift_register
+ *
+ */
+
+extern void __lfsr_needs_more_taps(void);
+
+static __always_inline u32 lfsr_taps(int bits)
+{
+	if (bits ==  1) return 0x0001;
+	if (bits ==  2) return 0x0001;
+	if (bits ==  3) return 0x0003;
+	if (bits ==  4) return 0x0009;
+	if (bits ==  5) return 0x0012;
+	if (bits ==  6) return 0x0021;
+	if (bits ==  7) return 0x0041;
+	if (bits ==  8) return 0x008E;
+	if (bits ==  9) return 0x0108;
+	if (bits == 10) return 0x0204;
+	if (bits == 11) return 0x0402;
+	if (bits == 12) return 0x0829;
+	if (bits == 13) return 0x100D;
+	if (bits == 14) return 0x2015;
+
+	/*
+	 * For more taps see:
+	 *   http://users.ece.cmu.edu/~koopman/lfsr/index.html
+	 */
+	__lfsr_needs_more_taps();
+
+	return 0;
+}
+
+static inline u32 lfsr(u32 val, int bits)
+{
+	u32 bit = val & 1;
+
+	val >>= 1;
+	if (bit)
+		val ^= lfsr_taps(bits);
+	return val;
+}
+
+#endif /* _LINUX_LFSR_H */
--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -2,6 +2,9 @@
 #error "do not include this file"
 #endif
 
+#include <linux/hash.h>
+#include <linux/lfsr.h>
+
 /*
  * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
  * of spinning them.
@@ -107,7 +110,120 @@ static void pv_kick_node(struct mcs_spin
 		pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+/*
+ * Hash table using open addressing with an LFSR probe sequence.
+ *
+ * Since we should not be holding locks from NMI context (very rare indeed) the
+ * max load factor is 0.75, which is around the point where open addressing
+ * breaks down.
+ *
+ * Instead of probing just the immediate bucket we probe all buckets in the
+ * same cacheline.
+ *
+ * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
+ *
+ */
+
+#define HB_RESERVED	((struct qspinlock *)1)
+
+struct pv_hash_bucket {
+	struct qspinlock *lock;
+	int cpu;
+};
+
+/*
+ * XXX dynamic allocate using nr_cpu_ids instead...
+ */
+#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
+
+#if PV_LOCK_HASH_BITS < 6
+#undef PV_LOCK_HASH_BITS
+#define PB_LOCK_HASH_BITS	6
+#endif
+
+#define PV_LOCK_HASH_SIZE	(1 << PV_LOCK_HASH_BITS)
+
+static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
+
+#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
+
+static inline u32 hash_align(u32 hash)
+{
+	return hash & ~(PV_HB_PER_LINE - 1);
+}
+
+static struct qspinlock **pv_hash(struct qspinlock *lock)
+{
+	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb, *end;
+
+	if (!hash)
+		hash = 1;
+
+	hb = &__pv_lock_hash[hash_align(hash)];
+	for (;;) {
+		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
+			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {
+				WRITE_ONCE(hb->cpu, smp_processor_id());
+				/*
+				 * Since we must read lock first and cpu
+				 * second, we must write cpu first and lock
+				 * second, therefore use HB_RESERVE to mark an
+				 * entry in use before writing the values.
+				 *
+				 * This can cause hb_hash_find() to not find a
+				 * cpu even though _Q_SLOW_VAL, this is not a
+				 * problem since we re-check l->locked before
+				 * going to sleep and the unlock will have
+				 * cleared l->locked already.
+				 */
+				smp_wmb(); /* matches rmb from pv_hash_find */
+				WRITE_ONCE(hb->lock, lock);
+				goto done;
+			}
+		}
+
+		hash = lfsr(hash, PV_LOCK_HASH_BITS);
+		hb = &__pv_lock_hash[hash_align(hash)];
+	}
+
+done:
+	return &hb->lock;
+}
+
+static int pv_hash_find(struct qspinlock *lock)
+{
+	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb, *end;
+	int cpu = -1;
+
+	if (!hash)
+		hash = 1;
+
+	hb = &__pv_lock_hash[hash_align(hash)];
+	for (;;) {
+		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
+			struct qspinlock *l = READ_ONCE(hb->lock);
+
+			/*
+			 * If we hit an unused bucket, there is no match.
+			 */
+			if (!l)
+				goto done;
+
+			if (l == lock) {
+				smp_rmb(); /* matches wmb from pv_hash() */
+				cpu = READ_ONCE(hb->cpu);
+				goto done;
+			}
+		}
+
+		hash = lfsr(hash, PV_LOCK_HASH_BITS);
+		hb = &__pv_lock_hash[hash_align(hash)];
+	}
+done:
+	return cpu;
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
@@ -116,6 +232,7 @@ static DEFINE_PER_CPU(struct qspinlock *
 static void pv_wait_head(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
+	struct qspinlock **lp = NULL;
 	int loop;
 
 	for (;;) {
@@ -126,13 +243,13 @@ static void pv_wait_head(struct qspinloc
 			cpu_relax();
 		}
 
-		this_cpu_write(__pv_lock_wait, lock);
+		lp = pv_hash(lock);
 		/*
-		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
+		 * lp  must be set before setting _Q_SLOW_VAL
 		 *
-		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
+		 * [S] lp = lock                [RmW] l = l->locked = 0
 		 *     MB                             MB
-		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
+		 * [S] l->locked = _Q_SLOW_VAL  [L]   lp
 		 *
 		 * Matches the xchg() in pv_queue_spin_unlock().
 		 */
@@ -142,7 +259,8 @@ static void pv_wait_head(struct qspinloc
 		pv_wait(&l->locked, _Q_SLOW_VAL);
 	}
 done:
-	this_cpu_write(__pv_lock_wait, NULL);
+	if (lp)
+		WRITE_ONCE(*lp, NULL);
 
 	/*
 	 * Lock is unlocked now; the caller will acquire it without waiting.
@@ -165,13 +283,10 @@ void __pv_queue_spin_unlock(struct qspin
 
 	/*
 	 * At this point the memory pointed at by lock can be freed/reused,
-	 * however we can still use the pointer value to search in our cpu
-	 * array.
-	 *
-	 * XXX: get rid of this loop
+	 * however we can still use the pointer value to search in our hash
+	 * table.
 	 */
-	for_each_possible_cpu(cpu) {
-		if (per_cpu(__pv_lock_wait, cpu) == lock)
-			pv_kick(cpu);
-	}
+	cpu = pv_hash_find(lock);
+	if (cpu >= 0)
+		pv_kick(cpu);
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 12:25         ` Peter Zijlstra
                           ` (2 preceding siblings ...)
  (?)
@ 2015-03-19 13:43         ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 13:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Thu, Mar 19, 2015 at 01:25:36PM +0100, Peter Zijlstra wrote:
> +static struct qspinlock **pv_hash(struct qspinlock *lock)
> +{
> +	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb = &__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
> +			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {

That should be: !cmpxchg(), bit disturbing that that booted.

> +				WRITE_ONCE(hb->cpu, smp_processor_id());
> +				/*
> +				 * Since we must read lock first and cpu
> +				 * second, we must write cpu first and lock
> +				 * second, therefore use HB_RESERVE to mark an
> +				 * entry in use before writing the values.
> +				 *
> +				 * This can cause hb_hash_find() to not find a
> +				 * cpu even though _Q_SLOW_VAL, this is not a
> +				 * problem since we re-check l->locked before
> +				 * going to sleep and the unlock will have
> +				 * cleared l->locked already.
> +				 */
> +				smp_wmb(); /* matches rmb from pv_hash_find */
> +				WRITE_ONCE(hb->lock, lock);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb = &__pv_lock_hash[hash_align(hash)];
> +	}
> +
> +done:
> +	return &hb->lock;
> +}

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 12:25         ` Peter Zijlstra
  (?)
@ 2015-03-19 13:43         ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 13:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Thu, Mar 19, 2015 at 01:25:36PM +0100, Peter Zijlstra wrote:
> +static struct qspinlock **pv_hash(struct qspinlock *lock)
> +{
> +	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb = &__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
> +			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {

That should be: !cmpxchg(), bit disturbing that that booted.

> +				WRITE_ONCE(hb->cpu, smp_processor_id());
> +				/*
> +				 * Since we must read lock first and cpu
> +				 * second, we must write cpu first and lock
> +				 * second, therefore use HB_RESERVE to mark an
> +				 * entry in use before writing the values.
> +				 *
> +				 * This can cause hb_hash_find() to not find a
> +				 * cpu even though _Q_SLOW_VAL, this is not a
> +				 * problem since we re-check l->locked before
> +				 * going to sleep and the unlock will have
> +				 * cleared l->locked already.
> +				 */
> +				smp_wmb(); /* matches rmb from pv_hash_find */
> +				WRITE_ONCE(hb->lock, lock);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb = &__pv_lock_hash[hash_align(hash)];
> +	}
> +
> +done:
> +	return &hb->lock;
> +}

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 12:25         ` Peter Zijlstra
  (?)
  (?)
@ 2015-03-19 13:43         ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 13:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Thu, Mar 19, 2015 at 01:25:36PM +0100, Peter Zijlstra wrote:
> +static struct qspinlock **pv_hash(struct qspinlock *lock)
> +{
> +	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb = &__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb < end; hb++) {
> +			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {

That should be: !cmpxchg(), bit disturbing that that booted.

> +				WRITE_ONCE(hb->cpu, smp_processor_id());
> +				/*
> +				 * Since we must read lock first and cpu
> +				 * second, we must write cpu first and lock
> +				 * second, therefore use HB_RESERVE to mark an
> +				 * entry in use before writing the values.
> +				 *
> +				 * This can cause hb_hash_find() to not find a
> +				 * cpu even though _Q_SLOW_VAL, this is not a
> +				 * problem since we re-check l->locked before
> +				 * going to sleep and the unlock will have
> +				 * cleared l->locked already.
> +				 */
> +				smp_wmb(); /* matches rmb from pv_hash_find */
> +				WRITE_ONCE(hb->lock, lock);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb = &__pv_lock_hash[hash_align(hash)];
> +	}
> +
> +done:
> +	return &hb->lock;
> +}

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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
@ 2015-03-19 18:01   ` David Vrabel
  -1 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-19 18:01 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: raghavendra.kt, kvm, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

On 16/03/15 13:16, Peter Zijlstra wrote:
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

This seems work for me, but I've not got time to give it a more thorough
testing.

You can fold this into your series.

There doesn't seem to be a way to disable QUEUE_SPINLOCKS when supported by
the arch, is this intentional?  If so, the existing ticketlock code could go.

David

8<------------------------------
x86/xen: paravirt support for qspinlocks

Provide the wait and kick ops necessary for paravirt-aware queue
spinlocks.

Signed-off-by: David Vrabel <david.vrabel@citrix.com>
---
 arch/x86/xen/spinlock.c |   40 +++++++++++++++++++++++++++++++++++++---
 1 file changed, 37 insertions(+), 3 deletions(-)

diff --git a/arch/x86/xen/spinlock.c b/arch/x86/xen/spinlock.c
index 956374c..b019b2a 100644
--- a/arch/x86/xen/spinlock.c
+++ b/arch/x86/xen/spinlock.c
@@ -95,17 +95,43 @@ static inline void spin_time_accum_blocked(u64 start)
 }
 #endif  /* CONFIG_XEN_DEBUG_FS */
 
+static DEFINE_PER_CPU(int, lock_kicker_irq) = -1;
+static DEFINE_PER_CPU(char *, irq_name);
+static bool xen_pvspin = true;
+
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+#include <asm/qspinlock.h>
+
+PV_CALLEE_SAVE_REGS_THUNK(__pv_queue_spin_unlock);
+
+static void xen_qlock_wait(u8 *ptr, u8 val)
+{
+	int irq = __this_cpu_read(lock_kicker_irq);
+
+	xen_clear_irq_pending(irq);
+
+	barrier();
+
+	if (READ_ONCE(*ptr) == val)
+		xen_poll_irq(irq);
+}
+
+static void xen_qlock_kick(int cpu)
+{
+	xen_send_IPI_one(cpu, XEN_SPIN_UNLOCK_VECTOR);
+}
+
+#else
+
 struct xen_lock_waiting {
 	struct arch_spinlock *lock;
 	__ticket_t want;
 };
 
-static DEFINE_PER_CPU(int, lock_kicker_irq) = -1;
-static DEFINE_PER_CPU(char *, irq_name);
 static DEFINE_PER_CPU(struct xen_lock_waiting, lock_waiting);
 static cpumask_t waiting_cpus;
 
-static bool xen_pvspin = true;
 __visible void xen_lock_spinning(struct arch_spinlock *lock, __ticket_t want)
 {
 	int irq = __this_cpu_read(lock_kicker_irq);
@@ -217,6 +243,7 @@ static void xen_unlock_kick(struct arch_spinlock *lock, __ticket_t next)
 		}
 	}
 }
+#endif /* !QUEUE_SPINLOCK */
 
 static irqreturn_t dummy_handler(int irq, void *dev_id)
 {
@@ -280,8 +307,15 @@ void __init xen_init_spinlocks(void)
 		return;
 	}
 	printk(KERN_DEBUG "xen: PV spinlocks enabled\n");
+#ifdef CONFIG_QUEUE_SPINLOCK
+	pv_lock_ops.queue_spin_lock_slowpath = __pv_queue_spin_lock_slowpath;
+	pv_lock_ops.queue_spin_unlock = PV_CALLEE_SAVE(__pv_queue_spin_unlock);
+	pv_lock_ops.wait = xen_qlock_wait;
+	pv_lock_ops.kick = xen_qlock_kick;
+#else
 	pv_lock_ops.lock_spinning = PV_CALLEE_SAVE(xen_lock_spinning);
 	pv_lock_ops.unlock_kick = xen_unlock_kick;
+#endif
 }
 
 /*
-- 
1.7.10.4


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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-19 18:01   ` David Vrabel
  0 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-19 18:01 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: raghavendra.kt, kvm, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

On 16/03/15 13:16, Peter Zijlstra wrote:
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

This seems work for me, but I've not got time to give it a more thorough
testing.

You can fold this into your series.

There doesn't seem to be a way to disable QUEUE_SPINLOCKS when supported by
the arch, is this intentional?  If so, the existing ticketlock code could go.

David

8<------------------------------
x86/xen: paravirt support for qspinlocks

Provide the wait and kick ops necessary for paravirt-aware queue
spinlocks.

Signed-off-by: David Vrabel <david.vrabel@citrix.com>
---
 arch/x86/xen/spinlock.c |   40 +++++++++++++++++++++++++++++++++++++---
 1 file changed, 37 insertions(+), 3 deletions(-)

diff --git a/arch/x86/xen/spinlock.c b/arch/x86/xen/spinlock.c
index 956374c..b019b2a 100644
--- a/arch/x86/xen/spinlock.c
+++ b/arch/x86/xen/spinlock.c
@@ -95,17 +95,43 @@ static inline void spin_time_accum_blocked(u64 start)
 }
 #endif  /* CONFIG_XEN_DEBUG_FS */
 
+static DEFINE_PER_CPU(int, lock_kicker_irq) = -1;
+static DEFINE_PER_CPU(char *, irq_name);
+static bool xen_pvspin = true;
+
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+#include <asm/qspinlock.h>
+
+PV_CALLEE_SAVE_REGS_THUNK(__pv_queue_spin_unlock);
+
+static void xen_qlock_wait(u8 *ptr, u8 val)
+{
+	int irq = __this_cpu_read(lock_kicker_irq);
+
+	xen_clear_irq_pending(irq);
+
+	barrier();
+
+	if (READ_ONCE(*ptr) == val)
+		xen_poll_irq(irq);
+}
+
+static void xen_qlock_kick(int cpu)
+{
+	xen_send_IPI_one(cpu, XEN_SPIN_UNLOCK_VECTOR);
+}
+
+#else
+
 struct xen_lock_waiting {
 	struct arch_spinlock *lock;
 	__ticket_t want;
 };
 
-static DEFINE_PER_CPU(int, lock_kicker_irq) = -1;
-static DEFINE_PER_CPU(char *, irq_name);
 static DEFINE_PER_CPU(struct xen_lock_waiting, lock_waiting);
 static cpumask_t waiting_cpus;
 
-static bool xen_pvspin = true;
 __visible void xen_lock_spinning(struct arch_spinlock *lock, __ticket_t want)
 {
 	int irq = __this_cpu_read(lock_kicker_irq);
@@ -217,6 +243,7 @@ static void xen_unlock_kick(struct arch_spinlock *lock, __ticket_t next)
 		}
 	}
 }
+#endif /* !QUEUE_SPINLOCK */
 
 static irqreturn_t dummy_handler(int irq, void *dev_id)
 {
@@ -280,8 +307,15 @@ void __init xen_init_spinlocks(void)
 		return;
 	}
 	printk(KERN_DEBUG "xen: PV spinlocks enabled\n");
+#ifdef CONFIG_QUEUE_SPINLOCK
+	pv_lock_ops.queue_spin_lock_slowpath = __pv_queue_spin_lock_slowpath;
+	pv_lock_ops.queue_spin_unlock = PV_CALLEE_SAVE(__pv_queue_spin_unlock);
+	pv_lock_ops.wait = xen_qlock_wait;
+	pv_lock_ops.kick = xen_qlock_kick;
+#else
 	pv_lock_ops.lock_spinning = PV_CALLEE_SAVE(xen_lock_spinning);
 	pv_lock_ops.unlock_kick = xen_unlock_kick;
+#endif
 }
 
 /*
-- 
1.7.10.4

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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (25 preceding siblings ...)
  (?)
@ 2015-03-19 18:01 ` David Vrabel
  -1 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-19 18:01 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: linux-arch, riel, x86, kvm, oleg, scott.norton, raghavendra.kt,
	paolo.bonzini, linux-kernel, virtualization, mingo, doug.hatch,
	hpa, luto, xen-devel, boris.ostrovsky, paulmck, torvalds, tglx,
	david.vrabel

On 16/03/15 13:16, Peter Zijlstra wrote:
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

This seems work for me, but I've not got time to give it a more thorough
testing.

You can fold this into your series.

There doesn't seem to be a way to disable QUEUE_SPINLOCKS when supported by
the arch, is this intentional?  If so, the existing ticketlock code could go.

David

8<------------------------------
x86/xen: paravirt support for qspinlocks

Provide the wait and kick ops necessary for paravirt-aware queue
spinlocks.

Signed-off-by: David Vrabel <david.vrabel@citrix.com>
---
 arch/x86/xen/spinlock.c |   40 +++++++++++++++++++++++++++++++++++++---
 1 file changed, 37 insertions(+), 3 deletions(-)

diff --git a/arch/x86/xen/spinlock.c b/arch/x86/xen/spinlock.c
index 956374c..b019b2a 100644
--- a/arch/x86/xen/spinlock.c
+++ b/arch/x86/xen/spinlock.c
@@ -95,17 +95,43 @@ static inline void spin_time_accum_blocked(u64 start)
 }
 #endif  /* CONFIG_XEN_DEBUG_FS */
 
+static DEFINE_PER_CPU(int, lock_kicker_irq) = -1;
+static DEFINE_PER_CPU(char *, irq_name);
+static bool xen_pvspin = true;
+
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+#include <asm/qspinlock.h>
+
+PV_CALLEE_SAVE_REGS_THUNK(__pv_queue_spin_unlock);
+
+static void xen_qlock_wait(u8 *ptr, u8 val)
+{
+	int irq = __this_cpu_read(lock_kicker_irq);
+
+	xen_clear_irq_pending(irq);
+
+	barrier();
+
+	if (READ_ONCE(*ptr) == val)
+		xen_poll_irq(irq);
+}
+
+static void xen_qlock_kick(int cpu)
+{
+	xen_send_IPI_one(cpu, XEN_SPIN_UNLOCK_VECTOR);
+}
+
+#else
+
 struct xen_lock_waiting {
 	struct arch_spinlock *lock;
 	__ticket_t want;
 };
 
-static DEFINE_PER_CPU(int, lock_kicker_irq) = -1;
-static DEFINE_PER_CPU(char *, irq_name);
 static DEFINE_PER_CPU(struct xen_lock_waiting, lock_waiting);
 static cpumask_t waiting_cpus;
 
-static bool xen_pvspin = true;
 __visible void xen_lock_spinning(struct arch_spinlock *lock, __ticket_t want)
 {
 	int irq = __this_cpu_read(lock_kicker_irq);
@@ -217,6 +243,7 @@ static void xen_unlock_kick(struct arch_spinlock *lock, __ticket_t next)
 		}
 	}
 }
+#endif /* !QUEUE_SPINLOCK */
 
 static irqreturn_t dummy_handler(int irq, void *dev_id)
 {
@@ -280,8 +307,15 @@ void __init xen_init_spinlocks(void)
 		return;
 	}
 	printk(KERN_DEBUG "xen: PV spinlocks enabled\n");
+#ifdef CONFIG_QUEUE_SPINLOCK
+	pv_lock_ops.queue_spin_lock_slowpath = __pv_queue_spin_lock_slowpath;
+	pv_lock_ops.queue_spin_unlock = PV_CALLEE_SAVE(__pv_queue_spin_unlock);
+	pv_lock_ops.wait = xen_qlock_wait;
+	pv_lock_ops.kick = xen_qlock_kick;
+#else
 	pv_lock_ops.lock_spinning = PV_CALLEE_SAVE(xen_lock_spinning);
 	pv_lock_ops.unlock_kick = xen_unlock_kick;
+#endif
 }
 
 /*
-- 
1.7.10.4

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (27 preceding siblings ...)
  (?)
@ 2015-03-19 18:01 ` David Vrabel
  -1 siblings, 0 replies; 136+ messages in thread
From: David Vrabel @ 2015-03-19 18:01 UTC (permalink / raw)
  To: Peter Zijlstra, Waiman.Long
  Cc: linux-arch, riel, x86, kvm, oleg, scott.norton, raghavendra.kt,
	paolo.bonzini, linux-kernel, virtualization, mingo, doug.hatch,
	hpa, luto, xen-devel, boris.ostrovsky, paulmck, torvalds, tglx,
	david.vrabel

On 16/03/15 13:16, Peter Zijlstra wrote:
> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).

This seems work for me, but I've not got time to give it a more thorough
testing.

You can fold this into your series.

There doesn't seem to be a way to disable QUEUE_SPINLOCKS when supported by
the arch, is this intentional?  If so, the existing ticketlock code could go.

David

8<------------------------------
x86/xen: paravirt support for qspinlocks

Provide the wait and kick ops necessary for paravirt-aware queue
spinlocks.

Signed-off-by: David Vrabel <david.vrabel@citrix.com>
---
 arch/x86/xen/spinlock.c |   40 +++++++++++++++++++++++++++++++++++++---
 1 file changed, 37 insertions(+), 3 deletions(-)

diff --git a/arch/x86/xen/spinlock.c b/arch/x86/xen/spinlock.c
index 956374c..b019b2a 100644
--- a/arch/x86/xen/spinlock.c
+++ b/arch/x86/xen/spinlock.c
@@ -95,17 +95,43 @@ static inline void spin_time_accum_blocked(u64 start)
 }
 #endif  /* CONFIG_XEN_DEBUG_FS */
 
+static DEFINE_PER_CPU(int, lock_kicker_irq) = -1;
+static DEFINE_PER_CPU(char *, irq_name);
+static bool xen_pvspin = true;
+
+#ifdef CONFIG_QUEUE_SPINLOCK
+
+#include <asm/qspinlock.h>
+
+PV_CALLEE_SAVE_REGS_THUNK(__pv_queue_spin_unlock);
+
+static void xen_qlock_wait(u8 *ptr, u8 val)
+{
+	int irq = __this_cpu_read(lock_kicker_irq);
+
+	xen_clear_irq_pending(irq);
+
+	barrier();
+
+	if (READ_ONCE(*ptr) == val)
+		xen_poll_irq(irq);
+}
+
+static void xen_qlock_kick(int cpu)
+{
+	xen_send_IPI_one(cpu, XEN_SPIN_UNLOCK_VECTOR);
+}
+
+#else
+
 struct xen_lock_waiting {
 	struct arch_spinlock *lock;
 	__ticket_t want;
 };
 
-static DEFINE_PER_CPU(int, lock_kicker_irq) = -1;
-static DEFINE_PER_CPU(char *, irq_name);
 static DEFINE_PER_CPU(struct xen_lock_waiting, lock_waiting);
 static cpumask_t waiting_cpus;
 
-static bool xen_pvspin = true;
 __visible void xen_lock_spinning(struct arch_spinlock *lock, __ticket_t want)
 {
 	int irq = __this_cpu_read(lock_kicker_irq);
@@ -217,6 +243,7 @@ static void xen_unlock_kick(struct arch_spinlock *lock, __ticket_t next)
 		}
 	}
 }
+#endif /* !QUEUE_SPINLOCK */
 
 static irqreturn_t dummy_handler(int irq, void *dev_id)
 {
@@ -280,8 +307,15 @@ void __init xen_init_spinlocks(void)
 		return;
 	}
 	printk(KERN_DEBUG "xen: PV spinlocks enabled\n");
+#ifdef CONFIG_QUEUE_SPINLOCK
+	pv_lock_ops.queue_spin_lock_slowpath = __pv_queue_spin_lock_slowpath;
+	pv_lock_ops.queue_spin_unlock = PV_CALLEE_SAVE(__pv_queue_spin_unlock);
+	pv_lock_ops.wait = xen_qlock_wait;
+	pv_lock_ops.kick = xen_qlock_kick;
+#else
 	pv_lock_ops.lock_spinning = PV_CALLEE_SAVE(xen_lock_spinning);
 	pv_lock_ops.unlock_kick = xen_unlock_kick;
+#endif
 }
 
 /*
-- 
1.7.10.4

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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
  2015-03-19 18:01   ` David Vrabel
@ 2015-03-19 18:32     ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 18:32 UTC (permalink / raw)
  To: David Vrabel
  Cc: Waiman.Long, raghavendra.kt, kvm, linux-kernel, hpa,
	boris.ostrovsky, linux-arch, x86, mingo, doug.hatch, xen-devel,
	paulmck, riel, scott.norton, paolo.bonzini, tglx, virtualization,
	oleg, luto, torvalds

On Thu, Mar 19, 2015 at 06:01:34PM +0000, David Vrabel wrote:
> This seems work for me, but I've not got time to give it a more thorough
> testing.
> 
> You can fold this into your series.

Thanks!

> There doesn't seem to be a way to disable QUEUE_SPINLOCKS when supported by
> the arch, is this intentional?  If so, the existing ticketlock code could go.

Yeah, its left as a rudiment such that if we find issues with the
qspinlock code we can 'revert' with a trivial patch. If no issues show
up we can rip out all the old code in a subsequent release.

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

* Re: [Xen-devel] [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-19 18:32     ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 18:32 UTC (permalink / raw)
  To: David Vrabel
  Cc: Waiman.Long, linux-arch, riel, x86, kvm, oleg, scott.norton,
	raghavendra.kt, paolo.bonzini, linux-kernel, virtualization,
	mingo, doug.hatch, hpa, luto, xen-devel, boris.ostrovsky,
	paulmck, torvalds, tglx

On Thu, Mar 19, 2015 at 06:01:34PM +0000, David Vrabel wrote:
> This seems work for me, but I've not got time to give it a more thorough
> testing.
> 
> You can fold this into your series.

Thanks!

> There doesn't seem to be a way to disable QUEUE_SPINLOCKS when supported by
> the arch, is this intentional?  If so, the existing ticketlock code could go.

Yeah, its left as a rudiment such that if we find issues with the
qspinlock code we can 'revert' with a trivial patch. If no issues show
up we can rip out all the old code in a subsequent release.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-19 18:01   ` David Vrabel
  (?)
  (?)
@ 2015-03-19 18:32   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-19 18:32 UTC (permalink / raw)
  To: David Vrabel
  Cc: Waiman.Long, linux-arch, riel, x86, kvm, oleg, scott.norton,
	raghavendra.kt, paolo.bonzini, linux-kernel, virtualization,
	mingo, doug.hatch, hpa, luto, xen-devel, boris.ostrovsky,
	paulmck, torvalds, tglx

On Thu, Mar 19, 2015 at 06:01:34PM +0000, David Vrabel wrote:
> This seems work for me, but I've not got time to give it a more thorough
> testing.
> 
> You can fold this into your series.

Thanks!

> There doesn't seem to be a way to disable QUEUE_SPINLOCKS when supported by
> the arch, is this intentional?  If so, the existing ticketlock code could go.

Yeah, its left as a rudiment such that if we find issues with the
qspinlock code we can 'revert' with a trivial patch. If no issues show
up we can rip out all the old code in a subsequent release.

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

* Re: [PATCH 9/9] qspinlock,x86,kvm: Implement KVM support for paravirt qspinlock
  2015-03-19 10:01       ` Peter Zijlstra
@ 2015-03-19 21:08         ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-19 21:08 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 03/19/2015 06:01 AM, Peter Zijlstra wrote:
> On Wed, Mar 18, 2015 at 10:45:55PM -0400, Waiman Long wrote:
>> On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
>> I do have some concern about this call site patching mechanism as the
>> modification is not atomic. The spin_unlock() calls are in many places in
>> the kernel. There is a possibility that a thread is calling a certain
>> spin_unlock call site while it is being patched by another one with the
>> alternative() function call.
>>
>> So far, I don't see any problem with bare metal where paravirt_patch_insns()
>> is used to patch it to the move instruction. However, in a virtual guest
>> enivornment where paravirt_patch_call() was used, there were situations
>> where the system panic because of page fault on some invalid memory in the
>> kthread. If you look at the paravirt_patch_call(), you will see:
>>
>>      :
>> b->opcode = 0xe8; /* call */
>> b->delta = delta;
>>
>> If another CPU reads the instruction at the call site at the right moment,
>> it will get the modified call instruction, but not the new delta value. It
>> will then jump to a random location. I believe that was causing the system
>> panic that I saw.
>>
>> So I think it is kind of risky to use it here unless we can guarantee that
>> call site patching is atomic wrt other CPUs.
> Just look at where the patching is done:
>
> init/main.c:start_kernel()
>    check_bugs()
>      alternative_instructions()
>        apply_paravirt()
>
> We're UP and not holding any locks, disable IRQs (see text_poke_early())
> and have NMIs 'disabled'.

You are probably right. The initial apply_paravirt() was done before the 
SMP boot. Subsequent ones were at kernel module load time. I put a 
counter in the __native_queue_spin_unlock() and it registered 26949 
unlock calls in a 16-cpu guest before it got patched out.

The panic that I observed before might be due to some coding error of my 
own.

-Longman

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

* Re: [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
@ 2015-03-19 21:08         ` Waiman Long
  0 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-19 21:08 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/19/2015 06:01 AM, Peter Zijlstra wrote:
> On Wed, Mar 18, 2015 at 10:45:55PM -0400, Waiman Long wrote:
>> On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
>> I do have some concern about this call site patching mechanism as the
>> modification is not atomic. The spin_unlock() calls are in many places in
>> the kernel. There is a possibility that a thread is calling a certain
>> spin_unlock call site while it is being patched by another one with the
>> alternative() function call.
>>
>> So far, I don't see any problem with bare metal where paravirt_patch_insns()
>> is used to patch it to the move instruction. However, in a virtual guest
>> enivornment where paravirt_patch_call() was used, there were situations
>> where the system panic because of page fault on some invalid memory in the
>> kthread. If you look at the paravirt_patch_call(), you will see:
>>
>>      :
>> b->opcode = 0xe8; /* call */
>> b->delta = delta;
>>
>> If another CPU reads the instruction at the call site at the right moment,
>> it will get the modified call instruction, but not the new delta value. It
>> will then jump to a random location. I believe that was causing the system
>> panic that I saw.
>>
>> So I think it is kind of risky to use it here unless we can guarantee that
>> call site patching is atomic wrt other CPUs.
> Just look at where the patching is done:
>
> init/main.c:start_kernel()
>    check_bugs()
>      alternative_instructions()
>        apply_paravirt()
>
> We're UP and not holding any locks, disable IRQs (see text_poke_early())
> and have NMIs 'disabled'.

You are probably right. The initial apply_paravirt() was done before the 
SMP boot. Subsequent ones were at kernel module load time. I put a 
counter in the __native_queue_spin_unlock() and it registered 26949 
unlock calls in a 16-cpu guest before it got patched out.

The panic that I observed before might be due to some coding error of my 
own.

-Longman

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

* Re: [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
  2015-03-19 10:01       ` Peter Zijlstra
  (?)
@ 2015-03-19 21:08       ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-19 21:08 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/19/2015 06:01 AM, Peter Zijlstra wrote:
> On Wed, Mar 18, 2015 at 10:45:55PM -0400, Waiman Long wrote:
>> On 03/16/2015 09:16 AM, Peter Zijlstra wrote:
>> I do have some concern about this call site patching mechanism as the
>> modification is not atomic. The spin_unlock() calls are in many places in
>> the kernel. There is a possibility that a thread is calling a certain
>> spin_unlock call site while it is being patched by another one with the
>> alternative() function call.
>>
>> So far, I don't see any problem with bare metal where paravirt_patch_insns()
>> is used to patch it to the move instruction. However, in a virtual guest
>> enivornment where paravirt_patch_call() was used, there were situations
>> where the system panic because of page fault on some invalid memory in the
>> kthread. If you look at the paravirt_patch_call(), you will see:
>>
>>      :
>> b->opcode = 0xe8; /* call */
>> b->delta = delta;
>>
>> If another CPU reads the instruction at the call site at the right moment,
>> it will get the modified call instruction, but not the new delta value. It
>> will then jump to a random location. I believe that was causing the system
>> panic that I saw.
>>
>> So I think it is kind of risky to use it here unless we can guarantee that
>> call site patching is atomic wrt other CPUs.
> Just look at where the patching is done:
>
> init/main.c:start_kernel()
>    check_bugs()
>      alternative_instructions()
>        apply_paravirt()
>
> We're UP and not holding any locks, disable IRQs (see text_poke_early())
> and have NMIs 'disabled'.

You are probably right. The initial apply_paravirt() was done before the 
SMP boot. Subsequent ones were at kernel module load time. I put a 
counter in the __native_queue_spin_unlock() and it registered 26949 
unlock calls in a 16-cpu guest before it got patched out.

The panic that I observed before might be due to some coding error of my 
own.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 12:25         ` Peter Zijlstra
                           ` (4 preceding siblings ...)
  (?)
@ 2015-03-19 23:25         ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-19 23:25 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 03/19/2015 08:25 AM, Peter Zijlstra wrote:
> On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
>> So I was now thinking of hashing the lock pointer; let me go and quickly
>> put something together.
> A little something like so; ideally we'd allocate the hashtable since
> NR_CPUS is kinda bloated, but it shows the idea I think.
>
> And while this has loops in (the rehashing thing) their fwd progress
> does not depend on other CPUs.
>
> And I suspect that for the typical lock contention scenarios its
> unlikely we ever really get into long rehashing chains.
>
> ---
>   include/linux/lfsr.h                |   49 ++++++++++++
>   kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
>   2 files changed, 178 insertions(+), 14 deletions(-)

This is a much better alternative.

> --- /dev/null
> +++ b/include/linux/lfsr.h
> @@ -0,0 +1,49 @@
> +#ifndef _LINUX_LFSR_H
> +#define _LINUX_LFSR_H
> +
> +/*
> + * Simple Binary Galois Linear Feedback Shift Register
> + *
> + * http://en.wikipedia.org/wiki/Linear_feedback_shift_register
> + *
> + */
> +
> +extern void __lfsr_needs_more_taps(void);
> +
> +static __always_inline u32 lfsr_taps(int bits)
> +{
> +	if (bits ==  1) return 0x0001;
> +	if (bits ==  2) return 0x0001;
> +	if (bits ==  3) return 0x0003;
> +	if (bits ==  4) return 0x0009;
> +	if (bits ==  5) return 0x0012;
> +	if (bits ==  6) return 0x0021;
> +	if (bits ==  7) return 0x0041;
> +	if (bits ==  8) return 0x008E;
> +	if (bits ==  9) return 0x0108;
> +	if (bits == 10) return 0x0204;
> +	if (bits == 11) return 0x0402;
> +	if (bits == 12) return 0x0829;
> +	if (bits == 13) return 0x100D;
> +	if (bits == 14) return 0x2015;
> +
> +	/*
> +	 * For more taps see:
> +	 *   http://users.ece.cmu.edu/~koopman/lfsr/index.html
> +	 */
> +	__lfsr_needs_more_taps();
> +
> +	return 0;
> +}
> +
> +static inline u32 lfsr(u32 val, int bits)
> +{
> +	u32 bit = val&  1;
> +
> +	val>>= 1;
> +	if (bit)
> +		val ^= lfsr_taps(bits);
> +	return val;
> +}
> +
> +#endif /* _LINUX_LFSR_H */
> --- a/kernel/locking/qspinlock_paravirt.h
> +++ b/kernel/locking/qspinlock_paravirt.h
> @@ -2,6 +2,9 @@
>   #error "do not include this file"
>   #endif
>
> +#include<linux/hash.h>
> +#include<linux/lfsr.h>
> +
>   /*
>    * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
>    * of spinning them.
> @@ -107,7 +110,120 @@ static void pv_kick_node(struct mcs_spin
>   		pv_kick(pn->cpu);
>   }
>
> -static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
> +/*
> + * Hash table using open addressing with an LFSR probe sequence.
> + *
> + * Since we should not be holding locks from NMI context (very rare indeed) the
> + * max load factor is 0.75, which is around the point where open addressing
> + * breaks down.
> + *
> + * Instead of probing just the immediate bucket we probe all buckets in the
> + * same cacheline.
> + *
> + * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
> + *
> + */
> +
> +#define HB_RESERVED	((struct qspinlock *)1)
> +
> +struct pv_hash_bucket {
> +	struct qspinlock *lock;
> +	int cpu;
> +};
> +
> +/*
> + * XXX dynamic allocate using nr_cpu_ids instead...
> + */
> +#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
> +

As said here, we should make it dynamically allocated depending on 
num_possible_cpus().

> +#if PV_LOCK_HASH_BITS<  6
> +#undef PV_LOCK_HASH_BITS
> +#define PB_LOCK_HASH_BITS	6
> +#endif
> +
> +#define PV_LOCK_HASH_SIZE	(1<<  PV_LOCK_HASH_BITS)
> +
> +static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
> +
> +#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
> +
> +static inline u32 hash_align(u32 hash)
> +{
> +	return hash&  ~(PV_HB_PER_LINE - 1);
> +}
> +
> +static struct qspinlock **pv_hash(struct qspinlock *lock)
> +{
> +	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {
> +				WRITE_ONCE(hb->cpu, smp_processor_id());
> +				/*
> +				 * Since we must read lock first and cpu
> +				 * second, we must write cpu first and lock
> +				 * second, therefore use HB_RESERVE to mark an
> +				 * entry in use before writing the values.
> +				 *
> +				 * This can cause hb_hash_find() to not find a
> +				 * cpu even though _Q_SLOW_VAL, this is not a
> +				 * problem since we re-check l->locked before
> +				 * going to sleep and the unlock will have
> +				 * cleared l->locked already.
> +				 */
> +				smp_wmb(); /* matches rmb from pv_hash_find */
> +				WRITE_ONCE(hb->lock, lock);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb =&__pv_lock_hash[hash_align(hash)];
> +	}
> +
> +done:
> +	return&hb->lock;
> +}
> +
> +static int pv_hash_find(struct qspinlock *lock)
> +{
> +	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +	int cpu = -1;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			struct qspinlock *l = READ_ONCE(hb->lock);
> +
> +			/*
> +			 * If we hit an unused bucket, there is no match.
> +			 */
> +			if (!l)
> +				goto done;
> +
> +			if (l == lock) {
> +				smp_rmb(); /* matches wmb from pv_hash() */
> +				cpu = READ_ONCE(hb->cpu);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb =&__pv_lock_hash[hash_align(hash)];
> +	}
> +done:
> +	return cpu;
> +}
>   

We should probably abstract out the pv_hash and pv_hash_find into 
generic functions that can be put into header like hash.h instead of 
doing it locally here.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 12:25         ` Peter Zijlstra
                           ` (5 preceding siblings ...)
  (?)
@ 2015-03-19 23:25         ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-19 23:25 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/19/2015 08:25 AM, Peter Zijlstra wrote:
> On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
>> So I was now thinking of hashing the lock pointer; let me go and quickly
>> put something together.
> A little something like so; ideally we'd allocate the hashtable since
> NR_CPUS is kinda bloated, but it shows the idea I think.
>
> And while this has loops in (the rehashing thing) their fwd progress
> does not depend on other CPUs.
>
> And I suspect that for the typical lock contention scenarios its
> unlikely we ever really get into long rehashing chains.
>
> ---
>   include/linux/lfsr.h                |   49 ++++++++++++
>   kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
>   2 files changed, 178 insertions(+), 14 deletions(-)

This is a much better alternative.

> --- /dev/null
> +++ b/include/linux/lfsr.h
> @@ -0,0 +1,49 @@
> +#ifndef _LINUX_LFSR_H
> +#define _LINUX_LFSR_H
> +
> +/*
> + * Simple Binary Galois Linear Feedback Shift Register
> + *
> + * http://en.wikipedia.org/wiki/Linear_feedback_shift_register
> + *
> + */
> +
> +extern void __lfsr_needs_more_taps(void);
> +
> +static __always_inline u32 lfsr_taps(int bits)
> +{
> +	if (bits ==  1) return 0x0001;
> +	if (bits ==  2) return 0x0001;
> +	if (bits ==  3) return 0x0003;
> +	if (bits ==  4) return 0x0009;
> +	if (bits ==  5) return 0x0012;
> +	if (bits ==  6) return 0x0021;
> +	if (bits ==  7) return 0x0041;
> +	if (bits ==  8) return 0x008E;
> +	if (bits ==  9) return 0x0108;
> +	if (bits == 10) return 0x0204;
> +	if (bits == 11) return 0x0402;
> +	if (bits == 12) return 0x0829;
> +	if (bits == 13) return 0x100D;
> +	if (bits == 14) return 0x2015;
> +
> +	/*
> +	 * For more taps see:
> +	 *   http://users.ece.cmu.edu/~koopman/lfsr/index.html
> +	 */
> +	__lfsr_needs_more_taps();
> +
> +	return 0;
> +}
> +
> +static inline u32 lfsr(u32 val, int bits)
> +{
> +	u32 bit = val&  1;
> +
> +	val>>= 1;
> +	if (bit)
> +		val ^= lfsr_taps(bits);
> +	return val;
> +}
> +
> +#endif /* _LINUX_LFSR_H */
> --- a/kernel/locking/qspinlock_paravirt.h
> +++ b/kernel/locking/qspinlock_paravirt.h
> @@ -2,6 +2,9 @@
>   #error "do not include this file"
>   #endif
>
> +#include<linux/hash.h>
> +#include<linux/lfsr.h>
> +
>   /*
>    * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
>    * of spinning them.
> @@ -107,7 +110,120 @@ static void pv_kick_node(struct mcs_spin
>   		pv_kick(pn->cpu);
>   }
>
> -static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
> +/*
> + * Hash table using open addressing with an LFSR probe sequence.
> + *
> + * Since we should not be holding locks from NMI context (very rare indeed) the
> + * max load factor is 0.75, which is around the point where open addressing
> + * breaks down.
> + *
> + * Instead of probing just the immediate bucket we probe all buckets in the
> + * same cacheline.
> + *
> + * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
> + *
> + */
> +
> +#define HB_RESERVED	((struct qspinlock *)1)
> +
> +struct pv_hash_bucket {
> +	struct qspinlock *lock;
> +	int cpu;
> +};
> +
> +/*
> + * XXX dynamic allocate using nr_cpu_ids instead...
> + */
> +#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
> +

As said here, we should make it dynamically allocated depending on 
num_possible_cpus().

> +#if PV_LOCK_HASH_BITS<  6
> +#undef PV_LOCK_HASH_BITS
> +#define PB_LOCK_HASH_BITS	6
> +#endif
> +
> +#define PV_LOCK_HASH_SIZE	(1<<  PV_LOCK_HASH_BITS)
> +
> +static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
> +
> +#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
> +
> +static inline u32 hash_align(u32 hash)
> +{
> +	return hash&  ~(PV_HB_PER_LINE - 1);
> +}
> +
> +static struct qspinlock **pv_hash(struct qspinlock *lock)
> +{
> +	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {
> +				WRITE_ONCE(hb->cpu, smp_processor_id());
> +				/*
> +				 * Since we must read lock first and cpu
> +				 * second, we must write cpu first and lock
> +				 * second, therefore use HB_RESERVE to mark an
> +				 * entry in use before writing the values.
> +				 *
> +				 * This can cause hb_hash_find() to not find a
> +				 * cpu even though _Q_SLOW_VAL, this is not a
> +				 * problem since we re-check l->locked before
> +				 * going to sleep and the unlock will have
> +				 * cleared l->locked already.
> +				 */
> +				smp_wmb(); /* matches rmb from pv_hash_find */
> +				WRITE_ONCE(hb->lock, lock);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb =&__pv_lock_hash[hash_align(hash)];
> +	}
> +
> +done:
> +	return&hb->lock;
> +}
> +
> +static int pv_hash_find(struct qspinlock *lock)
> +{
> +	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +	int cpu = -1;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			struct qspinlock *l = READ_ONCE(hb->lock);
> +
> +			/*
> +			 * If we hit an unused bucket, there is no match.
> +			 */
> +			if (!l)
> +				goto done;
> +
> +			if (l == lock) {
> +				smp_rmb(); /* matches wmb from pv_hash() */
> +				cpu = READ_ONCE(hb->cpu);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb =&__pv_lock_hash[hash_align(hash)];
> +	}
> +done:
> +	return cpu;
> +}
>   

We should probably abstract out the pv_hash and pv_hash_find into 
generic functions that can be put into header like hash.h instead of 
doing it locally here.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 12:25         ` Peter Zijlstra
                           ` (3 preceding siblings ...)
  (?)
@ 2015-03-19 23:25         ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-19 23:25 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/19/2015 08:25 AM, Peter Zijlstra wrote:
> On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
>> So I was now thinking of hashing the lock pointer; let me go and quickly
>> put something together.
> A little something like so; ideally we'd allocate the hashtable since
> NR_CPUS is kinda bloated, but it shows the idea I think.
>
> And while this has loops in (the rehashing thing) their fwd progress
> does not depend on other CPUs.
>
> And I suspect that for the typical lock contention scenarios its
> unlikely we ever really get into long rehashing chains.
>
> ---
>   include/linux/lfsr.h                |   49 ++++++++++++
>   kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
>   2 files changed, 178 insertions(+), 14 deletions(-)

This is a much better alternative.

> --- /dev/null
> +++ b/include/linux/lfsr.h
> @@ -0,0 +1,49 @@
> +#ifndef _LINUX_LFSR_H
> +#define _LINUX_LFSR_H
> +
> +/*
> + * Simple Binary Galois Linear Feedback Shift Register
> + *
> + * http://en.wikipedia.org/wiki/Linear_feedback_shift_register
> + *
> + */
> +
> +extern void __lfsr_needs_more_taps(void);
> +
> +static __always_inline u32 lfsr_taps(int bits)
> +{
> +	if (bits ==  1) return 0x0001;
> +	if (bits ==  2) return 0x0001;
> +	if (bits ==  3) return 0x0003;
> +	if (bits ==  4) return 0x0009;
> +	if (bits ==  5) return 0x0012;
> +	if (bits ==  6) return 0x0021;
> +	if (bits ==  7) return 0x0041;
> +	if (bits ==  8) return 0x008E;
> +	if (bits ==  9) return 0x0108;
> +	if (bits == 10) return 0x0204;
> +	if (bits == 11) return 0x0402;
> +	if (bits == 12) return 0x0829;
> +	if (bits == 13) return 0x100D;
> +	if (bits == 14) return 0x2015;
> +
> +	/*
> +	 * For more taps see:
> +	 *   http://users.ece.cmu.edu/~koopman/lfsr/index.html
> +	 */
> +	__lfsr_needs_more_taps();
> +
> +	return 0;
> +}
> +
> +static inline u32 lfsr(u32 val, int bits)
> +{
> +	u32 bit = val&  1;
> +
> +	val>>= 1;
> +	if (bit)
> +		val ^= lfsr_taps(bits);
> +	return val;
> +}
> +
> +#endif /* _LINUX_LFSR_H */
> --- a/kernel/locking/qspinlock_paravirt.h
> +++ b/kernel/locking/qspinlock_paravirt.h
> @@ -2,6 +2,9 @@
>   #error "do not include this file"
>   #endif
>
> +#include<linux/hash.h>
> +#include<linux/lfsr.h>
> +
>   /*
>    * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
>    * of spinning them.
> @@ -107,7 +110,120 @@ static void pv_kick_node(struct mcs_spin
>   		pv_kick(pn->cpu);
>   }
>
> -static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
> +/*
> + * Hash table using open addressing with an LFSR probe sequence.
> + *
> + * Since we should not be holding locks from NMI context (very rare indeed) the
> + * max load factor is 0.75, which is around the point where open addressing
> + * breaks down.
> + *
> + * Instead of probing just the immediate bucket we probe all buckets in the
> + * same cacheline.
> + *
> + * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
> + *
> + */
> +
> +#define HB_RESERVED	((struct qspinlock *)1)
> +
> +struct pv_hash_bucket {
> +	struct qspinlock *lock;
> +	int cpu;
> +};
> +
> +/*
> + * XXX dynamic allocate using nr_cpu_ids instead...
> + */
> +#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
> +

As said here, we should make it dynamically allocated depending on 
num_possible_cpus().

> +#if PV_LOCK_HASH_BITS<  6
> +#undef PV_LOCK_HASH_BITS
> +#define PB_LOCK_HASH_BITS	6
> +#endif
> +
> +#define PV_LOCK_HASH_SIZE	(1<<  PV_LOCK_HASH_BITS)
> +
> +static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
> +
> +#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
> +
> +static inline u32 hash_align(u32 hash)
> +{
> +	return hash&  ~(PV_HB_PER_LINE - 1);
> +}
> +
> +static struct qspinlock **pv_hash(struct qspinlock *lock)
> +{
> +	u32 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			if (cmpxchg(&hb->lock, NULL, HB_RESERVED)) {
> +				WRITE_ONCE(hb->cpu, smp_processor_id());
> +				/*
> +				 * Since we must read lock first and cpu
> +				 * second, we must write cpu first and lock
> +				 * second, therefore use HB_RESERVE to mark an
> +				 * entry in use before writing the values.
> +				 *
> +				 * This can cause hb_hash_find() to not find a
> +				 * cpu even though _Q_SLOW_VAL, this is not a
> +				 * problem since we re-check l->locked before
> +				 * going to sleep and the unlock will have
> +				 * cleared l->locked already.
> +				 */
> +				smp_wmb(); /* matches rmb from pv_hash_find */
> +				WRITE_ONCE(hb->lock, lock);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb =&__pv_lock_hash[hash_align(hash)];
> +	}
> +
> +done:
> +	return&hb->lock;
> +}
> +
> +static int pv_hash_find(struct qspinlock *lock)
> +{
> +	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +	int cpu = -1;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			struct qspinlock *l = READ_ONCE(hb->lock);
> +
> +			/*
> +			 * If we hit an unused bucket, there is no match.
> +			 */
> +			if (!l)
> +				goto done;
> +
> +			if (l == lock) {
> +				smp_rmb(); /* matches wmb from pv_hash() */
> +				cpu = READ_ONCE(hb->cpu);
> +				goto done;
> +			}
> +		}
> +
> +		hash = lfsr(hash, PV_LOCK_HASH_BITS);
> +		hb =&__pv_lock_hash[hash_align(hash)];
> +	}
> +done:
> +	return cpu;
> +}
>   

We should probably abstract out the pv_hash and pv_hash_find into 
generic functions that can be put into header like hash.h instead of 
doing it locally here.

-Longman

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

* Re: [PATCH 9/9] qspinlock,x86,kvm: Implement KVM support for paravirt qspinlock
  2015-03-19 21:08         ` [PATCH 9/9] qspinlock, x86, kvm: " Waiman Long
@ 2015-03-20  7:43           ` Raghavendra K T
  -1 siblings, 0 replies; 136+ messages in thread
From: Raghavendra K T @ 2015-03-20  7:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: Peter Zijlstra, tglx, mingo, hpa, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 03/20/2015 02:38 AM, Waiman Long wrote:
> On 03/19/2015 06:01 AM, Peter Zijlstra wrote:
[...]
> You are probably right. The initial apply_paravirt() was done before the
> SMP boot. Subsequent ones were at kernel module load time. I put a
> counter in the __native_queue_spin_unlock() and it registered 26949
> unlock calls in a 16-cpu guest before it got patched out.

because even printks take lock..


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

* Re: [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
@ 2015-03-20  7:43           ` Raghavendra K T
  0 siblings, 0 replies; 136+ messages in thread
From: Raghavendra K T @ 2015-03-20  7:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, kvm, konrad.wilk, Peter Zijlstra, scott.norton,
	x86, paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/20/2015 02:38 AM, Waiman Long wrote:
> On 03/19/2015 06:01 AM, Peter Zijlstra wrote:
[...]
> You are probably right. The initial apply_paravirt() was done before the
> SMP boot. Subsequent ones were at kernel module load time. I put a
> counter in the __native_queue_spin_unlock() and it registered 26949
> unlock calls in a 16-cpu guest before it got patched out.

because even printks take lock..

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

* Re: [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock
  2015-03-19 21:08         ` [PATCH 9/9] qspinlock, x86, kvm: " Waiman Long
  (?)
@ 2015-03-20  7:43         ` Raghavendra K T
  -1 siblings, 0 replies; 136+ messages in thread
From: Raghavendra K T @ 2015-03-20  7:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, kvm, Peter Zijlstra, scott.norton, x86,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/20/2015 02:38 AM, Waiman Long wrote:
> On 03/19/2015 06:01 AM, Peter Zijlstra wrote:
[...]
> You are probably right. The initial apply_paravirt() was done before the
> SMP boot. Subsequent ones were at kernel module load time. I put a
> counter in the __native_queue_spin_unlock() and it registered 26949
> unlock calls in a 16-cpu guest before it got patched out.

because even printks take lock..

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (28 preceding siblings ...)
  (?)
@ 2015-03-25 19:47 ` Konrad Rzeszutek Wilk
  2015-03-26 20:21   ` Peter Zijlstra
                     ` (4 more replies)
  -1 siblings, 5 replies; 136+ messages in thread
From: Konrad Rzeszutek Wilk @ 2015-03-25 19:47 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, tglx, mingo, hpa, paolo.bonzini, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Mon, Mar 16, 2015 at 02:16:13PM +0100, Peter Zijlstra wrote:
> Hi Waiman,
> 
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
> 
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
> 
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
> 
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
> 
> So while the paravirt code isn't the most optimal code ever conceived it does work.
> 
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.

Ah nice. That could be spun out as a seperate patch to optimize the existing
ticket locks I presume.

Now with the old pv ticketlock code an vCPU would only go to sleep once and
be woken up when it was its turn. With this new code it is woken up twice 
(and twice it goes to sleep). With an overcommit scenario this would imply
that we will have at least twice as many VMEXIT as with the previous code.

I presume when you did benchmarking this did not even register? Thought
I wonder if it would if you ran the benchmark for a week or so.

> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
> 
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
> 
> Of course; once this lands people can look at improving the paravirt nonsense.
> 

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (29 preceding siblings ...)
  (?)
@ 2015-03-25 19:47 ` Konrad Rzeszutek Wilk
  -1 siblings, 0 replies; 136+ messages in thread
From: Konrad Rzeszutek Wilk @ 2015-03-25 19:47 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, linux-arch, riel, x86, kvm, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Mon, Mar 16, 2015 at 02:16:13PM +0100, Peter Zijlstra wrote:
> Hi Waiman,
> 
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
> 
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
> 
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
> 
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
> 
> So while the paravirt code isn't the most optimal code ever conceived it does work.
> 
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.

Ah nice. That could be spun out as a seperate patch to optimize the existing
ticket locks I presume.

Now with the old pv ticketlock code an vCPU would only go to sleep once and
be woken up when it was its turn. With this new code it is woken up twice 
(and twice it goes to sleep). With an overcommit scenario this would imply
that we will have at least twice as many VMEXIT as with the previous code.

I presume when you did benchmarking this did not even register? Thought
I wonder if it would if you ran the benchmark for a week or so.

> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
> 
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
> 
> Of course; once this lands people can look at improving the paravirt nonsense.
> 

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (30 preceding siblings ...)
  (?)
@ 2015-03-25 19:47 ` Konrad Rzeszutek Wilk
  -1 siblings, 0 replies; 136+ messages in thread
From: Konrad Rzeszutek Wilk @ 2015-03-25 19:47 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, linux-arch, riel, x86, kvm, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Mon, Mar 16, 2015 at 02:16:13PM +0100, Peter Zijlstra wrote:
> Hi Waiman,
> 
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
> 
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
> 
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
> 
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
> 
> So while the paravirt code isn't the most optimal code ever conceived it does work.
> 
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.

Ah nice. That could be spun out as a seperate patch to optimize the existing
ticket locks I presume.

Now with the old pv ticketlock code an vCPU would only go to sleep once and
be woken up when it was its turn. With this new code it is woken up twice 
(and twice it goes to sleep). With an overcommit scenario this would imply
that we will have at least twice as many VMEXIT as with the previous code.

I presume when you did benchmarking this did not even register? Thought
I wonder if it would if you ran the benchmark for a week or so.

> 
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
> 
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
> 
> Of course; once this lands people can look at improving the paravirt nonsense.
> 

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-25 19:47 ` Konrad Rzeszutek Wilk
@ 2015-03-26 20:21     ` Peter Zijlstra
  2015-03-26 20:21     ` Peter Zijlstra
                       ` (3 subsequent siblings)
  4 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-26 20:21 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: Waiman.Long, tglx, mingo, hpa, paolo.bonzini, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
> Ah nice. That could be spun out as a seperate patch to optimize the existing
> ticket locks I presume.

Yes I suppose we can do something similar for the ticket and patch in
the right increment. We'd need to restructure the code a bit, but
its not fundamentally impossible.

We could equally apply the head hashing to the current ticket
implementation and avoid the current bitmap iteration.

> Now with the old pv ticketlock code an vCPU would only go to sleep once and
> be woken up when it was its turn. With this new code it is woken up twice 
> (and twice it goes to sleep). With an overcommit scenario this would imply
> that we will have at least twice as many VMEXIT as with the previous code.

An astute observation, I had not considered that.

> I presume when you did benchmarking this did not even register? Thought
> I wonder if it would if you ran the benchmark for a week or so.

You presume I benchmarked :-) I managed to boot something virt and run
hackbench in it. I wouldn't know a representative virt setup if I ran
into it.

The thing is, we want this qspinlock for real hardware because its
faster and I really want to avoid having to carry two spinlock
implementations -- although I suppose that if we really really have to
we could.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-26 20:21     ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-26 20:21 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: Waiman.Long, linux-arch, riel, x86, kvm, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
> Ah nice. That could be spun out as a seperate patch to optimize the existing
> ticket locks I presume.

Yes I suppose we can do something similar for the ticket and patch in
the right increment. We'd need to restructure the code a bit, but
its not fundamentally impossible.

We could equally apply the head hashing to the current ticket
implementation and avoid the current bitmap iteration.

> Now with the old pv ticketlock code an vCPU would only go to sleep once and
> be woken up when it was its turn. With this new code it is woken up twice 
> (and twice it goes to sleep). With an overcommit scenario this would imply
> that we will have at least twice as many VMEXIT as with the previous code.

An astute observation, I had not considered that.

> I presume when you did benchmarking this did not even register? Thought
> I wonder if it would if you ran the benchmark for a week or so.

You presume I benchmarked :-) I managed to boot something virt and run
hackbench in it. I wouldn't know a representative virt setup if I ran
into it.

The thing is, we want this qspinlock for real hardware because its
faster and I really want to avoid having to carry two spinlock
implementations -- although I suppose that if we really really have to
we could.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-25 19:47 ` Konrad Rzeszutek Wilk
@ 2015-03-26 20:21   ` Peter Zijlstra
  2015-03-26 20:21     ` Peter Zijlstra
                     ` (3 subsequent siblings)
  4 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-26 20:21 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: Waiman.Long, linux-arch, riel, x86, kvm, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
> Ah nice. That could be spun out as a seperate patch to optimize the existing
> ticket locks I presume.

Yes I suppose we can do something similar for the ticket and patch in
the right increment. We'd need to restructure the code a bit, but
its not fundamentally impossible.

We could equally apply the head hashing to the current ticket
implementation and avoid the current bitmap iteration.

> Now with the old pv ticketlock code an vCPU would only go to sleep once and
> be woken up when it was its turn. With this new code it is woken up twice 
> (and twice it goes to sleep). With an overcommit scenario this would imply
> that we will have at least twice as many VMEXIT as with the previous code.

An astute observation, I had not considered that.

> I presume when you did benchmarking this did not even register? Thought
> I wonder if it would if you ran the benchmark for a week or so.

You presume I benchmarked :-) I managed to boot something virt and run
hackbench in it. I wouldn't know a representative virt setup if I ran
into it.

The thing is, we want this qspinlock for real hardware because its
faster and I really want to avoid having to carry two spinlock
implementations -- although I suppose that if we really really have to
we could.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (31 preceding siblings ...)
  (?)
@ 2015-03-27  6:40 ` Raghavendra K T
  -1 siblings, 0 replies; 136+ messages in thread
From: Raghavendra K T @ 2015-03-27  6:40 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, tglx, mingo, hpa, paolo.bonzini, konrad.wilk,
	boris.ostrovsky, paulmck, riel, torvalds, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 03/16/2015 06:46 PM, Peter Zijlstra wrote:
> Hi Waiman,
>
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
>
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
>
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
>
> So while the paravirt code isn't the most optimal code ever conceived it does work.
>
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
>
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
>
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
>
> Of course; once this lands people can look at improving the paravirt nonsense.
>

last time I had reported some hangs in kvm case, and I can confirm that
the current set of patches works fine.

Feel free to add
Tested-by: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com> #kvm pv

As far as performance is concerned (with my 16core +ht machine having
16vcpu guests [ even w/ , w/o the lfsr hash patchset ]), I do not see
any significant observations to report, though I understand that we
could see much more benefit with large number of vcpus because of
possible reduction in cache bouncing.







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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (33 preceding siblings ...)
  (?)
@ 2015-03-27  6:40 ` Raghavendra K T
  -1 siblings, 0 replies; 136+ messages in thread
From: Raghavendra K T @ 2015-03-27  6:40 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, linux-arch, riel, kvm, konrad.wilk, scott.norton,
	x86, paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/16/2015 06:46 PM, Peter Zijlstra wrote:
> Hi Waiman,
>
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
>
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
>
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
>
> So while the paravirt code isn't the most optimal code ever conceived it does work.
>
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
>
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
>
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
>
> Of course; once this lands people can look at improving the paravirt nonsense.
>

last time I had reported some hangs in kvm case, and I can confirm that
the current set of patches works fine.

Feel free to add
Tested-by: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com> #kvm pv

As far as performance is concerned (with my 16core +ht machine having
16vcpu guests [ even w/ , w/o the lfsr hash patchset ]), I do not see
any significant observations to report, though I understand that we
could see much more benefit with large number of vcpus because of
possible reduction in cache bouncing.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-16 13:16 ` Peter Zijlstra
                   ` (32 preceding siblings ...)
  (?)
@ 2015-03-27  6:40 ` Raghavendra K T
  -1 siblings, 0 replies; 136+ messages in thread
From: Raghavendra K T @ 2015-03-27  6:40 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, linux-arch, riel, kvm, scott.norton, x86,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/16/2015 06:46 PM, Peter Zijlstra wrote:
> Hi Waiman,
>
> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>
> All the !paravirt patches are more or less the same as before (the only real
> change is the copyright lines in the first patch).
>
> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
> convoluted and I've no real way to test that but it should be stright fwd to
> make work.
>
> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
> it both booted and survived a hackbench run (perf bench sched messaging -g 20
> -l 5000).
>
> So while the paravirt code isn't the most optimal code ever conceived it does work.
>
> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
> for the native case, which should greatly reduce the cost of having
> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
>
> I feel that if someone were to do a Xen patch we can go ahead and merge this
> stuff (finally!).
>
> These patches do not implement the paravirt spinlock debug stats currently
> implemented (separately) by KVM and Xen, but that should not be too hard to do
> on top and in the 'generic' code -- no reason to duplicate all that.
>
> Of course; once this lands people can look at improving the paravirt nonsense.
>

last time I had reported some hangs in kvm case, and I can confirm that
the current set of patches works fine.

Feel free to add
Tested-by: Raghavendra K T <raghavendra.kt@linux.vnet.ibm.com> #kvm pv

As far as performance is concerned (with my 16core +ht machine having
16vcpu guests [ even w/ , w/o the lfsr hash patchset ]), I do not see
any significant observations to report, though I understand that we
could see much more benefit with large number of vcpus because of
possible reduction in cache bouncing.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-26 20:21     ` Peter Zijlstra
                       ` (2 preceding siblings ...)
  (?)
@ 2015-03-27 14:07     ` Konrad Rzeszutek Wilk
  2015-03-30 16:41       ` Waiman Long
                         ` (2 more replies)
  -1 siblings, 3 replies; 136+ messages in thread
From: Konrad Rzeszutek Wilk @ 2015-03-27 14:07 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, tglx, mingo, hpa, paolo.bonzini, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Thu, Mar 26, 2015 at 09:21:53PM +0100, Peter Zijlstra wrote:
> On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
> > Ah nice. That could be spun out as a seperate patch to optimize the existing
> > ticket locks I presume.
> 
> Yes I suppose we can do something similar for the ticket and patch in
> the right increment. We'd need to restructure the code a bit, but
> its not fundamentally impossible.
> 
> We could equally apply the head hashing to the current ticket
> implementation and avoid the current bitmap iteration.
> 
> > Now with the old pv ticketlock code an vCPU would only go to sleep once and
> > be woken up when it was its turn. With this new code it is woken up twice 
> > (and twice it goes to sleep). With an overcommit scenario this would imply
> > that we will have at least twice as many VMEXIT as with the previous code.
> 
> An astute observation, I had not considered that.

Thank you.
> 
> > I presume when you did benchmarking this did not even register? Thought
> > I wonder if it would if you ran the benchmark for a week or so.
> 
> You presume I benchmarked :-) I managed to boot something virt and run
> hackbench in it. I wouldn't know a representative virt setup if I ran
> into it.
> 
> The thing is, we want this qspinlock for real hardware because its
> faster and I really want to avoid having to carry two spinlock
> implementations -- although I suppose that if we really really have to
> we could.

In some way you already have that - for virtualized environments where you
don't have an PV mechanism you just use the byte spinlock - which is good.

And switching to PV ticketlock implementation after boot.. ugh. I feel your pain.

What if you used an PV bytelock implemenation? The code you posted already
'sprays' all the vCPUS to wake up. And that is exactly what you need for PV
bytelocks - well, you only need to wake up the vCPUS that have gone to sleep
waiting on an specific 'struct spinlock' and just stash those in an per-cpu
area. The old Xen spinlock code (Before 3.11?) had this.

Just an idea thought.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-26 20:21     ` Peter Zijlstra
  (?)
  (?)
@ 2015-03-27 14:07     ` Konrad Rzeszutek Wilk
  -1 siblings, 0 replies; 136+ messages in thread
From: Konrad Rzeszutek Wilk @ 2015-03-27 14:07 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, linux-arch, riel, x86, kvm, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Thu, Mar 26, 2015 at 09:21:53PM +0100, Peter Zijlstra wrote:
> On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
> > Ah nice. That could be spun out as a seperate patch to optimize the existing
> > ticket locks I presume.
> 
> Yes I suppose we can do something similar for the ticket and patch in
> the right increment. We'd need to restructure the code a bit, but
> its not fundamentally impossible.
> 
> We could equally apply the head hashing to the current ticket
> implementation and avoid the current bitmap iteration.
> 
> > Now with the old pv ticketlock code an vCPU would only go to sleep once and
> > be woken up when it was its turn. With this new code it is woken up twice 
> > (and twice it goes to sleep). With an overcommit scenario this would imply
> > that we will have at least twice as many VMEXIT as with the previous code.
> 
> An astute observation, I had not considered that.

Thank you.
> 
> > I presume when you did benchmarking this did not even register? Thought
> > I wonder if it would if you ran the benchmark for a week or so.
> 
> You presume I benchmarked :-) I managed to boot something virt and run
> hackbench in it. I wouldn't know a representative virt setup if I ran
> into it.
> 
> The thing is, we want this qspinlock for real hardware because its
> faster and I really want to avoid having to carry two spinlock
> implementations -- although I suppose that if we really really have to
> we could.

In some way you already have that - for virtualized environments where you
don't have an PV mechanism you just use the byte spinlock - which is good.

And switching to PV ticketlock implementation after boot.. ugh. I feel your pain.

What if you used an PV bytelock implemenation? The code you posted already
'sprays' all the vCPUS to wake up. And that is exactly what you need for PV
bytelocks - well, you only need to wake up the vCPUS that have gone to sleep
waiting on an specific 'struct spinlock' and just stash those in an per-cpu
area. The old Xen spinlock code (Before 3.11?) had this.

Just an idea thought.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-26 20:21     ` Peter Zijlstra
  (?)
@ 2015-03-27 14:07     ` Konrad Rzeszutek Wilk
  -1 siblings, 0 replies; 136+ messages in thread
From: Konrad Rzeszutek Wilk @ 2015-03-27 14:07 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Waiman.Long, linux-arch, riel, x86, kvm, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Thu, Mar 26, 2015 at 09:21:53PM +0100, Peter Zijlstra wrote:
> On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
> > Ah nice. That could be spun out as a seperate patch to optimize the existing
> > ticket locks I presume.
> 
> Yes I suppose we can do something similar for the ticket and patch in
> the right increment. We'd need to restructure the code a bit, but
> its not fundamentally impossible.
> 
> We could equally apply the head hashing to the current ticket
> implementation and avoid the current bitmap iteration.
> 
> > Now with the old pv ticketlock code an vCPU would only go to sleep once and
> > be woken up when it was its turn. With this new code it is woken up twice 
> > (and twice it goes to sleep). With an overcommit scenario this would imply
> > that we will have at least twice as many VMEXIT as with the previous code.
> 
> An astute observation, I had not considered that.

Thank you.
> 
> > I presume when you did benchmarking this did not even register? Thought
> > I wonder if it would if you ran the benchmark for a week or so.
> 
> You presume I benchmarked :-) I managed to boot something virt and run
> hackbench in it. I wouldn't know a representative virt setup if I ran
> into it.
> 
> The thing is, we want this qspinlock for real hardware because its
> faster and I really want to avoid having to carry two spinlock
> implementations -- although I suppose that if we really really have to
> we could.

In some way you already have that - for virtualized environments where you
don't have an PV mechanism you just use the byte spinlock - which is good.

And switching to PV ticketlock implementation after boot.. ugh. I feel your pain.

What if you used an PV bytelock implemenation? The code you posted already
'sprays' all the vCPUS to wake up. And that is exactly what you need for PV
bytelocks - well, you only need to wake up the vCPUS that have gone to sleep
waiting on an specific 'struct spinlock' and just stash those in an per-cpu
area. The old Xen spinlock code (Before 3.11?) had this.

Just an idea thought.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-25 19:47 ` Konrad Rzeszutek Wilk
                     ` (2 preceding siblings ...)
  2015-03-30 16:25   ` Waiman Long
@ 2015-03-30 16:25   ` Waiman Long
  2015-03-30 16:29       ` Peter Zijlstra
  2015-03-30 16:29     ` Peter Zijlstra
  2015-03-30 16:25   ` Waiman Long
  4 siblings, 2 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:25 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: Peter Zijlstra, tglx, mingo, hpa, paolo.bonzini, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 03/25/2015 03:47 PM, Konrad Rzeszutek Wilk wrote:
> On Mon, Mar 16, 2015 at 02:16:13PM +0100, Peter Zijlstra wrote:
>> Hi Waiman,
>>
>> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>>
>> All the !paravirt patches are more or less the same as before (the only real
>> change is the copyright lines in the first patch).
>>
>> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
>> convoluted and I've no real way to test that but it should be stright fwd to
>> make work.
>>
>> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
>> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
>> it both booted and survived a hackbench run (perf bench sched messaging -g 20
>> -l 5000).
>>
>> So while the paravirt code isn't the most optimal code ever conceived it does work.
>>
>> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
>> for the native case, which should greatly reduce the cost of having
>> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
> Ah nice. That could be spun out as a seperate patch to optimize the existing
> ticket locks I presume.

The goal is to replace ticket spinlock by queue spinlock. We may not 
want to support 2 different spinlock implementations in the kernel.

>
> Now with the old pv ticketlock code an vCPU would only go to sleep once and
> be woken up when it was its turn. With this new code it is woken up twice
> (and twice it goes to sleep). With an overcommit scenario this would imply
> that we will have at least twice as many VMEXIT as with the previous code.

I did it differently in my PV portion of the qspinlock patch. Instead of 
just waking up the CPU, the new lock holder will check if the new queue 
head has been halted. If so, it will set the slowpath flag for the 
halted queue head in the lock so as to wake it up at unlock time. This 
should eliminate your concern of dong twice as many VMEXIT in an 
overcommitted scenario.

BTW, I did some qspinlock vs. ticketspinlock benchmarks using AIM7 
high_systime workload on a 4-socket IvyBridge-EX system (60 cores, 120 
threads) with some interesting results.

In term of the performance benefit of this patch, I ran the
high_systime workload (which does a lot of fork() and exit())
at various load levels (500, 1000, 1500 and 2000 users) on a
4-socket IvyBridge-EX bare-metal system (60 cores, 120 threads)
with intel_pstate driver and performance scaling governor. The JPM
(jobs/minutes) and execution time results were as follows:

     Kernel          JPM        Execution Time
     ------          ---        --------------
At 500 users:
      3.19        118857.14        26.25s
     3.19-qspinlock    134889.75        23.13s
     % change     +13.5%            -11.9%

At 1000 users:
      3.19        204255.32        30.55s
      3.19-qspinlock    239631.34        26.04s
     % change     +17.3%            -14.8%

At 1500 users:
      3.19        177272.73        52.80s
      3.19-qspinlock    326132.40        28.70s
     % change     +84.0%            -45.6%

At 2000 users:
      3.19        196690.31        63.45s
      3.19-qspinlock    341730.56        36.52s
     % change     +73.7%            -42.4%

It turns out that this workload was causing quite a lot of spinlock
contention in the vanilla 3.19 kernel. The performance advantage of
this patch increases with heavier loads.

With the powersave governor, the JPM data were as follows:

     Users        3.19     3.19-qspinlock      % Change
     -----        ----    --------------      --------
      500      112635.38      132596.69       +17.7%
     1000      171240.40      240369.80       +40.4%
     1500      130507.53      324436.74      +148.6%
     2000      175972.93      341637.01       +94.1%

With the qspinlock patch, there wasn't too much difference in
performance between the 2 scaling governors. Without this patch,
the powersave governor was much slower than the performance governor.

By disabling the intel_pstate driver and use acpi_cpufreq instead,
the benchmark performance (JPM) at 1000 users level for the performance
and ondemand governors were:

       Governor          3.19    3.19-qspinlock       % Change
       --------          ----    --------------       --------
       performance   124949.94       219950.65        +76.0%
       ondemand          4838.90       206690.96        +4171%

The performance was just horrible when there was significant spinlock
contention with the ondemand governor. There was also significant
run-to-run variation.  A second run of the same benchmark gave a result
of 22115 JPMs. With the qspinlock patch, however, the performance was
much more stable under different cpufreq drivers and governors. That
is not the case with the default ticket spinlock implementation.

The %CPU times spent on spinlock contention (from perf) with the
performance governor and the intel_pstate driver were:

   Kernel Function        3.19 kernel    3.19-qspinlock kernel
   ---------------        -----------    ---------------------
At 500 users:
   _raw_spin_lock*          28.23%        2.25%
   queue_spin_lock_slowpath       N/A            4.05%

At 1000 users:
   _raw_spin_lock*          23.21%        2.25%
   queue_spin_lock_slowpath       N/A            4.42%

At 1500 users:
   _raw_spin_lock*          29.07%        2.24%
   queue_spin_lock_slowpath       N/A            4.49%

At 2000 users:
   _raw_spin_lock*          29.15%        2.26%
   queue_spin_lock_slowpath       N/A            4.82%

The top spinlock related entries in the perf profile for the 3.19
kernel at 1000 users were:

    7.40%    reaim  [kernel.kallsyms]  [k] _raw_spin_lock_irqsave
                |--58.96%-- rwsem_wake
                |--20.02%-- release_pages
                |--15.88%-- pagevec_lru_move_fn
                |--1.53%-- get_page_from_freelist
                |--0.78%-- __wake_up
                |--0.55%-- try_to_wake_up
                 --2.28%-- [...]
    3.13%    reaim  [kernel.kallsyms]  [k] _raw_spin_lock
                |--37.55%-- free_one_page
                |--17.47%-- __cache_free_alien
                |--4.95%-- __rcu_process_callbacks
                |--2.93%-- __pte_alloc
                |--2.68%-- __drain_alien_cache
                |--2.56%-- ext4_do_update_inode
                |--2.54%-- try_to_wake_up
                |--2.46%-- pgd_free
                |--2.32%-- cache_alloc_refill
                |--2.32%-- pgd_alloc
                |--2.32%-- free_pcppages_bulk
                |--1.88%-- do_wp_page
                |--1.77%-- handle_pte_fault
                |--1.58%-- do_anonymous_page
                |--1.56%-- rmqueue_bulk.clone.0
                |--1.35%-- copy_pte_range
                |--1.25%-- zap_pte_range
                |--1.13%-- cache_flusharray
                |--0.88%-- __pmd_alloc
                |--0.70%-- wake_up_new_task
                |--0.66%-- __pud_alloc
                |--0.59%-- ext4_discard_preallocations
                 --6.53%-- [...]

With the qspinlock patch, the perf profile at 1000 users was:

    3.25%    reaim  [kernel.kallsyms]  [k] queue_spin_lock_slowpath
                |--62.00%-- _raw_spin_lock_irqsave
                |          |--77.49%-- rwsem_wake
                |          |--11.99%-- release_pages
                |          |--4.34%-- pagevec_lru_move_fn
                |          |--1.93%-- get_page_from_freelist
                |          |--1.90%-- prepare_to_wait_exclusive
                |          |--1.29%-- __wake_up
                |          |--0.74%-- finish_wait
                |--11.63%-- _raw_spin_lock
                |          |--31.11%-- try_to_wake_up
                |          |--7.77%-- free_pcppages_bulk
                |          |--7.12%-- __drain_alien_cache
                |          |--6.17%-- rmqueue_bulk.clone.0
                |          |--4.17%-- __rcu_process_callbacks
                |          |--2.22%-- cache_alloc_refill
                |          |--2.15%-- wake_up_new_task
                |          |--1.80%-- ext4_do_update_inode
                |          |--1.52%-- cache_flusharray
                |          |--0.89%-- __mutex_unlock_slowpath
                |          |--0.64%-- ttwu_queue
                |--11.19%-- _raw_spin_lock_irq
                |          |--98.95%-- rwsem_down_write_failed
                |          |--0.93%-- __schedule
                |--7.91%-- queue_read_lock_slowpath
                |          _raw_read_lock
                |          |--96.79%-- do_wait
                |          |--2.44%-- do_prlimit
                |                     chrdev_open
                |                     do_dentry_open
                |                     vfs_open
                |                     do_last
                |                     path_openat
                |                     do_filp_open
                |                     do_sys_open
                |                     sys_open
                |                     system_call
                |                     __GI___libc_open
                |--7.05%-- queue_write_lock_slowpath
                |          _raw_write_lock_irq
                |          |--35.36%-- release_task
                |          |--32.76%-- copy_process
                |                     do_exit
                |                     do_group_exit
                |                     sys_exit_group
                |                     system_call
                 --0.22%-- [...]

This demonstrates the benefit of this patch for those applications
that run on multi-socket machines and can cause significant spinlock
contentions in the kernel.


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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-25 19:47 ` Konrad Rzeszutek Wilk
                     ` (3 preceding siblings ...)
  2015-03-30 16:25   ` Waiman Long
@ 2015-03-30 16:25   ` Waiman Long
  4 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:25 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: linux-arch, riel, x86, kvm, Peter Zijlstra, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/25/2015 03:47 PM, Konrad Rzeszutek Wilk wrote:
> On Mon, Mar 16, 2015 at 02:16:13PM +0100, Peter Zijlstra wrote:
>> Hi Waiman,
>>
>> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>>
>> All the !paravirt patches are more or less the same as before (the only real
>> change is the copyright lines in the first patch).
>>
>> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
>> convoluted and I've no real way to test that but it should be stright fwd to
>> make work.
>>
>> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
>> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
>> it both booted and survived a hackbench run (perf bench sched messaging -g 20
>> -l 5000).
>>
>> So while the paravirt code isn't the most optimal code ever conceived it does work.
>>
>> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
>> for the native case, which should greatly reduce the cost of having
>> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
> Ah nice. That could be spun out as a seperate patch to optimize the existing
> ticket locks I presume.

The goal is to replace ticket spinlock by queue spinlock. We may not 
want to support 2 different spinlock implementations in the kernel.

>
> Now with the old pv ticketlock code an vCPU would only go to sleep once and
> be woken up when it was its turn. With this new code it is woken up twice
> (and twice it goes to sleep). With an overcommit scenario this would imply
> that we will have at least twice as many VMEXIT as with the previous code.

I did it differently in my PV portion of the qspinlock patch. Instead of 
just waking up the CPU, the new lock holder will check if the new queue 
head has been halted. If so, it will set the slowpath flag for the 
halted queue head in the lock so as to wake it up at unlock time. This 
should eliminate your concern of dong twice as many VMEXIT in an 
overcommitted scenario.

BTW, I did some qspinlock vs. ticketspinlock benchmarks using AIM7 
high_systime workload on a 4-socket IvyBridge-EX system (60 cores, 120 
threads) with some interesting results.

In term of the performance benefit of this patch, I ran the
high_systime workload (which does a lot of fork() and exit())
at various load levels (500, 1000, 1500 and 2000 users) on a
4-socket IvyBridge-EX bare-metal system (60 cores, 120 threads)
with intel_pstate driver and performance scaling governor. The JPM
(jobs/minutes) and execution time results were as follows:

     Kernel          JPM        Execution Time
     ------          ---        --------------
At 500 users:
      3.19        118857.14        26.25s
     3.19-qspinlock    134889.75        23.13s
     % change     +13.5%            -11.9%

At 1000 users:
      3.19        204255.32        30.55s
      3.19-qspinlock    239631.34        26.04s
     % change     +17.3%            -14.8%

At 1500 users:
      3.19        177272.73        52.80s
      3.19-qspinlock    326132.40        28.70s
     % change     +84.0%            -45.6%

At 2000 users:
      3.19        196690.31        63.45s
      3.19-qspinlock    341730.56        36.52s
     % change     +73.7%            -42.4%

It turns out that this workload was causing quite a lot of spinlock
contention in the vanilla 3.19 kernel. The performance advantage of
this patch increases with heavier loads.

With the powersave governor, the JPM data were as follows:

     Users        3.19     3.19-qspinlock      % Change
     -----        ----    --------------      --------
      500      112635.38      132596.69       +17.7%
     1000      171240.40      240369.80       +40.4%
     1500      130507.53      324436.74      +148.6%
     2000      175972.93      341637.01       +94.1%

With the qspinlock patch, there wasn't too much difference in
performance between the 2 scaling governors. Without this patch,
the powersave governor was much slower than the performance governor.

By disabling the intel_pstate driver and use acpi_cpufreq instead,
the benchmark performance (JPM) at 1000 users level for the performance
and ondemand governors were:

       Governor          3.19    3.19-qspinlock       % Change
       --------          ----    --------------       --------
       performance   124949.94       219950.65        +76.0%
       ondemand          4838.90       206690.96        +4171%

The performance was just horrible when there was significant spinlock
contention with the ondemand governor. There was also significant
run-to-run variation.  A second run of the same benchmark gave a result
of 22115 JPMs. With the qspinlock patch, however, the performance was
much more stable under different cpufreq drivers and governors. That
is not the case with the default ticket spinlock implementation.

The %CPU times spent on spinlock contention (from perf) with the
performance governor and the intel_pstate driver were:

   Kernel Function        3.19 kernel    3.19-qspinlock kernel
   ---------------        -----------    ---------------------
At 500 users:
   _raw_spin_lock*          28.23%        2.25%
   queue_spin_lock_slowpath       N/A            4.05%

At 1000 users:
   _raw_spin_lock*          23.21%        2.25%
   queue_spin_lock_slowpath       N/A            4.42%

At 1500 users:
   _raw_spin_lock*          29.07%        2.24%
   queue_spin_lock_slowpath       N/A            4.49%

At 2000 users:
   _raw_spin_lock*          29.15%        2.26%
   queue_spin_lock_slowpath       N/A            4.82%

The top spinlock related entries in the perf profile for the 3.19
kernel at 1000 users were:

    7.40%    reaim  [kernel.kallsyms]  [k] _raw_spin_lock_irqsave
                |--58.96%-- rwsem_wake
                |--20.02%-- release_pages
                |--15.88%-- pagevec_lru_move_fn
                |--1.53%-- get_page_from_freelist
                |--0.78%-- __wake_up
                |--0.55%-- try_to_wake_up
                 --2.28%-- [...]
    3.13%    reaim  [kernel.kallsyms]  [k] _raw_spin_lock
                |--37.55%-- free_one_page
                |--17.47%-- __cache_free_alien
                |--4.95%-- __rcu_process_callbacks
                |--2.93%-- __pte_alloc
                |--2.68%-- __drain_alien_cache
                |--2.56%-- ext4_do_update_inode
                |--2.54%-- try_to_wake_up
                |--2.46%-- pgd_free
                |--2.32%-- cache_alloc_refill
                |--2.32%-- pgd_alloc
                |--2.32%-- free_pcppages_bulk
                |--1.88%-- do_wp_page
                |--1.77%-- handle_pte_fault
                |--1.58%-- do_anonymous_page
                |--1.56%-- rmqueue_bulk.clone.0
                |--1.35%-- copy_pte_range
                |--1.25%-- zap_pte_range
                |--1.13%-- cache_flusharray
                |--0.88%-- __pmd_alloc
                |--0.70%-- wake_up_new_task
                |--0.66%-- __pud_alloc
                |--0.59%-- ext4_discard_preallocations
                 --6.53%-- [...]

With the qspinlock patch, the perf profile at 1000 users was:

    3.25%    reaim  [kernel.kallsyms]  [k] queue_spin_lock_slowpath
                |--62.00%-- _raw_spin_lock_irqsave
                |          |--77.49%-- rwsem_wake
                |          |--11.99%-- release_pages
                |          |--4.34%-- pagevec_lru_move_fn
                |          |--1.93%-- get_page_from_freelist
                |          |--1.90%-- prepare_to_wait_exclusive
                |          |--1.29%-- __wake_up
                |          |--0.74%-- finish_wait
                |--11.63%-- _raw_spin_lock
                |          |--31.11%-- try_to_wake_up
                |          |--7.77%-- free_pcppages_bulk
                |          |--7.12%-- __drain_alien_cache
                |          |--6.17%-- rmqueue_bulk.clone.0
                |          |--4.17%-- __rcu_process_callbacks
                |          |--2.22%-- cache_alloc_refill
                |          |--2.15%-- wake_up_new_task
                |          |--1.80%-- ext4_do_update_inode
                |          |--1.52%-- cache_flusharray
                |          |--0.89%-- __mutex_unlock_slowpath
                |          |--0.64%-- ttwu_queue
                |--11.19%-- _raw_spin_lock_irq
                |          |--98.95%-- rwsem_down_write_failed
                |          |--0.93%-- __schedule
                |--7.91%-- queue_read_lock_slowpath
                |          _raw_read_lock
                |          |--96.79%-- do_wait
                |          |--2.44%-- do_prlimit
                |                     chrdev_open
                |                     do_dentry_open
                |                     vfs_open
                |                     do_last
                |                     path_openat
                |                     do_filp_open
                |                     do_sys_open
                |                     sys_open
                |                     system_call
                |                     __GI___libc_open
                |--7.05%-- queue_write_lock_slowpath
                |          _raw_write_lock_irq
                |          |--35.36%-- release_task
                |          |--32.76%-- copy_process
                |                     do_exit
                |                     do_group_exit
                |                     sys_exit_group
                |                     system_call
                 --0.22%-- [...]

This demonstrates the benefit of this patch for those applications
that run on multi-socket machines and can cause significant spinlock
contentions in the kernel.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-25 19:47 ` Konrad Rzeszutek Wilk
  2015-03-26 20:21   ` Peter Zijlstra
  2015-03-26 20:21     ` Peter Zijlstra
@ 2015-03-30 16:25   ` Waiman Long
  2015-03-30 16:25   ` Waiman Long
  2015-03-30 16:25   ` Waiman Long
  4 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:25 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: linux-arch, riel, x86, kvm, Peter Zijlstra, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/25/2015 03:47 PM, Konrad Rzeszutek Wilk wrote:
> On Mon, Mar 16, 2015 at 02:16:13PM +0100, Peter Zijlstra wrote:
>> Hi Waiman,
>>
>> As promised; here is the paravirt stuff I did during the trip to BOS last week.
>>
>> All the !paravirt patches are more or less the same as before (the only real
>> change is the copyright lines in the first patch).
>>
>> The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
>> convoluted and I've no real way to test that but it should be stright fwd to
>> make work.
>>
>> I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
>> overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
>> it both booted and survived a hackbench run (perf bench sched messaging -g 20
>> -l 5000).
>>
>> So while the paravirt code isn't the most optimal code ever conceived it does work.
>>
>> Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
>> for the native case, which should greatly reduce the cost of having
>> CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.
> Ah nice. That could be spun out as a seperate patch to optimize the existing
> ticket locks I presume.

The goal is to replace ticket spinlock by queue spinlock. We may not 
want to support 2 different spinlock implementations in the kernel.

>
> Now with the old pv ticketlock code an vCPU would only go to sleep once and
> be woken up when it was its turn. With this new code it is woken up twice
> (and twice it goes to sleep). With an overcommit scenario this would imply
> that we will have at least twice as many VMEXIT as with the previous code.

I did it differently in my PV portion of the qspinlock patch. Instead of 
just waking up the CPU, the new lock holder will check if the new queue 
head has been halted. If so, it will set the slowpath flag for the 
halted queue head in the lock so as to wake it up at unlock time. This 
should eliminate your concern of dong twice as many VMEXIT in an 
overcommitted scenario.

BTW, I did some qspinlock vs. ticketspinlock benchmarks using AIM7 
high_systime workload on a 4-socket IvyBridge-EX system (60 cores, 120 
threads) with some interesting results.

In term of the performance benefit of this patch, I ran the
high_systime workload (which does a lot of fork() and exit())
at various load levels (500, 1000, 1500 and 2000 users) on a
4-socket IvyBridge-EX bare-metal system (60 cores, 120 threads)
with intel_pstate driver and performance scaling governor. The JPM
(jobs/minutes) and execution time results were as follows:

     Kernel          JPM        Execution Time
     ------          ---        --------------
At 500 users:
      3.19        118857.14        26.25s
     3.19-qspinlock    134889.75        23.13s
     % change     +13.5%            -11.9%

At 1000 users:
      3.19        204255.32        30.55s
      3.19-qspinlock    239631.34        26.04s
     % change     +17.3%            -14.8%

At 1500 users:
      3.19        177272.73        52.80s
      3.19-qspinlock    326132.40        28.70s
     % change     +84.0%            -45.6%

At 2000 users:
      3.19        196690.31        63.45s
      3.19-qspinlock    341730.56        36.52s
     % change     +73.7%            -42.4%

It turns out that this workload was causing quite a lot of spinlock
contention in the vanilla 3.19 kernel. The performance advantage of
this patch increases with heavier loads.

With the powersave governor, the JPM data were as follows:

     Users        3.19     3.19-qspinlock      % Change
     -----        ----    --------------      --------
      500      112635.38      132596.69       +17.7%
     1000      171240.40      240369.80       +40.4%
     1500      130507.53      324436.74      +148.6%
     2000      175972.93      341637.01       +94.1%

With the qspinlock patch, there wasn't too much difference in
performance between the 2 scaling governors. Without this patch,
the powersave governor was much slower than the performance governor.

By disabling the intel_pstate driver and use acpi_cpufreq instead,
the benchmark performance (JPM) at 1000 users level for the performance
and ondemand governors were:

       Governor          3.19    3.19-qspinlock       % Change
       --------          ----    --------------       --------
       performance   124949.94       219950.65        +76.0%
       ondemand          4838.90       206690.96        +4171%

The performance was just horrible when there was significant spinlock
contention with the ondemand governor. There was also significant
run-to-run variation.  A second run of the same benchmark gave a result
of 22115 JPMs. With the qspinlock patch, however, the performance was
much more stable under different cpufreq drivers and governors. That
is not the case with the default ticket spinlock implementation.

The %CPU times spent on spinlock contention (from perf) with the
performance governor and the intel_pstate driver were:

   Kernel Function        3.19 kernel    3.19-qspinlock kernel
   ---------------        -----------    ---------------------
At 500 users:
   _raw_spin_lock*          28.23%        2.25%
   queue_spin_lock_slowpath       N/A            4.05%

At 1000 users:
   _raw_spin_lock*          23.21%        2.25%
   queue_spin_lock_slowpath       N/A            4.42%

At 1500 users:
   _raw_spin_lock*          29.07%        2.24%
   queue_spin_lock_slowpath       N/A            4.49%

At 2000 users:
   _raw_spin_lock*          29.15%        2.26%
   queue_spin_lock_slowpath       N/A            4.82%

The top spinlock related entries in the perf profile for the 3.19
kernel at 1000 users were:

    7.40%    reaim  [kernel.kallsyms]  [k] _raw_spin_lock_irqsave
                |--58.96%-- rwsem_wake
                |--20.02%-- release_pages
                |--15.88%-- pagevec_lru_move_fn
                |--1.53%-- get_page_from_freelist
                |--0.78%-- __wake_up
                |--0.55%-- try_to_wake_up
                 --2.28%-- [...]
    3.13%    reaim  [kernel.kallsyms]  [k] _raw_spin_lock
                |--37.55%-- free_one_page
                |--17.47%-- __cache_free_alien
                |--4.95%-- __rcu_process_callbacks
                |--2.93%-- __pte_alloc
                |--2.68%-- __drain_alien_cache
                |--2.56%-- ext4_do_update_inode
                |--2.54%-- try_to_wake_up
                |--2.46%-- pgd_free
                |--2.32%-- cache_alloc_refill
                |--2.32%-- pgd_alloc
                |--2.32%-- free_pcppages_bulk
                |--1.88%-- do_wp_page
                |--1.77%-- handle_pte_fault
                |--1.58%-- do_anonymous_page
                |--1.56%-- rmqueue_bulk.clone.0
                |--1.35%-- copy_pte_range
                |--1.25%-- zap_pte_range
                |--1.13%-- cache_flusharray
                |--0.88%-- __pmd_alloc
                |--0.70%-- wake_up_new_task
                |--0.66%-- __pud_alloc
                |--0.59%-- ext4_discard_preallocations
                 --6.53%-- [...]

With the qspinlock patch, the perf profile at 1000 users was:

    3.25%    reaim  [kernel.kallsyms]  [k] queue_spin_lock_slowpath
                |--62.00%-- _raw_spin_lock_irqsave
                |          |--77.49%-- rwsem_wake
                |          |--11.99%-- release_pages
                |          |--4.34%-- pagevec_lru_move_fn
                |          |--1.93%-- get_page_from_freelist
                |          |--1.90%-- prepare_to_wait_exclusive
                |          |--1.29%-- __wake_up
                |          |--0.74%-- finish_wait
                |--11.63%-- _raw_spin_lock
                |          |--31.11%-- try_to_wake_up
                |          |--7.77%-- free_pcppages_bulk
                |          |--7.12%-- __drain_alien_cache
                |          |--6.17%-- rmqueue_bulk.clone.0
                |          |--4.17%-- __rcu_process_callbacks
                |          |--2.22%-- cache_alloc_refill
                |          |--2.15%-- wake_up_new_task
                |          |--1.80%-- ext4_do_update_inode
                |          |--1.52%-- cache_flusharray
                |          |--0.89%-- __mutex_unlock_slowpath
                |          |--0.64%-- ttwu_queue
                |--11.19%-- _raw_spin_lock_irq
                |          |--98.95%-- rwsem_down_write_failed
                |          |--0.93%-- __schedule
                |--7.91%-- queue_read_lock_slowpath
                |          _raw_read_lock
                |          |--96.79%-- do_wait
                |          |--2.44%-- do_prlimit
                |                     chrdev_open
                |                     do_dentry_open
                |                     vfs_open
                |                     do_last
                |                     path_openat
                |                     do_filp_open
                |                     do_sys_open
                |                     sys_open
                |                     system_call
                |                     __GI___libc_open
                |--7.05%-- queue_write_lock_slowpath
                |          _raw_write_lock_irq
                |          |--35.36%-- release_task
                |          |--32.76%-- copy_process
                |                     do_exit
                |                     do_group_exit
                |                     sys_exit_group
                |                     system_call
                 --0.22%-- [...]

This demonstrates the benefit of this patch for those applications
that run on multi-socket machines and can cause significant spinlock
contentions in the kernel.

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-30 16:25   ` Waiman Long
@ 2015-03-30 16:29       ` Peter Zijlstra
  2015-03-30 16:29     ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-30 16:29 UTC (permalink / raw)
  To: Waiman Long
  Cc: Konrad Rzeszutek Wilk, tglx, mingo, hpa, paolo.bonzini,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

On Mon, Mar 30, 2015 at 12:25:12PM -0400, Waiman Long wrote:
> I did it differently in my PV portion of the qspinlock patch. Instead of
> just waking up the CPU, the new lock holder will check if the new queue head
> has been halted. If so, it will set the slowpath flag for the halted queue
> head in the lock so as to wake it up at unlock time. This should eliminate
> your concern of dong twice as many VMEXIT in an overcommitted scenario.

We can still do that on top of all this right? As you might have
realized I'm a fan of gradual complexity :-)

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

* Re: [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-30 16:29       ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-30 16:29 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, Konrad Rzeszutek Wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Mon, Mar 30, 2015 at 12:25:12PM -0400, Waiman Long wrote:
> I did it differently in my PV portion of the qspinlock patch. Instead of
> just waking up the CPU, the new lock holder will check if the new queue head
> has been halted. If so, it will set the slowpath flag for the halted queue
> head in the lock so as to wake it up at unlock time. This should eliminate
> your concern of dong twice as many VMEXIT in an overcommitted scenario.

We can still do that on top of all this right? As you might have
realized I'm a fan of gradual complexity :-)

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-30 16:25   ` Waiman Long
  2015-03-30 16:29       ` Peter Zijlstra
@ 2015-03-30 16:29     ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-30 16:29 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Mon, Mar 30, 2015 at 12:25:12PM -0400, Waiman Long wrote:
> I did it differently in my PV portion of the qspinlock patch. Instead of
> just waking up the CPU, the new lock holder will check if the new queue head
> has been halted. If so, it will set the slowpath flag for the halted queue
> head in the lock so as to wake it up at unlock time. This should eliminate
> your concern of dong twice as many VMEXIT in an overcommitted scenario.

We can still do that on top of all this right? As you might have
realized I'm a fan of gradual complexity :-)

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-27 14:07     ` Konrad Rzeszutek Wilk
  2015-03-30 16:41       ` Waiman Long
@ 2015-03-30 16:41       ` Waiman Long
  2015-03-30 16:41       ` Waiman Long
  2 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:41 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: Peter Zijlstra, tglx, mingo, hpa, paolo.bonzini, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 03/27/2015 10:07 AM, Konrad Rzeszutek Wilk wrote:
> On Thu, Mar 26, 2015 at 09:21:53PM +0100, Peter Zijlstra wrote:
>> On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
>>> Ah nice. That could be spun out as a seperate patch to optimize the existing
>>> ticket locks I presume.
>> Yes I suppose we can do something similar for the ticket and patch in
>> the right increment. We'd need to restructure the code a bit, but
>> its not fundamentally impossible.
>>
>> We could equally apply the head hashing to the current ticket
>> implementation and avoid the current bitmap iteration.
>>
>>> Now with the old pv ticketlock code an vCPU would only go to sleep once and
>>> be woken up when it was its turn. With this new code it is woken up twice
>>> (and twice it goes to sleep). With an overcommit scenario this would imply
>>> that we will have at least twice as many VMEXIT as with the previous code.
>> An astute observation, I had not considered that.
> Thank you.
>>> I presume when you did benchmarking this did not even register? Thought
>>> I wonder if it would if you ran the benchmark for a week or so.
>> You presume I benchmarked :-) I managed to boot something virt and run
>> hackbench in it. I wouldn't know a representative virt setup if I ran
>> into it.
>>
>> The thing is, we want this qspinlock for real hardware because its
>> faster and I really want to avoid having to carry two spinlock
>> implementations -- although I suppose that if we really really have to
>> we could.
> In some way you already have that - for virtualized environments where you
> don't have an PV mechanism you just use the byte spinlock - which is good.
>
> And switching to PV ticketlock implementation after boot.. ugh. I feel your pain.
>
> What if you used an PV bytelock implemenation? The code you posted already
> 'sprays' all the vCPUS to wake up. And that is exactly what you need for PV
> bytelocks - well, you only need to wake up the vCPUS that have gone to sleep
> waiting on an specific 'struct spinlock' and just stash those in an per-cpu
> area. The old Xen spinlock code (Before 3.11?) had this.
>
> Just an idea thought.

The current code should have just waken up one sleeping vCPU. We 
shouldn't want to wake up all of them and have almost all except one go 
back to sleep. I think the PV bytelock you suggest is workable. It 
should also simplify the implementation. It is just a matter of how much 
we value the fairness attribute of the PV ticket or queue spinlock 
implementation that we have.

-Longman

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-27 14:07     ` Konrad Rzeszutek Wilk
  2015-03-30 16:41       ` Waiman Long
  2015-03-30 16:41       ` Waiman Long
@ 2015-03-30 16:41       ` Waiman Long
  2 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:41 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: linux-arch, riel, x86, kvm, Peter Zijlstra, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/27/2015 10:07 AM, Konrad Rzeszutek Wilk wrote:
> On Thu, Mar 26, 2015 at 09:21:53PM +0100, Peter Zijlstra wrote:
>> On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
>>> Ah nice. That could be spun out as a seperate patch to optimize the existing
>>> ticket locks I presume.
>> Yes I suppose we can do something similar for the ticket and patch in
>> the right increment. We'd need to restructure the code a bit, but
>> its not fundamentally impossible.
>>
>> We could equally apply the head hashing to the current ticket
>> implementation and avoid the current bitmap iteration.
>>
>>> Now with the old pv ticketlock code an vCPU would only go to sleep once and
>>> be woken up when it was its turn. With this new code it is woken up twice
>>> (and twice it goes to sleep). With an overcommit scenario this would imply
>>> that we will have at least twice as many VMEXIT as with the previous code.
>> An astute observation, I had not considered that.
> Thank you.
>>> I presume when you did benchmarking this did not even register? Thought
>>> I wonder if it would if you ran the benchmark for a week or so.
>> You presume I benchmarked :-) I managed to boot something virt and run
>> hackbench in it. I wouldn't know a representative virt setup if I ran
>> into it.
>>
>> The thing is, we want this qspinlock for real hardware because its
>> faster and I really want to avoid having to carry two spinlock
>> implementations -- although I suppose that if we really really have to
>> we could.
> In some way you already have that - for virtualized environments where you
> don't have an PV mechanism you just use the byte spinlock - which is good.
>
> And switching to PV ticketlock implementation after boot.. ugh. I feel your pain.
>
> What if you used an PV bytelock implemenation? The code you posted already
> 'sprays' all the vCPUS to wake up. And that is exactly what you need for PV
> bytelocks - well, you only need to wake up the vCPUS that have gone to sleep
> waiting on an specific 'struct spinlock' and just stash those in an per-cpu
> area. The old Xen spinlock code (Before 3.11?) had this.
>
> Just an idea thought.

The current code should have just waken up one sleeping vCPU. We 
shouldn't want to wake up all of them and have almost all except one go 
back to sleep. I think the PV bytelock you suggest is workable. It 
should also simplify the implementation. It is just a matter of how much 
we value the fairness attribute of the PV ticket or queue spinlock 
implementation that we have.

-Longman

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-27 14:07     ` Konrad Rzeszutek Wilk
@ 2015-03-30 16:41       ` Waiman Long
  2015-03-30 16:41       ` Waiman Long
  2015-03-30 16:41       ` Waiman Long
  2 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:41 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: linux-arch, riel, x86, kvm, Peter Zijlstra, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/27/2015 10:07 AM, Konrad Rzeszutek Wilk wrote:
> On Thu, Mar 26, 2015 at 09:21:53PM +0100, Peter Zijlstra wrote:
>> On Wed, Mar 25, 2015 at 03:47:39PM -0400, Konrad Rzeszutek Wilk wrote:
>>> Ah nice. That could be spun out as a seperate patch to optimize the existing
>>> ticket locks I presume.
>> Yes I suppose we can do something similar for the ticket and patch in
>> the right increment. We'd need to restructure the code a bit, but
>> its not fundamentally impossible.
>>
>> We could equally apply the head hashing to the current ticket
>> implementation and avoid the current bitmap iteration.
>>
>>> Now with the old pv ticketlock code an vCPU would only go to sleep once and
>>> be woken up when it was its turn. With this new code it is woken up twice
>>> (and twice it goes to sleep). With an overcommit scenario this would imply
>>> that we will have at least twice as many VMEXIT as with the previous code.
>> An astute observation, I had not considered that.
> Thank you.
>>> I presume when you did benchmarking this did not even register? Thought
>>> I wonder if it would if you ran the benchmark for a week or so.
>> You presume I benchmarked :-) I managed to boot something virt and run
>> hackbench in it. I wouldn't know a representative virt setup if I ran
>> into it.
>>
>> The thing is, we want this qspinlock for real hardware because its
>> faster and I really want to avoid having to carry two spinlock
>> implementations -- although I suppose that if we really really have to
>> we could.
> In some way you already have that - for virtualized environments where you
> don't have an PV mechanism you just use the byte spinlock - which is good.
>
> And switching to PV ticketlock implementation after boot.. ugh. I feel your pain.
>
> What if you used an PV bytelock implemenation? The code you posted already
> 'sprays' all the vCPUS to wake up. And that is exactly what you need for PV
> bytelocks - well, you only need to wake up the vCPUS that have gone to sleep
> waiting on an specific 'struct spinlock' and just stash those in an per-cpu
> area. The old Xen spinlock code (Before 3.11?) had this.
>
> Just an idea thought.

The current code should have just waken up one sleeping vCPU. We 
shouldn't want to wake up all of them and have almost all except one go 
back to sleep. I think the PV bytelock you suggest is workable. It 
should also simplify the implementation. It is just a matter of how much 
we value the fairness attribute of the PV ticket or queue spinlock 
implementation that we have.

-Longman

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-30 16:29       ` Peter Zijlstra
@ 2015-03-30 16:43         ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:43 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Konrad Rzeszutek Wilk, tglx, mingo, hpa, paolo.bonzini,
	boris.ostrovsky, paulmck, riel, torvalds, raghavendra.kt,
	david.vrabel, oleg, scott.norton, doug.hatch, linux-arch, x86,
	linux-kernel, virtualization, xen-devel, kvm, luto

On 03/30/2015 12:29 PM, Peter Zijlstra wrote:
> On Mon, Mar 30, 2015 at 12:25:12PM -0400, Waiman Long wrote:
>> I did it differently in my PV portion of the qspinlock patch. Instead of
>> just waking up the CPU, the new lock holder will check if the new queue head
>> has been halted. If so, it will set the slowpath flag for the halted queue
>> head in the lock so as to wake it up at unlock time. This should eliminate
>> your concern of dong twice as many VMEXIT in an overcommitted scenario.
> We can still do that on top of all this right? As you might have
> realized I'm a fan of gradual complexity :-)

Of course. I am just saying that the concern can be addressed with some 
additional code change.

-Longman

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

* Re: [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-30 16:43         ` Waiman Long
  0 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:43 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, Konrad Rzeszutek Wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/30/2015 12:29 PM, Peter Zijlstra wrote:
> On Mon, Mar 30, 2015 at 12:25:12PM -0400, Waiman Long wrote:
>> I did it differently in my PV portion of the qspinlock patch. Instead of
>> just waking up the CPU, the new lock holder will check if the new queue head
>> has been halted. If so, it will set the slowpath flag for the halted queue
>> head in the lock so as to wake it up at unlock time. This should eliminate
>> your concern of dong twice as many VMEXIT in an overcommitted scenario.
> We can still do that on top of all this right? As you might have
> realized I'm a fan of gradual complexity :-)

Of course. I am just saying that the concern can be addressed with some 
additional code change.

-Longman

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

* Re: [PATCH 0/9] qspinlock stuff -v15
  2015-03-30 16:29       ` Peter Zijlstra
  (?)
@ 2015-03-30 16:43       ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-03-30 16:43 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/30/2015 12:29 PM, Peter Zijlstra wrote:
> On Mon, Mar 30, 2015 at 12:25:12PM -0400, Waiman Long wrote:
>> I did it differently in my PV portion of the qspinlock patch. Instead of
>> just waking up the CPU, the new lock holder will check if the new queue head
>> has been halted. If so, it will set the slowpath flag for the halted queue
>> head in the lock so as to wake it up at unlock time. This should eliminate
>> your concern of dong twice as many VMEXIT in an overcommitted scenario.
> We can still do that on top of all this right? As you might have
> realized I'm a fan of gradual complexity :-)

Of course. I am just saying that the concern can be addressed with some 
additional code change.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 12:25         ` Peter Zijlstra
@ 2015-04-01 16:20           ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 16:20 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 03/19/2015 08:25 AM, Peter Zijlstra wrote:
> On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
>> So I was now thinking of hashing the lock pointer; let me go and quickly
>> put something together.
> A little something like so; ideally we'd allocate the hashtable since
> NR_CPUS is kinda bloated, but it shows the idea I think.
>
> And while this has loops in (the rehashing thing) their fwd progress
> does not depend on other CPUs.
>
> And I suspect that for the typical lock contention scenarios its
> unlikely we ever really get into long rehashing chains.
>
> ---
>   include/linux/lfsr.h                |   49 ++++++++++++
>   kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
>   2 files changed, 178 insertions(+), 14 deletions(-)
>
> --- /dev/null
>
> +
> +static int pv_hash_find(struct qspinlock *lock)
> +{
> +	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +	int cpu = -1;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			struct qspinlock *l = READ_ONCE(hb->lock);
> +
> +			/*
> +			 * If we hit an unused bucket, there is no match.
> +			 */
> +			if (!l)
> +				goto done;

After more careful reading, I think the assumption that the presence of 
an unused bucket means there is no match is not true. Consider the scenario:

1. cpu 0 puts lock1 into hb[0]
2. cpu 1 puts lock2 into hb[1]
3. cpu 2 clears hb[0]
4. cpu 3 looks for lock2 and doesn't find it

I was thinking about putting some USED flag in the buckets, but then we 
will eventually fill them all up as used. If we put the entries into a 
hashed linked list, we have to deal with the complicated synchronization 
issues with link list update.

At this point, I am thinking using back your previous idea of passing 
the queue head information down the queue. I am now convinced that the 
unlock call site patching should work. So I will incorporate that in my 
next update.

Please let me know if you think my reasoning is not correct.

Thanks,
Longman


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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-04-01 16:20           ` Waiman Long
  0 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 16:20 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 03/19/2015 08:25 AM, Peter Zijlstra wrote:
> On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
>> So I was now thinking of hashing the lock pointer; let me go and quickly
>> put something together.
> A little something like so; ideally we'd allocate the hashtable since
> NR_CPUS is kinda bloated, but it shows the idea I think.
>
> And while this has loops in (the rehashing thing) their fwd progress
> does not depend on other CPUs.
>
> And I suspect that for the typical lock contention scenarios its
> unlikely we ever really get into long rehashing chains.
>
> ---
>   include/linux/lfsr.h                |   49 ++++++++++++
>   kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
>   2 files changed, 178 insertions(+), 14 deletions(-)
>
> --- /dev/null
>
> +
> +static int pv_hash_find(struct qspinlock *lock)
> +{
> +	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +	int cpu = -1;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			struct qspinlock *l = READ_ONCE(hb->lock);
> +
> +			/*
> +			 * If we hit an unused bucket, there is no match.
> +			 */
> +			if (!l)
> +				goto done;

After more careful reading, I think the assumption that the presence of 
an unused bucket means there is no match is not true. Consider the scenario:

1. cpu 0 puts lock1 into hb[0]
2. cpu 1 puts lock2 into hb[1]
3. cpu 2 clears hb[0]
4. cpu 3 looks for lock2 and doesn't find it

I was thinking about putting some USED flag in the buckets, but then we 
will eventually fill them all up as used. If we put the entries into a 
hashed linked list, we have to deal with the complicated synchronization 
issues with link list update.

At this point, I am thinking using back your previous idea of passing 
the queue head information down the queue. I am now convinced that the 
unlock call site patching should work. So I will incorporate that in my 
next update.

Please let me know if you think my reasoning is not correct.

Thanks,
Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-03-19 12:25         ` Peter Zijlstra
                           ` (6 preceding siblings ...)
  (?)
@ 2015-04-01 16:20         ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 16:20 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 03/19/2015 08:25 AM, Peter Zijlstra wrote:
> On Thu, Mar 19, 2015 at 11:12:42AM +0100, Peter Zijlstra wrote:
>> So I was now thinking of hashing the lock pointer; let me go and quickly
>> put something together.
> A little something like so; ideally we'd allocate the hashtable since
> NR_CPUS is kinda bloated, but it shows the idea I think.
>
> And while this has loops in (the rehashing thing) their fwd progress
> does not depend on other CPUs.
>
> And I suspect that for the typical lock contention scenarios its
> unlikely we ever really get into long rehashing chains.
>
> ---
>   include/linux/lfsr.h                |   49 ++++++++++++
>   kernel/locking/qspinlock_paravirt.h |  143 ++++++++++++++++++++++++++++++++----
>   2 files changed, 178 insertions(+), 14 deletions(-)
>
> --- /dev/null
>
> +
> +static int pv_hash_find(struct qspinlock *lock)
> +{
> +	u64 hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb, *end;
> +	int cpu = -1;
> +
> +	if (!hash)
> +		hash = 1;
> +
> +	hb =&__pv_lock_hash[hash_align(hash)];
> +	for (;;) {
> +		for (end = hb + PV_HB_PER_LINE; hb<  end; hb++) {
> +			struct qspinlock *l = READ_ONCE(hb->lock);
> +
> +			/*
> +			 * If we hit an unused bucket, there is no match.
> +			 */
> +			if (!l)
> +				goto done;

After more careful reading, I think the assumption that the presence of 
an unused bucket means there is no match is not true. Consider the scenario:

1. cpu 0 puts lock1 into hb[0]
2. cpu 1 puts lock2 into hb[1]
3. cpu 2 clears hb[0]
4. cpu 3 looks for lock2 and doesn't find it

I was thinking about putting some USED flag in the buckets, but then we 
will eventually fill them all up as used. If we put the entries into a 
hashed linked list, we have to deal with the complicated synchronization 
issues with link list update.

At this point, I am thinking using back your previous idea of passing 
the queue head information down the queue. I am now convinced that the 
unlock call site patching should work. So I will incorporate that in my 
next update.

Please let me know if you think my reasoning is not correct.

Thanks,
Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 16:20           ` Waiman Long
                             ` (2 preceding siblings ...)
  (?)
@ 2015-04-01 17:12           ` Peter Zijlstra
  2015-04-01 17:42               ` Peter Zijlstra
                               ` (4 more replies)
  -1 siblings, 5 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 17:12 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
> After more careful reading, I think the assumption that the presence of an
> unused bucket means there is no match is not true. Consider the scenario:
> 
> 1. cpu 0 puts lock1 into hb[0]
> 2. cpu 1 puts lock2 into hb[1]
> 3. cpu 2 clears hb[0]
> 4. cpu 3 looks for lock2 and doesn't find it

Hmm, yes. The only way I can see that being true is if we assume entries
are never taken out again.

The wikipedia page could use some clarification here, this is not clear.

> At this point, I am thinking using back your previous idea of passing the
> queue head information down the queue.

Having to scan the entire array for a lookup sure sucks, but the wait
loops involved in the other idea can get us in the exact predicament we
were trying to get out, because their forward progress depends on other
CPUs.

Hohumm.. time to think more I think ;-)

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 16:20           ` Waiman Long
  (?)
  (?)
@ 2015-04-01 17:12           ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 17:12 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
> After more careful reading, I think the assumption that the presence of an
> unused bucket means there is no match is not true. Consider the scenario:
> 
> 1. cpu 0 puts lock1 into hb[0]
> 2. cpu 1 puts lock2 into hb[1]
> 3. cpu 2 clears hb[0]
> 4. cpu 3 looks for lock2 and doesn't find it

Hmm, yes. The only way I can see that being true is if we assume entries
are never taken out again.

The wikipedia page could use some clarification here, this is not clear.

> At this point, I am thinking using back your previous idea of passing the
> queue head information down the queue.

Having to scan the entire array for a lookup sure sucks, but the wait
loops involved in the other idea can get us in the exact predicament we
were trying to get out, because their forward progress depends on other
CPUs.

Hohumm.. time to think more I think ;-)

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 16:20           ` Waiman Long
  (?)
@ 2015-04-01 17:12           ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 17:12 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
> After more careful reading, I think the assumption that the presence of an
> unused bucket means there is no match is not true. Consider the scenario:
> 
> 1. cpu 0 puts lock1 into hb[0]
> 2. cpu 1 puts lock2 into hb[1]
> 3. cpu 2 clears hb[0]
> 4. cpu 3 looks for lock2 and doesn't find it

Hmm, yes. The only way I can see that being true is if we assume entries
are never taken out again.

The wikipedia page could use some clarification here, this is not clear.

> At this point, I am thinking using back your previous idea of passing the
> queue head information down the queue.

Having to scan the entire array for a lookup sure sucks, but the wait
loops involved in the other idea can get us in the exact predicament we
were trying to get out, because their forward progress depends on other
CPUs.

Hohumm.. time to think more I think ;-)

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 17:12           ` Peter Zijlstra
@ 2015-04-01 17:42               ` Peter Zijlstra
  2015-04-01 17:42             ` Peter Zijlstra
                                 ` (3 subsequent siblings)
  4 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 17:42 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Wed, Apr 01, 2015 at 07:12:23PM +0200, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
> > After more careful reading, I think the assumption that the presence of an
> > unused bucket means there is no match is not true. Consider the scenario:
> > 
> > 1. cpu 0 puts lock1 into hb[0]
> > 2. cpu 1 puts lock2 into hb[1]
> > 3. cpu 2 clears hb[0]
> > 4. cpu 3 looks for lock2 and doesn't find it
> 
> Hmm, yes. The only way I can see that being true is if we assume entries
> are never taken out again.
> 
> The wikipedia page could use some clarification here, this is not clear.
> 
> > At this point, I am thinking using back your previous idea of passing the
> > queue head information down the queue.
> 
> Having to scan the entire array for a lookup sure sucks, but the wait
> loops involved in the other idea can get us in the exact predicament we
> were trying to get out, because their forward progress depends on other
> CPUs.
> 
> Hohumm.. time to think more I think ;-)

So bear with me, I've not really pondered this well so it could be full
of holes (again).

After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
spin_unlock() must do the hash lookup, right? We can make the lookup
unhash.

If the cmpxchg() fails the unlock will not do the lookup and we must
unhash.


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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-04-01 17:42               ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 17:42 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Apr 01, 2015 at 07:12:23PM +0200, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
> > After more careful reading, I think the assumption that the presence of an
> > unused bucket means there is no match is not true. Consider the scenario:
> > 
> > 1. cpu 0 puts lock1 into hb[0]
> > 2. cpu 1 puts lock2 into hb[1]
> > 3. cpu 2 clears hb[0]
> > 4. cpu 3 looks for lock2 and doesn't find it
> 
> Hmm, yes. The only way I can see that being true is if we assume entries
> are never taken out again.
> 
> The wikipedia page could use some clarification here, this is not clear.
> 
> > At this point, I am thinking using back your previous idea of passing the
> > queue head information down the queue.
> 
> Having to scan the entire array for a lookup sure sucks, but the wait
> loops involved in the other idea can get us in the exact predicament we
> were trying to get out, because their forward progress depends on other
> CPUs.
> 
> Hohumm.. time to think more I think ;-)

So bear with me, I've not really pondered this well so it could be full
of holes (again).

After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
spin_unlock() must do the hash lookup, right? We can make the lookup
unhash.

If the cmpxchg() fails the unlock will not do the lookup and we must
unhash.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 17:12           ` Peter Zijlstra
  2015-04-01 17:42               ` Peter Zijlstra
@ 2015-04-01 17:42             ` Peter Zijlstra
  2015-04-01 20:10             ` Waiman Long
                               ` (2 subsequent siblings)
  4 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 17:42 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Wed, Apr 01, 2015 at 07:12:23PM +0200, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
> > After more careful reading, I think the assumption that the presence of an
> > unused bucket means there is no match is not true. Consider the scenario:
> > 
> > 1. cpu 0 puts lock1 into hb[0]
> > 2. cpu 1 puts lock2 into hb[1]
> > 3. cpu 2 clears hb[0]
> > 4. cpu 3 looks for lock2 and doesn't find it
> 
> Hmm, yes. The only way I can see that being true is if we assume entries
> are never taken out again.
> 
> The wikipedia page could use some clarification here, this is not clear.
> 
> > At this point, I am thinking using back your previous idea of passing the
> > queue head information down the queue.
> 
> Having to scan the entire array for a lookup sure sucks, but the wait
> loops involved in the other idea can get us in the exact predicament we
> were trying to get out, because their forward progress depends on other
> CPUs.
> 
> Hohumm.. time to think more I think ;-)

So bear with me, I've not really pondered this well so it could be full
of holes (again).

After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
spin_unlock() must do the hash lookup, right? We can make the lookup
unhash.

If the cmpxchg() fails the unlock will not do the lookup and we must
unhash.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 17:42               ` Peter Zijlstra
@ 2015-04-01 18:17                 ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 18:17 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
> > Hohumm.. time to think more I think ;-)
> 
> So bear with me, I've not really pondered this well so it could be full
> of holes (again).
> 
> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
> spin_unlock() must do the hash lookup, right? We can make the lookup
> unhash.
> 
> If the cmpxchg() fails the unlock will not do the lookup and we must
> unhash.

The idea being that the result is that any lookup is guaranteed to find
an entry, which reduces our worst case lookup cost to whatever the worst
case insertion cost was.


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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-04-01 18:17                 ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 18:17 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
> > Hohumm.. time to think more I think ;-)
> 
> So bear with me, I've not really pondered this well so it could be full
> of holes (again).
> 
> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
> spin_unlock() must do the hash lookup, right? We can make the lookup
> unhash.
> 
> If the cmpxchg() fails the unlock will not do the lookup and we must
> unhash.

The idea being that the result is that any lookup is guaranteed to find
an entry, which reduces our worst case lookup cost to whatever the worst
case insertion cost was.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 17:42               ` Peter Zijlstra
  (?)
  (?)
@ 2015-04-01 18:17               ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 18:17 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
> > Hohumm.. time to think more I think ;-)
> 
> So bear with me, I've not really pondered this well so it could be full
> of holes (again).
> 
> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
> spin_unlock() must do the hash lookup, right? We can make the lookup
> unhash.
> 
> If the cmpxchg() fails the unlock will not do the lookup and we must
> unhash.

The idea being that the result is that any lookup is guaranteed to find
an entry, which reduces our worst case lookup cost to whatever the worst
case insertion cost was.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 18:54                   ` Waiman Long
  (?)
  (?)
@ 2015-04-01 18:48                   ` Peter Zijlstra
  2015-04-01 19:58                     ` Waiman Long
                                       ` (2 more replies)
  -1 siblings, 3 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 18:48 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Wed, Apr 01, 2015 at 02:54:45PM -0400, Waiman Long wrote:
> On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
> >On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
> >>>Hohumm.. time to think more I think ;-)
> >>So bear with me, I've not really pondered this well so it could be full
> >>of holes (again).
> >>
> >>After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
> >>spin_unlock() must do the hash lookup, right? We can make the lookup
> >>unhash.
> >>
> >>If the cmpxchg() fails the unlock will not do the lookup and we must
> >>unhash.
> >The idea being that the result is that any lookup is guaranteed to find
> >an entry, which reduces our worst case lookup cost to whatever the worst
> >case insertion cost was.
> >
> 
> I think it doesn't matter who did the unhashing. Multiple independent locks
> can be hashed to the same value. Since they can be unhashed independently,
> there is no way to know whether you have checked all the possible buckets.

oh but the crux is that you guarantee a lookup will find an entry. it will
never need to iterate the entire array.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 18:54                   ` Waiman Long
  (?)
@ 2015-04-01 18:48                   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 18:48 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Apr 01, 2015 at 02:54:45PM -0400, Waiman Long wrote:
> On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
> >On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
> >>>Hohumm.. time to think more I think ;-)
> >>So bear with me, I've not really pondered this well so it could be full
> >>of holes (again).
> >>
> >>After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
> >>spin_unlock() must do the hash lookup, right? We can make the lookup
> >>unhash.
> >>
> >>If the cmpxchg() fails the unlock will not do the lookup and we must
> >>unhash.
> >The idea being that the result is that any lookup is guaranteed to find
> >an entry, which reduces our worst case lookup cost to whatever the worst
> >case insertion cost was.
> >
> 
> I think it doesn't matter who did the unhashing. Multiple independent locks
> can be hashed to the same value. Since they can be unhashed independently,
> there is no way to know whether you have checked all the possible buckets.

oh but the crux is that you guarantee a lookup will find an entry. it will
never need to iterate the entire array.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 18:54                   ` Waiman Long
                                     ` (2 preceding siblings ...)
  (?)
@ 2015-04-01 18:48                   ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 18:48 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Wed, Apr 01, 2015 at 02:54:45PM -0400, Waiman Long wrote:
> On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
> >On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
> >>>Hohumm.. time to think more I think ;-)
> >>So bear with me, I've not really pondered this well so it could be full
> >>of holes (again).
> >>
> >>After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
> >>spin_unlock() must do the hash lookup, right? We can make the lookup
> >>unhash.
> >>
> >>If the cmpxchg() fails the unlock will not do the lookup and we must
> >>unhash.
> >The idea being that the result is that any lookup is guaranteed to find
> >an entry, which reduces our worst case lookup cost to whatever the worst
> >case insertion cost was.
> >
> 
> I think it doesn't matter who did the unhashing. Multiple independent locks
> can be hashed to the same value. Since they can be unhashed independently,
> there is no way to know whether you have checked all the possible buckets.

oh but the crux is that you guarantee a lookup will find an entry. it will
never need to iterate the entire array.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 18:17                 ` Peter Zijlstra
@ 2015-04-01 18:54                   ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 18:54 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
>>> Hohumm.. time to think more I think ;-)
>> So bear with me, I've not really pondered this well so it could be full
>> of holes (again).
>>
>> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
>> spin_unlock() must do the hash lookup, right? We can make the lookup
>> unhash.
>>
>> If the cmpxchg() fails the unlock will not do the lookup and we must
>> unhash.
> The idea being that the result is that any lookup is guaranteed to find
> an entry, which reduces our worst case lookup cost to whatever the worst
> case insertion cost was.
>

I think it doesn't matter who did the unhashing. Multiple independent 
locks can be hashed to the same value. Since they can be unhashed 
independently, there is no way to know whether you have checked all the 
possible buckets.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-04-01 18:54                   ` Waiman Long
  0 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 18:54 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
>>> Hohumm.. time to think more I think ;-)
>> So bear with me, I've not really pondered this well so it could be full
>> of holes (again).
>>
>> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
>> spin_unlock() must do the hash lookup, right? We can make the lookup
>> unhash.
>>
>> If the cmpxchg() fails the unlock will not do the lookup and we must
>> unhash.
> The idea being that the result is that any lookup is guaranteed to find
> an entry, which reduces our worst case lookup cost to whatever the worst
> case insertion cost was.
>

I think it doesn't matter who did the unhashing. Multiple independent 
locks can be hashed to the same value. Since they can be unhashed 
independently, there is no way to know whether you have checked all the 
possible buckets.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 18:17                 ` Peter Zijlstra
  (?)
  (?)
@ 2015-04-01 18:54                 ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 18:54 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
>>> Hohumm.. time to think more I think ;-)
>> So bear with me, I've not really pondered this well so it could be full
>> of holes (again).
>>
>> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
>> spin_unlock() must do the hash lookup, right? We can make the lookup
>> unhash.
>>
>> If the cmpxchg() fails the unlock will not do the lookup and we must
>> unhash.
> The idea being that the result is that any lookup is guaranteed to find
> an entry, which reduces our worst case lookup cost to whatever the worst
> case insertion cost was.
>

I think it doesn't matter who did the unhashing. Multiple independent 
locks can be hashed to the same value. Since they can be unhashed 
independently, there is no way to know whether you have checked all the 
possible buckets.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 18:48                   ` Peter Zijlstra
@ 2015-04-01 19:58                     ` Waiman Long
  2015-04-01 21:03                       ` Peter Zijlstra
  2015-04-01 21:03                         ` Peter Zijlstra
  2015-04-01 19:58                     ` Waiman Long
  2015-04-01 19:58                     ` Waiman Long
  2 siblings, 2 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 19:58 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 02:54:45PM -0400, Waiman Long wrote:
>> On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
>>> On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
>>>>> Hohumm.. time to think more I think ;-)
>>>> So bear with me, I've not really pondered this well so it could be full
>>>> of holes (again).
>>>>
>>>> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
>>>> spin_unlock() must do the hash lookup, right? We can make the lookup
>>>> unhash.
>>>>
>>>> If the cmpxchg() fails the unlock will not do the lookup and we must
>>>> unhash.
>>> The idea being that the result is that any lookup is guaranteed to find
>>> an entry, which reduces our worst case lookup cost to whatever the worst
>>> case insertion cost was.
>>>
>> I think it doesn't matter who did the unhashing. Multiple independent locks
>> can be hashed to the same value. Since they can be unhashed independently,
>> there is no way to know whether you have checked all the possible buckets.
> oh but the crux is that you guarantee a lookup will find an entry. it will
> never need to iterate the entire array.

I am sorry that I don't quite get what you mean here. My point is that 
in the hashing step, a cpu will need to scan an empty bucket to put the 
lock in. In the interim, an previously used bucket before the empty one 
may get freed. In the lookup step for that lock, the scanning will stop 
because of an empty bucket in front of the target one.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 18:48                   ` Peter Zijlstra
  2015-04-01 19:58                     ` Waiman Long
  2015-04-01 19:58                     ` Waiman Long
@ 2015-04-01 19:58                     ` Waiman Long
  2 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 19:58 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 02:54:45PM -0400, Waiman Long wrote:
>> On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
>>> On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
>>>>> Hohumm.. time to think more I think ;-)
>>>> So bear with me, I've not really pondered this well so it could be full
>>>> of holes (again).
>>>>
>>>> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
>>>> spin_unlock() must do the hash lookup, right? We can make the lookup
>>>> unhash.
>>>>
>>>> If the cmpxchg() fails the unlock will not do the lookup and we must
>>>> unhash.
>>> The idea being that the result is that any lookup is guaranteed to find
>>> an entry, which reduces our worst case lookup cost to whatever the worst
>>> case insertion cost was.
>>>
>> I think it doesn't matter who did the unhashing. Multiple independent locks
>> can be hashed to the same value. Since they can be unhashed independently,
>> there is no way to know whether you have checked all the possible buckets.
> oh but the crux is that you guarantee a lookup will find an entry. it will
> never need to iterate the entire array.

I am sorry that I don't quite get what you mean here. My point is that 
in the hashing step, a cpu will need to scan an empty bucket to put the 
lock in. In the interim, an previously used bucket before the empty one 
may get freed. In the lookup step for that lock, the scanning will stop 
because of an empty bucket in front of the target one.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 18:48                   ` Peter Zijlstra
  2015-04-01 19:58                     ` Waiman Long
@ 2015-04-01 19:58                     ` Waiman Long
  2015-04-01 19:58                     ` Waiman Long
  2 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 19:58 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 02:54:45PM -0400, Waiman Long wrote:
>> On 04/01/2015 02:17 PM, Peter Zijlstra wrote:
>>> On Wed, Apr 01, 2015 at 07:42:39PM +0200, Peter Zijlstra wrote:
>>>>> Hohumm.. time to think more I think ;-)
>>>> So bear with me, I've not really pondered this well so it could be full
>>>> of holes (again).
>>>>
>>>> After the cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL) succeeds the
>>>> spin_unlock() must do the hash lookup, right? We can make the lookup
>>>> unhash.
>>>>
>>>> If the cmpxchg() fails the unlock will not do the lookup and we must
>>>> unhash.
>>> The idea being that the result is that any lookup is guaranteed to find
>>> an entry, which reduces our worst case lookup cost to whatever the worst
>>> case insertion cost was.
>>>
>> I think it doesn't matter who did the unhashing. Multiple independent locks
>> can be hashed to the same value. Since they can be unhashed independently,
>> there is no way to know whether you have checked all the possible buckets.
> oh but the crux is that you guarantee a lookup will find an entry. it will
> never need to iterate the entire array.

I am sorry that I don't quite get what you mean here. My point is that 
in the hashing step, a cpu will need to scan an empty bucket to put the 
lock in. In the interim, an previously used bucket before the empty one 
may get freed. In the lookup step for that lock, the scanning will stop 
because of an empty bucket in front of the target one.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 17:12           ` Peter Zijlstra
  2015-04-01 17:42               ` Peter Zijlstra
  2015-04-01 17:42             ` Peter Zijlstra
@ 2015-04-01 20:10             ` Waiman Long
  2015-04-01 20:10             ` Waiman Long
  2015-04-01 20:10             ` Waiman Long
  4 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 20:10 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 04/01/2015 01:12 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
>> After more careful reading, I think the assumption that the presence of an
>> unused bucket means there is no match is not true. Consider the scenario:
>>
>> 1. cpu 0 puts lock1 into hb[0]
>> 2. cpu 1 puts lock2 into hb[1]
>> 3. cpu 2 clears hb[0]
>> 4. cpu 3 looks for lock2 and doesn't find it
> Hmm, yes. The only way I can see that being true is if we assume entries
> are never taken out again.
>
> The wikipedia page could use some clarification here, this is not clear.
>
>> At this point, I am thinking using back your previous idea of passing the
>> queue head information down the queue.
> Having to scan the entire array for a lookup sure sucks, but the wait
> loops involved in the other idea can get us in the exact predicament we
> were trying to get out, because their forward progress depends on other
> CPUs.

For the waiting loop, the worst case is when a new CPU get queued right 
before we write the head value to the previous tail node. In the case, 
the maximum number of retries is equal to the total number of CPUs - 2. 
But that should rarely happen.

I do find a way to guarantee forward progress in a few steps. I will try 
the normal way once. If that fails, I will insert the head node to the 
tail once again after saving the next pointer. After modifying the 
previous tail node, cmpxchg will be used to restore the previous tail. 
If that fails, we just have to wait until the next pointer is updated 
and write it out to the previous tail node. We can now restore the next 
pointer and move forward.

Let me know if that looks reasonable to you.

-Longman


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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 17:12           ` Peter Zijlstra
                               ` (2 preceding siblings ...)
  2015-04-01 20:10             ` Waiman Long
@ 2015-04-01 20:10             ` Waiman Long
  2015-04-01 20:10             ` Waiman Long
  4 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 20:10 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 04/01/2015 01:12 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
>> After more careful reading, I think the assumption that the presence of an
>> unused bucket means there is no match is not true. Consider the scenario:
>>
>> 1. cpu 0 puts lock1 into hb[0]
>> 2. cpu 1 puts lock2 into hb[1]
>> 3. cpu 2 clears hb[0]
>> 4. cpu 3 looks for lock2 and doesn't find it
> Hmm, yes. The only way I can see that being true is if we assume entries
> are never taken out again.
>
> The wikipedia page could use some clarification here, this is not clear.
>
>> At this point, I am thinking using back your previous idea of passing the
>> queue head information down the queue.
> Having to scan the entire array for a lookup sure sucks, but the wait
> loops involved in the other idea can get us in the exact predicament we
> were trying to get out, because their forward progress depends on other
> CPUs.

For the waiting loop, the worst case is when a new CPU get queued right 
before we write the head value to the previous tail node. In the case, 
the maximum number of retries is equal to the total number of CPUs - 2. 
But that should rarely happen.

I do find a way to guarantee forward progress in a few steps. I will try 
the normal way once. If that fails, I will insert the head node to the 
tail once again after saving the next pointer. After modifying the 
previous tail node, cmpxchg will be used to restore the previous tail. 
If that fails, we just have to wait until the next pointer is updated 
and write it out to the previous tail node. We can now restore the next 
pointer and move forward.

Let me know if that looks reasonable to you.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 17:12           ` Peter Zijlstra
                               ` (3 preceding siblings ...)
  2015-04-01 20:10             ` Waiman Long
@ 2015-04-01 20:10             ` Waiman Long
  4 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-01 20:10 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 04/01/2015 01:12 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 12:20:30PM -0400, Waiman Long wrote:
>> After more careful reading, I think the assumption that the presence of an
>> unused bucket means there is no match is not true. Consider the scenario:
>>
>> 1. cpu 0 puts lock1 into hb[0]
>> 2. cpu 1 puts lock2 into hb[1]
>> 3. cpu 2 clears hb[0]
>> 4. cpu 3 looks for lock2 and doesn't find it
> Hmm, yes. The only way I can see that being true is if we assume entries
> are never taken out again.
>
> The wikipedia page could use some clarification here, this is not clear.
>
>> At this point, I am thinking using back your previous idea of passing the
>> queue head information down the queue.
> Having to scan the entire array for a lookup sure sucks, but the wait
> loops involved in the other idea can get us in the exact predicament we
> were trying to get out, because their forward progress depends on other
> CPUs.

For the waiting loop, the worst case is when a new CPU get queued right 
before we write the head value to the previous tail node. In the case, 
the maximum number of retries is equal to the total number of CPUs - 2. 
But that should rarely happen.

I do find a way to guarantee forward progress in a few steps. I will try 
the normal way once. If that fails, I will insert the head node to the 
tail once again after saving the next pointer. After modifying the 
previous tail node, cmpxchg will be used to restore the previous tail. 
If that fails, we just have to wait until the next pointer is updated 
and write it out to the previous tail node. We can now restore the next 
pointer and move forward.

Let me know if that looks reasonable to you.

-Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 19:58                     ` Waiman Long
@ 2015-04-01 21:03                         ` Peter Zijlstra
  2015-04-01 21:03                         ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 21:03 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
> On 04/01/2015 02:48 PM, Peter Zijlstra wrote:

> I am sorry that I don't quite get what you mean here. My point is that in
> the hashing step, a cpu will need to scan an empty bucket to put the lock
> in. In the interim, an previously used bucket before the empty one may get
> freed. In the lookup step for that lock, the scanning will stop because of
> an empty bucket in front of the target one.

Right, that's broken. So we need to do something else to limit the
lookup, because without that break, a lookup that needs to iterate the
entire array in order to determine -ENOENT, which is expensive.

So my alternative proposal is that IFF we can guarantee that every
lookup will succeed -- the entry we're looking for is always there, we
don't need the break on empty but can probe until we find the entry.
This will be bound in cost to the same number if probes we required for
insertion and avoids the full array scan.

Now I think we can indeed do this, if as said earlier we do not clear
the bucket on insert if the cmpxchg succeeds, in that case the unlock
will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
the entry. And we then need the unlock to clear the entry.

Does that explain this? Or should I try again with code?

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-04-01 21:03                         ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 21:03 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
> On 04/01/2015 02:48 PM, Peter Zijlstra wrote:

> I am sorry that I don't quite get what you mean here. My point is that in
> the hashing step, a cpu will need to scan an empty bucket to put the lock
> in. In the interim, an previously used bucket before the empty one may get
> freed. In the lookup step for that lock, the scanning will stop because of
> an empty bucket in front of the target one.

Right, that's broken. So we need to do something else to limit the
lookup, because without that break, a lookup that needs to iterate the
entire array in order to determine -ENOENT, which is expensive.

So my alternative proposal is that IFF we can guarantee that every
lookup will succeed -- the entry we're looking for is always there, we
don't need the break on empty but can probe until we find the entry.
This will be bound in cost to the same number if probes we required for
insertion and avoids the full array scan.

Now I think we can indeed do this, if as said earlier we do not clear
the bucket on insert if the cmpxchg succeeds, in that case the unlock
will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
the entry. And we then need the unlock to clear the entry.

Does that explain this? Or should I try again with code?

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 19:58                     ` Waiman Long
@ 2015-04-01 21:03                       ` Peter Zijlstra
  2015-04-01 21:03                         ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-01 21:03 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
> On 04/01/2015 02:48 PM, Peter Zijlstra wrote:

> I am sorry that I don't quite get what you mean here. My point is that in
> the hashing step, a cpu will need to scan an empty bucket to put the lock
> in. In the interim, an previously used bucket before the empty one may get
> freed. In the lookup step for that lock, the scanning will stop because of
> an empty bucket in front of the target one.

Right, that's broken. So we need to do something else to limit the
lookup, because without that break, a lookup that needs to iterate the
entire array in order to determine -ENOENT, which is expensive.

So my alternative proposal is that IFF we can guarantee that every
lookup will succeed -- the entry we're looking for is always there, we
don't need the break on empty but can probe until we find the entry.
This will be bound in cost to the same number if probes we required for
insertion and avoids the full array scan.

Now I think we can indeed do this, if as said earlier we do not clear
the bucket on insert if the cmpxchg succeeds, in that case the unlock
will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
the entry. And we then need the unlock to clear the entry.

Does that explain this? Or should I try again with code?

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 21:03                         ` Peter Zijlstra
  (?)
@ 2015-04-02 16:28                         ` Waiman Long
  2015-04-02 17:20                             ` Peter Zijlstra
  2015-04-02 17:20                           ` Peter Zijlstra
  -1 siblings, 2 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-02 16:28 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 04/01/2015 05:03 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
>> On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
>> I am sorry that I don't quite get what you mean here. My point is that in
>> the hashing step, a cpu will need to scan an empty bucket to put the lock
>> in. In the interim, an previously used bucket before the empty one may get
>> freed. In the lookup step for that lock, the scanning will stop because of
>> an empty bucket in front of the target one.
> Right, that's broken. So we need to do something else to limit the
> lookup, because without that break, a lookup that needs to iterate the
> entire array in order to determine -ENOENT, which is expensive.
>
> So my alternative proposal is that IFF we can guarantee that every
> lookup will succeed -- the entry we're looking for is always there, we
> don't need the break on empty but can probe until we find the entry.
> This will be bound in cost to the same number if probes we required for
> insertion and avoids the full array scan.
>
> Now I think we can indeed do this, if as said earlier we do not clear
> the bucket on insert if the cmpxchg succeeds, in that case the unlock
> will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
> the entry. And we then need the unlock to clear the entry.
> _Q_SLOW_VAL
> Does that explain this? Or should I try again with code?

OK, I got your proposal now. However, there is still the issue that 
setting the _Q_SLOW_VAL flag and the hash bucket are not atomic wrt each 
other. It is possible a CPU has set the _Q_SLOW_VAL flag but not yet 
filling in the hash bucket while another one is trying to look for it. 
So we need to have some kind of synchronization mechanism to let the 
lookup CPU know when is a good time to look up.

One possibility is to delay setting _Q_SLOW_VAL until the hash bucket is 
set up. Maybe we can make that work.

Cheers,
Longman



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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 21:03                         ` Peter Zijlstra
                                           ` (2 preceding siblings ...)
  (?)
@ 2015-04-02 16:28                         ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-02 16:28 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 04/01/2015 05:03 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
>> On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
>> I am sorry that I don't quite get what you mean here. My point is that in
>> the hashing step, a cpu will need to scan an empty bucket to put the lock
>> in. In the interim, an previously used bucket before the empty one may get
>> freed. In the lookup step for that lock, the scanning will stop because of
>> an empty bucket in front of the target one.
> Right, that's broken. So we need to do something else to limit the
> lookup, because without that break, a lookup that needs to iterate the
> entire array in order to determine -ENOENT, which is expensive.
>
> So my alternative proposal is that IFF we can guarantee that every
> lookup will succeed -- the entry we're looking for is always there, we
> don't need the break on empty but can probe until we find the entry.
> This will be bound in cost to the same number if probes we required for
> insertion and avoids the full array scan.
>
> Now I think we can indeed do this, if as said earlier we do not clear
> the bucket on insert if the cmpxchg succeeds, in that case the unlock
> will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
> the entry. And we then need the unlock to clear the entry.
> _Q_SLOW_VAL
> Does that explain this? Or should I try again with code?

OK, I got your proposal now. However, there is still the issue that 
setting the _Q_SLOW_VAL flag and the hash bucket are not atomic wrt each 
other. It is possible a CPU has set the _Q_SLOW_VAL flag but not yet 
filling in the hash bucket while another one is trying to look for it. 
So we need to have some kind of synchronization mechanism to let the 
lookup CPU know when is a good time to look up.

One possibility is to delay setting _Q_SLOW_VAL until the hash bucket is 
set up. Maybe we can make that work.

Cheers,
Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-01 21:03                         ` Peter Zijlstra
  (?)
  (?)
@ 2015-04-02 16:28                         ` Waiman Long
  -1 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-02 16:28 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 04/01/2015 05:03 PM, Peter Zijlstra wrote:
> On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
>> On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
>> I am sorry that I don't quite get what you mean here. My point is that in
>> the hashing step, a cpu will need to scan an empty bucket to put the lock
>> in. In the interim, an previously used bucket before the empty one may get
>> freed. In the lookup step for that lock, the scanning will stop because of
>> an empty bucket in front of the target one.
> Right, that's broken. So we need to do something else to limit the
> lookup, because without that break, a lookup that needs to iterate the
> entire array in order to determine -ENOENT, which is expensive.
>
> So my alternative proposal is that IFF we can guarantee that every
> lookup will succeed -- the entry we're looking for is always there, we
> don't need the break on empty but can probe until we find the entry.
> This will be bound in cost to the same number if probes we required for
> insertion and avoids the full array scan.
>
> Now I think we can indeed do this, if as said earlier we do not clear
> the bucket on insert if the cmpxchg succeeds, in that case the unlock
> will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
> the entry. And we then need the unlock to clear the entry.
> _Q_SLOW_VAL
> Does that explain this? Or should I try again with code?

OK, I got your proposal now. However, there is still the issue that 
setting the _Q_SLOW_VAL flag and the hash bucket are not atomic wrt each 
other. It is possible a CPU has set the _Q_SLOW_VAL flag but not yet 
filling in the hash bucket while another one is trying to look for it. 
So we need to have some kind of synchronization mechanism to let the 
lookup CPU know when is a good time to look up.

One possibility is to delay setting _Q_SLOW_VAL until the hash bucket is 
set up. Maybe we can make that work.

Cheers,
Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 16:28                         ` Waiman Long
@ 2015-04-02 17:20                             ` Peter Zijlstra
  2015-04-02 17:20                           ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-02 17:20 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Thu, Apr 02, 2015 at 12:28:30PM -0400, Waiman Long wrote:
> On 04/01/2015 05:03 PM, Peter Zijlstra wrote:
> >On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
> >>On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
> >>I am sorry that I don't quite get what you mean here. My point is that in
> >>the hashing step, a cpu will need to scan an empty bucket to put the lock
> >>in. In the interim, an previously used bucket before the empty one may get
> >>freed. In the lookup step for that lock, the scanning will stop because of
> >>an empty bucket in front of the target one.
> >Right, that's broken. So we need to do something else to limit the
> >lookup, because without that break, a lookup that needs to iterate the
> >entire array in order to determine -ENOENT, which is expensive.
> >
> >So my alternative proposal is that IFF we can guarantee that every
> >lookup will succeed -- the entry we're looking for is always there, we
> >don't need the break on empty but can probe until we find the entry.
> >This will be bound in cost to the same number if probes we required for
> >insertion and avoids the full array scan.
> >
> >Now I think we can indeed do this, if as said earlier we do not clear
> >the bucket on insert if the cmpxchg succeeds, in that case the unlock
> >will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
> >the entry. And we then need the unlock to clear the entry.
> >_Q_SLOW_VAL
> >Does that explain this? Or should I try again with code?
> 
> OK, I got your proposal now. However, there is still the issue that setting
> the _Q_SLOW_VAL flag and the hash bucket are not atomic wrt each other.

So? They're strictly ordered, that's sufficient. We first hash the lock,
then we set _Q_SLOW_VAL. There's a full memory barrier between them.

> It
> is possible a CPU has set the _Q_SLOW_VAL flag but not yet filling in the
> hash bucket while another one is trying to look for it.

Nope. The unlock side does an xchg() on the locked value first, xchg
also implies a full barrier, so that guarantees that if we observe
_Q_SLOW_VAL we must also observe the hash bucket with the lock value.

> So we need to have
> some kind of synchronization mechanism to let the lookup CPU know when is a
> good time to look up.

No, its all already ordered and working.

pv_wait_head():

	pv_hash()
	/* MB as per cmpxchg */
	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);

VS

__pv_queue_spin_unlock():

	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
		return;

	/* MB as per xchg */
	pv_hash_find(lock);



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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-04-02 17:20                             ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-02 17:20 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Thu, Apr 02, 2015 at 12:28:30PM -0400, Waiman Long wrote:
> On 04/01/2015 05:03 PM, Peter Zijlstra wrote:
> >On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
> >>On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
> >>I am sorry that I don't quite get what you mean here. My point is that in
> >>the hashing step, a cpu will need to scan an empty bucket to put the lock
> >>in. In the interim, an previously used bucket before the empty one may get
> >>freed. In the lookup step for that lock, the scanning will stop because of
> >>an empty bucket in front of the target one.
> >Right, that's broken. So we need to do something else to limit the
> >lookup, because without that break, a lookup that needs to iterate the
> >entire array in order to determine -ENOENT, which is expensive.
> >
> >So my alternative proposal is that IFF we can guarantee that every
> >lookup will succeed -- the entry we're looking for is always there, we
> >don't need the break on empty but can probe until we find the entry.
> >This will be bound in cost to the same number if probes we required for
> >insertion and avoids the full array scan.
> >
> >Now I think we can indeed do this, if as said earlier we do not clear
> >the bucket on insert if the cmpxchg succeeds, in that case the unlock
> >will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
> >the entry. And we then need the unlock to clear the entry.
> >_Q_SLOW_VAL
> >Does that explain this? Or should I try again with code?
> 
> OK, I got your proposal now. However, there is still the issue that setting
> the _Q_SLOW_VAL flag and the hash bucket are not atomic wrt each other.

So? They're strictly ordered, that's sufficient. We first hash the lock,
then we set _Q_SLOW_VAL. There's a full memory barrier between them.

> It
> is possible a CPU has set the _Q_SLOW_VAL flag but not yet filling in the
> hash bucket while another one is trying to look for it.

Nope. The unlock side does an xchg() on the locked value first, xchg
also implies a full barrier, so that guarantees that if we observe
_Q_SLOW_VAL we must also observe the hash bucket with the lock value.

> So we need to have
> some kind of synchronization mechanism to let the lookup CPU know when is a
> good time to look up.

No, its all already ordered and working.

pv_wait_head():

	pv_hash()
	/* MB as per cmpxchg */
	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);

VS

__pv_queue_spin_unlock():

	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
		return;

	/* MB as per xchg */
	pv_hash_find(lock);

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 16:28                         ` Waiman Long
  2015-04-02 17:20                             ` Peter Zijlstra
@ 2015-04-02 17:20                           ` Peter Zijlstra
  1 sibling, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-02 17:20 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Thu, Apr 02, 2015 at 12:28:30PM -0400, Waiman Long wrote:
> On 04/01/2015 05:03 PM, Peter Zijlstra wrote:
> >On Wed, Apr 01, 2015 at 03:58:58PM -0400, Waiman Long wrote:
> >>On 04/01/2015 02:48 PM, Peter Zijlstra wrote:
> >>I am sorry that I don't quite get what you mean here. My point is that in
> >>the hashing step, a cpu will need to scan an empty bucket to put the lock
> >>in. In the interim, an previously used bucket before the empty one may get
> >>freed. In the lookup step for that lock, the scanning will stop because of
> >>an empty bucket in front of the target one.
> >Right, that's broken. So we need to do something else to limit the
> >lookup, because without that break, a lookup that needs to iterate the
> >entire array in order to determine -ENOENT, which is expensive.
> >
> >So my alternative proposal is that IFF we can guarantee that every
> >lookup will succeed -- the entry we're looking for is always there, we
> >don't need the break on empty but can probe until we find the entry.
> >This will be bound in cost to the same number if probes we required for
> >insertion and avoids the full array scan.
> >
> >Now I think we can indeed do this, if as said earlier we do not clear
> >the bucket on insert if the cmpxchg succeeds, in that case the unlock
> >will observe _Q_SLOW_VAL and do the lookup, the lookup will then find
> >the entry. And we then need the unlock to clear the entry.
> >_Q_SLOW_VAL
> >Does that explain this? Or should I try again with code?
> 
> OK, I got your proposal now. However, there is still the issue that setting
> the _Q_SLOW_VAL flag and the hash bucket are not atomic wrt each other.

So? They're strictly ordered, that's sufficient. We first hash the lock,
then we set _Q_SLOW_VAL. There's a full memory barrier between them.

> It
> is possible a CPU has set the _Q_SLOW_VAL flag but not yet filling in the
> hash bucket while another one is trying to look for it.

Nope. The unlock side does an xchg() on the locked value first, xchg
also implies a full barrier, so that guarantees that if we observe
_Q_SLOW_VAL we must also observe the hash bucket with the lock value.

> So we need to have
> some kind of synchronization mechanism to let the lookup CPU know when is a
> good time to look up.

No, its all already ordered and working.

pv_wait_head():

	pv_hash()
	/* MB as per cmpxchg */
	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);

VS

__pv_queue_spin_unlock():

	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
		return;

	/* MB as per xchg */
	pv_hash_find(lock);

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 17:20                             ` Peter Zijlstra
  (?)
  (?)
@ 2015-04-02 19:48                             ` Peter Zijlstra
  2015-04-03  3:39                                 ` Waiman Long
                                                 ` (3 more replies)
  -1 siblings, 4 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-02 19:48 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Thu, Apr 02, 2015 at 07:20:57PM +0200, Peter Zijlstra wrote:
> pv_wait_head():
> 
> 	pv_hash()
> 	/* MB as per cmpxchg */
> 	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
> 
> VS
> 
> __pv_queue_spin_unlock():
> 
> 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
> 		return;
> 
> 	/* MB as per xchg */
> 	pv_hash_find(lock);
> 
> 


Something like so.. compile tested only.

I took out the LFSR because that was likely over engineering from my
side :-)

--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -2,6 +2,8 @@
 #error "do not include this file"
 #endif
 
+#include <linux/hash.h>
+
 /*
  * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
  * of spinning them.
@@ -107,7 +109,84 @@ static void pv_kick_node(struct mcs_spin
 		pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+/*
+ * Hash table using open addressing with an linear probe sequence.
+ *
+ * Since we should not be holding locks from NMI context (very rare indeed) the
+ * max load factor is 0.75, which is around the point where open adressing
+ * breaks down.
+ *
+ * Instead of probing just the immediate bucket we probe all buckets in the
+ * same cacheline.
+ *
+ * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
+ *
+ */
+
+struct pv_hash_bucket {
+	struct qspinlock *lock;
+	int cpu;
+};
+
+/*
+ * XXX dynamic allocate using nr_cpu_ids instead...
+ */
+#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
+
+#if PV_LOCK_HASH_BITS < 6
+#undef PV_LOCK_HASH_BITS
+#define PB_LOCK_HASH_BITS	6
+#endif
+
+#define PV_LOCK_HASH_SIZE	(1 << PV_LOCK_HASH_BITS)
+
+static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
+
+#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
+
+static inline u32 hash_align(u32 hash)
+{
+	return hash & ~(PV_HB_PER_LINE - 1);
+}
+
+#define for_each_hash_bucket(hb, off, hash)					\
+	for (hash = hash_align(hash), off = 0, hb = &__pv_lock_hash[hash + off];\
+	    off < PV_LOCK_HASH_SIZE;						\
+	    off++, hb = &__pv_lock_hash[(hash + off) % PV_LOCK_HASH_SIZE])
+
+static struct pv_hash_bucket *pv_hash_insert(struct qspinlock *lock)
+{
+	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb;
+
+	for_each_hash_bucket(hb, offset, hash) {
+		if (!cmpxchg(&hb->lock, NULL, lock)) {
+			WRITE_ONCE(hb->cpu, smp_processor_id());
+			return hb;
+		}
+	}
+
+	/*
+	 * Hard assumes there is an empty bucket somewhere.
+	 */
+	BUG();
+}
+
+static struct pv_hash_bucket *pv_hash_find(struct qspinlock *lock)
+{
+	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb;
+
+	for_each_hash_bucket(hb, offset, hash) {
+		if (READ_ONCE(hb->lock) == lock)
+			return hb;
+	}
+
+	/*
+	 * Hard assumes we _WILL_ find a match.
+	 */
+	BUG();
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
@@ -116,7 +195,9 @@ static DEFINE_PER_CPU(struct qspinlock *
 static void pv_wait_head(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
+	struct pv_hash_bucket *hb = NULL;
 	int loop;
+	u8 o;
 
 	for (;;) {
 		for (loop = SPIN_THRESHOLD; loop; loop--) {
@@ -126,29 +207,47 @@ static void pv_wait_head(struct qspinloc
 			cpu_relax();
 		}
 
-		this_cpu_write(__pv_lock_wait, lock);
-		/*
-		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
-		 *
-		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
-		 *     MB                             MB
-		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
-		 *
-		 * Matches the xchg() in pv_queue_spin_unlock().
-		 */
-		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
-			goto done;
+		if (!hb) {
+			hb = pv_hash_insert(lock);
+			/*
+			 * hb  must be set before setting _Q_SLOW_VAL
+			 *
+			 * [S]   hb <- lock               [RmW] l = l->locked = 0
+			 *       MB                             MB
+			 * [RmW] l->locked ?= _Q_SLOW_VAL [L]   hb
+			 *
+			 * Matches the xchg() in pv_queue_spin_unlock().
+			 */
+			o = cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
+			if (!o) {
+				/*
+				 * The lock got unlocked before we could set
+				 * _Q_SLOW_VAL, we must unhash ourselves.
+				 */
+				WRITE_ONCE(hb->lock, NULL);
+				goto done;
+			}
+			BUG_ON(o != _Q_LOCKED_VAL);
+			/*
+			 * At this point _Q_SLOW_VAL is visible and the unlock
+			 * will do the lookup. The lookup hard relies on the
+			 * entry being visible -- which it should be. Unlock
+			 * will unhash for us.
+			 */
+		}
 
 		pv_wait(&l->locked, _Q_SLOW_VAL);
+		/*
+		 * We can get spurious wakeups from interrupts, cycle back.
+		 */
 	}
 done:
-	this_cpu_write(__pv_lock_wait, NULL);
-
 	/*
 	 * Lock is unlocked now; the caller will acquire it without waiting.
 	 * As with pv_wait_node() we rely on the caller to do a load-acquire
 	 * for us.
 	 */
+	return;
 }
 
 /*
@@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
 void __pv_queue_spin_unlock(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
-	int cpu;
+	struct pv_hash_bucket *hb;
 
 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
 		return;
 
 	/*
 	 * At this point the memory pointed at by lock can be freed/reused,
-	 * however we can still use the pointer value to search in our cpu
-	 * array.
+	 * however we can still use the pointer value to search in our hash
+	 * table.
 	 *
-	 * XXX: get rid of this loop
+	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
+	 * bucket. See pv_wait_head().
 	 */
-	for_each_possible_cpu(cpu) {
-		if (per_cpu(__pv_lock_wait, cpu) == lock)
-			pv_kick(cpu);
-	}
+	hb = pv_hash_find(lock);
+	pv_kick(hb->cpu);
+	WRITE_ONCE(hb->lock, NULL); /* unhash */
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 17:20                             ` Peter Zijlstra
  (?)
@ 2015-04-02 19:48                             ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-02 19:48 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Thu, Apr 02, 2015 at 07:20:57PM +0200, Peter Zijlstra wrote:
> pv_wait_head():
> 
> 	pv_hash()
> 	/* MB as per cmpxchg */
> 	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
> 
> VS
> 
> __pv_queue_spin_unlock():
> 
> 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
> 		return;
> 
> 	/* MB as per xchg */
> 	pv_hash_find(lock);
> 
> 


Something like so.. compile tested only.

I took out the LFSR because that was likely over engineering from my
side :-)

--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -2,6 +2,8 @@
 #error "do not include this file"
 #endif
 
+#include <linux/hash.h>
+
 /*
  * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
  * of spinning them.
@@ -107,7 +109,84 @@ static void pv_kick_node(struct mcs_spin
 		pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+/*
+ * Hash table using open addressing with an linear probe sequence.
+ *
+ * Since we should not be holding locks from NMI context (very rare indeed) the
+ * max load factor is 0.75, which is around the point where open adressing
+ * breaks down.
+ *
+ * Instead of probing just the immediate bucket we probe all buckets in the
+ * same cacheline.
+ *
+ * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
+ *
+ */
+
+struct pv_hash_bucket {
+	struct qspinlock *lock;
+	int cpu;
+};
+
+/*
+ * XXX dynamic allocate using nr_cpu_ids instead...
+ */
+#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
+
+#if PV_LOCK_HASH_BITS < 6
+#undef PV_LOCK_HASH_BITS
+#define PB_LOCK_HASH_BITS	6
+#endif
+
+#define PV_LOCK_HASH_SIZE	(1 << PV_LOCK_HASH_BITS)
+
+static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
+
+#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
+
+static inline u32 hash_align(u32 hash)
+{
+	return hash & ~(PV_HB_PER_LINE - 1);
+}
+
+#define for_each_hash_bucket(hb, off, hash)					\
+	for (hash = hash_align(hash), off = 0, hb = &__pv_lock_hash[hash + off];\
+	    off < PV_LOCK_HASH_SIZE;						\
+	    off++, hb = &__pv_lock_hash[(hash + off) % PV_LOCK_HASH_SIZE])
+
+static struct pv_hash_bucket *pv_hash_insert(struct qspinlock *lock)
+{
+	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb;
+
+	for_each_hash_bucket(hb, offset, hash) {
+		if (!cmpxchg(&hb->lock, NULL, lock)) {
+			WRITE_ONCE(hb->cpu, smp_processor_id());
+			return hb;
+		}
+	}
+
+	/*
+	 * Hard assumes there is an empty bucket somewhere.
+	 */
+	BUG();
+}
+
+static struct pv_hash_bucket *pv_hash_find(struct qspinlock *lock)
+{
+	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb;
+
+	for_each_hash_bucket(hb, offset, hash) {
+		if (READ_ONCE(hb->lock) == lock)
+			return hb;
+	}
+
+	/*
+	 * Hard assumes we _WILL_ find a match.
+	 */
+	BUG();
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
@@ -116,7 +195,9 @@ static DEFINE_PER_CPU(struct qspinlock *
 static void pv_wait_head(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
+	struct pv_hash_bucket *hb = NULL;
 	int loop;
+	u8 o;
 
 	for (;;) {
 		for (loop = SPIN_THRESHOLD; loop; loop--) {
@@ -126,29 +207,47 @@ static void pv_wait_head(struct qspinloc
 			cpu_relax();
 		}
 
-		this_cpu_write(__pv_lock_wait, lock);
-		/*
-		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
-		 *
-		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
-		 *     MB                             MB
-		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
-		 *
-		 * Matches the xchg() in pv_queue_spin_unlock().
-		 */
-		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
-			goto done;
+		if (!hb) {
+			hb = pv_hash_insert(lock);
+			/*
+			 * hb  must be set before setting _Q_SLOW_VAL
+			 *
+			 * [S]   hb <- lock               [RmW] l = l->locked = 0
+			 *       MB                             MB
+			 * [RmW] l->locked ?= _Q_SLOW_VAL [L]   hb
+			 *
+			 * Matches the xchg() in pv_queue_spin_unlock().
+			 */
+			o = cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
+			if (!o) {
+				/*
+				 * The lock got unlocked before we could set
+				 * _Q_SLOW_VAL, we must unhash ourselves.
+				 */
+				WRITE_ONCE(hb->lock, NULL);
+				goto done;
+			}
+			BUG_ON(o != _Q_LOCKED_VAL);
+			/*
+			 * At this point _Q_SLOW_VAL is visible and the unlock
+			 * will do the lookup. The lookup hard relies on the
+			 * entry being visible -- which it should be. Unlock
+			 * will unhash for us.
+			 */
+		}
 
 		pv_wait(&l->locked, _Q_SLOW_VAL);
+		/*
+		 * We can get spurious wakeups from interrupts, cycle back.
+		 */
 	}
 done:
-	this_cpu_write(__pv_lock_wait, NULL);
-
 	/*
 	 * Lock is unlocked now; the caller will acquire it without waiting.
 	 * As with pv_wait_node() we rely on the caller to do a load-acquire
 	 * for us.
 	 */
+	return;
 }
 
 /*
@@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
 void __pv_queue_spin_unlock(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
-	int cpu;
+	struct pv_hash_bucket *hb;
 
 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
 		return;
 
 	/*
 	 * At this point the memory pointed at by lock can be freed/reused,
-	 * however we can still use the pointer value to search in our cpu
-	 * array.
+	 * however we can still use the pointer value to search in our hash
+	 * table.
 	 *
-	 * XXX: get rid of this loop
+	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
+	 * bucket. See pv_wait_head().
 	 */
-	for_each_possible_cpu(cpu) {
-		if (per_cpu(__pv_lock_wait, cpu) == lock)
-			pv_kick(cpu);
-	}
+	hb = pv_hash_find(lock);
+	pv_kick(hb->cpu);
+	WRITE_ONCE(hb->lock, NULL); /* unhash */
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 17:20                             ` Peter Zijlstra
                                               ` (2 preceding siblings ...)
  (?)
@ 2015-04-02 19:48                             ` Peter Zijlstra
  -1 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-02 19:48 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Thu, Apr 02, 2015 at 07:20:57PM +0200, Peter Zijlstra wrote:
> pv_wait_head():
> 
> 	pv_hash()
> 	/* MB as per cmpxchg */
> 	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
> 
> VS
> 
> __pv_queue_spin_unlock():
> 
> 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
> 		return;
> 
> 	/* MB as per xchg */
> 	pv_hash_find(lock);
> 
> 


Something like so.. compile tested only.

I took out the LFSR because that was likely over engineering from my
side :-)

--- a/kernel/locking/qspinlock_paravirt.h
+++ b/kernel/locking/qspinlock_paravirt.h
@@ -2,6 +2,8 @@
 #error "do not include this file"
 #endif
 
+#include <linux/hash.h>
+
 /*
  * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
  * of spinning them.
@@ -107,7 +109,84 @@ static void pv_kick_node(struct mcs_spin
 		pv_kick(pn->cpu);
 }
 
-static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
+/*
+ * Hash table using open addressing with an linear probe sequence.
+ *
+ * Since we should not be holding locks from NMI context (very rare indeed) the
+ * max load factor is 0.75, which is around the point where open adressing
+ * breaks down.
+ *
+ * Instead of probing just the immediate bucket we probe all buckets in the
+ * same cacheline.
+ *
+ * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
+ *
+ */
+
+struct pv_hash_bucket {
+	struct qspinlock *lock;
+	int cpu;
+};
+
+/*
+ * XXX dynamic allocate using nr_cpu_ids instead...
+ */
+#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
+
+#if PV_LOCK_HASH_BITS < 6
+#undef PV_LOCK_HASH_BITS
+#define PB_LOCK_HASH_BITS	6
+#endif
+
+#define PV_LOCK_HASH_SIZE	(1 << PV_LOCK_HASH_BITS)
+
+static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
+
+#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
+
+static inline u32 hash_align(u32 hash)
+{
+	return hash & ~(PV_HB_PER_LINE - 1);
+}
+
+#define for_each_hash_bucket(hb, off, hash)					\
+	for (hash = hash_align(hash), off = 0, hb = &__pv_lock_hash[hash + off];\
+	    off < PV_LOCK_HASH_SIZE;						\
+	    off++, hb = &__pv_lock_hash[(hash + off) % PV_LOCK_HASH_SIZE])
+
+static struct pv_hash_bucket *pv_hash_insert(struct qspinlock *lock)
+{
+	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb;
+
+	for_each_hash_bucket(hb, offset, hash) {
+		if (!cmpxchg(&hb->lock, NULL, lock)) {
+			WRITE_ONCE(hb->cpu, smp_processor_id());
+			return hb;
+		}
+	}
+
+	/*
+	 * Hard assumes there is an empty bucket somewhere.
+	 */
+	BUG();
+}
+
+static struct pv_hash_bucket *pv_hash_find(struct qspinlock *lock)
+{
+	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
+	struct pv_hash_bucket *hb;
+
+	for_each_hash_bucket(hb, offset, hash) {
+		if (READ_ONCE(hb->lock) == lock)
+			return hb;
+	}
+
+	/*
+	 * Hard assumes we _WILL_ find a match.
+	 */
+	BUG();
+}
 
 /*
  * Wait for l->locked to become clear; halt the vcpu after a short spin.
@@ -116,7 +195,9 @@ static DEFINE_PER_CPU(struct qspinlock *
 static void pv_wait_head(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
+	struct pv_hash_bucket *hb = NULL;
 	int loop;
+	u8 o;
 
 	for (;;) {
 		for (loop = SPIN_THRESHOLD; loop; loop--) {
@@ -126,29 +207,47 @@ static void pv_wait_head(struct qspinloc
 			cpu_relax();
 		}
 
-		this_cpu_write(__pv_lock_wait, lock);
-		/*
-		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
-		 *
-		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
-		 *     MB                             MB
-		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
-		 *
-		 * Matches the xchg() in pv_queue_spin_unlock().
-		 */
-		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
-			goto done;
+		if (!hb) {
+			hb = pv_hash_insert(lock);
+			/*
+			 * hb  must be set before setting _Q_SLOW_VAL
+			 *
+			 * [S]   hb <- lock               [RmW] l = l->locked = 0
+			 *       MB                             MB
+			 * [RmW] l->locked ?= _Q_SLOW_VAL [L]   hb
+			 *
+			 * Matches the xchg() in pv_queue_spin_unlock().
+			 */
+			o = cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
+			if (!o) {
+				/*
+				 * The lock got unlocked before we could set
+				 * _Q_SLOW_VAL, we must unhash ourselves.
+				 */
+				WRITE_ONCE(hb->lock, NULL);
+				goto done;
+			}
+			BUG_ON(o != _Q_LOCKED_VAL);
+			/*
+			 * At this point _Q_SLOW_VAL is visible and the unlock
+			 * will do the lookup. The lookup hard relies on the
+			 * entry being visible -- which it should be. Unlock
+			 * will unhash for us.
+			 */
+		}
 
 		pv_wait(&l->locked, _Q_SLOW_VAL);
+		/*
+		 * We can get spurious wakeups from interrupts, cycle back.
+		 */
 	}
 done:
-	this_cpu_write(__pv_lock_wait, NULL);
-
 	/*
 	 * Lock is unlocked now; the caller will acquire it without waiting.
 	 * As with pv_wait_node() we rely on the caller to do a load-acquire
 	 * for us.
 	 */
+	return;
 }
 
 /*
@@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
 void __pv_queue_spin_unlock(struct qspinlock *lock)
 {
 	struct __qspinlock *l = (void *)lock;
-	int cpu;
+	struct pv_hash_bucket *hb;
 
 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
 		return;
 
 	/*
 	 * At this point the memory pointed at by lock can be freed/reused,
-	 * however we can still use the pointer value to search in our cpu
-	 * array.
+	 * however we can still use the pointer value to search in our hash
+	 * table.
 	 *
-	 * XXX: get rid of this loop
+	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
+	 * bucket. See pv_wait_head().
 	 */
-	for_each_possible_cpu(cpu) {
-		if (per_cpu(__pv_lock_wait, cpu) == lock)
-			pv_kick(cpu);
-	}
+	hb = pv_hash_find(lock);
+	pv_kick(hb->cpu);
+	WRITE_ONCE(hb->lock, NULL); /* unhash */
 }

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 19:48                             ` Peter Zijlstra
@ 2015-04-03  3:39                                 ` Waiman Long
  2015-04-03  3:39                               ` Waiman Long
                                                   ` (2 subsequent siblings)
  3 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-03  3:39 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On 04/02/2015 03:48 PM, Peter Zijlstra wrote:
> On Thu, Apr 02, 2015 at 07:20:57PM +0200, Peter Zijlstra wrote:
>> pv_wait_head():
>>
>> 	pv_hash()
>> 	/* MB as per cmpxchg */
>> 	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
>>
>> VS
>>
>> __pv_queue_spin_unlock():
>>
>> 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>> 		return;
>>
>> 	/* MB as per xchg */
>> 	pv_hash_find(lock);
>>
>>
>
> Something like so.. compile tested only.
>
> I took out the LFSR because that was likely over engineering from my
> side :-)
>
> --- a/kernel/locking/qspinlock_paravirt.h
> +++ b/kernel/locking/qspinlock_paravirt.h
> @@ -2,6 +2,8 @@
>   #error "do not include this file"
>   #endif
>
> +#include<linux/hash.h>
> +
>   /*
>    * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
>    * of spinning them.
> @@ -107,7 +109,84 @@ static void pv_kick_node(struct mcs_spin
>   		pv_kick(pn->cpu);
>   }
>
> -static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
> +/*
> + * Hash table using open addressing with an linear probe sequence.
> + *
> + * Since we should not be holding locks from NMI context (very rare indeed) the
> + * max load factor is 0.75, which is around the point where open adressing
> + * breaks down.
> + *
> + * Instead of probing just the immediate bucket we probe all buckets in the
> + * same cacheline.
> + *
> + * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
> + *
> + */
> +
> +struct pv_hash_bucket {
> +	struct qspinlock *lock;
> +	int cpu;
> +};
> +
> +/*
> + * XXX dynamic allocate using nr_cpu_ids instead...
> + */
> +#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
> +
> +#if PV_LOCK_HASH_BITS<  6
> +#undef PV_LOCK_HASH_BITS
> +#define PB_LOCK_HASH_BITS	6
> +#endif
> +
> +#define PV_LOCK_HASH_SIZE	(1<<  PV_LOCK_HASH_BITS)
> +
> +static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
> +
> +#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
> +
> +static inline u32 hash_align(u32 hash)
> +{
> +	return hash&  ~(PV_HB_PER_LINE - 1);
> +}
> +
> +#define for_each_hash_bucket(hb, off, hash)					\
> +	for (hash = hash_align(hash), off = 0, hb =&__pv_lock_hash[hash + off];\
> +	    off<  PV_LOCK_HASH_SIZE;						\
> +	    off++, hb =&__pv_lock_hash[(hash + off) % PV_LOCK_HASH_SIZE])
> +
> +static struct pv_hash_bucket *pv_hash_insert(struct qspinlock *lock)
> +{
> +	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb;
> +
> +	for_each_hash_bucket(hb, offset, hash) {
> +		if (!cmpxchg(&hb->lock, NULL, lock)) {
> +			WRITE_ONCE(hb->cpu, smp_processor_id());
> +			return hb;
> +		}
> +	}
> +
> +	/*
> +	 * Hard assumes there is an empty bucket somewhere.
> +	 */
> +	BUG();
> +}
> +
> +static struct pv_hash_bucket *pv_hash_find(struct qspinlock *lock)
> +{
> +	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb;
> +
> +	for_each_hash_bucket(hb, offset, hash) {
> +		if (READ_ONCE(hb->lock) == lock)
> +			return hb;
> +	}
> +
> +	/*
> +	 * Hard assumes we _WILL_ find a match.
> +	 */
> +	BUG();
> +}
>
>   /*
>    * Wait for l->locked to become clear; halt the vcpu after a short spin.
> @@ -116,7 +195,9 @@ static DEFINE_PER_CPU(struct qspinlock *
>   static void pv_wait_head(struct qspinlock *lock)
>   {
>   	struct __qspinlock *l = (void *)lock;
> +	struct pv_hash_bucket *hb = NULL;
>   	int loop;
> +	u8 o;
>
>   	for (;;) {
>   		for (loop = SPIN_THRESHOLD; loop; loop--) {
> @@ -126,29 +207,47 @@ static void pv_wait_head(struct qspinloc
>   			cpu_relax();
>   		}
>
> -		this_cpu_write(__pv_lock_wait, lock);
> -		/*
> -		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
> -		 *
> -		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
> -		 *     MB                             MB
> -		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
> -		 *
> -		 * Matches the xchg() in pv_queue_spin_unlock().
> -		 */
> -		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
> -			goto done;
> +		if (!hb) {
> +			hb = pv_hash_insert(lock);
> +			/*
> +			 * hb  must be set before setting _Q_SLOW_VAL
> +			 *
> +			 * [S]   hb<- lock               [RmW] l = l->locked = 0
> +			 *       MB                             MB
> +			 * [RmW] l->locked ?= _Q_SLOW_VAL [L]   hb
> +			 *
> +			 * Matches the xchg() in pv_queue_spin_unlock().
> +			 */
> +			o = cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
> +			if (!o) {
> +				/*
> +				 * The lock got unlocked before we could set
> +				 * _Q_SLOW_VAL, we must unhash ourselves.
> +				 */
> +				WRITE_ONCE(hb->lock, NULL);
> +				goto done;
> +			}
> +			BUG_ON(o != _Q_LOCKED_VAL);
> +			/*
> +			 * At this point _Q_SLOW_VAL is visible and the unlock
> +			 * will do the lookup. The lookup hard relies on the
> +			 * entry being visible -- which it should be. Unlock
> +			 * will unhash for us.
> +			 */
> +		}
>
>   		pv_wait(&l->locked, _Q_SLOW_VAL);
> +		/*
> +		 * We can get spurious wakeups from interrupts, cycle back.
> +		 */
>   	}
>   done:
> -	this_cpu_write(__pv_lock_wait, NULL);
> -
>   	/*
>   	 * Lock is unlocked now; the caller will acquire it without waiting.
>   	 * As with pv_wait_node() we rely on the caller to do a load-acquire
>   	 * for us.
>   	 */
> +	return;
>   }
>
>   /*
> @@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
>   void __pv_queue_spin_unlock(struct qspinlock *lock)
>   {
>   	struct __qspinlock *l = (void *)lock;
> -	int cpu;
> +	struct pv_hash_bucket *hb;
>
>   	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>   		return;
>
>   	/*
>   	 * At this point the memory pointed at by lock can be freed/reused,
> -	 * however we can still use the pointer value to search in our cpu
> -	 * array.
> +	 * however we can still use the pointer value to search in our hash
> +	 * table.
>   	 *
> -	 * XXX: get rid of this loop
> +	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
> +	 * bucket. See pv_wait_head().
>   	 */
> -	for_each_possible_cpu(cpu) {
> -		if (per_cpu(__pv_lock_wait, cpu) == lock)
> -			pv_kick(cpu);
> -	}
> +	hb = pv_hash_find(lock);
> +	pv_kick(hb->cpu);
> +	WRITE_ONCE(hb->lock, NULL); /* unhash */
>   }

Thanks for the code. I am working on my own version, too. Will need to 
run some tests to verify the correctness of the code. Hopefully I have 
something for you to review early next week.

Cheers,
Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-04-03  3:39                                 ` Waiman Long
  0 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-03  3:39 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On 04/02/2015 03:48 PM, Peter Zijlstra wrote:
> On Thu, Apr 02, 2015 at 07:20:57PM +0200, Peter Zijlstra wrote:
>> pv_wait_head():
>>
>> 	pv_hash()
>> 	/* MB as per cmpxchg */
>> 	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
>>
>> VS
>>
>> __pv_queue_spin_unlock():
>>
>> 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>> 		return;
>>
>> 	/* MB as per xchg */
>> 	pv_hash_find(lock);
>>
>>
>
> Something like so.. compile tested only.
>
> I took out the LFSR because that was likely over engineering from my
> side :-)
>
> --- a/kernel/locking/qspinlock_paravirt.h
> +++ b/kernel/locking/qspinlock_paravirt.h
> @@ -2,6 +2,8 @@
>   #error "do not include this file"
>   #endif
>
> +#include<linux/hash.h>
> +
>   /*
>    * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
>    * of spinning them.
> @@ -107,7 +109,84 @@ static void pv_kick_node(struct mcs_spin
>   		pv_kick(pn->cpu);
>   }
>
> -static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
> +/*
> + * Hash table using open addressing with an linear probe sequence.
> + *
> + * Since we should not be holding locks from NMI context (very rare indeed) the
> + * max load factor is 0.75, which is around the point where open adressing
> + * breaks down.
> + *
> + * Instead of probing just the immediate bucket we probe all buckets in the
> + * same cacheline.
> + *
> + * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
> + *
> + */
> +
> +struct pv_hash_bucket {
> +	struct qspinlock *lock;
> +	int cpu;
> +};
> +
> +/*
> + * XXX dynamic allocate using nr_cpu_ids instead...
> + */
> +#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
> +
> +#if PV_LOCK_HASH_BITS<  6
> +#undef PV_LOCK_HASH_BITS
> +#define PB_LOCK_HASH_BITS	6
> +#endif
> +
> +#define PV_LOCK_HASH_SIZE	(1<<  PV_LOCK_HASH_BITS)
> +
> +static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
> +
> +#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
> +
> +static inline u32 hash_align(u32 hash)
> +{
> +	return hash&  ~(PV_HB_PER_LINE - 1);
> +}
> +
> +#define for_each_hash_bucket(hb, off, hash)					\
> +	for (hash = hash_align(hash), off = 0, hb =&__pv_lock_hash[hash + off];\
> +	    off<  PV_LOCK_HASH_SIZE;						\
> +	    off++, hb =&__pv_lock_hash[(hash + off) % PV_LOCK_HASH_SIZE])
> +
> +static struct pv_hash_bucket *pv_hash_insert(struct qspinlock *lock)
> +{
> +	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb;
> +
> +	for_each_hash_bucket(hb, offset, hash) {
> +		if (!cmpxchg(&hb->lock, NULL, lock)) {
> +			WRITE_ONCE(hb->cpu, smp_processor_id());
> +			return hb;
> +		}
> +	}
> +
> +	/*
> +	 * Hard assumes there is an empty bucket somewhere.
> +	 */
> +	BUG();
> +}
> +
> +static struct pv_hash_bucket *pv_hash_find(struct qspinlock *lock)
> +{
> +	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb;
> +
> +	for_each_hash_bucket(hb, offset, hash) {
> +		if (READ_ONCE(hb->lock) == lock)
> +			return hb;
> +	}
> +
> +	/*
> +	 * Hard assumes we _WILL_ find a match.
> +	 */
> +	BUG();
> +}
>
>   /*
>    * Wait for l->locked to become clear; halt the vcpu after a short spin.
> @@ -116,7 +195,9 @@ static DEFINE_PER_CPU(struct qspinlock *
>   static void pv_wait_head(struct qspinlock *lock)
>   {
>   	struct __qspinlock *l = (void *)lock;
> +	struct pv_hash_bucket *hb = NULL;
>   	int loop;
> +	u8 o;
>
>   	for (;;) {
>   		for (loop = SPIN_THRESHOLD; loop; loop--) {
> @@ -126,29 +207,47 @@ static void pv_wait_head(struct qspinloc
>   			cpu_relax();
>   		}
>
> -		this_cpu_write(__pv_lock_wait, lock);
> -		/*
> -		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
> -		 *
> -		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
> -		 *     MB                             MB
> -		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
> -		 *
> -		 * Matches the xchg() in pv_queue_spin_unlock().
> -		 */
> -		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
> -			goto done;
> +		if (!hb) {
> +			hb = pv_hash_insert(lock);
> +			/*
> +			 * hb  must be set before setting _Q_SLOW_VAL
> +			 *
> +			 * [S]   hb<- lock               [RmW] l = l->locked = 0
> +			 *       MB                             MB
> +			 * [RmW] l->locked ?= _Q_SLOW_VAL [L]   hb
> +			 *
> +			 * Matches the xchg() in pv_queue_spin_unlock().
> +			 */
> +			o = cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
> +			if (!o) {
> +				/*
> +				 * The lock got unlocked before we could set
> +				 * _Q_SLOW_VAL, we must unhash ourselves.
> +				 */
> +				WRITE_ONCE(hb->lock, NULL);
> +				goto done;
> +			}
> +			BUG_ON(o != _Q_LOCKED_VAL);
> +			/*
> +			 * At this point _Q_SLOW_VAL is visible and the unlock
> +			 * will do the lookup. The lookup hard relies on the
> +			 * entry being visible -- which it should be. Unlock
> +			 * will unhash for us.
> +			 */
> +		}
>
>   		pv_wait(&l->locked, _Q_SLOW_VAL);
> +		/*
> +		 * We can get spurious wakeups from interrupts, cycle back.
> +		 */
>   	}
>   done:
> -	this_cpu_write(__pv_lock_wait, NULL);
> -
>   	/*
>   	 * Lock is unlocked now; the caller will acquire it without waiting.
>   	 * As with pv_wait_node() we rely on the caller to do a load-acquire
>   	 * for us.
>   	 */
> +	return;
>   }
>
>   /*
> @@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
>   void __pv_queue_spin_unlock(struct qspinlock *lock)
>   {
>   	struct __qspinlock *l = (void *)lock;
> -	int cpu;
> +	struct pv_hash_bucket *hb;
>
>   	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>   		return;
>
>   	/*
>   	 * At this point the memory pointed at by lock can be freed/reused,
> -	 * however we can still use the pointer value to search in our cpu
> -	 * array.
> +	 * however we can still use the pointer value to search in our hash
> +	 * table.
>   	 *
> -	 * XXX: get rid of this loop
> +	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
> +	 * bucket. See pv_wait_head().
>   	 */
> -	for_each_possible_cpu(cpu) {
> -		if (per_cpu(__pv_lock_wait, cpu) == lock)
> -			pv_kick(cpu);
> -	}
> +	hb = pv_hash_find(lock);
> +	pv_kick(hb->cpu);
> +	WRITE_ONCE(hb->lock, NULL); /* unhash */
>   }

Thanks for the code. I am working on my own version, too. Will need to 
run some tests to verify the correctness of the code. Hopefully I have 
something for you to review early next week.

Cheers,
Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 19:48                             ` Peter Zijlstra
  2015-04-03  3:39                                 ` Waiman Long
@ 2015-04-03  3:39                               ` Waiman Long
  2015-04-03 13:43                               ` Peter Zijlstra
  2015-04-03 13:43                                 ` Peter Zijlstra
  3 siblings, 0 replies; 136+ messages in thread
From: Waiman Long @ 2015-04-03  3:39 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On 04/02/2015 03:48 PM, Peter Zijlstra wrote:
> On Thu, Apr 02, 2015 at 07:20:57PM +0200, Peter Zijlstra wrote:
>> pv_wait_head():
>>
>> 	pv_hash()
>> 	/* MB as per cmpxchg */
>> 	cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
>>
>> VS
>>
>> __pv_queue_spin_unlock():
>>
>> 	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>> 		return;
>>
>> 	/* MB as per xchg */
>> 	pv_hash_find(lock);
>>
>>
>
> Something like so.. compile tested only.
>
> I took out the LFSR because that was likely over engineering from my
> side :-)
>
> --- a/kernel/locking/qspinlock_paravirt.h
> +++ b/kernel/locking/qspinlock_paravirt.h
> @@ -2,6 +2,8 @@
>   #error "do not include this file"
>   #endif
>
> +#include<linux/hash.h>
> +
>   /*
>    * Implement paravirt qspinlocks; the general idea is to halt the vcpus instead
>    * of spinning them.
> @@ -107,7 +109,84 @@ static void pv_kick_node(struct mcs_spin
>   		pv_kick(pn->cpu);
>   }
>
> -static DEFINE_PER_CPU(struct qspinlock *, __pv_lock_wait);
> +/*
> + * Hash table using open addressing with an linear probe sequence.
> + *
> + * Since we should not be holding locks from NMI context (very rare indeed) the
> + * max load factor is 0.75, which is around the point where open adressing
> + * breaks down.
> + *
> + * Instead of probing just the immediate bucket we probe all buckets in the
> + * same cacheline.
> + *
> + * http://en.wikipedia.org/wiki/Hash_table#Open_addressing
> + *
> + */
> +
> +struct pv_hash_bucket {
> +	struct qspinlock *lock;
> +	int cpu;
> +};
> +
> +/*
> + * XXX dynamic allocate using nr_cpu_ids instead...
> + */
> +#define PV_LOCK_HASH_BITS	(2 + NR_CPUS_BITS)
> +
> +#if PV_LOCK_HASH_BITS<  6
> +#undef PV_LOCK_HASH_BITS
> +#define PB_LOCK_HASH_BITS	6
> +#endif
> +
> +#define PV_LOCK_HASH_SIZE	(1<<  PV_LOCK_HASH_BITS)
> +
> +static struct pv_hash_bucket __pv_lock_hash[PV_LOCK_HASH_SIZE] ____cacheline_aligned;
> +
> +#define PV_HB_PER_LINE		(SMP_CACHE_BYTES / sizeof(struct pv_hash_bucket))
> +
> +static inline u32 hash_align(u32 hash)
> +{
> +	return hash&  ~(PV_HB_PER_LINE - 1);
> +}
> +
> +#define for_each_hash_bucket(hb, off, hash)					\
> +	for (hash = hash_align(hash), off = 0, hb =&__pv_lock_hash[hash + off];\
> +	    off<  PV_LOCK_HASH_SIZE;						\
> +	    off++, hb =&__pv_lock_hash[(hash + off) % PV_LOCK_HASH_SIZE])
> +
> +static struct pv_hash_bucket *pv_hash_insert(struct qspinlock *lock)
> +{
> +	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb;
> +
> +	for_each_hash_bucket(hb, offset, hash) {
> +		if (!cmpxchg(&hb->lock, NULL, lock)) {
> +			WRITE_ONCE(hb->cpu, smp_processor_id());
> +			return hb;
> +		}
> +	}
> +
> +	/*
> +	 * Hard assumes there is an empty bucket somewhere.
> +	 */
> +	BUG();
> +}
> +
> +static struct pv_hash_bucket *pv_hash_find(struct qspinlock *lock)
> +{
> +	u32 offset, hash = hash_ptr(lock, PV_LOCK_HASH_BITS);
> +	struct pv_hash_bucket *hb;
> +
> +	for_each_hash_bucket(hb, offset, hash) {
> +		if (READ_ONCE(hb->lock) == lock)
> +			return hb;
> +	}
> +
> +	/*
> +	 * Hard assumes we _WILL_ find a match.
> +	 */
> +	BUG();
> +}
>
>   /*
>    * Wait for l->locked to become clear; halt the vcpu after a short spin.
> @@ -116,7 +195,9 @@ static DEFINE_PER_CPU(struct qspinlock *
>   static void pv_wait_head(struct qspinlock *lock)
>   {
>   	struct __qspinlock *l = (void *)lock;
> +	struct pv_hash_bucket *hb = NULL;
>   	int loop;
> +	u8 o;
>
>   	for (;;) {
>   		for (loop = SPIN_THRESHOLD; loop; loop--) {
> @@ -126,29 +207,47 @@ static void pv_wait_head(struct qspinloc
>   			cpu_relax();
>   		}
>
> -		this_cpu_write(__pv_lock_wait, lock);
> -		/*
> -		 * __pv_lock_wait must be set before setting _Q_SLOW_VAL
> -		 *
> -		 * [S] __pv_lock_wait = lock    [RmW] l = l->locked = 0
> -		 *     MB                             MB
> -		 * [S] l->locked = _Q_SLOW_VAL  [L]   __pv_lock_wait
> -		 *
> -		 * Matches the xchg() in pv_queue_spin_unlock().
> -		 */
> -		if (!cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL))
> -			goto done;
> +		if (!hb) {
> +			hb = pv_hash_insert(lock);
> +			/*
> +			 * hb  must be set before setting _Q_SLOW_VAL
> +			 *
> +			 * [S]   hb<- lock               [RmW] l = l->locked = 0
> +			 *       MB                             MB
> +			 * [RmW] l->locked ?= _Q_SLOW_VAL [L]   hb
> +			 *
> +			 * Matches the xchg() in pv_queue_spin_unlock().
> +			 */
> +			o = cmpxchg(&l->locked, _Q_LOCKED_VAL, _Q_SLOW_VAL);
> +			if (!o) {
> +				/*
> +				 * The lock got unlocked before we could set
> +				 * _Q_SLOW_VAL, we must unhash ourselves.
> +				 */
> +				WRITE_ONCE(hb->lock, NULL);
> +				goto done;
> +			}
> +			BUG_ON(o != _Q_LOCKED_VAL);
> +			/*
> +			 * At this point _Q_SLOW_VAL is visible and the unlock
> +			 * will do the lookup. The lookup hard relies on the
> +			 * entry being visible -- which it should be. Unlock
> +			 * will unhash for us.
> +			 */
> +		}
>
>   		pv_wait(&l->locked, _Q_SLOW_VAL);
> +		/*
> +		 * We can get spurious wakeups from interrupts, cycle back.
> +		 */
>   	}
>   done:
> -	this_cpu_write(__pv_lock_wait, NULL);
> -
>   	/*
>   	 * Lock is unlocked now; the caller will acquire it without waiting.
>   	 * As with pv_wait_node() we rely on the caller to do a load-acquire
>   	 * for us.
>   	 */
> +	return;
>   }
>
>   /*
> @@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
>   void __pv_queue_spin_unlock(struct qspinlock *lock)
>   {
>   	struct __qspinlock *l = (void *)lock;
> -	int cpu;
> +	struct pv_hash_bucket *hb;
>
>   	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>   		return;
>
>   	/*
>   	 * At this point the memory pointed at by lock can be freed/reused,
> -	 * however we can still use the pointer value to search in our cpu
> -	 * array.
> +	 * however we can still use the pointer value to search in our hash
> +	 * table.
>   	 *
> -	 * XXX: get rid of this loop
> +	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
> +	 * bucket. See pv_wait_head().
>   	 */
> -	for_each_possible_cpu(cpu) {
> -		if (per_cpu(__pv_lock_wait, cpu) == lock)
> -			pv_kick(cpu);
> -	}
> +	hb = pv_hash_find(lock);
> +	pv_kick(hb->cpu);
> +	WRITE_ONCE(hb->lock, NULL); /* unhash */
>   }

Thanks for the code. I am working on my own version, too. Will need to 
run some tests to verify the correctness of the code. Hopefully I have 
something for you to review early next week.

Cheers,
Longman

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 19:48                             ` Peter Zijlstra
@ 2015-04-03 13:43                                 ` Peter Zijlstra
  2015-04-03  3:39                               ` Waiman Long
                                                   ` (2 subsequent siblings)
  3 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-03 13:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: tglx, mingo, hpa, paolo.bonzini, konrad.wilk, boris.ostrovsky,
	paulmck, riel, torvalds, raghavendra.kt, david.vrabel, oleg,
	scott.norton, doug.hatch, linux-arch, x86, linux-kernel,
	virtualization, xen-devel, kvm, luto

On Thu, Apr 02, 2015 at 09:48:34PM +0200, Peter Zijlstra wrote:
> @@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
>  void __pv_queue_spin_unlock(struct qspinlock *lock)
>  {
>  	struct __qspinlock *l = (void *)lock;
> +	struct pv_hash_bucket *hb;
>  
>  	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>  		return;
>  
>  	/*
>  	 * At this point the memory pointed at by lock can be freed/reused,
> +	 * however we can still use the pointer value to search in our hash
> +	 * table.
>  	 *
> +	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
> +	 * bucket. See pv_wait_head().
>  	 */
> +	hb = pv_hash_find(lock);
> +	pv_kick(hb->cpu);
> +	WRITE_ONCE(hb->lock, NULL); /* unhash */
>  }

So I _think_ I found a problem with this approach :/

If, as per the above, the lock does indeed get freed, it can get
re-allocated and stuck in the hash table (again) before we get the
lookup and unhash it.

I'll have to ponder that a bit more.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
@ 2015-04-03 13:43                                 ` Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-03 13:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, konrad.wilk, scott.norton,
	raghavendra.kt, paolo.bonzini, oleg, linux-kernel, mingo,
	david.vrabel, hpa, luto, xen-devel, tglx, paulmck, torvalds,
	boris.ostrovsky, virtualization, doug.hatch

On Thu, Apr 02, 2015 at 09:48:34PM +0200, Peter Zijlstra wrote:
> @@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
>  void __pv_queue_spin_unlock(struct qspinlock *lock)
>  {
>  	struct __qspinlock *l = (void *)lock;
> +	struct pv_hash_bucket *hb;
>  
>  	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>  		return;
>  
>  	/*
>  	 * At this point the memory pointed at by lock can be freed/reused,
> +	 * however we can still use the pointer value to search in our hash
> +	 * table.
>  	 *
> +	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
> +	 * bucket. See pv_wait_head().
>  	 */
> +	hb = pv_hash_find(lock);
> +	pv_kick(hb->cpu);
> +	WRITE_ONCE(hb->lock, NULL); /* unhash */
>  }

So I _think_ I found a problem with this approach :/

If, as per the above, the lock does indeed get freed, it can get
re-allocated and stuck in the hash table (again) before we get the
lookup and unhash it.

I'll have to ponder that a bit more.

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

* Re: [PATCH 8/9] qspinlock: Generic paravirt support
  2015-04-02 19:48                             ` Peter Zijlstra
  2015-04-03  3:39                                 ` Waiman Long
  2015-04-03  3:39                               ` Waiman Long
@ 2015-04-03 13:43                               ` Peter Zijlstra
  2015-04-03 13:43                                 ` Peter Zijlstra
  3 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-04-03 13:43 UTC (permalink / raw)
  To: Waiman Long
  Cc: linux-arch, riel, x86, kvm, scott.norton, raghavendra.kt,
	paolo.bonzini, oleg, linux-kernel, mingo, david.vrabel, hpa,
	luto, xen-devel, tglx, paulmck, torvalds, boris.ostrovsky,
	virtualization, doug.hatch

On Thu, Apr 02, 2015 at 09:48:34PM +0200, Peter Zijlstra wrote:
> @@ -158,20 +257,20 @@ static void pv_wait_head(struct qspinloc
>  void __pv_queue_spin_unlock(struct qspinlock *lock)
>  {
>  	struct __qspinlock *l = (void *)lock;
> +	struct pv_hash_bucket *hb;
>  
>  	if (xchg(&l->locked, 0) != _Q_SLOW_VAL)
>  		return;
>  
>  	/*
>  	 * At this point the memory pointed at by lock can be freed/reused,
> +	 * however we can still use the pointer value to search in our hash
> +	 * table.
>  	 *
> +	 * Also, if we observe _Q_SLOW_VAL we _must_ now observe 'our' hash
> +	 * bucket. See pv_wait_head().
>  	 */
> +	hb = pv_hash_find(lock);
> +	pv_kick(hb->cpu);
> +	WRITE_ONCE(hb->lock, NULL); /* unhash */
>  }

So I _think_ I found a problem with this approach :/

If, as per the above, the lock does indeed get freed, it can get
re-allocated and stuck in the hash table (again) before we get the
lookup and unhash it.

I'll have to ponder that a bit more.

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

* [PATCH 0/9] qspinlock stuff -v15
@ 2015-03-16 13:16 Peter Zijlstra
  0 siblings, 0 replies; 136+ messages in thread
From: Peter Zijlstra @ 2015-03-16 13:16 UTC (permalink / raw)
  To: Waiman.Long
  Cc: raghavendra.kt, kvm, peterz, linux-kernel, hpa, boris.ostrovsky,
	linux-arch, x86, mingo, doug.hatch, xen-devel, paulmck, riel,
	scott.norton, paolo.bonzini, tglx, virtualization, oleg, luto,
	david.vrabel, torvalds

Hi Waiman,

As promised; here is the paravirt stuff I did during the trip to BOS last week.

All the !paravirt patches are more or less the same as before (the only real
change is the copyright lines in the first patch).

The paravirt stuff is 'simple' and KVM only -- the Xen code was a little more
convoluted and I've no real way to test that but it should be stright fwd to
make work.

I ran this using the virtme tool (thanks Andy) on my laptop with a 4x
overcommit on vcpus (16 vcpus as compared to the 4 my laptop actually has) and
it both booted and survived a hackbench run (perf bench sched messaging -g 20
-l 5000).

So while the paravirt code isn't the most optimal code ever conceived it does work.

Also, the paravirt patching includes replacing the call with "movb $0, %arg1"
for the native case, which should greatly reduce the cost of having
CONFIG_PARAVIRT_SPINLOCKS enabled on actual hardware.

I feel that if someone were to do a Xen patch we can go ahead and merge this
stuff (finally!).

These patches do not implement the paravirt spinlock debug stats currently
implemented (separately) by KVM and Xen, but that should not be too hard to do
on top and in the 'generic' code -- no reason to duplicate all that.

Of course; once this lands people can look at improving the paravirt nonsense.

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

end of thread, other threads:[~2015-04-03 13:43 UTC | newest]

Thread overview: 136+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-03-16 13:16 [PATCH 0/9] qspinlock stuff -v15 Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 1/9] qspinlock: A simple generic 4-byte queue spinlock Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 2/9] qspinlock, x86: Enable x86-64 to use " Peter Zijlstra
2015-03-16 13:16   ` Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 3/9] qspinlock: Add pending bit Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 4/9] qspinlock: Extract out code snippets for the next patch Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16   ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 5/9] qspinlock: Optimize for smaller NR_CPUS Peter Zijlstra
2015-03-16 13:16   ` Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 6/9] qspinlock: Use a simple write to grab the lock Peter Zijlstra
2015-03-16 13:16   ` Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 7/9] qspinlock: Revert to test-and-set on hypervisors Peter Zijlstra
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16   ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 8/9] qspinlock: Generic paravirt support Peter Zijlstra
2015-03-16 13:16   ` Peter Zijlstra
2015-03-18 20:50   ` Waiman Long
2015-03-19 10:12     ` Peter Zijlstra
2015-03-19 10:12     ` Peter Zijlstra
2015-03-19 10:12     ` Peter Zijlstra
2015-03-19 12:25       ` Peter Zijlstra
2015-03-19 12:25         ` Peter Zijlstra
2015-03-19 13:43         ` Peter Zijlstra
2015-03-19 13:43         ` Peter Zijlstra
2015-03-19 13:43         ` Peter Zijlstra
2015-03-19 23:25         ` Waiman Long
2015-03-19 23:25         ` Waiman Long
2015-03-19 23:25         ` Waiman Long
2015-04-01 16:20         ` Waiman Long
2015-04-01 16:20         ` Waiman Long
2015-04-01 16:20           ` Waiman Long
2015-04-01 17:12           ` Peter Zijlstra
2015-04-01 17:12           ` Peter Zijlstra
2015-04-01 17:12           ` Peter Zijlstra
2015-04-01 17:42             ` Peter Zijlstra
2015-04-01 17:42               ` Peter Zijlstra
2015-04-01 18:17               ` Peter Zijlstra
2015-04-01 18:17                 ` Peter Zijlstra
2015-04-01 18:54                 ` Waiman Long
2015-04-01 18:54                   ` Waiman Long
2015-04-01 18:48                   ` Peter Zijlstra
2015-04-01 18:48                   ` Peter Zijlstra
2015-04-01 19:58                     ` Waiman Long
2015-04-01 21:03                       ` Peter Zijlstra
2015-04-01 21:03                       ` Peter Zijlstra
2015-04-01 21:03                         ` Peter Zijlstra
2015-04-02 16:28                         ` Waiman Long
2015-04-02 17:20                           ` Peter Zijlstra
2015-04-02 17:20                             ` Peter Zijlstra
2015-04-02 19:48                             ` Peter Zijlstra
2015-04-02 19:48                             ` Peter Zijlstra
2015-04-03  3:39                               ` Waiman Long
2015-04-03  3:39                                 ` Waiman Long
2015-04-03  3:39                               ` Waiman Long
2015-04-03 13:43                               ` Peter Zijlstra
2015-04-03 13:43                               ` Peter Zijlstra
2015-04-03 13:43                                 ` Peter Zijlstra
2015-04-02 19:48                             ` Peter Zijlstra
2015-04-02 17:20                           ` Peter Zijlstra
2015-04-02 16:28                         ` Waiman Long
2015-04-02 16:28                         ` Waiman Long
2015-04-01 19:58                     ` Waiman Long
2015-04-01 19:58                     ` Waiman Long
2015-04-01 18:48                   ` Peter Zijlstra
2015-04-01 18:54                 ` Waiman Long
2015-04-01 18:17               ` Peter Zijlstra
2015-04-01 17:42             ` Peter Zijlstra
2015-04-01 20:10             ` Waiman Long
2015-04-01 20:10             ` Waiman Long
2015-04-01 20:10             ` Waiman Long
2015-03-19 12:25       ` Peter Zijlstra
2015-03-18 20:50   ` Waiman Long
2015-03-16 13:16 ` Peter Zijlstra
2015-03-16 13:16 ` [PATCH 9/9] qspinlock, x86, kvm: Implement KVM support for paravirt qspinlock Peter Zijlstra
2015-03-16 13:16 ` [PATCH 9/9] qspinlock,x86,kvm: " Peter Zijlstra
2015-03-16 13:16   ` [PATCH 9/9] qspinlock, x86, kvm: " Peter Zijlstra
2015-03-19  2:45   ` Waiman Long
2015-03-19 10:01     ` Peter Zijlstra
2015-03-19 10:01     ` [PATCH 9/9] qspinlock,x86,kvm: " Peter Zijlstra
2015-03-19 10:01       ` Peter Zijlstra
2015-03-19 21:08       ` [PATCH 9/9] qspinlock, x86, kvm: " Waiman Long
2015-03-19 21:08       ` [PATCH 9/9] qspinlock,x86,kvm: " Waiman Long
2015-03-19 21:08         ` [PATCH 9/9] qspinlock, x86, kvm: " Waiman Long
2015-03-20  7:43         ` Raghavendra K T
2015-03-20  7:43         ` [PATCH 9/9] qspinlock,x86,kvm: " Raghavendra K T
2015-03-20  7:43           ` [PATCH 9/9] qspinlock, x86, kvm: " Raghavendra K T
2015-03-19  2:45   ` Waiman Long
2015-03-16 14:08 ` [PATCH 0/9] qspinlock stuff -v15 David Vrabel
2015-03-16 14:08 ` [Xen-devel] " David Vrabel
2015-03-16 14:08   ` David Vrabel
2015-03-16 14:08   ` David Vrabel
2015-03-16 14:08   ` David Vrabel
2015-03-18 20:36 ` Waiman Long
2015-03-18 20:36 ` Waiman Long
2015-03-18 20:36 ` Waiman Long
2015-03-19 18:01 ` [Xen-devel] " David Vrabel
2015-03-19 18:01 ` David Vrabel
2015-03-19 18:01   ` David Vrabel
2015-03-19 18:32   ` Peter Zijlstra
2015-03-19 18:32     ` Peter Zijlstra
2015-03-19 18:32   ` Peter Zijlstra
2015-03-19 18:01 ` David Vrabel
2015-03-25 19:47 ` Konrad Rzeszutek Wilk
2015-03-26 20:21   ` Peter Zijlstra
2015-03-26 20:21   ` Peter Zijlstra
2015-03-26 20:21     ` Peter Zijlstra
2015-03-27 14:07     ` Konrad Rzeszutek Wilk
2015-03-27 14:07     ` Konrad Rzeszutek Wilk
2015-03-27 14:07     ` Konrad Rzeszutek Wilk
2015-03-30 16:41       ` Waiman Long
2015-03-30 16:41       ` Waiman Long
2015-03-30 16:41       ` Waiman Long
2015-03-30 16:25   ` Waiman Long
2015-03-30 16:25   ` Waiman Long
2015-03-30 16:29     ` Peter Zijlstra
2015-03-30 16:29       ` Peter Zijlstra
2015-03-30 16:43       ` Waiman Long
2015-03-30 16:43       ` Waiman Long
2015-03-30 16:43         ` Waiman Long
2015-03-30 16:29     ` Peter Zijlstra
2015-03-30 16:25   ` Waiman Long
2015-03-25 19:47 ` Konrad Rzeszutek Wilk
2015-03-25 19:47 ` Konrad Rzeszutek Wilk
2015-03-27  6:40 ` Raghavendra K T
2015-03-27  6:40 ` Raghavendra K T
2015-03-27  6:40 ` Raghavendra K T
  -- strict thread matches above, loose matches on Subject: below --
2015-03-16 13:16 Peter Zijlstra

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.