2014-01-28 18:20:10

by Waiman Long

[permalink] [raw]
Subject: [PATCH v3 0/2] qspinlock: Introducing a 4-byte queue spinlock

v2->v3:
- Simplify the code by using numerous mode only without an unfair option.
- Use the latest smp_load_acquire()/smp_store_release() barriers.
- Move the queue spinlock code to kernel/locking.
- Make the use of queue spinlock the default for x86-64 without user
configuration.
- Additional performance tuning.

v1->v2:
- Add some more comments to document what the code does.
- Add a numerous CPU mode to support >= 16K CPUs
- Add a configuration option to allow lock stealing which can further
improve performance in many cases.
- Enable wakeup of queue head CPU at unlock time for non-numerous
CPU mode.

This patch set introduces a queue-based spinlock implementation that
can replace the default ticket spinlock without increasing the size
of the spinlock data structure. As a result, critical kernel data
structures that embed spinlock won't increase in size and breaking
data alignments.

The queue spinlock has slight better performance than the ticket
spinlock in uncontended case. Its performance can be much better
with moderate to heavy contention. This patch has the potential of
improving the performance of all the workloads that have moderate to
heavy spinlock contention.

The queue spinlock is especially suitable for NUMA machines with at
least 2 sockets, though noticeable performance benefit probably won't
show up in machines with less than 4 sockets.

The purpose of this patch set is not to solve any particular spinlock
contention problems. Those need to be solved by refactoring the code
to make more efficient use of the lock or finer granularity ones. The
main purpose is to make the lock contention problems more tolerable
until someone can spend the time and effort to fix them.

Waiman Long (2):
qspinlock: Introducing a 4-byte queue spinlock implementation
qspinlock, x86: Enable x86-64 to use queue spinlock

arch/x86/Kconfig | 1 +
arch/x86/include/asm/spinlock.h | 2 +
arch/x86/include/asm/spinlock_types.h | 4 +
include/asm-generic/qspinlock.h | 156 +++++++++++++
kernel/Kconfig.locks | 7 +
kernel/locking/Makefile | 1 +
kernel/locking/qspinlock.c | 398 +++++++++++++++++++++++++++++++++
7 files changed, 569 insertions(+), 0 deletions(-)
create mode 100644 include/asm-generic/qspinlock.h
create mode 100644 kernel/locking/qspinlock.c


2014-01-28 18:20:11

by Waiman Long

[permalink] [raw]
Subject: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

This patch introduces a new 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. Only in light to
moderate contention where the average queue depth is around 1-2 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.

The idea behind this spinlock implementation is the fact that spinlocks
are acquired with preemption disabled. In other words, 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. Of course, interrupt handler can try
to acquire one spinlock while the interrupted user process is in the
process of getting another spinlock. 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 16-bit size. Together with
the 1-byte lock bit, this queue spinlock implementation will only
need 4 bytes to hold all the information that it needs.

The current queue node address encoding of the 4-byte word is as
follows:
Bits 0-7 : the locked byte
Bits 8-9 : queue node index in the per-cpu array (4 entries)
Bits 10-31: cpu number + 1 (max cpus = 4M -1)

In the extremely unlikely case that all the queue node entries are
used up, the current code will fall back to busy spinning without
waiting in a queue with warning message.

For single-thread performance (no contention), a 256K lock/unlock
loop was run on a 2.4Ghz Westmere x86-64 CPU. The following table
shows the average time (in ns) for a single lock/unlock sequence
(including the looping and timing overhead):

Lock Type Time (ns)
--------- ---------
Ticket spinlock 14.1
Queue spinlock (Normal) 8.8*

So the queue spinlock is much faster than the ticket spinlock, even
though the overhead of locking and unlocking should be pretty small
when there is no contention. (* That is with the barrier.h patch that
extends __native_word() to char and short.)

The AIM7 benchmark was run on a 8-socket 80-core DL980 with Westmere
x86-64 CPUs with XFS filesystem on a ramdisk and HT off to evaluate
the performance impact of this patch on a 3.13 kernel.

+------------+----------+-----------------+---------+
| Kernel | 3.13 JPM | 3.13 with | %Change |
| | | qspinlock patch | |
+------------+----------+-----------------+---------+
| 10-100 users |
+------------+----------+-----------------+---------+
|custom | 357459 | 363109 | +1.58% |
|dbase | 496847 | 498801 | +0.39% |
|disk | 2925312 | 2771387 | -5.26% |
|five_sec | 166612 | 169215 | +1.56% |
|fserver | 382129 | 383279 | +0.30% |
|high_systime| 16356 | 16380 | +0.15% |
|short | 4521978 | 4257363 | -5.85% |
+------------+----------+-----------------+---------+
| 200-1000 users |
+------------+----------+-----------------+---------+
|custom | 449070 | 447711 | -0.30% |
|dbase | 845029 | 853362 | +0.99% |
|disk | 2725249 | 4892907 | +79.54% |
|five_sec | 169410 | 170638 | +0.72% |
|fserver | 489662 | 491828 | +0.44% |
|high_systime| 142823 | 143790 | +0.68% |
|short | 7435288 | 9016171 | +21.26% |
+------------+----------+-----------------+---------+
| 1100-2000 users |
+------------+----------+-----------------+---------+
|custom | 432470 | 432570 | +0.02% |
|dbase | 889289 | 890026 | +0.08% |
|disk | 2565138 | 5008732 | +95.26% |
|five_sec | 169141 | 170034 | +0.53% |
|fserver | 498569 | 500701 | +0.43% |
|high_systime| 229913 | 245866 | +6.94% |
|short | 8496794 | 8281918 | -2.53% |
+------------+----------+-----------------+---------+

The workload with the most gain was the disk workload. Without the
patch, the perf profile at 1500 users looked like:

26.19% reaim [kernel.kallsyms] [k] _raw_spin_lock
|--47.28%-- evict
|--46.87%-- inode_sb_list_add
|--1.24%-- xlog_cil_insert_items
|--0.68%-- __remove_inode_hash
|--0.67%-- inode_wait_for_writeback
--3.26%-- [...]
22.96% swapper [kernel.kallsyms] [k] cpu_idle_loop
5.56% reaim [kernel.kallsyms] [k] mutex_spin_on_owner
4.87% reaim [kernel.kallsyms] [k] update_cfs_rq_blocked_load
2.04% reaim [kernel.kallsyms] [k] mspin_lock
1.30% reaim [kernel.kallsyms] [k] memcpy
1.08% reaim [unknown] [.] 0x0000003c52009447

There was pretty high spinlock contention on the inode_sb_list_lock
and maybe the inode's i_lock.

With the patch, the perf profile at 1500 users became:

26.82% swapper [kernel.kallsyms] [k] cpu_idle_loop
4.66% reaim [kernel.kallsyms] [k] mutex_spin_on_owner
3.97% reaim [kernel.kallsyms] [k] update_cfs_rq_blocked_load
2.40% reaim [kernel.kallsyms] [k] queue_spin_lock_slowpath
|--88.31%-- _raw_spin_lock
| |--36.02%-- inode_sb_list_add
| |--35.09%-- evict
| |--16.89%-- xlog_cil_insert_items
| |--6.30%-- try_to_wake_up
| |--2.20%-- _xfs_buf_find
| |--0.75%-- __remove_inode_hash
| |--0.72%-- __mutex_lock_slowpath
| |--0.53%-- load_balance
|--6.02%-- _raw_spin_lock_irqsave
| |--74.75%-- down_trylock
| |--9.69%-- rcu_check_quiescent_state
| |--7.47%-- down
| |--3.57%-- up
| |--1.67%-- rwsem_wake
| |--1.00%-- remove_wait_queue
| |--0.56%-- pagevec_lru_move_fn
|--5.39%-- _raw_spin_lock_irq
| |--82.05%-- rwsem_down_read_failed
| |--10.48%-- rwsem_down_write_failed
| |--4.24%-- __down
| |--2.74%-- __schedule
--0.28%-- [...]
2.20% reaim [kernel.kallsyms] [k] memcpy
1.84% reaim [unknown] [.] 0x000000000041517b
1.77% reaim [kernel.kallsyms] [k] _raw_spin_lock
|--21.08%-- xlog_cil_insert_items
|--10.14%-- xfs_icsb_modify_counters
|--7.20%-- xfs_iget_cache_hit
|--6.56%-- inode_sb_list_add
|--5.49%-- _xfs_buf_find
|--5.25%-- evict
|--5.03%-- __remove_inode_hash
|--4.64%-- __mutex_lock_slowpath
|--3.78%-- selinux_inode_free_security
|--2.95%-- xfs_inode_is_filestream
|--2.35%-- try_to_wake_up
|--2.07%-- xfs_inode_set_reclaim_tag
|--1.52%-- list_lru_add
|--1.16%-- xfs_inode_clear_eofblocks_tag
:
1.30% reaim [kernel.kallsyms] [k] effective_load
1.27% reaim [kernel.kallsyms] [k] mspin_lock
1.10% reaim [kernel.kallsyms] [k] security_compute_sid

On the ext4 filesystem, the disk workload improved from 416281 JPM
to 899101 JPM (+116%) with the patch. In this case, the contended
spinlock is the mb_cache_spinlock.

Signed-off-by: Waiman Long <[email protected]>
---
include/asm-generic/qspinlock.h | 156 +++++++++++++++
kernel/Kconfig.locks | 7 +
kernel/locking/Makefile | 1 +
kernel/locking/qspinlock.c | 398 +++++++++++++++++++++++++++++++++++++++
4 files changed, 562 insertions(+), 0 deletions(-)
create mode 100644 include/asm-generic/qspinlock.h
create mode 100644 kernel/locking/qspinlock.c

diff --git a/include/asm-generic/qspinlock.h b/include/asm-generic/qspinlock.h
new file mode 100644
index 0000000..536e568
--- /dev/null
+++ b/include/asm-generic/qspinlock.h
@@ -0,0 +1,156 @@
+/*
+ * 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 <[email protected]>
+ */
+#ifndef __ASM_GENERIC_QSPINLOCK_H
+#define __ASM_GENERIC_QSPINLOCK_H
+
+#include <linux/types.h>
+#include <linux/atomic.h>
+#include <asm/cmpxchg.h>
+#include <asm/barrier.h>
+#include <asm/processor.h>
+#include <asm/byteorder.h>
+
+#if !defined(__LITTLE_ENDIAN) && !defined(__BIG_ENDIAN)
+#error "Missing either LITTLE_ENDIAN or BIG_ENDIAN definition."
+#endif
+
+/*
+ * The queue spinlock data structure - a 32-bit word
+ * Bits 0-7 : Set if locked
+ * Bits 8-31: Queue code
+ */
+typedef struct qspinlock {
+ union {
+ u32 qlcode; /* Queue spinlock code */
+ atomic_t qlcode_a; /* Atomic_t version */
+ struct {
+#ifdef __LITTLE_ENDIAN
+ u8 locked; /* 8-bit lock word */
+#else
+ u16 qcode16; /* MS wait queue code */
+ u8 qcode8; /* LS wait queue code */
+ u8 locked; /* 8-bit lock word */
+#endif
+ };
+ };
+} arch_spinlock_t;
+
+#define _QSPINLOCK_LOCKED 1
+#define _QCODE_OFFSET 8
+#define _QCODE(lock) (ACCESS_ONCE((lock)->qlcode) >> _QCODE_OFFSET)
+
+/*
+ * External function declarations
+ */
+extern void queue_spin_lock_slowpath(struct qspinlock *lock);
+
+/**
+ * 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 ACCESS_ONCE(lock->locked);
+}
+
+/**
+ * queue_spin_value_unlocked - is the spinlock structure unlocked?
+ * @lock: queue spinlock structure
+ * Return: 1 if it is unlocked, 0 otherwise
+ */
+static __always_inline int queue_spin_value_unlocked(struct qspinlock lock)
+{
+ return !lock.locked;
+}
+
+/**
+ * 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 _QCODE(lock);
+}
+/**
+ * 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 (!ACCESS_ONCE(lock->qlcode) &&
+ (cmpxchg(&lock->qlcode, 0, _QSPINLOCK_LOCKED) == 0))
+ return 1;
+ return 0;
+}
+
+/**
+ * queue_spin_lock - acquire a queue spinlock
+ * @lock: Pointer to queue spinlock structure
+ */
+static __always_inline void queue_spin_lock(struct qspinlock *lock)
+{
+ /*
+ * To reduce memory access to only once for the cold cache case,
+ * a direct cmpxchg() is performed in the fastpath to optimize the
+ * uncontended case. The contended performance, however, may suffer
+ * a bit because of that.
+ */
+ if (likely(cmpxchg(&lock->qlcode, 0, _QSPINLOCK_LOCKED) == 0))
+ return;
+ queue_spin_lock_slowpath(lock);
+}
+
+/**
+ * queue_spin_unlock - release a queue spinlock
+ * @lock : Pointer to queue spinlock structure
+ */
+static __always_inline void queue_spin_unlock(struct qspinlock *lock)
+{
+ /*
+ * If the writer field is atomic, it can be cleared directly.
+ * Otherwise, an atomic subtraction will be used to clear it.
+ */
+ if (__native_word(lock->locked))
+ smp_store_release(&lock->locked, 0);
+ else {
+ smp_mb__before_atomic_dec();
+ atomic_sub(_QSPINLOCK_LOCKED, &lock->qlcode_a);
+ }
+}
+
+/*
+ * Initializier
+ */
+#define __ARCH_SPIN_LOCK_UNLOCKED { { 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)
+
+#endif /* __ASM_GENERIC_QSPINLOCK_H */
diff --git a/kernel/Kconfig.locks b/kernel/Kconfig.locks
index d2b32ac..f185584 100644
--- a/kernel/Kconfig.locks
+++ b/kernel/Kconfig.locks
@@ -223,3 +223,10 @@ endif
config MUTEX_SPIN_ON_OWNER
def_bool y
depends on SMP && !DEBUG_MUTEXES
+
+config ARCH_USE_QUEUE_SPINLOCK
+ bool
+
+config QUEUE_SPINLOCK
+ def_bool y if ARCH_USE_QUEUE_SPINLOCK
+ depends on SMP && !PARAVIRT_SPINLOCKS
diff --git a/kernel/locking/Makefile b/kernel/locking/Makefile
index baab8e5..1a17380 100644
--- a/kernel/locking/Makefile
+++ b/kernel/locking/Makefile
@@ -23,3 +23,4 @@ obj-$(CONFIG_DEBUG_SPINLOCK) += spinlock_debug.o
obj-$(CONFIG_RWSEM_GENERIC_SPINLOCK) += rwsem-spinlock.o
obj-$(CONFIG_RWSEM_XCHGADD_ALGORITHM) += rwsem-xadd.o
obj-$(CONFIG_PERCPU_RWSEM) += percpu-rwsem.o
+obj-$(CONFIG_QUEUE_SPINLOCK) += qspinlock.o
diff --git a/kernel/locking/qspinlock.c b/kernel/locking/qspinlock.c
new file mode 100644
index 0000000..da4b6cc
--- /dev/null
+++ b/kernel/locking/qspinlock.c
@@ -0,0 +1,398 @@
+/*
+ * 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 <[email protected]>
+ */
+#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-generic/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 with twists
+ * to make it fit the following constraints:
+ * 1. A max spinlock size of 4 bytes
+ * 2. Good fastpath performance
+ * 3. No change in the locking APIs
+ *
+ * The queue spinlock fastpath is as simple as it can get, all the heavy
+ * lifting is done in the lock slowpath. The main idea behind this queue
+ * spinlock implementation is to keep the spinlock size at 4 bytes while
+ * at the same time implement a queue structure to queue up the waiting
+ * lock spinners.
+ *
+ * Since preemption is disabled before getting the lock, a given CPU will
+ * only need to use one queue node structure in a non-interrupt context.
+ * A percpu queue node structure will be allocated for this purpose and the
+ * cpu number will be put into the queue spinlock structure to indicate the
+ * tail of the queue.
+ *
+ * To handle spinlock acquisition at interrupt context (softirq or hardirq),
+ * the queue node structure is actually an array for supporting nested spin
+ * locking operations in interrupt handlers. If all the entries in the
+ * array are used up, a warning message will be printed (as that shouldn't
+ * happen in normal circumstances) and the lock spinner will fall back to
+ * busy spinning instead of waiting in a queue.
+ */
+
+/*
+#ifndef CONFIG_DEBUG_SPINLOCK
+#define CONFIG_DEBUG_SPINLOCK 1
+#endif
+ */
+
+/*
+ * The queue node structure
+ *
+ * The used flag is used for synchronization between processs and interrupt
+ * contexts of the same CPU. So it should be set first at initialization and
+ * cleared last in the cleanup code.
+ */
+struct qnode {
+ u32 used; /* Used flag */
+ u32 wait; /* Waiting flag */
+#ifdef CONFIG_DEBUG_SPINLOCK
+ u32 cpu_nr; /* CPU number */
+ void *lock; /* Lock address */
+#endif
+ struct qnode *next; /* Next queue node addr */
+};
+
+/*
+ * The 24-bit queue node code is divided into the following 2 fields:
+ * Bits 0-1 : queue node index (4 nodes)
+ * Bits 2-23: CPU number + 1 (4M - 1 CPUs)
+ *
+ * A queue node code of 0 indicates that no one is waiting for the lock.
+ * As the value 0 cannot be used as a valid CPU number. We need to add
+ * 1 to it before putting it into the queue code.
+ */
+#define MAX_QNODES 4
+#define GET_QN_IDX(code) (((code) >> 8) & 3)
+#define GET_CPU_NR(code) (((code) >> 10) - 1)
+#define SET_QCODE(cpu, idx) ((((cpu) + 1) << 10) | ((idx) << 8) |\
+ _QSPINLOCK_LOCKED)
+
+/*
+ * Per-CPU queue node structures
+ */
+static DEFINE_PER_CPU(struct qnode [MAX_QNODES], qnodes) ____cacheline_aligned
+ = { { 0 } };
+
+/**
+ * xlate_qcode - translate the queue code into the queue node address
+ * @qcode: Queue code to be translated
+ * Return: The corresponding queue node address
+ */
+static inline struct qnode *xlate_qcode(u32 qcode)
+{
+ u32 cpu_nr = GET_CPU_NR(qcode);
+ u8 qn_idx = GET_QN_IDX(qcode);
+
+ return per_cpu_ptr(&qnodes[qn_idx], cpu_nr);
+}
+
+/*
+ * Debugging macros & functions
+ */
+#ifdef CONFIG_DEBUG_SPINLOCK
+#define ASSERT(cond) BUG_ON(!(cond))
+#define SET_NODE(var, val) do { node->(var) = (val); } while (0)
+
+/**
+ * assert_nextnode - assert that the next node is valid
+ * @next: Pointer to the next node
+ * @lock: Pointer to queue spinlock structure
+ * @cpu_nr: CPU number
+ */
+static noinline void
+assert_nextnode(struct qnode *next, struct qspinlock *lock, u32 cpu_nr)
+{
+ ASSERT(next->cpu_nr != cpu_nr);
+ ASSERT(next->lock == (void *)lock);
+ ASSERT(next->wait);
+ ASSERT(next->used);
+}
+
+
+/**
+ * assert_prevnode - assert the previous node is valid
+ * @prev: Pointer to the previous node
+ * @lock: Pointer to queue spinlock structure
+ * @cpu_nr: CPU number
+ */
+static noinline void
+assert_prevnode(struct qnode *prev, struct qspinlock *lock, u32 cpu_nr)
+{
+ ASSERT(prev->cpu_nr != cpu_nr);
+ ASSERT(prev->lock == (void *)lock);
+ ASSERT(prev->used);
+ ASSERT(prev->next == NULL);
+}
+
+#else
+#define ASSERT(cond)
+#define SET_NODE(var, val)
+#define assert_nextnode(next, lock, cpu_nr)
+#define assert_prevnode(prev, lock, cpu_nr)
+#endif
+
+/**
+ * unfair_trylock - try to acquire the lock ignoring the qcode
+ * @lock: Pointer to queue spinlock structure
+ * Return: 1 if lock acquired, 0 if failed
+ */
+static __always_inline int unfair_trylock(struct qspinlock *lock)
+{
+ u32 qlcode = ACCESS_ONCE(lock->qlcode);
+
+ if (qlcode & _QSPINLOCK_LOCKED)
+ return 0;
+ if (__native_word(lock->locked)) {
+ if (cmpxchg(&lock->locked, 0, _QSPINLOCK_LOCKED) == 0)
+ return 1;
+ } else if (cmpxchg(&lock->qlcode, qlcode, qlcode|_QSPINLOCK_LOCKED)
+ == qlcode)
+ return 1;
+ return 0;
+}
+
+/**
+ * init_node - initialize the queue node
+ * @node: Pointer to queue node structure
+ * @lock: Pointer to queue spinlock structure
+ * @cpu_nr: CPU number
+ *
+ * The used flag must be set before other fields. Since the used flag will
+ * always be checked by the same CPU, even though it may be in a different
+ * interrupt context, no explicit barrier instruction other than a compiler
+ * barrier will be needed.
+ */
+static inline void
+init_node(struct qnode *node, struct qspinlock *lock, u32 cpu_nr)
+{
+ ASSERT(!node->used);
+ node->used = true;
+ barrier();
+
+ ASSERT(!node->lock);
+ ASSERT(!node->next);
+ ASSERT(!node->wait);
+ node->wait = true;
+ node->next = NULL;
+ SET_NODE(cpu_nr, cpu_nr);
+ SET_NODE(lock, (void *)lock);
+}
+
+/**
+ * cleanup_node - Clean up the queue node
+ * @node: Pointer to queue node structure
+ * @cpu_nr: CPU number
+ *
+ * The used flag must be the last one to be cleared.
+ */
+static inline void cleanup_node(struct qnode *node, u16 cpu_nr)
+{
+ node->next = NULL;
+ node->wait = false;
+ SET_NODE(lock, NULL);
+ ASSERT(cpu_nr == smp_processor_id());
+ barrier();
+ node->used = false;
+}
+
+/**
+ * queue_spin_lock_slowpath - acquire the queue spinlock
+ * @lock: Pointer to queue spinlock structure
+ */
+void queue_spin_lock_slowpath(struct qspinlock *lock)
+{
+ unsigned int cpu_nr, qn_idx;
+ struct qnode *node, *next = NULL;
+ u32 prev_qcode, my_qcode;
+
+ /*
+ * Get the queue node
+ */
+ cpu_nr = smp_processor_id();
+ node = this_cpu_ptr(&qnodes[0]);
+ qn_idx = 0;
+
+ if (unlikely(node->used)) {
+ /*
+ * This node has been used, try to find an empty queue
+ * node entry.
+ */
+ for (qn_idx = 1; qn_idx < MAX_QNODES; qn_idx++)
+ if (!node[qn_idx].used)
+ break;
+ if (unlikely(qn_idx == MAX_QNODES)) {
+ /*
+ * This shouldn't happen, print a warning message
+ * & busy spinning on the lock.
+ */
+ printk_sched(
+ "qspinlock: queue node table exhausted at cpu %d!\n",
+ cpu_nr);
+ while (!unfair_trylock(lock))
+ arch_mutex_cpu_relax();
+ return;
+ }
+ /* Adjust node pointer */
+ node += qn_idx;
+ }
+
+ /*
+ * Set up the new cpu code to be exchanged
+ */
+ my_qcode = SET_QCODE(cpu_nr, qn_idx);
+
+ /*
+ * The lock may be available at this point, try again before waiting
+ * in a queue.
+ */
+ if (queue_spin_trylock(lock))
+ return;
+
+ /*
+ * Initialize the queue node
+ */
+ init_node(node, lock, cpu_nr);
+
+ /*
+ * Exchange current copy of the queue node code
+ */
+ prev_qcode = xchg(&lock->qlcode, my_qcode);
+ /*
+ * It is possible that we may accidentally steal the lock. If this is
+ * the case, we need to either release it if not the head of the queue
+ * or get the lock and be done with it.
+ */
+ if (unlikely(!(prev_qcode & _QSPINLOCK_LOCKED))) {
+ if (prev_qcode == 0) {
+ /*
+ * Got the lock since it is at the head of the queue
+ * Now try to atomically clear the queue code.
+ */
+ if (cmpxchg(&lock->qlcode, my_qcode, _QSPINLOCK_LOCKED)
+ == my_qcode)
+ goto release_node;
+ /*
+ * The cmpxchg fails only if one or more processes
+ * are added to the queue. In this case, we need to
+ * notify the next one to be the head of the queue.
+ */
+ goto notify_next;
+ }
+ /*
+ * Accidentally steal the lock, release the lock and
+ * let the queue head get it.
+ */
+ queue_spin_unlock(lock);
+ } else
+ prev_qcode &= ~_QSPINLOCK_LOCKED; /* Clear the lock bit */
+ my_qcode &= ~_QSPINLOCK_LOCKED;
+
+ if (prev_qcode) {
+ /*
+ * Not at the queue head, get the address of the previous node
+ * and set up the "next" fields of the that node.
+ */
+ struct qnode *prev = xlate_qcode(prev_qcode);
+
+ assert_prevnode(prev, lock, cpu_nr);
+ ACCESS_ONCE(prev->next) = node;
+ /*
+ * Wait until the waiting flag is off
+ */
+ while (smp_load_acquire(&node->wait))
+ arch_mutex_cpu_relax();
+ }
+
+ /*
+ * At the head of the wait queue now
+ */
+ while (true) {
+ u32 qcode;
+
+ if (unlikely(!next)) {
+ /*
+ * Try to get the next node address & clean up
+ * current node data structure now if the next node
+ * address had been set.
+ */
+ next = ACCESS_ONCE(node->next);
+ if (next) {
+ assert_nextnode(next, lock, cpu_nr);
+ cleanup_node(node, cpu_nr);
+ node = NULL;
+ }
+ }
+ qcode = ACCESS_ONCE(lock->qlcode);
+ if (qcode & _QSPINLOCK_LOCKED)
+ ; /* Lock not available yet */
+ else if (qcode != my_qcode) {
+ /*
+ * Just get the lock with other spinners waiting
+ * in the queue.
+ */
+ if (unfair_trylock(lock))
+ /* May still need to notify the next node */
+ goto notify_next;
+ } else {
+ /*
+ * Get the lock & clear the queue code simultaneously
+ */
+ if (cmpxchg(&lock->qlcode, my_qcode,
+ _QSPINLOCK_LOCKED) == my_qcode)
+ /* No need to notify next one */
+ goto release_node;
+ }
+ arch_mutex_cpu_relax();
+ }
+
+notify_next:
+ /*
+ * If the next pointer is not set, we need to wait and notify the
+ * next one in line to do busy spinning.
+ */
+ if (unlikely(!next)) {
+ /*
+ * Wait until the next one in queue set up the next field
+ */
+ while (!(next = ACCESS_ONCE(node->next)))
+ arch_mutex_cpu_relax();
+ assert_nextnode(next, lock, cpu_nr);
+ }
+ /*
+ * The next one in queue is now at the head
+ */
+ smp_store_release(&next->wait, false);
+
+release_node:
+ if (node)
+ cleanup_node(node, cpu_nr);
+}
+EXPORT_SYMBOL(queue_spin_lock_slowpath);
--
1.7.1

2014-01-28 18:21:11

by Waiman Long

[permalink] [raw]
Subject: [PATCH v3 2/2] qspinlock, x86: Enable x86-64 to use queue spinlock

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.

Signed-off-by: Waiman Long <[email protected]>
---
arch/x86/Kconfig | 1 +
arch/x86/include/asm/spinlock.h | 2 ++
arch/x86/include/asm/spinlock_types.h | 4 ++++
3 files changed, 7 insertions(+), 0 deletions(-)

diff --git a/arch/x86/Kconfig b/arch/x86/Kconfig
index 03db714..50692b4 100644
--- a/arch/x86/Kconfig
+++ b/arch/x86/Kconfig
@@ -17,6 +17,7 @@ config X86_64
depends on 64BIT
select X86_DEV_DMA_OPS
select ARCH_USE_CMPXCHG_LOCKREF
+ select ARCH_USE_QUEUE_SPINLOCK

### Arch settings
config X86
diff --git a/arch/x86/include/asm/spinlock.h b/arch/x86/include/asm/spinlock.h
index bf156de..cfe51a6 100644
--- a/arch/x86/include/asm/spinlock.h
+++ b/arch/x86/include/asm/spinlock.h
@@ -43,6 +43,7 @@
extern struct static_key paravirt_ticketlocks_enabled;
static __always_inline bool static_key_false(struct static_key *key);

+#ifndef CONFIG_QUEUE_SPINLOCK
#ifdef CONFIG_PARAVIRT_SPINLOCKS

static inline void __ticket_enter_slowpath(arch_spinlock_t *lock)
@@ -181,6 +182,7 @@ static __always_inline void arch_spin_lock_flags(arch_spinlock_t *lock,
{
arch_spin_lock(lock);
}
+#endif /* !CONFIG_QUEUE_SPINLOCK */

static inline void arch_spin_unlock_wait(arch_spinlock_t *lock)
{
diff --git a/arch/x86/include/asm/spinlock_types.h b/arch/x86/include/asm/spinlock_types.h
index 4f1bea1..f501a2a 100644
--- a/arch/x86/include/asm/spinlock_types.h
+++ b/arch/x86/include/asm/spinlock_types.h
@@ -11,6 +11,9 @@
#define TICKET_SLOWPATH_FLAG ((__ticket_t)0)
#endif

+#ifdef CONFIG_QUEUE_SPINLOCK
+#include <asm-generic/qspinlock.h>
+#else
#if (CONFIG_NR_CPUS < (256 / __TICKET_LOCK_INC))
typedef u8 __ticket_t;
typedef u16 __ticketpair_t;
@@ -33,6 +36,7 @@ typedef struct arch_spinlock {
} arch_spinlock_t;

#define __ARCH_SPIN_LOCK_UNLOCKED { { 0 } }
+#endif /* CONFIG_QUEUE_SPINLOCK */

#include <asm/rwlock.h>

--
1.7.1

2014-01-29 00:20:52

by Andi Kleen

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation


So the 1-2 threads case is the standard case on a small
system, isn't it? This may well cause regressions.

> In the extremely unlikely case that all the queue node entries are
> used up, the current code will fall back to busy spinning without
> waiting in a queue with warning message.

Traditionally we had some code which could take thousands
of locks in rare cases (e.g. all locks in a hash table or all locks of
a big reader lock)

The biggest offender was the mm for changing mmu
notifiers, but I believe that's a mutex now.
lglocks presumably still can do it on large enough
systems. I wouldn't be surprised if there is
other code which e.g. make take all locks in a table.

I don't think the warning is valid and will
likely trigger in some obscure cases.

-Andi

2014-01-29 02:57:44

by George Spelvin

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

> So the 1-2 threads case is the standard case on a small
> system, isn't it? This may well cause regressions.

Well, the common case should be uncontended, which is faster.
But yes, testing would be nice.

>> In the extremely unlikely case that all the queue node entries are
>> used up, the current code will fall back to busy spinning without
>> waiting in a queue with warning message.

> Traditionally we had some code which could take thousands
> of locks in rare cases (e.g. all locks in a hash table or all locks of
> a big reader lock)

Doesn't apply; the question implies a misunderstanding of what's
happening. The entry is only needed while spinning waiting for
the lock. Once the lock has been acquired, it may be recycled.

The thread may *hold* thousands of locks; the entries only apply
to locks being *waited for*.

>From process context a thread may only be waiting for one at a time.
Additional entries are only needed in case a processor takes an interrupt
while spinning, and the interrupt handler wants to take a lock, too.

If that lock also has to be waited for, and during the wait you take a
nested interrupt or NMI, a third level might happen.

The chances of this being nested more than 4 deep seem sufficiently
minute.

2014-01-29 17:58:25

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 01/28/2014 07:20 PM, Andi Kleen wrote:
> So the 1-2 threads case is the standard case on a small
> system, isn't it? This may well cause regressions.
>

Yes, it is possible that in a lightly contended case, the queue spinlock
maybe a bit slower because of the slowpath overhead. I observed some
slight slowdown in some of the lightly contended workloads. I will run
more test in a smaller 2-socket system or even a 1-socket system to see
if there is observed regression.

>> In the extremely unlikely case that all the queue node entries are
>> used up, the current code will fall back to busy spinning without
>> waiting in a queue with warning message.
> Traditionally we had some code which could take thousands
> of locks in rare cases (e.g. all locks in a hash table or all locks of
> a big reader lock)
>
> The biggest offender was the mm for changing mmu
> notifiers, but I believe that's a mutex now.
> lglocks presumably still can do it on large enough
> systems. I wouldn't be surprised if there is
> other code which e.g. make take all locks in a table.
>
> I don't think the warning is valid and will
> likely trigger in some obscure cases.
>
> -Andi

As explained by George, the queue node is only needed when the thread is
waiting to acquire the lock. Once it gets the lock, the node can be
released and be reused.

-Longman

2014-01-30 08:49:41

by Raghavendra K T

[permalink] [raw]
Subject: Re: [PATCH v3 0/2] qspinlock: Introducing a 4-byte queue spinlock

On 01/28/2014 11:49 PM, Waiman Long wrote:
> v2->v3:
> - Simplify the code by using numerous mode only without an unfair option.
> - Use the latest smp_load_acquire()/smp_store_release() barriers.
> - Move the queue spinlock code to kernel/locking.
> - Make the use of queue spinlock the default for x86-64 without user
> configuration.
> - Additional performance tuning.
>

Could you please point me to any dependency patches I am missing?
I am getting the compilation error (latest linus tree with
9b0cd304f26b9fca140de15deeac2bf357d1f388)

Got something like below

make[1]: Nothing to be done for `all'.
CHK include/config/kernel.release
CHK include/generated/uapi/linux/version.h
CHK include/generated/utsrelease.h
make[1]: Nothing to be done for `relocs'.
CC kernel/bounds.s
In file included from include/asm-generic/getorder.h:7:0,
from /test/master/arch/x86/include/asm/page.h:71,
from /test/master/arch/x86/include/asm/processor.h:17,
from /test/master/arch/x86/include/asm/atomic.h:6,
from include/linux/atomic.h:4,
from include/asm-generic/qspinlock.h:22,
from
/test/master/arch/x86/include/asm/spinlock_types.h:15,
from
/test/master/arch/x86/include/asm/paravirt_types.h:331,
from /test/master/arch/x86/include/asm/ptrace.h:65,
from /test/master/arch/x86/include/asm/alternative.h:8,
from /test/master/arch/x86/include/asm/bitops.h:16,
from include/linux/bitops.h:33,
from include/linux/kernel.h:10,
from include/asm-generic/bug.h:13,
from /test/master/arch/x86/include/asm/bug.h:38,
from include/linux/bug.h:4,
from include/linux/page-flags.h:9,
from kernel/bounds.c:9:
include/linux/log2.h: In function ?__ilog2_u32?:
include/linux/log2.h:34:2: error: implicit declaration of function ?fls?
[-Werror=implicit-function-declaration]
include/linux/log2.h: In function ?__ilog2_u64?:
include/linux/log2.h:42:2: error: implicit declaration of function
?fls64? [-Werror=implicit-function-declaration]
include/linux/log2.h: In function ?__roundup_pow_of_two?:
.
.
[trimmed]

2014-01-30 15:39:00

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 0/2] qspinlock: Introducing a 4-byte queue spinlock

On 01/30/2014 03:55 AM, Raghavendra K T wrote:
> On 01/28/2014 11:49 PM, Waiman Long wrote:
>> v2->v3:
>> - Simplify the code by using numerous mode only without an unfair
>> option.
>> - Use the latest smp_load_acquire()/smp_store_release() barriers.
>> - Move the queue spinlock code to kernel/locking.
>> - Make the use of queue spinlock the default for x86-64 without user
>> configuration.
>> - Additional performance tuning.
>>
>
> Could you please point me to any dependency patches I am missing?
> I am getting the compilation error (latest linus tree with
> 9b0cd304f26b9fca140de15deeac2bf357d1f388)
>
> Got something like below
>
> make[1]: Nothing to be done for `all'.
> CHK include/config/kernel.release
> CHK include/generated/uapi/linux/version.h
> CHK include/generated/utsrelease.h
> make[1]: Nothing to be done for `relocs'.
> CC kernel/bounds.s
> In file included from include/asm-generic/getorder.h:7:0,
> from /test/master/arch/x86/include/asm/page.h:71,
> from /test/master/arch/x86/include/asm/processor.h:17,
> from /test/master/arch/x86/include/asm/atomic.h:6,
> from include/linux/atomic.h:4,
> from include/asm-generic/qspinlock.h:22,
> from
> /test/master/arch/x86/include/asm/spinlock_types.h:15,
> from
> /test/master/arch/x86/include/asm/paravirt_types.h:331,
> from /test/master/arch/x86/include/asm/ptrace.h:65,
> from /test/master/arch/x86/include/asm/alternative.h:8,
> from /test/master/arch/x86/include/asm/bitops.h:16,
> from include/linux/bitops.h:33,
> from include/linux/kernel.h:10,
> from include/asm-generic/bug.h:13,
> from /test/master/arch/x86/include/asm/bug.h:38,
> from include/linux/bug.h:4,
> from include/linux/page-flags.h:9,
> from kernel/bounds.c:9:
> include/linux/log2.h: In function ?__ilog2_u32?:
> include/linux/log2.h:34:2: error: implicit declaration of function
> ?fls? [-Werror=implicit-function-declaration]
> include/linux/log2.h: In function ?__ilog2_u64?:
> include/linux/log2.h:42:2: error: implicit declaration of function
> ?fls64? [-Werror=implicit-function-declaration]
> include/linux/log2.h: In function ?__roundup_pow_of_two?:
> .
> .
> [trimmed]
>

The code will need to be compiled in either the latest linux tree as the
patches have dependency on some new memory barrier that are in 3.14.
Alternatively, you can compile with the latest tip tree.

If you have already done that, please send me your config file so that I
can reproduce the compilation error in my build environment.

Thanks,
Longman

2014-01-30 17:45:11

by Rik van Riel

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 01/28/2014 01:19 PM, Waiman Long wrote:
> This patch introduces a new 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. Only in light to
> moderate contention where the average queue depth is around 1-2 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.

>
> Signed-off-by: Waiman Long <[email protected]>

Acked-by: Rik van Riel <[email protected]>

2014-01-30 17:46:42

by Rik van Riel

[permalink] [raw]
Subject: Re: [PATCH v3 2/2] qspinlock, x86: Enable x86-64 to use queue spinlock

On 01/28/2014 01:19 PM, Waiman Long wrote:
> 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.
>
> Signed-off-by: Waiman Long <[email protected]>

Acked-by: Rik van Riel <[email protected]>

2014-01-30 18:43:43

by Raghavendra K T

[permalink] [raw]
Subject: Re: [PATCH v3 0/2] qspinlock: Introducing a 4-byte queue spinlock

On 01/30/2014 09:08 PM, Waiman Long wrote:
> On 01/30/2014 03:55 AM, Raghavendra K T wrote:
>> On 01/28/2014 11:49 PM, Waiman Long wrote:
[...]
from kernel/bounds.c:9:
>> include/linux/log2.h: In function ?__ilog2_u32?:
>> include/linux/log2.h:34:2: error: implicit declaration of function
>> ?fls? [-Werror=implicit-function-declaration]
>> include/linux/log2.h: In function ?__ilog2_u64?:
>> include/linux/log2.h:42:2: error: implicit declaration of function
>> ?fls64? [-Werror=implicit-function-declaration]
>> include/linux/log2.h: In function ?__roundup_pow_of_two?:
>> .
>> .
>> [trimmed]
>>
>
> The code will need to be compiled in either the latest linux tree as the
> patches have dependency on some new memory barrier that are in 3.14.
> Alternatively, you can compile with the latest tip tree.
>
> If you have already done that, please send me your config file so that I
> can reproduce the compilation error in my build environment.
>

I did compile on latest linux tree which had barrier.txt. May be I 'll
recheck if I am missing something obvious.

Here is .config attached


Attachments:
config-qspin (96.45 kB)

2014-01-30 19:00:35

by Tim Chen

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On Tue, 2014-01-28 at 13:19 -0500, Waiman Long wrote:

> +/**
> + * queue_spin_lock_slowpath - acquire the queue spinlock
> + * @lock: Pointer to queue spinlock structure
> + */
> +void queue_spin_lock_slowpath(struct qspinlock *lock)
> +{
> + unsigned int cpu_nr, qn_idx;
> + struct qnode *node, *next = NULL;
> + u32 prev_qcode, my_qcode;
> +
> + /*
> + * Get the queue node
> + */
> + cpu_nr = smp_processor_id();
> + node = this_cpu_ptr(&qnodes[0]);
> + qn_idx = 0;
> +
> + if (unlikely(node->used)) {
> + /*
> + * This node has been used, try to find an empty queue
> + * node entry.
> + */
> + for (qn_idx = 1; qn_idx < MAX_QNODES; qn_idx++)
> + if (!node[qn_idx].used)
> + break;
> + if (unlikely(qn_idx == MAX_QNODES)) {
> + /*
> + * This shouldn't happen, print a warning message
> + * & busy spinning on the lock.
> + */
> + printk_sched(
> + "qspinlock: queue node table exhausted at cpu %d!\n",
> + cpu_nr);
> + while (!unfair_trylock(lock))
> + arch_mutex_cpu_relax();
> + return;
> + }
> + /* Adjust node pointer */
> + node += qn_idx;
> + }
> +
> + /*
> + * Set up the new cpu code to be exchanged
> + */
> + my_qcode = SET_QCODE(cpu_nr, qn_idx);
> +

If we get interrupted here before we have a chance to set the used flag,
the interrupt handler could pick up the same qnode if it tries to
acquire queued spin lock. Then we could overwrite the qcode we have set
here.

Perhaps an exchange operation for the used flag to prevent this race
condition?

Tim

> + /*
> + * The lock may be available at this point, try again before waiting
> + * in a queue.
> + */
> + if (queue_spin_trylock(lock))
> + return;
> +
> + /*
> + * Initialize the queue node
> + */
> + init_node(node, lock, cpu_nr);
> +
> + /*
> + * Exchange current copy of the queue node code
> + */
> + prev_qcode = xchg(&lock->qlcode, my_qcode);
> + /*
> + * It is possible that we may accidentally steal the lock. If this is
> + * the case, we need to either release it if not the head of the queue
> + * or get the lock and be done with it.
> + */
> + if (unlikely(!(prev_qcode & _QSPINLOCK_LOCKED))) {
> + if (prev_qcode == 0) {
> + /*
> + * Got the lock since it is at the head of the queue
> + * Now try to atomically clear the queue code.
> + */
> + if (cmpxchg(&lock->qlcode, my_qcode, _QSPINLOCK_LOCKED)
> + == my_qcode)
> + goto release_node;
> + /*
> + * The cmpxchg fails only if one or more processes
> + * are added to the queue. In this case, we need to
> + * notify the next one to be the head of the queue.
> + */
> + goto notify_next;
> + }
> + /*
> + * Accidentally steal the lock, release the lock and
> + * let the queue head get it.
> + */
> + queue_spin_unlock(lock);
> + } else
> + prev_qcode &= ~_QSPINLOCK_LOCKED; /* Clear the lock bit */
> + my_qcode &= ~_QSPINLOCK_LOCKED;
> +
> + if (prev_qcode) {
> + /*
> + * Not at the queue head, get the address of the previous node
> + * and set up the "next" fields of the that node.
> + */
> + struct qnode *prev = xlate_qcode(prev_qcode);
> +
> + assert_prevnode(prev, lock, cpu_nr);
> + ACCESS_ONCE(prev->next) = node;
> + /*
> + * Wait until the waiting flag is off
> + */
> + while (smp_load_acquire(&node->wait))
> + arch_mutex_cpu_relax();
> + }
> +
> + /*
> + * At the head of the wait queue now
> + */
> + while (true) {
> + u32 qcode;
> +
> + if (unlikely(!next)) {
> + /*
> + * Try to get the next node address & clean up
> + * current node data structure now if the next node
> + * address had been set.
> + */
> + next = ACCESS_ONCE(node->next);
> + if (next) {
> + assert_nextnode(next, lock, cpu_nr);
> + cleanup_node(node, cpu_nr);
> + node = NULL;
> + }
> + }
> + qcode = ACCESS_ONCE(lock->qlcode);
> + if (qcode & _QSPINLOCK_LOCKED)
> + ; /* Lock not available yet */
> + else if (qcode != my_qcode) {
> + /*
> + * Just get the lock with other spinners waiting
> + * in the queue.
> + */
> + if (unfair_trylock(lock))
> + /* May still need to notify the next node */
> + goto notify_next;
> + } else {
> + /*
> + * Get the lock & clear the queue code simultaneously
> + */
> + if (cmpxchg(&lock->qlcode, my_qcode,
> + _QSPINLOCK_LOCKED) == my_qcode)
> + /* No need to notify next one */
> + goto release_node;
> + }
> + arch_mutex_cpu_relax();
> + }
> +
> +notify_next:
> + /*
> + * If the next pointer is not set, we need to wait and notify the
> + * next one in line to do busy spinning.
> + */
> + if (unlikely(!next)) {
> + /*
> + * Wait until the next one in queue set up the next field
> + */
> + while (!(next = ACCESS_ONCE(node->next)))
> + arch_mutex_cpu_relax();
> + assert_nextnode(next, lock, cpu_nr);
> + }
> + /*
> + * The next one in queue is now at the head
> + */
> + smp_store_release(&next->wait, false);
> +
> +release_node:
> + if (node)
> + cleanup_node(node, cpu_nr);
> +}
> +EXPORT_SYMBOL(queue_spin_lock_slowpath);

2014-01-30 19:29:04

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On Thu, Jan 30, 2014 at 11:00:30AM -0800, Tim Chen wrote:
> On Tue, 2014-01-28 at 13:19 -0500, Waiman Long wrote:
>
> > +/**
> > + * queue_spin_lock_slowpath - acquire the queue spinlock
> > + * @lock: Pointer to queue spinlock structure
> > + */
> > +void queue_spin_lock_slowpath(struct qspinlock *lock)
> > +{
> > + unsigned int cpu_nr, qn_idx;
> > + struct qnode *node, *next = NULL;
> > + u32 prev_qcode, my_qcode;
> > +
> > + /*
> > + * Get the queue node
> > + */
> > + cpu_nr = smp_processor_id();
> > + node = this_cpu_ptr(&qnodes[0]);
> > + qn_idx = 0;
> > +
> > + if (unlikely(node->used)) {
> > + /*
> > + * This node has been used, try to find an empty queue
> > + * node entry.
> > + */
> > + for (qn_idx = 1; qn_idx < MAX_QNODES; qn_idx++)
> > + if (!node[qn_idx].used)
> > + break;
> > + if (unlikely(qn_idx == MAX_QNODES)) {
> > + /*
> > + * This shouldn't happen, print a warning message
> > + * & busy spinning on the lock.
> > + */
> > + printk_sched(
> > + "qspinlock: queue node table exhausted at cpu %d!\n",
> > + cpu_nr);
> > + while (!unfair_trylock(lock))
> > + arch_mutex_cpu_relax();
> > + return;
> > + }
> > + /* Adjust node pointer */
> > + node += qn_idx;
> > + }
> > +
> > + /*
> > + * Set up the new cpu code to be exchanged
> > + */
> > + my_qcode = SET_QCODE(cpu_nr, qn_idx);
> > +
>
> If we get interrupted here before we have a chance to set the used flag,
> the interrupt handler could pick up the same qnode if it tries to
> acquire queued spin lock. Then we could overwrite the qcode we have set
> here.
>
> Perhaps an exchange operation for the used flag to prevent this race
> condition?

I don't get why we need the used thing at all; something like:

struct qna {
int cnt;
struct qnode nodes[4];
};

DEFINE_PER_CPU(struct qna, qna);

struct qnode *get_qnode(void)
{
struct qna *qna = this_cpu_ptr(&qna);

return qna->nodes[qna->cnt++]; /* RMW */
}

void put_qnode(struct qnode *qnode)
{
struct qna *qna = this_cpu_ptr(&qna);
qna->cnt--;
}

Should do fine, right?

If we interrupt the RMW above the interrupted context hasn't yet used
the queue and once we return its free again, so all should be well even
on load-store archs.

The nodes array might as well be 3, because NMIs should never contend on
a spinlock, so all we're left with is task, softirq and hardirq context.

2014-01-30 19:36:03

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On Tue, Jan 28, 2014 at 01:19:10PM -0500, Waiman Long wrote:
> This patch introduces a new 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. Only in light to
> moderate contention where the average queue depth is around 1-2 will
> this queue spinlock be potentially a bit slower due to the higher
> slowpath overhead.

But light to moderate contention should be the most common case. If your
spinlock is very heavily contended, it should be broken up.

I would really like more performance numbers for this on single and dual
socket machines and 'normal' workloads (ha! as if those exist), ie.
kbuild is all very important to us :-)

2014-01-30 22:27:09

by Tim Chen

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation


> > > + /*
> > > + * Set up the new cpu code to be exchanged
> > > + */
> > > + my_qcode = SET_QCODE(cpu_nr, qn_idx);
> > > +
> >
> > If we get interrupted here before we have a chance to set the used flag,
> > the interrupt handler could pick up the same qnode if it tries to
> > acquire queued spin lock. Then we could overwrite the qcode we have set
> > here.
> >
> > Perhaps an exchange operation for the used flag to prevent this race
> > condition?
>
> I don't get why we need the used thing at all; something like:
>
> struct qna {
> int cnt;
> struct qnode nodes[4];
> };
>
> DEFINE_PER_CPU(struct qna, qna);
>
> struct qnode *get_qnode(void)
> {
> struct qna *qna = this_cpu_ptr(&qna);
>
> return qna->nodes[qna->cnt++]; /* RMW */
> }
>
> void put_qnode(struct qnode *qnode)
> {
> struct qna *qna = this_cpu_ptr(&qna);
> qna->cnt--;
> }
>
> Should do fine, right?
>
> If we interrupt the RMW above the interrupted context hasn't yet used
> the queue and once we return its free again, so all should be well even
> on load-store archs.

Agreed. This approach is more efficient and avoid the overhead
searching for unused node and setting used flag.

Tim

>
> The nodes array might as well be 3, because NMIs should never contend on
> a spinlock, so all we're left with is task, softirq and hardirq context.

2014-01-31 15:09:01

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On Tue, Jan 28, 2014 at 01:19:10PM -0500, Waiman Long wrote:
> For single-thread performance (no contention), a 256K lock/unlock
> loop was run on a 2.4Ghz Westmere x86-64 CPU. The following table
> shows the average time (in ns) for a single lock/unlock sequence
> (including the looping and timing overhead):
>
> Lock Type Time (ns)
> --------- ---------
> Ticket spinlock 14.1
> Queue spinlock (Normal) 8.8*

What CONFIG_NR_CPUS ?

Because for CONFIG_NR_CPUS < 128 (or 256 if you got !PARAVIRT), the fast
path code should be:

ticket:

mov $0x100,eax
lock xadd %ax,(%rbx)
cmp %al,%ah
jne ...

although my GCC is being silly and writes:

mov $0x100,eax
lock xadd %ax,(%rbx)
movzbl %ah,%edx
cmp %al,%dl
jne ...

Which seems rather like a waste of a perfectly good cycle.

With a bigger NR_CPUS you do indeed need more ops:

mov $0x10000,%edx
lock xadd %edx,(%rbx)
mov %edx,%ecx
shr $0x10,%ecx
cmp %dx,%cx
jne ...


Whereas for the straight cmpxchg() you'd get something relatively simple
like:

mov %edx,%eax
lock cmpxchg %ecx,(%rbx)
cmp %edx,%eax
jne ...



Anyway, as soon as you get some (light) contention you're going to tank
because you have to pull in extra cachelines, which is sad.


I suppose we could from the ticket code more and optimize the
uncontended path, but that'll make the contended path more expensive
again, although probably not as bad as hitting a new cacheline.

2014-01-31 18:16:30

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 01/30/2014 02:00 PM, Tim Chen wrote:
> On Tue, 2014-01-28 at 13:19 -0500, Waiman Long wrote:
>
>> +/**
>> + * queue_spin_lock_slowpath - acquire the queue spinlock
>> + * @lock: Pointer to queue spinlock structure
>> + */
>> +void queue_spin_lock_slowpath(struct qspinlock *lock)
>> +{
>> + unsigned int cpu_nr, qn_idx;
>> + struct qnode *node, *next = NULL;
>> + u32 prev_qcode, my_qcode;
>> +
>> + /*
>> + * Get the queue node
>> + */
>> + cpu_nr = smp_processor_id();
>> + node = this_cpu_ptr(&qnodes[0]);
>> + qn_idx = 0;
>> +
>> + if (unlikely(node->used)) {
>> + /*
>> + * This node has been used, try to find an empty queue
>> + * node entry.
>> + */
>> + for (qn_idx = 1; qn_idx< MAX_QNODES; qn_idx++)
>> + if (!node[qn_idx].used)
>> + break;
>> + if (unlikely(qn_idx == MAX_QNODES)) {
>> + /*
>> + * This shouldn't happen, print a warning message
>> + *& busy spinning on the lock.
>> + */
>> + printk_sched(
>> + "qspinlock: queue node table exhausted at cpu %d!\n",
>> + cpu_nr);
>> + while (!unfair_trylock(lock))
>> + arch_mutex_cpu_relax();
>> + return;
>> + }
>> + /* Adjust node pointer */
>> + node += qn_idx;
>> + }
>> +
>> + /*
>> + * Set up the new cpu code to be exchanged
>> + */
>> + my_qcode = SET_QCODE(cpu_nr, qn_idx);
>> +
> If we get interrupted here before we have a chance to set the used flag,
> the interrupt handler could pick up the same qnode if it tries to
> acquire queued spin lock. Then we could overwrite the qcode we have set
> here.
>
> Perhaps an exchange operation for the used flag to prevent this race
> condition?
>
> Tim

That actually is fine. I am assuming that whenever an interrupt handler
needs to acquire a spinlock, it can use the same queue node as the
interrupted function as long as it can finish the lock acquisition and
release queue node back to the pool before returning to the interrupted
function. The only case where an interrupt handler cannot use the queue
node is when useful data were already there indicated by the setting of
the used flag.

I will add comment to clarify this possible scenario.

-Longman

2014-01-31 18:26:40

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 01/30/2014 02:28 PM, Peter Zijlstra wrote:
> On Thu, Jan 30, 2014 at 11:00:30AM -0800, Tim Chen wrote:
>> On Tue, 2014-01-28 at 13:19 -0500, Waiman Long wrote:
>>
>>> +/**
>>> + * queue_spin_lock_slowpath - acquire the queue spinlock
>>> + * @lock: Pointer to queue spinlock structure
>>> + */
>>> +void queue_spin_lock_slowpath(struct qspinlock *lock)
>>> +{
>>> + unsigned int cpu_nr, qn_idx;
>>> + struct qnode *node, *next = NULL;
>>> + u32 prev_qcode, my_qcode;
>>> +
>>> + /*
>>> + * Get the queue node
>>> + */
>>> + cpu_nr = smp_processor_id();
>>> + node = this_cpu_ptr(&qnodes[0]);
>>> + qn_idx = 0;
>>> +
>>> + if (unlikely(node->used)) {
>>> + /*
>>> + * This node has been used, try to find an empty queue
>>> + * node entry.
>>> + */
>>> + for (qn_idx = 1; qn_idx< MAX_QNODES; qn_idx++)
>>> + if (!node[qn_idx].used)
>>> + break;
>>> + if (unlikely(qn_idx == MAX_QNODES)) {
>>> + /*
>>> + * This shouldn't happen, print a warning message
>>> + *& busy spinning on the lock.
>>> + */
>>> + printk_sched(
>>> + "qspinlock: queue node table exhausted at cpu %d!\n",
>>> + cpu_nr);
>>> + while (!unfair_trylock(lock))
>>> + arch_mutex_cpu_relax();
>>> + return;
>>> + }
>>> + /* Adjust node pointer */
>>> + node += qn_idx;
>>> + }
>>> +
>>> + /*
>>> + * Set up the new cpu code to be exchanged
>>> + */
>>> + my_qcode = SET_QCODE(cpu_nr, qn_idx);
>>> +
>> If we get interrupted here before we have a chance to set the used flag,
>> the interrupt handler could pick up the same qnode if it tries to
>> acquire queued spin lock. Then we could overwrite the qcode we have set
>> here.
>>
>> Perhaps an exchange operation for the used flag to prevent this race
>> condition?
> I don't get why we need the used thing at all; something like:
>
> struct qna {
> int cnt;
> struct qnode nodes[4];
> };
>
> DEFINE_PER_CPU(struct qna, qna);
>
> struct qnode *get_qnode(void)
> {
> struct qna *qna = this_cpu_ptr(&qna);
>
> return qna->nodes[qna->cnt++]; /* RMW */
> }
>
> void put_qnode(struct qnode *qnode)
> {
> struct qna *qna = this_cpu_ptr(&qna);
> qna->cnt--;
> }
>
> Should do fine, right?

Yes, we can do something like that. However I think put_qnode() needs to
use atomic dec as well. As a result, we will need 2 additional atomic
operations per slowpath invocation. The code may look simpler, but I
don't think it will be faster than what I am currently doing as the
cases where the used flag is set will be relatively rare.

>
> If we interrupt the RMW above the interrupted context hasn't yet used
> the queue and once we return its free again, so all should be well even
> on load-store archs.
>
> The nodes array might as well be 3, because NMIs should never contend on
> a spinlock, so all we're left with is task, softirq and hardirq context.

I am not so sure about NMI not taking a spinlock. I seem to remember
seeing code that did that. Actually, I think the NMI code is trying to
printk something which, in turn, need to acquire a spinlock.

-Longman

2014-01-31 18:28:14

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 01/30/2014 02:35 PM, Peter Zijlstra wrote:
> On Tue, Jan 28, 2014 at 01:19:10PM -0500, Waiman Long wrote:
>> This patch introduces a new 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. Only in light to
>> moderate contention where the average queue depth is around 1-2 will
>> this queue spinlock be potentially a bit slower due to the higher
>> slowpath overhead.
> But light to moderate contention should be the most common case. If your
> spinlock is very heavily contended, it should be broken up.
>
> I would really like more performance numbers for this on single and dual
> socket machines and 'normal' workloads (ha! as if those exist), ie.
> kbuild is all very important to us :-)

Yes, I am planning to get more performance data on low end machines. The
v3 patch was sent out to spur discussion on this topic while I am trying
to get more data.

-Longman

2014-01-31 19:14:42

by George Spelvin

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

> Yes, we can do something like that. However I think put_qnode() needs to
> use atomic dec as well. As a result, we will need 2 additional atomic
> operations per slowpath invocation. The code may look simpler, but I
> don't think it will be faster than what I am currently doing as the
> cases where the used flag is set will be relatively rare.

The increment does *not* have to be atomic.

First of all, note that the only reader that matters is a local interrupt;
other processors never access the variable at all, so what they see
is irrelevant.

"Okay, so I use a non-atomic RMW instruction; what about non-x86
processors without op-to-memory?"

Well, they're okay, too. The only requriement is that the write to
qna->cnt must be visible to the local processor (barrier()) before the
qna->nodes[] slot is used.

Remember, a local interrupt may use a slot temporarily, but will always
return qna->cnt to its original value before returning. So there's
nothing wrong with

- Load qna->cnt to register
- Increment register
- Store register to qna->cnt

Because an interrupt, although it may temporarily modify qna->cnt, will
restore it before returning so this code will never see any modification.

Just like using the stack below the %rsp, the only requirement is to
ensure that the qna->cnt increment is visble *to the local processor's
interrupt handler* before actually using the slot.

The effect of the interrupt handler is that it may corrupt, at any
time and without warning, any slot not marked in use via qna->cnt.
But that's not a difficult thing to deal with, and does *not* require
atomic operations.

2014-01-31 19:24:41

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 01/31/2014 10:08 AM, Peter Zijlstra wrote:
> On Tue, Jan 28, 2014 at 01:19:10PM -0500, Waiman Long wrote:
>> For single-thread performance (no contention), a 256K lock/unlock
>> loop was run on a 2.4Ghz Westmere x86-64 CPU. The following table
>> shows the average time (in ns) for a single lock/unlock sequence
>> (including the looping and timing overhead):
>>
>> Lock Type Time (ns)
>> --------- ---------
>> Ticket spinlock 14.1
>> Queue spinlock (Normal) 8.8*
> What CONFIG_NR_CPUS ?

I was testing on a RHEL6.4 system which has a CONFIG_NR_CPUS of 4096.

>
> Because for CONFIG_NR_CPUS< 128 (or 256 if you got !PARAVIRT), the fast
> path code should be:
>
> ticket:
>
> mov $0x100,eax
> lock xadd %ax,(%rbx)
> cmp %al,%ah
> jne ...
>
> although my GCC is being silly and writes:
>
> mov $0x100,eax
> lock xadd %ax,(%rbx)
> movzbl %ah,%edx
> cmp %al,%dl
> jne ...
>
> Which seems rather like a waste of a perfectly good cycle.
>
> With a bigger NR_CPUS you do indeed need more ops:
>
> mov $0x10000,%edx
> lock xadd %edx,(%rbx)
> mov %edx,%ecx
> shr $0x10,%ecx
> cmp %dx,%cx
> jne ...
>
>
> Whereas for the straight cmpxchg() you'd get something relatively simple
> like:
>
> mov %edx,%eax
> lock cmpxchg %ecx,(%rbx)
> cmp %edx,%eax
> jne ...

I believe the speeds of the lock functions are about the same. However,
qspinlock has a much simpler unlock function which probably account of
most of the speed gain.

> Anyway, as soon as you get some (light) contention you're going to tank
> because you have to pull in extra cachelines, which is sad.

Light contention is the only case where the qspinlock may not perform as
good as the ticket spinlock. I know this is the most common case.
However, I would argue that the slowdown, if any, will not be really
noticeable. This is what I will try to find out.


> I suppose we could from the ticket code more and optimize the
> uncontended path, but that'll make the contended path more expensive
> again, although probably not as bad as hitting a new cacheline.

I don't get what you are trying to say.

Right now, I am using only bit 0 as a lock bit. I can use bit 4, for
instance, as a pending locker bit and spin until bit 0 is clear. So if
there is only 1 other task spinning, it won't need to fetch another
cacheline. However, it will slow down the uncontended path as I can't
assign a 0 byte to free the lock. I have to use an atomic subtraction or
clear bit instead.

-Longman

2014-01-31 19:28:54

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 01/31/2014 02:14 PM, George Spelvin wrote:
>> Yes, we can do something like that. However I think put_qnode() needs to
>> use atomic dec as well. As a result, we will need 2 additional atomic
>> operations per slowpath invocation. The code may look simpler, but I
>> don't think it will be faster than what I am currently doing as the
>> cases where the used flag is set will be relatively rare.
> The increment does *not* have to be atomic.
>
> First of all, note that the only reader that matters is a local interrupt;
> other processors never access the variable at all, so what they see
> is irrelevant.
>
> "Okay, so I use a non-atomic RMW instruction; what about non-x86
> processors without op-to-memory?"
>
> Well, they're okay, too. The only requriement is that the write to
> qna->cnt must be visible to the local processor (barrier()) before the
> qna->nodes[] slot is used.
>
> Remember, a local interrupt may use a slot temporarily, but will always
> return qna->cnt to its original value before returning. So there's
> nothing wrong with
>
> - Load qna->cnt to register
> - Increment register
> - Store register to qna->cnt
>
> Because an interrupt, although it may temporarily modify qna->cnt, will
> restore it before returning so this code will never see any modification.
>
> Just like using the stack below the %rsp, the only requirement is to
> ensure that the qna->cnt increment is visble *to the local processor's
> interrupt handler* before actually using the slot.
>
> The effect of the interrupt handler is that it may corrupt, at any
> time and without warning, any slot not marked in use via qna->cnt.
> But that's not a difficult thing to deal with, and does *not* require
> atomic operations.

George, you are right. I am thinking too much from the general
perspective of RMW instruction.

-Longman

2014-01-31 19:46:21

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On Fri, Jan 31, 2014 at 01:26:29PM -0500, Waiman Long wrote:
> >I don't get why we need the used thing at all; something like:
> >
> >struct qna {
> > int cnt;
> > struct qnode nodes[4];
> >};
> >
> >DEFINE_PER_CPU(struct qna, qna);
> >
> >struct qnode *get_qnode(void)
> >{
> > struct qna *qna = this_cpu_ptr(&qna);
> >
> > return qna->nodes[qna->cnt++]; /* RMW */
> >}
> >
> >void put_qnode(struct qnode *qnode)
> >{
> > struct qna *qna = this_cpu_ptr(&qna);
> > qna->cnt--;
> >}
> >
> >Should do fine, right?
>
> Yes, we can do something like that. However I think put_qnode() needs to use
> atomic dec as well. As a result, we will need 2 additional atomic operations
> per slowpath invocation. The code may look simpler, but I don't think it
> will be faster than what I am currently doing as the cases where the used
> flag is set will be relatively rare.

No, put doesn't need an atomic; nor is it as well; because the inc
doesn't need an atomic either.

> >If we interrupt the RMW above the interrupted context hasn't yet used
> >the queue and once we return its free again, so all should be well even
> >on load-store archs.
> >
> >The nodes array might as well be 3, because NMIs should never contend on
> >a spinlock, so all we're left with is task, softirq and hardirq context.
>
> I am not so sure about NMI not taking a spinlock. I seem to remember seeing
> code that did that. Actually, I think the NMI code is trying to printk
> something which, in turn, need to acquire a spinlock.

Yeah I know, terribly broken that, I've been waiting for that to explode
:-)

2014-01-31 19:52:00

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On Fri, Jan 31, 2014 at 02:24:33PM -0500, Waiman Long wrote:
> On 01/31/2014 10:08 AM, Peter Zijlstra wrote:
> >Whereas for the straight cmpxchg() you'd get something relatively simple
> >like:
> >
> > mov %edx,%eax
> > lock cmpxchg %ecx,(%rbx)
> > cmp %edx,%eax
> > jne ...
>
> I believe the speeds of the lock functions are about the same. However,
> qspinlock has a much simpler unlock function which probably account of most
> of the speed gain.

The unlock path for ticket locks is a single "add[bw] $0x1,()", that should be
as fast as the single "movb 0,()" you have.

> >I suppose we could from the ticket code more and optimize the
> >uncontended path, but that'll make the contended path more expensive
> >again, although probably not as bad as hitting a new cacheline.
>
> I don't get what you are trying to say.

I said we could probably make the ticket lock function faster for the
uncontended case by making the contended case slightly more expensive.

2014-02-03 08:45:20

by Raghavendra K T

[permalink] [raw]
Subject: Re: [PATCH v3 0/2] qspinlock: Introducing a 4-byte queue spinlock

On 01/31/2014 12:19 AM, Raghavendra K T wrote:
> On 01/30/2014 09:08 PM, Waiman Long wrote:
>> On 01/30/2014 03:55 AM, Raghavendra K T wrote:
>>> On 01/28/2014 11:49 PM, Waiman Long wrote:
> [...]
> from kernel/bounds.c:9:
>>> include/linux/log2.h: In function ‘__ilog2_u32’:
>>> include/linux/log2.h:34:2: error: implicit declaration of function
>>> ‘fls’ [-Werror=implicit-function-declaration]
>>> include/linux/log2.h: In function ‘__ilog2_u64’:
>>> include/linux/log2.h:42:2: error: implicit declaration of function
>>> ‘fls64’ [-Werror=implicit-function-declaration]
>>> include/linux/log2.h: In function ‘__roundup_pow_of_two’:
>>> .
>>> .
>>> [trimmed]
>>>
>>
>> The code will need to be compiled in either the latest linux tree as the
>> patches have dependency on some new memory barrier that are in 3.14.
>> Alternatively, you can compile with the latest tip tree.
>>
>> If you have already done that, please send me your config file so that I
>> can reproduce the compilation error in my build environment.
>>
>
> I did compile on latest linux tree which had barrier.txt. May be I 'll
> recheck if I am missing something obvious.
>
> Here is .config attached

An Update:
The reason for failure was that I was compiling for guest. and
CONFIG_PARAVIRT was enabled.


2014-02-03 08:45:56

by Raghavendra K T

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 01/28/2014 11:49 PM, Waiman Long wrote:
> +#endif /* __ASM_GENERIC_QSPINLOCK_H */
> diff --git a/kernel/Kconfig.locks b/kernel/Kconfig.locks
> index d2b32ac..f185584 100644
> --- a/kernel/Kconfig.locks
> +++ b/kernel/Kconfig.locks
> @@ -223,3 +223,10 @@ endif
> config MUTEX_SPIN_ON_OWNER
> def_bool y
> depends on SMP && !DEBUG_MUTEXES
> +
> +config ARCH_USE_QUEUE_SPINLOCK
> + bool
> +
> +config QUEUE_SPINLOCK
> + def_bool y if ARCH_USE_QUEUE_SPINLOCK
> + depends on SMP && !PARAVIRT_SPINLOCKS

This should have been !CONFIG_PARAVIRT instead of !PARAVIRT_SPINLOCKS ?

2014-02-03 11:41:12

by Peter Zijlstra

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On Fri, Jan 31, 2014 at 02:24:33PM -0500, Waiman Long wrote:
>
> Light contention is the only case where the qspinlock may not perform as
> good as the ticket spinlock. I know this is the most common case. However, I
> would argue that the slowdown, if any, will not be really noticeable. This
> is what I will try to find out.

Please also explain why things are slower/faster. Thomas tried to get
you to do so a few months back, but you kept deflecting.

2014-02-06 03:09:52

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 0/2] qspinlock: Introducing a 4-byte queue spinlock

On 02/03/2014 03:51 AM, Raghavendra K T wrote:
> On 01/31/2014 12:19 AM, Raghavendra K T wrote:
>> On 01/30/2014 09:08 PM, Waiman Long wrote:
>>> On 01/30/2014 03:55 AM, Raghavendra K T wrote:
>>>> On 01/28/2014 11:49 PM, Waiman Long wrote:
>> [...]
>> from kernel/bounds.c:9:
>>>> include/linux/log2.h: In function ‘__ilog2_u32’:
>>>> include/linux/log2.h:34:2: error: implicit declaration of function
>>>> ‘fls’ [-Werror=implicit-function-declaration]
>>>> include/linux/log2.h: In function ‘__ilog2_u64’:
>>>> include/linux/log2.h:42:2: error: implicit declaration of function
>>>> ‘fls64’ [-Werror=implicit-function-declaration]
>>>> include/linux/log2.h: In function ‘__roundup_pow_of_two’:
>>>> .
>>>> .
>>>> [trimmed]
>>>>
>>>
>>> The code will need to be compiled in either the latest linux tree as
>>> the
>>> patches have dependency on some new memory barrier that are in 3.14.
>>> Alternatively, you can compile with the latest tip tree.
>>>
>>> If you have already done that, please send me your config file so
>>> that I
>>> can reproduce the compilation error in my build environment.
>>>
>>
>> I did compile on latest linux tree which had barrier.txt. May be I 'll
>> recheck if I am missing something obvious.
>>
>> Here is .config attached
>
> An Update:
> The reason for failure was that I was compiling for guest. and
> CONFIG_PARAVIRT was enabled.
>
>
>
Thank for the information. I will update my patch to correct that problem.

-Longman

2014-02-06 03:11:02

by Waiman Long

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On 02/03/2014 06:40 AM, Peter Zijlstra wrote:
> On Fri, Jan 31, 2014 at 02:24:33PM -0500, Waiman Long wrote:
>> Light contention is the only case where the qspinlock may not perform as
>> good as the ticket spinlock. I know this is the most common case. However, I
>> would argue that the slowdown, if any, will not be really noticeable. This
>> is what I will try to find out.
> Please also explain why things are slower/faster. Thomas tried to get
> you to do so a few months back, but you kept deflecting.

It is not easy to have a test case to test light contention. I am trying
to create custom test case to get that data.

-Longman

2014-02-07 18:17:48

by Paul E. McKenney

[permalink] [raw]
Subject: Re: [PATCH v3 1/2] qspinlock: Introducing a 4-byte queue spinlock implementation

On Wed, Feb 05, 2014 at 10:10:34PM -0500, Waiman Long wrote:
> On 02/03/2014 06:40 AM, Peter Zijlstra wrote:
> >On Fri, Jan 31, 2014 at 02:24:33PM -0500, Waiman Long wrote:
> >>Light contention is the only case where the qspinlock may not perform as
> >>good as the ticket spinlock. I know this is the most common case. However, I
> >>would argue that the slowdown, if any, will not be really noticeable. This
> >>is what I will try to find out.
> >Please also explain why things are slower/faster. Thomas tried to get
> >you to do so a few months back, but you kept deflecting.
>
> It is not easy to have a test case to test light contention. I am
> trying to create custom test case to get that data.

Here are some ways of doing that:

1. Provide (say) a thousand locks for each thread, so that you
have all of these locks in one big array of locks. Have each
thread loop, where each pass through the loop acquires
and releases a randomly selected lock. Then measure the
acquisition/release throughput.

2. As #1 above, but vary the number of locks per thread in order to
vary the level of contention in a controlled manner. Note that
the cache-miss probability is (N-1)/N, where where N is the
number of threads, at least assuming each thread gets its own CPU.

3. Provide each thread with its own lock and have each thread
loop, where each pass through the loop acquires and releases
the thread's lock. This eliminates both contention and
cache misses.

4. As #1 above, but randomly acquire some other thread's lock with
controlled probability to introduce controlled levels of both
contention and cache misses.

5. As #4 above, but provide each thread with multiple locks
randomly selected to allow cache miss rate to be increased
independently of contention.

All approaches require extremely efficient random-number generators,
for example, independent per-thread generators.

Thanx, Paul