All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 0/4] generic smp helpers vs kmalloc
@ 2009-02-16 16:38 Peter Zijlstra
  2009-02-16 16:38 ` [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many() Peter Zijlstra
                   ` (3 more replies)
  0 siblings, 4 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 16:38 UTC (permalink / raw)
  To: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt
  Cc: linux-kernel, Oleg Nesterov, Peter Zijlstra

Steve's recent patches brought up the need (or rather, lack thereof) of the
kmalloc() usage in smp_call_function_single().

This patch set does the same for the multi-cpu case and fully removes kmalloc.

Tested on a Q9450 with a kernel build.


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

* [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()
  2009-02-16 16:38 [PATCH 0/4] generic smp helpers vs kmalloc Peter Zijlstra
@ 2009-02-16 16:38 ` Peter Zijlstra
  2009-02-16 19:10   ` Oleg Nesterov
                     ` (2 more replies)
  2009-02-16 16:38 ` [PATCH 2/4] generic-smp: remove kmalloc usage Peter Zijlstra
                   ` (2 subsequent siblings)
  3 siblings, 3 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 16:38 UTC (permalink / raw)
  To: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt
  Cc: linux-kernel, Oleg Nesterov, Peter Zijlstra

[-- Attachment #1: smp-many-fallback.patch --]
[-- Type: text/plain, Size: 7572 bytes --]

In preparation of removing the kmalloc() calls from the generic-ipi code
get rid of the single ipi fallback for smp_call_function_many().

Because we cannot get around carrying the cpumask in the data -- imagine
2 such calls with different but overlapping masks -- put in a full mask.

Also, since we cannot simply remove an item from the global queue (another
cpu might be observing it), a quiesence of sorts needs to be observed. The
current code uses regular RCU for that purpose.

However, since we'll be wanting to quickly reuse an item, we need something
with a much faster turn-around. We do this by simply observing the global
queue quiesence. Since there are a limited number of elements, it will auto
force a quiecent state if we wait for it.

Signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
---
 kernel/smp.c |  112 +++++++++++++++++++++++++++++++++++++++--------------------
 1 file changed, 75 insertions(+), 37 deletions(-)

Index: linux-2.6/kernel/smp.c
===================================================================
--- linux-2.6.orig/kernel/smp.c
+++ linux-2.6/kernel/smp.c
@@ -12,8 +12,23 @@
 #include <linux/smp.h>
 
 static DEFINE_PER_CPU(struct call_single_queue, call_single_queue);
-static LIST_HEAD(call_function_queue);
-__cacheline_aligned_in_smp DEFINE_SPINLOCK(call_function_lock);
+
+static struct {
+	struct list_head	queue;
+	spinlock_t		lock;
+	/*
+	 * mini-RCU state machine
+	 *
+	 * @counter counts the number of entries on the queue, and @free_list
+	 * contains the items to free when the queue is idle.
+	 */
+	unsigned int		counter;
+	struct list_head	free_list;
+} call_function __cacheline_aligned_in_smp = {
+	.queue = LIST_HEAD_INIT(call_function.queue),
+	.lock  = __SPIN_LOCK_UNLOCKED(call_function.lock),
+	.free_list = LIST_HEAD_INIT(call_function.free_list),
+};
 
 enum {
 	CSD_FLAG_WAIT		= 0x01,
@@ -25,8 +40,11 @@ struct call_function_data {
 	struct call_single_data csd;
 	spinlock_t lock;
 	unsigned int refs;
-	struct rcu_head rcu_head;
-	unsigned long cpumask_bits[];
+	union {
+		struct rcu_head rcu_head;
+		struct list_head free_list;
+	};
+	struct cpumask cpumask;
 };
 
 struct call_single_queue {
@@ -107,17 +125,17 @@ void generic_smp_call_function_interrupt
 	 * It's ok to use list_for_each_rcu() here even though we may delete
 	 * 'pos', since list_del_rcu() doesn't clear ->next
 	 */
-	rcu_read_lock();
-	list_for_each_entry_rcu(data, &call_function_queue, csd.list) {
+	list_for_each_entry_rcu(data, &call_function.queue, csd.list) {
+		LIST_HEAD(free_list);
 		int refs;
 
-		if (!cpumask_test_cpu(cpu, to_cpumask(data->cpumask_bits)))
+		if (!cpumask_test_cpu(cpu, &data->cpumask))
 			continue;
 
 		data->csd.func(data->csd.info);
 
 		spin_lock(&data->lock);
-		cpumask_clear_cpu(cpu, to_cpumask(data->cpumask_bits));
+		cpumask_clear_cpu(cpu, &data->cpumask);
 		WARN_ON(data->refs == 0);
 		data->refs--;
 		refs = data->refs;
@@ -126,9 +144,12 @@ void generic_smp_call_function_interrupt
 		if (refs)
 			continue;
 
-		spin_lock(&call_function_lock);
+		spin_lock(&call_function.lock);
+		list_add(&data->free_list, &call_function.free_list);
 		list_del_rcu(&data->csd.list);
-		spin_unlock(&call_function_lock);
+		if (!--call_function.counter)
+			list_splice_init(&call_function.free_list, &free_list);
+		spin_unlock(&call_function.lock);
 
 		if (data->csd.flags & CSD_FLAG_WAIT) {
 			/*
@@ -140,8 +161,22 @@ void generic_smp_call_function_interrupt
 		}
 		if (data->csd.flags & CSD_FLAG_ALLOC)
 			call_rcu(&data->rcu_head, rcu_free_call_data);
+
+		while (!list_empty(&free_list)) {
+			struct call_function_data *free;
+
+			free = list_first_entry(&free_list,
+					struct call_function_data, free_list);
+
+			list_del(&data->free_list);
+			/*
+			 * serialize stores to data with the flags
+			 * clear
+			 */
+			smp_wmb();
+			free->csd.flags &= ~CSD_FLAG_LOCK;
+		}
 	}
-	rcu_read_unlock();
 
 	put_cpu();
 }
@@ -302,6 +337,8 @@ void __smp_call_function_single(int cpu,
 	arch_send_call_function_ipi(*(maskp))
 #endif
 
+static DEFINE_PER_CPU(struct call_function_data, cfd_data);
+
 /**
  * smp_call_function_many(): Run a function on a set of other CPUs.
  * @mask: The set of cpus to run on (only runs on online subset).
@@ -323,14 +360,14 @@ void smp_call_function_many(const struct
 {
 	struct call_function_data *data;
 	unsigned long flags;
-	int cpu, next_cpu;
+	int cpu, next_cpu, me = smp_processor_id();
 
 	/* Can deadlock when called with interrupts disabled */
 	WARN_ON(irqs_disabled());
 
 	/* So, what's a CPU they want?  Ignoring this one. */
 	cpu = cpumask_first_and(mask, cpu_online_mask);
-	if (cpu == smp_processor_id())
+	if (cpu == me)
 		cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
 	/* No online cpus?  We're done. */
 	if (cpu >= nr_cpu_ids)
@@ -338,7 +375,7 @@ void smp_call_function_many(const struct
 
 	/* Do we have another CPU which isn't us? */
 	next_cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
-	if (next_cpu == smp_processor_id())
+	if (next_cpu == me)
 		next_cpu = cpumask_next_and(next_cpu, mask, cpu_online_mask);
 
 	/* Fastpath: do that cpu by itself. */
@@ -347,31 +384,32 @@ void smp_call_function_many(const struct
 		return;
 	}
 
-	data = kmalloc(sizeof(*data) + cpumask_size(), GFP_ATOMIC);
-	if (unlikely(!data)) {
-		/* Slow path. */
-		for_each_online_cpu(cpu) {
-			if (cpu == smp_processor_id())
-				continue;
-			if (cpumask_test_cpu(cpu, mask))
-				smp_call_function_single(cpu, func, info, wait);
-		}
-		return;
+	data = kmalloc(sizeof(*data), GFP_ATOMIC);
+	if (data)
+		data->csd.flags = CSD_FLAG_ALLOC;
+	else {
+		data = &per_cpu(cfd_data, me);
+		/*
+		 * We need to wait for all previous users to go away.
+		 */
+		while (data->csd.flags & CSD_FLAG_LOCK)
+			cpu_relax();
+		data->csd.flags = CSD_FLAG_LOCK;
 	}
 
 	spin_lock_init(&data->lock);
-	data->csd.flags = CSD_FLAG_ALLOC;
 	if (wait)
 		data->csd.flags |= CSD_FLAG_WAIT;
 	data->csd.func = func;
 	data->csd.info = info;
-	cpumask_and(to_cpumask(data->cpumask_bits), mask, cpu_online_mask);
-	cpumask_clear_cpu(smp_processor_id(), to_cpumask(data->cpumask_bits));
-	data->refs = cpumask_weight(to_cpumask(data->cpumask_bits));
-
-	spin_lock_irqsave(&call_function_lock, flags);
-	list_add_tail_rcu(&data->csd.list, &call_function_queue);
-	spin_unlock_irqrestore(&call_function_lock, flags);
+	cpumask_and(&data->cpumask, mask, cpu_online_mask);
+	cpumask_clear_cpu(smp_processor_id(), &data->cpumask);
+	data->refs = cpumask_weight(&data->cpumask);
+
+	spin_lock_irqsave(&call_function.lock, flags);
+	call_function.counter++;
+	list_add_tail_rcu(&data->csd.list, &call_function.queue);
+	spin_unlock_irqrestore(&call_function.lock, flags);
 
 	/*
 	 * Make the list addition visible before sending the ipi.
@@ -379,7 +417,7 @@ void smp_call_function_many(const struct
 	smp_mb();
 
 	/* Send a message to all CPUs in the map */
-	arch_send_call_function_ipi_mask(to_cpumask(data->cpumask_bits));
+	arch_send_call_function_ipi_mask(&data->cpumask);
 
 	/* optionally wait for the CPUs to complete */
 	if (wait)
@@ -413,20 +451,20 @@ EXPORT_SYMBOL(smp_call_function);
 
 void ipi_call_lock(void)
 {
-	spin_lock(&call_function_lock);
+	spin_lock(&call_function.lock);
 }
 
 void ipi_call_unlock(void)
 {
-	spin_unlock(&call_function_lock);
+	spin_unlock(&call_function.lock);
 }
 
 void ipi_call_lock_irq(void)
 {
-	spin_lock_irq(&call_function_lock);
+	spin_lock_irq(&call_function.lock);
 }
 
 void ipi_call_unlock_irq(void)
 {
-	spin_unlock_irq(&call_function_lock);
+	spin_unlock_irq(&call_function.lock);
 }

-- 


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

* [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-16 16:38 [PATCH 0/4] generic smp helpers vs kmalloc Peter Zijlstra
  2009-02-16 16:38 ` [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many() Peter Zijlstra
@ 2009-02-16 16:38 ` Peter Zijlstra
  2009-02-17  0:40   ` Linus Torvalds
  2009-02-16 16:38 ` [PATCH 3/4] generic-smp: properly allocate the cpumasks Peter Zijlstra
  2009-02-16 16:38 ` [PATCH 4/4] generic-smp: clean up some of the csd->flags fiddling Peter Zijlstra
  3 siblings, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 16:38 UTC (permalink / raw)
  To: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt
  Cc: linux-kernel, Oleg Nesterov, Peter Zijlstra

[-- Attachment #1: smp-remove-kmalloc.patch --]
[-- Type: text/plain, Size: 3905 bytes --]

Now that there is no strict need for kmalloc anymore, and nobody seems to
rely it for the queueing behaviour, remove it.

Signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
---
 kernel/smp.c |   66 ++++++++++++++++-------------------------------------------
 1 file changed, 19 insertions(+), 47 deletions(-)

Index: linux-2.6/kernel/smp.c
===================================================================
--- linux-2.6.orig/kernel/smp.c
+++ linux-2.6/kernel/smp.c
@@ -32,18 +32,14 @@ static struct {
 
 enum {
 	CSD_FLAG_WAIT		= 0x01,
-	CSD_FLAG_ALLOC		= 0x02,
-	CSD_FLAG_LOCK		= 0x04,
+	CSD_FLAG_LOCK		= 0x02,
 };
 
 struct call_function_data {
 	struct call_single_data csd;
 	spinlock_t lock;
 	unsigned int refs;
-	union {
-		struct rcu_head rcu_head;
-		struct list_head free_list;
-	};
+	struct list_head free_list;
 	struct cpumask cpumask;
 };
 
@@ -103,15 +99,6 @@ static void generic_exec_single(int cpu,
 		csd_flag_wait(data);
 }
 
-static void rcu_free_call_data(struct rcu_head *head)
-{
-	struct call_function_data *data;
-
-	data = container_of(head, struct call_function_data, rcu_head);
-
-	kfree(data);
-}
-
 /*
  * Invoked by arch to handle an IPI for call function. Must be called with
  * interrupts disabled.
@@ -159,8 +146,6 @@ void generic_smp_call_function_interrupt
 			smp_wmb();
 			data->csd.flags &= ~CSD_FLAG_WAIT;
 		}
-		if (data->csd.flags & CSD_FLAG_ALLOC)
-			call_rcu(&data->rcu_head, rcu_free_call_data);
 
 		while (!list_empty(&free_list)) {
 			struct call_function_data *free;
@@ -225,8 +210,7 @@ void generic_smp_call_function_single_in
 			} else if (data_flags & CSD_FLAG_LOCK) {
 				smp_wmb();
 				data->flags &= ~CSD_FLAG_LOCK;
-			} else if (data_flags & CSD_FLAG_ALLOC)
-				kfree(data);
+			}
 		}
 		/*
 		 * See comment on outer loop
@@ -271,13 +255,11 @@ int smp_call_function_single(int cpu, vo
 			/*
 			 * We are calling a function on a single CPU
 			 * and we are not going to wait for it to finish.
-			 * We first try to allocate the data, but if we
-			 * fail, we fall back to use a per cpu data to pass
-			 * the information to that CPU. Since all callers
-			 * of this code will use the same data, we must
-			 * synchronize the callers to prevent a new caller
-			 * from corrupting the data before the callee
-			 * can access it.
+			 * We use a per cpu data to pass the information to
+			 * that CPU. Since all callers of this code will
+			 * use the same data, we must synchronize the
+			 * callers to prevent a new caller from corrupting
+			 * the data before the callee can access it.
 			 *
 			 * The CSD_FLAG_LOCK is used to let us know when
 			 * the IPI handler is done with the data.
@@ -287,15 +269,10 @@ int smp_call_function_single(int cpu, vo
 			 * will make sure the callee is done with the
 			 * data before a new caller will use it.
 			 */
-			data = kmalloc(sizeof(*data), GFP_ATOMIC);
-			if (data)
-				data->flags = CSD_FLAG_ALLOC;
-			else {
-				data = &per_cpu(csd_data, me);
-				while (data->flags & CSD_FLAG_LOCK)
-					cpu_relax();
-				data->flags = CSD_FLAG_LOCK;
-			}
+			data = &per_cpu(csd_data, me);
+			while (data->flags & CSD_FLAG_LOCK)
+				cpu_relax();
+			data->flags = CSD_FLAG_LOCK;
 		} else {
 			data = &d;
 			data->flags = CSD_FLAG_WAIT;
@@ -384,18 +361,13 @@ void smp_call_function_many(const struct
 		return;
 	}
 
-	data = kmalloc(sizeof(*data), GFP_ATOMIC);
-	if (data)
-		data->csd.flags = CSD_FLAG_ALLOC;
-	else {
-		data = &per_cpu(cfd_data, me);
-		/*
-		 * We need to wait for all previous users to go away.
-		 */
-		while (data->csd.flags & CSD_FLAG_LOCK)
-			cpu_relax();
-		data->csd.flags = CSD_FLAG_LOCK;
-	}
+	data = &per_cpu(cfd_data, me);
+	/*
+	 * We need to wait for all previous users to go away.
+	 */
+	while (data->csd.flags & CSD_FLAG_LOCK)
+		cpu_relax();
+	data->csd.flags = CSD_FLAG_LOCK;
 
 	spin_lock_init(&data->lock);
 	if (wait)

-- 


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

* [PATCH 3/4] generic-smp: properly allocate the cpumasks
  2009-02-16 16:38 [PATCH 0/4] generic smp helpers vs kmalloc Peter Zijlstra
  2009-02-16 16:38 ` [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many() Peter Zijlstra
  2009-02-16 16:38 ` [PATCH 2/4] generic-smp: remove kmalloc usage Peter Zijlstra
@ 2009-02-16 16:38 ` Peter Zijlstra
  2009-02-16 23:17   ` Rusty Russell
  2009-02-16 16:38 ` [PATCH 4/4] generic-smp: clean up some of the csd->flags fiddling Peter Zijlstra
  3 siblings, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 16:38 UTC (permalink / raw)
  To: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt
  Cc: linux-kernel, Oleg Nesterov, Peter Zijlstra

[-- Attachment #1: smp-proper-cpu-mask.patch --]
[-- Type: text/plain, Size: 3694 bytes --]

Its not deemed proper to have full cpumasks in data anymore, we have to
dynamically allocate them based on runtime determined dimensions of the
machine in question.

Signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
---
 kernel/smp.c |   54 +++++++++++++++++++++++++++++++++++++++++++++---------
 1 file changed, 45 insertions(+), 9 deletions(-)

Index: linux-2.6/kernel/smp.c
===================================================================
--- linux-2.6.orig/kernel/smp.c
+++ linux-2.6/kernel/smp.c
@@ -10,6 +10,7 @@
 #include <linux/rcupdate.h>
 #include <linux/rculist.h>
 #include <linux/smp.h>
+#include <linux/cpu.h>
 
 static DEFINE_PER_CPU(struct call_single_queue, call_single_queue);
 
@@ -40,7 +41,7 @@ struct call_function_data {
 	spinlock_t lock;
 	unsigned int refs;
 	struct list_head free_list;
-	struct cpumask cpumask;
+	cpumask_var_t cpumask;
 };
 
 struct call_single_queue {
@@ -48,8 +49,41 @@ struct call_single_queue {
 	spinlock_t lock;
 };
 
+static DEFINE_PER_CPU(struct call_function_data, cfd_data);
+
+static int
+hotplug_cfd(struct notifier_block *nfb, unsigned long action, void *hcpu)
+{
+	long cpu = (long)hcpu;
+	struct call_function_data *cfd = &per_cpu(cfd_data, cpu);
+
+	switch (action) {
+	case CPU_UP_PREPARE:
+	case CPU_UP_PREPARE_FROZEN:
+		if (!alloc_cpumask_var_node(&cfd->cpumask, GFP_KERNEL,
+				cpu_to_node(cpu)))
+			return NOTIFY_BAD;
+		break;
+
+	case CPU_UP_CANCELED:
+	case CPU_UP_CANCELED_FROZEN:
+
+	case CPU_DEAD:
+	case CPU_DEAD_FROZEN:
+		free_cpumask_var(cfd->cpumask);
+		break;
+	};
+
+	return NOTIFY_OK;
+}
+
+static struct notifier_block __cpuinitdata hotplug_cfd_notifier = {
+	.notifier_call = hotplug_cfd,
+};
+
 static int __cpuinit init_call_single_data(void)
 {
+	void *cpu = (void *)(long)smp_processor_id();
 	int i;
 
 	for_each_possible_cpu(i) {
@@ -58,6 +92,10 @@ static int __cpuinit init_call_single_da
 		spin_lock_init(&q->lock);
 		INIT_LIST_HEAD(&q->list);
 	}
+
+	hotplug_cfd(&hotplug_cfd_notifier, CPU_UP_PREPARE, cpu);
+	register_cpu_notifier(&hotplug_cfd_notifier);
+
 	return 0;
 }
 early_initcall(init_call_single_data);
@@ -116,13 +154,13 @@ void generic_smp_call_function_interrupt
 		LIST_HEAD(free_list);
 		int refs;
 
-		if (!cpumask_test_cpu(cpu, &data->cpumask))
+		if (!cpumask_test_cpu(cpu, data->cpumask))
 			continue;
 
 		data->csd.func(data->csd.info);
 
 		spin_lock(&data->lock);
-		cpumask_clear_cpu(cpu, &data->cpumask);
+		cpumask_clear_cpu(cpu, data->cpumask);
 		WARN_ON(data->refs == 0);
 		data->refs--;
 		refs = data->refs;
@@ -313,8 +351,6 @@ void __smp_call_function_single(int cpu,
 	arch_send_call_function_ipi(*(maskp))
 #endif
 
-static DEFINE_PER_CPU(struct call_function_data, cfd_data);
-
 /**
  * smp_call_function_many(): Run a function on a set of other CPUs.
  * @mask: The set of cpus to run on (only runs on online subset).
@@ -373,9 +409,9 @@ void smp_call_function_many(const struct
 		data->csd.flags |= CSD_FLAG_WAIT;
 	data->csd.func = func;
 	data->csd.info = info;
-	cpumask_and(&data->cpumask, mask, cpu_online_mask);
-	cpumask_clear_cpu(smp_processor_id(), &data->cpumask);
-	data->refs = cpumask_weight(&data->cpumask);
+	cpumask_and(data->cpumask, mask, cpu_online_mask);
+	cpumask_clear_cpu(smp_processor_id(), data->cpumask);
+	data->refs = cpumask_weight(data->cpumask);
 
 	spin_lock_irqsave(&call_function.lock, flags);
 	call_function.counter++;
@@ -388,7 +424,7 @@ void smp_call_function_many(const struct
 	smp_mb();
 
 	/* Send a message to all CPUs in the map */
-	arch_send_call_function_ipi_mask(&data->cpumask);
+	arch_send_call_function_ipi_mask(data->cpumask);
 
 	/* optionally wait for the CPUs to complete */
 	if (wait)

-- 


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

* [PATCH 4/4] generic-smp: clean up some of the csd->flags fiddling
  2009-02-16 16:38 [PATCH 0/4] generic smp helpers vs kmalloc Peter Zijlstra
                   ` (2 preceding siblings ...)
  2009-02-16 16:38 ` [PATCH 3/4] generic-smp: properly allocate the cpumasks Peter Zijlstra
@ 2009-02-16 16:38 ` Peter Zijlstra
  3 siblings, 0 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 16:38 UTC (permalink / raw)
  To: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt
  Cc: linux-kernel, Oleg Nesterov, Peter Zijlstra

[-- Attachment #1: smp-cleanup.patch --]
[-- Type: text/plain, Size: 4672 bytes --]

Break out the WAIT and LOCK bit operations into functions and provide some
extra comments.

Signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
---
 kernel/smp.c |  100 +++++++++++++++++++++++++++++++++++------------------------
 1 file changed, 60 insertions(+), 40 deletions(-)

Index: linux-2.6/kernel/smp.c
===================================================================
--- linux-2.6.orig/kernel/smp.c
+++ linux-2.6/kernel/smp.c
@@ -100,14 +100,50 @@ static int __cpuinit init_call_single_da
 }
 early_initcall(init_call_single_data);
 
-static void csd_flag_wait(struct call_single_data *data)
+/*
+ * csd_wait/csd_complete are used for synchronous ipi calls
+ */
+static void csd_wait_prepare(struct call_single_data *data)
+{
+	data->flags |= CSD_FLAG_WAIT;
+}
+
+static void csd_complete(struct call_single_data *data)
+{
+	/*
+	 * Serialize stores to data with the flag clear and wakeup.
+	 */
+	smp_wmb();
+	data->flags &= ~CSD_FLAG_WAIT;
+}
+
+static void csd_wait(struct call_single_data *data)
+{
+	while (data->flags & CSD_FLAG_WAIT)
+		cpu_relax();
+}
+
+/*
+ * csd_lock/csd_unlock used to serialize access to per-cpu csd resources
+ *
+ * For non-synchronous ipi calls the csd can still be in use by the previous
+ * function call. For multi-cpu calls its even more interesting as we'll have
+ * to ensure no other cpu is observing our csd.
+ */
+static void csd_lock(struct call_single_data *data)
 {
-	/* Wait for response */
-	do {
-		if (!(data->flags & CSD_FLAG_WAIT))
-			break;
+	while (data->flags & CSD_FLAG_LOCK)
 		cpu_relax();
-	} while (1);
+	data->flags = CSD_FLAG_LOCK;
+}
+
+static void csd_unlock(struct call_single_data *data)
+{
+	/*
+	 * Serialize stores to data with the flags clear.
+	 */
+	smp_wmb();
+	data->flags &= ~CSD_FLAG_LOCK;
 }
 
 /*
@@ -134,7 +170,7 @@ static void generic_exec_single(int cpu,
 		arch_send_call_function_single_ipi(cpu);
 
 	if (wait)
-		csd_flag_wait(data);
+		csd_wait(data);
 }
 
 /*
@@ -172,18 +208,17 @@ void generic_smp_call_function_interrupt
 		spin_lock(&call_function.lock);
 		list_add(&data->free_list, &call_function.free_list);
 		list_del_rcu(&data->csd.list);
+		/*
+		 * When the global queue is empty, its guaranteed that no cpu
+		 * is still observing any entry on the free_list, therefore
+		 * we can go ahead and unlock them.
+		 */
 		if (!--call_function.counter)
 			list_splice_init(&call_function.free_list, &free_list);
 		spin_unlock(&call_function.lock);
 
-		if (data->csd.flags & CSD_FLAG_WAIT) {
-			/*
-			 * serialize stores to data with the flag clear
-			 * and wakeup
-			 */
-			smp_wmb();
-			data->csd.flags &= ~CSD_FLAG_WAIT;
-		}
+		if (data->csd.flags & CSD_FLAG_WAIT)
+			csd_complete(&data->csd);
 
 		while (!list_empty(&free_list)) {
 			struct call_function_data *free;
@@ -192,12 +227,7 @@ void generic_smp_call_function_interrupt
 					struct call_function_data, free_list);
 
 			list_del(&data->free_list);
-			/*
-			 * serialize stores to data with the flags
-			 * clear
-			 */
-			smp_wmb();
-			free->csd.flags &= ~CSD_FLAG_LOCK;
+			csd_unlock(&free->csd);
 		}
 	}
 
@@ -242,13 +272,10 @@ void generic_smp_call_function_single_in
 
 			data->func(data->info);
 
-			if (data_flags & CSD_FLAG_WAIT) {
-				smp_wmb();
-				data->flags &= ~CSD_FLAG_WAIT;
-			} else if (data_flags & CSD_FLAG_LOCK) {
-				smp_wmb();
-				data->flags &= ~CSD_FLAG_LOCK;
-			}
+			if (data_flags & CSD_FLAG_WAIT)
+				csd_complete(data);
+			else if (data_flags & CSD_FLAG_LOCK)
+				csd_unlock(data);
 		}
 		/*
 		 * See comment on outer loop
@@ -308,12 +335,10 @@ int smp_call_function_single(int cpu, vo
 			 * data before a new caller will use it.
 			 */
 			data = &per_cpu(csd_data, me);
-			while (data->flags & CSD_FLAG_LOCK)
-				cpu_relax();
-			data->flags = CSD_FLAG_LOCK;
+			csd_lock(data);
 		} else {
 			data = &d;
-			data->flags = CSD_FLAG_WAIT;
+			csd_wait_prepare(data);
 		}
 
 		data->func = func;
@@ -398,16 +423,11 @@ void smp_call_function_many(const struct
 	}
 
 	data = &per_cpu(cfd_data, me);
-	/*
-	 * We need to wait for all previous users to go away.
-	 */
-	while (data->csd.flags & CSD_FLAG_LOCK)
-		cpu_relax();
-	data->csd.flags = CSD_FLAG_LOCK;
+	csd_lock(&data->csd);
 
 	spin_lock_init(&data->lock);
 	if (wait)
-		data->csd.flags |= CSD_FLAG_WAIT;
+		csd_wait_prepare(&data->csd);
 	data->csd.func = func;
 	data->csd.info = info;
 	cpumask_and(data->cpumask, mask, cpu_online_mask);
@@ -429,7 +449,7 @@ void smp_call_function_many(const struct
 
 	/* optionally wait for the CPUs to complete */
 	if (wait)
-		csd_flag_wait(&data->csd);
+		csd_wait(&data->csd);
 }
 EXPORT_SYMBOL(smp_call_function_many);
 

-- 


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

* Re: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()
  2009-02-16 16:38 ` [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many() Peter Zijlstra
@ 2009-02-16 19:10   ` Oleg Nesterov
  2009-02-16 19:41     ` Peter Zijlstra
  2009-02-17 12:25     ` Oleg Nesterov
  2009-02-16 20:49   ` Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()) Oleg Nesterov
  2009-02-17 15:40   ` [PATCH] generic-smp: remove kmalloc() Peter Zijlstra
  2 siblings, 2 replies; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-16 19:10 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/16, Peter Zijlstra wrote:
>
> @@ -347,31 +384,32 @@ void smp_call_function_many(const struct
>  		return;
>  	}
>
> -	data = kmalloc(sizeof(*data) + cpumask_size(), GFP_ATOMIC);
> -	if (unlikely(!data)) {
> -		/* Slow path. */
> -		for_each_online_cpu(cpu) {
> -			if (cpu == smp_processor_id())
> -				continue;
> -			if (cpumask_test_cpu(cpu, mask))
> -				smp_call_function_single(cpu, func, info, wait);
> -		}
> -		return;
> +	data = kmalloc(sizeof(*data), GFP_ATOMIC);
> +	if (data)
> +		data->csd.flags = CSD_FLAG_ALLOC;
> +	else {
> +		data = &per_cpu(cfd_data, me);
> +		/*
> +		 * We need to wait for all previous users to go away.
> +		 */
> +		while (data->csd.flags & CSD_FLAG_LOCK)
> +			cpu_relax();
> +		data->csd.flags = CSD_FLAG_LOCK;
>  	}
>
>  	spin_lock_init(&data->lock);
> -	data->csd.flags = CSD_FLAG_ALLOC;
>  	if (wait)
>  		data->csd.flags |= CSD_FLAG_WAIT;
>  	data->csd.func = func;
>  	data->csd.info = info;
> -	cpumask_and(to_cpumask(data->cpumask_bits), mask, cpu_online_mask);
> -	cpumask_clear_cpu(smp_processor_id(), to_cpumask(data->cpumask_bits));
> -	data->refs = cpumask_weight(to_cpumask(data->cpumask_bits));
> -
> -	spin_lock_irqsave(&call_function_lock, flags);
> -	list_add_tail_rcu(&data->csd.list, &call_function_queue);
> -	spin_unlock_irqrestore(&call_function_lock, flags);
> +	cpumask_and(&data->cpumask, mask, cpu_online_mask);
> +	cpumask_clear_cpu(smp_processor_id(), &data->cpumask);

(perhaps it makes sense to use "me" instead of smp_processor_id())

> +	data->refs = cpumask_weight(&data->cpumask);
> +
> +	spin_lock_irqsave(&call_function.lock, flags);
> +	call_function.counter++;
> +	list_add_tail_rcu(&data->csd.list, &call_function.queue);
> +	spin_unlock_irqrestore(&call_function.lock, flags);

What if the initialization above leaks into the critical section?

I mean, generic_smp_call_function_interrupt() running on another CPU
can see the result of list_add_tail_rcu() and cpumask_and(data->cpumask)
but not (say) "data->refs = ...".


Actually I don't understand the counter/free_list logic.

	generic_smp_call_function_interrupt:

			/*
			 * When the global queue is empty, its guaranteed that no cpu
			 * is still observing any entry on the free_list, therefore
			 * we can go ahead and unlock them.
			 */
			if (!--call_function.counter)
				list_splice_init(&call_function.free_list, &free_list);

I can't see why "its guaranteed that no cpu ...". Let's suppose CPU 0
"hangs" for some reason in generic_smp_call_function_interrupt() right
before "if (!cpumask_test_cpu(cpu, data->cpumask))" test. Then it is
possible that another CPU removes the single entry (which doesn't have
CPU 0 in data->cpumask) from call_function.queue.

Now, if that entry is re-added, we can have a problem if CPU 0 sees
the partly initialized entry.

But why do we need counter/free_list at all?
Can't we do the following:

	- initialize call_function_data.lock at boot time

	- change smp_call_function_many() to initialize cfd_data
	  under ->lock

	- change generic_smp_call_function_interrupt() to do

		list_for_each_entry_rcu(data) {

			if (!cpumask_test_cpu(cpu, data->cpumask))
				continue;

			spin_lock(&data->lock);
			if (!cpumask_test_cpu(cpu, data->cpumask)) {
				spin_unlock(data->lock);
				continue;
			}

			cpumask_clear_cpu(cpu, data->cpumask);
			refs = --data->refs;

			func = data->func;
			info = data->info;
			spin_unlock(&data->lock);

			func(info);

			if (refs)
				continue;

			...
		}

Afaics, it is OK if smp_call_function_many() sees the "unneeded" entry on
list, the only thing we must ensure is that we can't "misunderstand" this
entry.

No?


Off-topic question, looks like smp_call_function_single() must not be
called from interrupt/bh handler, right? But the comment says nothing.

And,
	smp_call_function_single:

		/* Can deadlock when called with interrupts disabled */
		WARN_ON(irqs_disabled());
	
Just curious, we can only deadlock if wait = T, right?

Oleg.


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

* Re: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()
  2009-02-16 19:10   ` Oleg Nesterov
@ 2009-02-16 19:41     ` Peter Zijlstra
  2009-02-16 20:30       ` Oleg Nesterov
  2009-02-17 12:25     ` Oleg Nesterov
  1 sibling, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 19:41 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Mon, 2009-02-16 at 20:10 +0100, Oleg Nesterov wrote:
> On 02/16, Peter Zijlstra wrote:
> >
> > @@ -347,31 +384,32 @@ void smp_call_function_many(const struct
> >  		return;
> >  	}
> >
> > -	data = kmalloc(sizeof(*data) + cpumask_size(), GFP_ATOMIC);
> > -	if (unlikely(!data)) {
> > -		/* Slow path. */
> > -		for_each_online_cpu(cpu) {
> > -			if (cpu == smp_processor_id())
> > -				continue;
> > -			if (cpumask_test_cpu(cpu, mask))
> > -				smp_call_function_single(cpu, func, info, wait);
> > -		}
> > -		return;
> > +	data = kmalloc(sizeof(*data), GFP_ATOMIC);
> > +	if (data)
> > +		data->csd.flags = CSD_FLAG_ALLOC;
> > +	else {
> > +		data = &per_cpu(cfd_data, me);
> > +		/*
> > +		 * We need to wait for all previous users to go away.
> > +		 */
> > +		while (data->csd.flags & CSD_FLAG_LOCK)
> > +			cpu_relax();
> > +		data->csd.flags = CSD_FLAG_LOCK;
> >  	}
> >
> >  	spin_lock_init(&data->lock);
> > -	data->csd.flags = CSD_FLAG_ALLOC;
> >  	if (wait)
> >  		data->csd.flags |= CSD_FLAG_WAIT;
> >  	data->csd.func = func;
> >  	data->csd.info = info;
> > -	cpumask_and(to_cpumask(data->cpumask_bits), mask, cpu_online_mask);
> > -	cpumask_clear_cpu(smp_processor_id(), to_cpumask(data->cpumask_bits));
> > -	data->refs = cpumask_weight(to_cpumask(data->cpumask_bits));
> > -
> > -	spin_lock_irqsave(&call_function_lock, flags);
> > -	list_add_tail_rcu(&data->csd.list, &call_function_queue);
> > -	spin_unlock_irqrestore(&call_function_lock, flags);
> > +	cpumask_and(&data->cpumask, mask, cpu_online_mask);
> > +	cpumask_clear_cpu(smp_processor_id(), &data->cpumask);
> 
> (perhaps it makes sense to use "me" instead of smp_processor_id())

Ah, missed one it seems, thanks ;-)

> > +	data->refs = cpumask_weight(&data->cpumask);
> > +
> > +	spin_lock_irqsave(&call_function.lock, flags);
> > +	call_function.counter++;
> > +	list_add_tail_rcu(&data->csd.list, &call_function.queue);
> > +	spin_unlock_irqrestore(&call_function.lock, flags);
> 
> What if the initialization above leaks into the critical section?
> 
> I mean, generic_smp_call_function_interrupt() running on another CPU
> can see the result of list_add_tail_rcu() and cpumask_and(data->cpumask)
> but not (say) "data->refs = ...".

Ho humm, nice :-)

So best would be to put that data initialization under data->lock. This
would be a bug in the original code too.

> Actually I don't understand the counter/free_list logic.
> 
> 	generic_smp_call_function_interrupt:
> 
> 			/*
> 			 * When the global queue is empty, its guaranteed that no cpu
> 			 * is still observing any entry on the free_list, therefore
> 			 * we can go ahead and unlock them.
> 			 */
> 			if (!--call_function.counter)
> 				list_splice_init(&call_function.free_list, &free_list);
> 
> I can't see why "its guaranteed that no cpu ...". Let's suppose CPU 0
> "hangs" for some reason in generic_smp_call_function_interrupt() right
> before "if (!cpumask_test_cpu(cpu, data->cpumask))" test. Then it is
> possible that another CPU removes the single entry (which doesn't have
> CPU 0 in data->cpumask) from call_function.queue.

Then call_function.counter wouldn't be 0, and we would not release the
list entries.

> Now, if that entry is re-added, we can have a problem if CPU 0 sees
> the partly initialized entry.

Right, so the scenario is that a cpu observes the list entry after we do
list_del_rcu() -- most likely a cpu not in the original mask, taversing
the list for a next entry -- we have to wait for some quiescent state to
ensure nobody is still referencing it.

We cannot use regular RCU, because its quiesent state takes forever to
happen, therefore this implementes a simple counting rcu for the queue
domain only.

When the list is empty, there's nobody seeing any elements, ergo we can
release the entries and re-use them.

> But why do we need counter/free_list at all?
> Can't we do the following:
> 
> 	- initialize call_function_data.lock at boot time
> 
> 	- change smp_call_function_many() to initialize cfd_data
> 	  under ->lock
> 
> 	- change generic_smp_call_function_interrupt() to do
> 
> 		list_for_each_entry_rcu(data) {
> 
> 			if (!cpumask_test_cpu(cpu, data->cpumask))
> 				continue;
> 
> 			spin_lock(&data->lock);
> 			if (!cpumask_test_cpu(cpu, data->cpumask)) {
> 				spin_unlock(data->lock);
> 				continue;
> 			}
> 
> 			cpumask_clear_cpu(cpu, data->cpumask);
> 			refs = --data->refs;
> 
> 			func = data->func;
> 			info = data->info;
> 			spin_unlock(&data->lock);
> 
> 			func(info);
> 
> 			if (refs)
> 				continue;
> 
> 			...
> 		}
> 
> Afaics, it is OK if smp_call_function_many() sees the "unneeded" entry on
> list, the only thing we must ensure is that we can't "misunderstand" this
> entry.
> 
> No?

Hmm, could that not leed to loops, and or missed entries in the
list-iteration?

The saves approach seemed to wait until sure nobody observed the entry
before re-using it.

> 
> Off-topic question, looks like smp_call_function_single() must not be
> called from interrupt/bh handler, right? But the comment says nothing.
> 
> And,
> 	smp_call_function_single:
> 
> 		/* Can deadlock when called with interrupts disabled */
> 		WARN_ON(irqs_disabled());
> 	
> Just curious, we can only deadlock if wait = T, right?

Right.


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

* Re: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()
  2009-02-16 19:41     ` Peter Zijlstra
@ 2009-02-16 20:30       ` Oleg Nesterov
  2009-02-16 20:55         ` Peter Zijlstra
  0 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-16 20:30 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/16, Peter Zijlstra wrote:
>
> On Mon, 2009-02-16 at 20:10 +0100, Oleg Nesterov wrote:
> > Actually I don't understand the counter/free_list logic.
> >
> > 	generic_smp_call_function_interrupt:
> >
> > 			/*
> > 			 * When the global queue is empty, its guaranteed that no cpu
> > 			 * is still observing any entry on the free_list, therefore
> > 			 * we can go ahead and unlock them.
> > 			 */
> > 			if (!--call_function.counter)
> > 				list_splice_init(&call_function.free_list, &free_list);
> >
> > I can't see why "its guaranteed that no cpu ...". Let's suppose CPU 0
> > "hangs" for some reason in generic_smp_call_function_interrupt() right
> > before "if (!cpumask_test_cpu(cpu, data->cpumask))" test. Then it is
> > possible that another CPU removes the single entry (which doesn't have
> > CPU 0 in data->cpumask) from call_function.queue.
>
> Then call_function.counter wouldn't be 0, and we would not release the
> list entries.

Why it wouldn't be 0? IOW, do you mean that the spurious CALL_FUNCTION_VECTOR
is not possible?

OK, let's suppose CPUs 1 and 2 both do smp_call_function_many(cpumask_of(0)).

CPU_1 does arch_send_call_function_ipi_mask() and returns.

CPU_2 inserts cfd_data[2] and hangs before arch_send_call_function_ipi_mask().

CPU_0 sees the interrupt and removes both entries.

CPU_2 proceeds, and sends IPI to CPU_0 again.

Now, when CPU_0 sees CALL_FUNCTION_VECTOR interrupt and calls
generic_smp_call_function_interrupt(), there is no work for this CPU,
so the above race is possible even with counter/free_list.

The new entry can be inserted (counter becomes 1) and quickly removed
(counter becomes 0) while CPU 0 still sees it on list.

Unless I misread the patch of course...

> > Now, if that entry is re-added, we can have a problem if CPU 0 sees
> > the partly initialized entry.
>
> Right, so the scenario is that a cpu observes the list entry after we do
> list_del_rcu() -- most likely a cpu not in the original mask, taversing
> the list for a next entry -- we have to wait for some quiescent state to
> ensure nobody is still referencing it.

Yes I see. But if we change generic_smp_call_function_interrupt() to
re-check cpumask_test_cpu(cpu, data->cpumask) under data->lock then
we don't have to wait for quiescent state, afaics. And we have to
take this lock anyway.

Because smp_call_function_many() does mb(), but I can't see how it
can help. Some CPU from ->cpumask can already execute
generic_smp_call_function_interrupt() before we do
arch_send_call_function_ipi_mask().


> > But why do we need counter/free_list at all?
> > Can't we do the following:
> >
> > 	- initialize call_function_data.lock at boot time
> >
> > 	- change smp_call_function_many() to initialize cfd_data
> > 	  under ->lock
> >
> > 	- change generic_smp_call_function_interrupt() to do
> >
> > 		list_for_each_entry_rcu(data) {
> >
> > 			if (!cpumask_test_cpu(cpu, data->cpumask))
> > 				continue;
> >
> > 			spin_lock(&data->lock);
> > 			if (!cpumask_test_cpu(cpu, data->cpumask)) {
> > 				spin_unlock(data->lock);
> > 				continue;
> > 			}
> >
> > 			cpumask_clear_cpu(cpu, data->cpumask);
> > 			refs = --data->refs;
> >
> > 			func = data->func;
> > 			info = data->info;
> > 			spin_unlock(&data->lock);
> >
> > 			func(info);
> >
> > 			if (refs)
> > 				continue;
> >
> > 			...
> > 		}
> >
> > Afaics, it is OK if smp_call_function_many() sees the "unneeded" entry on
> > list, the only thing we must ensure is that we can't "misunderstand" this
> > entry.
> >
> > No?
>
> Hmm, could that not leed to loops, and or missed entries in the
> list-iteration?

How? when we lock data->lock and see this cpu in the ->cpumask,
we can be sure we can proceed?

Oleg.


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

* Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-16 16:38 ` [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many() Peter Zijlstra
  2009-02-16 19:10   ` Oleg Nesterov
@ 2009-02-16 20:49   ` Oleg Nesterov
  2009-02-16 21:03     ` Peter Zijlstra
  2009-02-17 15:40   ` [PATCH] generic-smp: remove kmalloc() Peter Zijlstra
  2 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-16 20:49 UTC (permalink / raw)
  To: Jens Axboe, Peter Zijlstra, Suresh Siddha
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

I am trying to understand the barriers in smp.c, please help!

"generic-ipi: fix the smp_mb() placement" commit
561920a0d2bb6d63343e83acfd784c0a77bd28d1 added smp_read_barrier_depends()
to generic_smp_call_function_single_interrupt().

Why it is needed? The comment says:

	/*
	 * Need to see other stores to list head for checking whether
	 * list is empty without holding q->lock
	 */
	smp_read_barrier_depends();
	while (!list_empty(&q->list)) {

But we can't miss the addition to the call_single_queue.list,
if generic_exec_single() sees list_empty(&dst->list) it sends
another IPI?


This commit also removed the barrier from csd_flag_wait(), is this OK?
Without the barrier, csd_flag_wait() can return before we see the result
of data->func() ?

IOW,
	int VAR = 0;

	void func(coid *unused)
	{
		VAR = 1;
	}

Now,

	smp_call_function_single(0, func, NULL, 1);
	BUG_ON(VAR == 0);

afaics, the BUG_ON() above is possible. Is this OK ?

Oleg.


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

* Re: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()
  2009-02-16 20:30       ` Oleg Nesterov
@ 2009-02-16 20:55         ` Peter Zijlstra
  2009-02-16 21:22           ` Oleg Nesterov
  0 siblings, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 20:55 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Mon, 2009-02-16 at 21:30 +0100, Oleg Nesterov wrote:
> On 02/16, Peter Zijlstra wrote:
> >
> > On Mon, 2009-02-16 at 20:10 +0100, Oleg Nesterov wrote:
> > > Actually I don't understand the counter/free_list logic.
> > >
> > > 	generic_smp_call_function_interrupt:
> > >
> > > 			/*
> > > 			 * When the global queue is empty, its guaranteed that no cpu
> > > 			 * is still observing any entry on the free_list, therefore
> > > 			 * we can go ahead and unlock them.
> > > 			 */
> > > 			if (!--call_function.counter)
> > > 				list_splice_init(&call_function.free_list, &free_list);
> > >
> > > I can't see why "its guaranteed that no cpu ...". Let's suppose CPU 0
> > > "hangs" for some reason in generic_smp_call_function_interrupt() right
> > > before "if (!cpumask_test_cpu(cpu, data->cpumask))" test. Then it is
> > > possible that another CPU removes the single entry (which doesn't have
> > > CPU 0 in data->cpumask) from call_function.queue.
> >
> > Then call_function.counter wouldn't be 0, and we would not release the
> > list entries.
> 
> Why it wouldn't be 0? IOW, do you mean that the spurious CALL_FUNCTION_VECTOR
> is not possible?

Ah, I think I see what you mean, you're right. I should not be counting
the number of entries in the queue, for when the last one gets removed
does not correlate to another observing entries.

> > > Now, if that entry is re-added, we can have a problem if CPU 0 sees
> > > the partly initialized entry.
> >
> > Right, so the scenario is that a cpu observes the list entry after we do
> > list_del_rcu() -- most likely a cpu not in the original mask, taversing
> > the list for a next entry -- we have to wait for some quiescent state to
> > ensure nobody is still referencing it.
> 
> Yes I see. But if we change generic_smp_call_function_interrupt() to
> re-check cpumask_test_cpu(cpu, data->cpumask) under data->lock then
> we don't have to wait for quiescent state, afaics. And we have to
> take this lock anyway.

Suppose entries A,B,C,D are queued, and some cpu1 is traversing the list
to execute C and is currently at B waiting for data->lock. Concurrently
cpu2 has completed B and removes it from the list, when cpu3 takes B and
inserts it at the end.

So we end up with A->C->D->B, and cpu1 will find its at the end of the
list and stop, never having seen C, the one it was meant to execute.

One possible solution would be to always queue new entries at the head,
but that would be unfair wrt function execution -- do we care?


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-16 20:49   ` Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()) Oleg Nesterov
@ 2009-02-16 21:03     ` Peter Zijlstra
  2009-02-16 21:32       ` Oleg Nesterov
  0 siblings, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 21:03 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Jens Axboe, Suresh Siddha, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Mon, 2009-02-16 at 21:49 +0100, Oleg Nesterov wrote:
> I am trying to understand the barriers in smp.c, please help!
> 
> "generic-ipi: fix the smp_mb() placement" commit
> 561920a0d2bb6d63343e83acfd784c0a77bd28d1 added smp_read_barrier_depends()
> to generic_smp_call_function_single_interrupt().
> 
> Why it is needed? The comment says:
> 
> 	/*
> 	 * Need to see other stores to list head for checking whether
> 	 * list is empty without holding q->lock
> 	 */
> 	smp_read_barrier_depends();
> 	while (!list_empty(&q->list)) {
> 
> But we can't miss the addition to the call_single_queue.list,
> if generic_exec_single() sees list_empty(&dst->list) it sends
> another IPI?

I was about to write a response, but found it to be a justification for
the read_barrier_depends() at the end of the loop. 

> This commit also removed the barrier from csd_flag_wait(), is this OK?
> Without the barrier, csd_flag_wait() can return before we see the result
> of data->func() ?
> 
> IOW,
> 	int VAR = 0;
> 
> 	void func(coid *unused)
> 	{
> 		VAR = 1;
> 	}
> 
> Now,
> 
> 	smp_call_function_single(0, func, NULL, 1);
> 	BUG_ON(VAR == 0);
> 
> afaics, the BUG_ON() above is possible. Is this OK ?

Would it not be the caller's responsibility to provide the needed
serialization in this case?


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

* Re: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()
  2009-02-16 20:55         ` Peter Zijlstra
@ 2009-02-16 21:22           ` Oleg Nesterov
  0 siblings, 0 replies; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-16 21:22 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/16, Peter Zijlstra wrote:
>
> On Mon, 2009-02-16 at 21:30 +0100, Oleg Nesterov wrote:
> >
> > Yes I see. But if we change generic_smp_call_function_interrupt() to
> > re-check cpumask_test_cpu(cpu, data->cpumask) under data->lock then
> > we don't have to wait for quiescent state, afaics. And we have to
> > take this lock anyway.
>
> Suppose entries A,B,C,D are queued, and some cpu1 is traversing the list
> to execute C and is currently at B waiting for data->lock. Concurrently
> cpu2 has completed B and removes it from the list, when cpu3 takes B and
> inserts it at the end.

Ah indeed, I see. Thanks Peter.

Oleg.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-16 21:03     ` Peter Zijlstra
@ 2009-02-16 21:32       ` Oleg Nesterov
  2009-02-16 21:45         ` Peter Zijlstra
  0 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-16 21:32 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Jens Axboe, Suresh Siddha, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On 02/16, Peter Zijlstra wrote:
>
> On Mon, 2009-02-16 at 21:49 +0100, Oleg Nesterov wrote:
> > I am trying to understand the barriers in smp.c, please help!
> >
> > "generic-ipi: fix the smp_mb() placement" commit
> > 561920a0d2bb6d63343e83acfd784c0a77bd28d1 added smp_read_barrier_depends()
> > to generic_smp_call_function_single_interrupt().
> >
> > Why it is needed? The comment says:
> >
> > 	/*
> > 	 * Need to see other stores to list head for checking whether
> > 	 * list is empty without holding q->lock
> > 	 */
> > 	smp_read_barrier_depends();
> > 	while (!list_empty(&q->list)) {
> >
> > But we can't miss the addition to the call_single_queue.list,
> > if generic_exec_single() sees list_empty(&dst->list) it sends
> > another IPI?
>
> I was about to write a response, but found it to be a justification for
> the read_barrier_depends() at the end of the loop.

I forgot to mention I don't understand the read_barrier_depends() at the
end of the loop as well ;)

> > This commit also removed the barrier from csd_flag_wait(), is this OK?
> > Without the barrier, csd_flag_wait() can return before we see the result
> > of data->func() ?
> >
> > IOW,
> > 	int VAR = 0;
> >
> > 	void func(coid *unused)
> > 	{
> > 		VAR = 1;
> > 	}
> >
> > Now,
> >
> > 	smp_call_function_single(0, func, NULL, 1);
> > 	BUG_ON(VAR == 0);
> >
> > afaics, the BUG_ON() above is possible. Is this OK ?
>
> Would it not be the caller's responsibility to provide the needed
> serialization in this case?

Yes, yes, I don't claim this is necessary wrong. I am just asking,
is the lack of "implicit" serialization is "by design" or this was
an oversight.

Oleg.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-16 21:32       ` Oleg Nesterov
@ 2009-02-16 21:45         ` Peter Zijlstra
  2009-02-16 22:02           ` Oleg Nesterov
  0 siblings, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 21:45 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Jens Axboe, Suresh Siddha, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Mon, 2009-02-16 at 22:32 +0100, Oleg Nesterov wrote:
> > I was about to write a response, but found it to be a justification for
> > the read_barrier_depends() at the end of the loop.
> 
> I forgot to mention I don't understand the read_barrier_depends() at the
> end of the loop as well ;)

Suppose cpu0 adds to csd to cpu1:


 cpu0:                 cpu1:

add entry1
mb();
send ipi
                      run ipi handler
                      read_barrier_depends()
                      while (!list_empty())    [A]
                        do foo

add entry2
mb();
[no ipi -- we still observe entry1]

                        remove foo
                        read_barrier_depends()
                      while (!list_empty())      [B]


The read_barrier_depends() matches the mb() on the other cpu, without
which the 'new' entry might not be observed.

So it turns out the initial one is needed as well.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-16 21:45         ` Peter Zijlstra
@ 2009-02-16 22:02           ` Oleg Nesterov
  2009-02-16 22:24             ` Peter Zijlstra
  0 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-16 22:02 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Jens Axboe, Suresh Siddha, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On 02/16, Peter Zijlstra wrote:
>
> On Mon, 2009-02-16 at 22:32 +0100, Oleg Nesterov wrote:
> > > I was about to write a response, but found it to be a justification for
> > > the read_barrier_depends() at the end of the loop.
> >
> > I forgot to mention I don't understand the read_barrier_depends() at the
> > end of the loop as well ;)
>
> Suppose cpu0 adds to csd to cpu1:
>
>
>  cpu0:                 cpu1:
>
> add entry1
> mb();
> send ipi
>                       run ipi handler
>                       read_barrier_depends()
>                       while (!list_empty())    [A]
>                         do foo
>
> add entry2
> mb();
> [no ipi -- we still observe entry1]
>
>                         remove foo
>                         read_barrier_depends()
>                       while (!list_empty())      [B]

Still can't understand.

cpu1 (generic_smp_call_function_single_interrupt) does
list_replace_init(q->lock), this lock is also taken by
generic_exec_single().

Either cpu1 sees entry2 on list, or cpu0 sees list_empty()
and sends ipi.

> The read_barrier_depends() matches the mb() on the other cpu, without
> which the 'new' entry might not be observed.

And that mb() looks unneeded too. Again, because
generic_smp_call_function_single_interrupt() takes call_single_queue.lock
before it uses "data".


Even if I missed something (very possible), then I can't understand
why we need rmb() only on alpha.

Oleg.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-16 22:02           ` Oleg Nesterov
@ 2009-02-16 22:24             ` Peter Zijlstra
  2009-02-16 23:19               ` Oleg Nesterov
  0 siblings, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-16 22:24 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Jens Axboe, Suresh Siddha, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Mon, 2009-02-16 at 23:02 +0100, Oleg Nesterov wrote:
> On 02/16, Peter Zijlstra wrote:
> >
> > On Mon, 2009-02-16 at 22:32 +0100, Oleg Nesterov wrote:
> > > > I was about to write a response, but found it to be a justification for
> > > > the read_barrier_depends() at the end of the loop.
> > >
> > > I forgot to mention I don't understand the read_barrier_depends() at the
> > > end of the loop as well ;)
> >
> > Suppose cpu0 adds to csd to cpu1:
> >
> >
> >  cpu0:                 cpu1:
> >
> > add entry1
> > mb();
> > send ipi
> >                       run ipi handler
> >                       read_barrier_depends()
> >                       while (!list_empty())    [A]
> >                         do foo
> >
> > add entry2
> > mb();
> > [no ipi -- we still observe entry1]
> >
> >                         remove foo
> >                         read_barrier_depends()
> >                       while (!list_empty())      [B]
> 
> Still can't understand.
> 
> cpu1 (generic_smp_call_function_single_interrupt) does
> list_replace_init(q->lock), this lock is also taken by
> generic_exec_single().
> 
> Either cpu1 sees entry2 on list, or cpu0 sees list_empty()
> and sends ipi.

cpu0:		cpu1:

spin_lock_irqsave(&dst->lock, flags);
ipi = list_empty(&dst->list);
list_add_tail(&data->list, &dst->list);
spin_unlock_irqrestore(&dst->lock, flags);

ipi ----->

		while (!list_empty(&q->list)) {
                	unsigned int data_flags;

                	spin_lock(&q->lock);
               		list_replace_init(&q->list, &list);
	                spin_unlock(&q->lock);


Strictly speaking the unlock() is semi-permeable, allowing the read of
q->list to enter the critical section, allowing us to observe an empty
list, never getting to q->lock on cpu1.

The mb()/rbd() pair seems to avoid that.

> > The read_barrier_depends() matches the mb() on the other cpu, without
> > which the 'new' entry might not be observed.
> 
> And that mb() looks unneeded too. Again, because
> generic_smp_call_function_single_interrupt() takes call_single_queue.lock
> before it uses "data".
> 
> 
> Even if I missed something (very possible), then I can't understand
> why we need rmb() only on alpha.

Because only alpha is insane enough to do speculative reads? Dunno
really :-)


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

* Re: [PATCH 3/4] generic-smp: properly allocate the cpumasks
  2009-02-16 16:38 ` [PATCH 3/4] generic-smp: properly allocate the cpumasks Peter Zijlstra
@ 2009-02-16 23:17   ` Rusty Russell
  0 siblings, 0 replies; 103+ messages in thread
From: Rusty Russell @ 2009-02-16 23:17 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Steven Rostedt, linux-kernel, Oleg Nesterov

On Tuesday 17 February 2009 03:08:50 Peter Zijlstra wrote:
> Its not deemed proper to have full cpumasks in data anymore, we have to
> dynamically allocate them based on runtime determined dimensions of the
> machine in question.

Wow, thanks.

But FYI I would have been happy with allocating them for each possible CPU,
rather than doing the hotplug callback thing.

If you really *can* have 4096 CPUS, I'm not so upset about spending 2M; it's
when you've got 2 CPUs and the distro has set CONFIG_NR_CPUS=4096 that I
care about.

I'm not saying to fix it, I'm just saying you went beyond the call of duty.

Kudos,
Rusty.

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-16 22:24             ` Peter Zijlstra
@ 2009-02-16 23:19               ` Oleg Nesterov
  2009-02-17  9:29                 ` Peter Zijlstra
  0 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-16 23:19 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Jens Axboe, Suresh Siddha, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On 02/16, Peter Zijlstra wrote:
>
> On Mon, 2009-02-16 at 23:02 +0100, Oleg Nesterov wrote:
> > On 02/16, Peter Zijlstra wrote:
> > >
> > > On Mon, 2009-02-16 at 22:32 +0100, Oleg Nesterov wrote:
> > > > > I was about to write a response, but found it to be a justification for
> > > > > the read_barrier_depends() at the end of the loop.
> > > >
> > > > I forgot to mention I don't understand the read_barrier_depends() at the
> > > > end of the loop as well ;)
> > >
> > > Suppose cpu0 adds to csd to cpu1:
> > >
> > >
> > >  cpu0:                 cpu1:
> > >
> > > add entry1
> > > mb();
> > > send ipi
> > >                       run ipi handler
> > >                       read_barrier_depends()
> > >                       while (!list_empty())    [A]
> > >                         do foo
> > >
> > > add entry2
> > > mb();
> > > [no ipi -- we still observe entry1]
> > >
> > >                         remove foo
> > >                         read_barrier_depends()
> > >                       while (!list_empty())      [B]
> >
> > Still can't understand.
> >
> > cpu1 (generic_smp_call_function_single_interrupt) does
> > list_replace_init(q->lock), this lock is also taken by
> > generic_exec_single().
> >
> > Either cpu1 sees entry2 on list, or cpu0 sees list_empty()
> > and sends ipi.
>
> cpu0:		cpu1:
>
> spin_lock_irqsave(&dst->lock, flags);
> ipi = list_empty(&dst->list);
> list_add_tail(&data->list, &dst->list);
> spin_unlock_irqrestore(&dst->lock, flags);
>
> ipi ----->
>
> 		while (!list_empty(&q->list)) {
>                 	unsigned int data_flags;
>
>                 	spin_lock(&q->lock);
>                		list_replace_init(&q->list, &list);
> 	                spin_unlock(&q->lock);
>
>
> Strictly speaking the unlock() is semi-permeable, allowing the read of
> q->list to enter the critical section, allowing us to observe an empty
> list, never getting to q->lock on cpu1.

Hmm. If we take &q->lock, then we alread saw !list_empty() ?

And the question is, how can we miss list_empty() == F before spin_lock().

> > > The read_barrier_depends() matches the mb() on the other cpu, without
> > > which the 'new' entry might not be observed.
> >
> > And that mb() looks unneeded too. Again, because
> > generic_smp_call_function_single_interrupt() takes call_single_queue.lock
> > before it uses "data".

to clarify, I meant it is not needed unless we are going to send the IPI.
IOW, I think we can do

	if (ipi) {
		/* Make the list addition visible before sending the ipi. */
		wmb();
		arch_send_call_function_single_ipi(cpu);
	}

> > Even if I missed something (very possible), then I can't understand
> > why we need rmb() only on alpha.
>
> Because only alpha is insane enough to do speculative reads? Dunno
> really :-)

Perhaps...

It would be nice to have a comment which explains how can we miss the
first addition without read_barrier_depends(). And why only on alpha.

And arch/alpha/kernel/smp.c:handle_ipi() does mb() itself...

Confused.

Oleg.


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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-16 16:38 ` [PATCH 2/4] generic-smp: remove kmalloc usage Peter Zijlstra
@ 2009-02-17  0:40   ` Linus Torvalds
  2009-02-17  8:24     ` Peter Zijlstra
  0 siblings, 1 reply; 103+ messages in thread
From: Linus Torvalds @ 2009-02-17  0:40 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Nick Piggin, Jens Axboe, Paul E. McKenney, Ingo Molnar,
	Rusty Russell, Steven Rostedt, linux-kernel, Oleg Nesterov



On Mon, 16 Feb 2009, Peter Zijlstra wrote:
>
> Now that there is no strict need for kmalloc anymore, and nobody seems to
> rely it for the queueing behaviour, remove it.

Peter, I really hate this series.

Why?

In 1/4 you introduce that cfd RCU thing, and then in 2/4 you remove it 
again.

I realize that you seem to do that in order to do some incremental 
step-wise changes, but quite frankly, it just complicates the whole series 
and makes the patches much harder to read and follow.

Why don't you just combine patches 1&2? That split-up seems to just 
confuse things. At least it confuses me. Why does it happen?

		Linus

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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-17  0:40   ` Linus Torvalds
@ 2009-02-17  8:24     ` Peter Zijlstra
  2009-02-17  9:43       ` Ingo Molnar
  2009-02-17 15:44       ` Linus Torvalds
  0 siblings, 2 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17  8:24 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Nick Piggin, Jens Axboe, Paul E. McKenney, Ingo Molnar,
	Rusty Russell, Steven Rostedt, linux-kernel, Oleg Nesterov

On Mon, 2009-02-16 at 16:40 -0800, Linus Torvalds wrote:
> 
> On Mon, 16 Feb 2009, Peter Zijlstra wrote:
> >
> > Now that there is no strict need for kmalloc anymore, and nobody seems to
> > rely it for the queueing behaviour, remove it.
> 
> Peter, I really hate this series.
> 
> Why?
> 
> In 1/4 you introduce that cfd RCU thing, and then in 2/4 you remove it 
> again.

Ah, no, I don't actually. I remove the kmalloc+call_rcu stuff in 2, not
the newly cfd mini rcu thing.

> I realize that you seem to do that in order to do some incremental 
> step-wise changes, but quite frankly, it just complicates the whole series 
> and makes the patches much harder to read and follow.
> 
> Why don't you just combine patches 1&2? That split-up seems to just 
> confuse things. At least it confuses me. Why does it happen?

The idea was to remove the necessity for kmalloc() in patch 1, and then
remove kmalloc() in patch 2.

If you prefer I can fold them, no problem.

But as you might have seen, Oleg has been punching holes in my #1, so I
guess I'm back to the drawing board no matter what :-)


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-16 23:19               ` Oleg Nesterov
@ 2009-02-17  9:29                 ` Peter Zijlstra
  2009-02-17 10:11                   ` Nick Piggin
                                     ` (2 more replies)
  0 siblings, 3 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17  9:29 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Jens Axboe, Suresh Siddha, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Tue, 2009-02-17 at 00:19 +0100, Oleg Nesterov wrote:
> On 02/16, Peter Zijlstra wrote:
> >
> > On Mon, 2009-02-16 at 23:02 +0100, Oleg Nesterov wrote:
> > > On 02/16, Peter Zijlstra wrote:
> > > >
> > > > On Mon, 2009-02-16 at 22:32 +0100, Oleg Nesterov wrote:
> > > > > > I was about to write a response, but found it to be a justification for
> > > > > > the read_barrier_depends() at the end of the loop.
> > > > >
> > > > > I forgot to mention I don't understand the read_barrier_depends() at the
> > > > > end of the loop as well ;)
> > > >
> > > > Suppose cpu0 adds to csd to cpu1:
> > > >
> > > >
> > > >  cpu0:                 cpu1:
> > > >
> > > > add entry1
> > > > mb();
> > > > send ipi
> > > >                       run ipi handler
> > > >                       read_barrier_depends()
> > > >                       while (!list_empty())    [A]
> > > >                         do foo
> > > >
> > > > add entry2
> > > > mb();
> > > > [no ipi -- we still observe entry1]
> > > >
> > > >                         remove foo
> > > >                         read_barrier_depends()
> > > >                       while (!list_empty())      [B]
> > >
> > > Still can't understand.
> > >
> > > cpu1 (generic_smp_call_function_single_interrupt) does
> > > list_replace_init(q->lock), this lock is also taken by
> > > generic_exec_single().
> > >
> > > Either cpu1 sees entry2 on list, or cpu0 sees list_empty()
> > > and sends ipi.
> >
> > cpu0:		cpu1:
> >
> > spin_lock_irqsave(&dst->lock, flags);
> > ipi = list_empty(&dst->list);
> > list_add_tail(&data->list, &dst->list);
> > spin_unlock_irqrestore(&dst->lock, flags);
> >
> > ipi ----->
> >
> > 		while (!list_empty(&q->list)) {
> >                 	unsigned int data_flags;
> >
> >                 	spin_lock(&q->lock);
> >                		list_replace_init(&q->list, &list);
> > 	                spin_unlock(&q->lock);
> >
> >
> > Strictly speaking the unlock() is semi-permeable, allowing the read of
> > q->list to enter the critical section, allowing us to observe an empty
> > list, never getting to q->lock on cpu1.
> 
> Hmm. If we take &q->lock, then we alread saw !list_empty() ?

That's how I read the above code.

> And the question is, how can we miss list_empty() == F before spin_lock().

Confusion... my explanation above covers exactly this case. The reads
determining list_empty() can slip into the q->lock section on the other
cpu, and observe an empty list.

> > > Even if I missed something (very possible), then I can't understand
> > > why we need rmb() only on alpha.
> >
> > Because only alpha is insane enough to do speculative reads? Dunno
> > really :-)
> 
> Perhaps...
> 
> It would be nice to have a comment which explains how can we miss the
> first addition without read_barrier_depends(). And why only on alpha.

Paul, care to once again enlighten us? The best I can remember is that
alpha has split caches, and the rmb is needed for them to become
coherent -- no other arch is crazy in exactly that way.

But note that read_barrier_depends() is not quite a NOP for !alpha, it
does that ACCESS_ONCE() thing, which very much makes a difference, even
on x86.

> And arch/alpha/kernel/smp.c:handle_ipi() does mb() itself...

Right, but arguing by our memory model, we cannot assume that.


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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-17  8:24     ` Peter Zijlstra
@ 2009-02-17  9:43       ` Ingo Molnar
  2009-02-17  9:49         ` Peter Zijlstra
  2009-02-18  4:50         ` Rusty Russell
  2009-02-17 15:44       ` Linus Torvalds
  1 sibling, 2 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-17  9:43 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Rusty Russell, Steven Rostedt, linux-kernel, Oleg Nesterov


* Peter Zijlstra <a.p.zijlstra@chello.nl> wrote:

> On Mon, 2009-02-16 at 16:40 -0800, Linus Torvalds wrote:
> > 
> > On Mon, 16 Feb 2009, Peter Zijlstra wrote:
> > >
> > > Now that there is no strict need for kmalloc anymore, and nobody seems to
> > > rely it for the queueing behaviour, remove it.
> > 
> > Peter, I really hate this series.
> > 
> > Why?
> > 
> > In 1/4 you introduce that cfd RCU thing, and then in 2/4 you remove it 
> > again.
> 
> Ah, no, I don't actually. I remove the kmalloc+call_rcu stuff in 2, not
> the newly cfd mini rcu thing.
> 
> > I realize that you seem to do that in order to do some incremental 
> > step-wise changes, but quite frankly, it just complicates the whole series 
> > and makes the patches much harder to read and follow.
> > 
> > Why don't you just combine patches 1&2? That split-up seems to just 
> > confuse things. At least it confuses me. Why does it happen?
> 
> The idea was to remove the necessity for kmalloc() in patch 1, 
> and then remove kmalloc() in patch 2.
> 
> If you prefer I can fold them, no problem.
> 
> But as you might have seen, Oleg has been punching holes in my 
> #1, so I guess I'm back to the drawing board no matter what 
> :-)

I think the kmalloc() is clearly ugly, we should remove it, and 
if someone wants to add it we want to see _hard numbers_ that 
it's worth the ugliness. I.e. lets go with the two patches i 
posted, they are obvious and tested.

We should not bend backwards trying to preserve that kmalloc() 
[and prove that it's safe and race-free] - i.e. the burden of 
proof is on the person insisting that it's needed, not on the 
person wanting to remove it.

	Ingo

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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-17  9:43       ` Ingo Molnar
@ 2009-02-17  9:49         ` Peter Zijlstra
  2009-02-17 10:56           ` Ingo Molnar
  2009-02-18  4:50         ` Rusty Russell
  1 sibling, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17  9:49 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Rusty Russell, Steven Rostedt, linux-kernel, Oleg Nesterov

On Tue, 2009-02-17 at 10:43 +0100, Ingo Molnar wrote:
> 
> We should not bend backwards trying to preserve that kmalloc() 
> [and prove that it's safe and race-free] - i.e. the burden of 
> proof is on the person insisting that it's needed, not on the 
> person wanting to remove it.

I'm not getting it, I am removing it. The only thing I want to preserve
is the multi-cast ipi and not be forced into the multiple single-ipi
path.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17  9:29                 ` Peter Zijlstra
@ 2009-02-17 10:11                   ` Nick Piggin
  2009-02-17 10:27                     ` Peter Zijlstra
  2009-02-17 12:40                   ` Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()) Peter Zijlstra
  2009-02-17 15:43                   ` Paul E. McKenney
  2 siblings, 1 reply; 103+ messages in thread
From: Nick Piggin @ 2009-02-17 10:11 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Oleg Nesterov, Jens Axboe, Suresh Siddha, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Tue, Feb 17, 2009 at 10:29:34AM +0100, Peter Zijlstra wrote:
> On Tue, 2009-02-17 at 00:19 +0100, Oleg Nesterov wrote:
> > > cpu0:		cpu1:
> > >
> > > spin_lock_irqsave(&dst->lock, flags);
> > > ipi = list_empty(&dst->list);
> > > list_add_tail(&data->list, &dst->list);
> > > spin_unlock_irqrestore(&dst->lock, flags);
> > >
> > > ipi ----->
> > >
> > > 		while (!list_empty(&q->list)) {
> > >                 	unsigned int data_flags;
> > >
> > >                 	spin_lock(&q->lock);
> > >                		list_replace_init(&q->list, &list);
> > > 	                spin_unlock(&q->lock);
> > >
> > >
> > > Strictly speaking the unlock() is semi-permeable, allowing the read of
> > > q->list to enter the critical section, allowing us to observe an empty
> > > list, never getting to q->lock on cpu1.
> > 
> > Hmm. If we take &q->lock, then we alread saw !list_empty() ?
> 
> That's how I read the above code.
> 
> > And the question is, how can we miss list_empty() == F before spin_lock().
> 
> Confusion... my explanation above covers exactly this case. The reads
> determining list_empty() can slip into the q->lock section on the other
> cpu, and observe an empty list.

But in that case, cpu0 should see list_empty and send another IPI,
because our load of list_empty has moved before the unlock of the
lock, so there can't be another item concurrently put on the list.

But hmm, why even bother with all this complexity? Why not just
remove the outer loop completely? Do the lock and the list_replace_init
unconditionally. It would turn tricky lockless code into simple locked
code... we've already taken an interrupt anyway, so chances are pretty
high that we have work here to do, right?


> > > > Even if I missed something (very possible), then I can't understand
> > > > why we need rmb() only on alpha.
> > >
> > > Because only alpha is insane enough to do speculative reads? Dunno
> > > really :-)
> > 
> > Perhaps...
> > 
> > It would be nice to have a comment which explains how can we miss the
> > first addition without read_barrier_depends(). And why only on alpha.
> 
> Paul, care to once again enlighten us? The best I can remember is that
> alpha has split caches, and the rmb is needed for them to become
> coherent -- no other arch is crazy in exactly that way.

Other architectures can do speculative reads, and several (sparc, ia64,
powerpc AFAIK) do have release barrier semantics for their unlocks so
they could equally have loads passing spin_unlock.

Alpha has split cache I guess processing external cache coherency requests
independently. So it can see a pair of ordered stores coming from one
CPU (or CPUs if you want to get tricky I guess) in the wrong order.
This is not because of the loads being executed speculatively out of
order, although the end result is similar. The big difference why it
isn't hidden behind a regular smp_rmb() is because no CPU supported
by Linux does speculative loads over data dependencies, so we only
define smp_rmb to order loads that have no dependencies or only control
dependencies.

But you probably don't have to care about caches if it makes reasoning
easier. For all purposes it can be treated as Alpha speculatively
executing data dependent loads out of order I think eg.

x = *ptr;
y = array[x];

Then the CPU misses cache on the first load and thinks hmm, every
time I have loaded this it has been 10, so I'll do the 2nd load with
that value and retry if it turns out the prediction was wrong when
the 1st load completes. In the case that the 2nd load completes
first, and the prediction of the 1st load was correct, then the loads
were executed out of order... equivalent to out of order speculative
loads with control deps I guess.

But what actually happens on Alpha is not speculation but simply
the stores get seen in the wrong order.



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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 10:11                   ` Nick Piggin
@ 2009-02-17 10:27                     ` Peter Zijlstra
  2009-02-17 10:39                       ` Nick Piggin
  2009-02-17 11:26                       ` Nick Piggin
  0 siblings, 2 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 10:27 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Oleg Nesterov, Jens Axboe, Suresh Siddha, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Tue, 2009-02-17 at 11:11 +0100, Nick Piggin wrote:

> But in that case, cpu0 should see list_empty and send another IPI,
> because our load of list_empty has moved before the unlock of the
> lock, so there can't be another item concurrently put on the list.

Suppose a first smp_call_function_single()

So cpu0 does:

 spin_lock(dst->lock);
 ipi = list_empty(dst->list);
 list_add_tail(data->list, dst->list);
 spin_unlock(dst->lock);

 if (ipi) /* true */
   send_single_ipi(cpu);

then cpu1 does:

 while (!list_empty(q->list))

and observes no entries, quits the ipi handler, and stuff is stuck.

cpu0 will observe a non-empty queue and will not raise another ipi, cpu1
got the ipi, but observed no work and hence will not remove it.

> But hmm, why even bother with all this complexity? Why not just
> remove the outer loop completely? Do the lock and the list_replace_init
> unconditionally. It would turn tricky lockless code into simple locked
> code... we've already taken an interrupt anyway, so chances are pretty
> high that we have work here to do, right?

Well, that's a practical suggestion, and I agree.

It was just fun arguing with Oleg ;-)


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 10:27                     ` Peter Zijlstra
@ 2009-02-17 10:39                       ` Nick Piggin
  2009-02-17 11:26                       ` Nick Piggin
  1 sibling, 0 replies; 103+ messages in thread
From: Nick Piggin @ 2009-02-17 10:39 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Oleg Nesterov, Jens Axboe, Suresh Siddha, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Tue, Feb 17, 2009 at 11:27:33AM +0100, Peter Zijlstra wrote:
> On Tue, 2009-02-17 at 11:11 +0100, Nick Piggin wrote:
> 
> > But in that case, cpu0 should see list_empty and send another IPI,
> > because our load of list_empty has moved before the unlock of the
> > lock, so there can't be another item concurrently put on the list.
> 
> Suppose a first smp_call_function_single()
> 
> So cpu0 does:
> 
>  spin_lock(dst->lock);
>  ipi = list_empty(dst->list);
>  list_add_tail(data->list, dst->list);
>  spin_unlock(dst->lock);
> 
>  if (ipi) /* true */
>    send_single_ipi(cpu);
> 
> then cpu1 does:
> 
>  while (!list_empty(q->list))
> 
> and observes no entries, quits the ipi handler, and stuff is stuck.
> 
> cpu0 will observe a non-empty queue and will not raise another ipi, cpu1
> got the ipi, but observed no work and hence will not remove it.

Yes that's a valid case, but different from your one of the load
passing the spin_unlock inside the loop.

This one is interesting because we (the generic code) don't actually
quite know what we're ordering against. An IPI in some architecture
certainly could pass the cache coherency stream. But if that were the
case, then we have no generic primitives to handle it so I think it is
better to be enforced in arch code. Ie. cpu1 should always evaluate
to true in generic code with no additional barriers (and assuming that
a previous running IPI handler on cpu1 hasn't cleaned the list earlier).


> > But hmm, why even bother with all this complexity? Why not just
> > remove the outer loop completely? Do the lock and the list_replace_init
> > unconditionally. It would turn tricky lockless code into simple locked
> > code... we've already taken an interrupt anyway, so chances are pretty
> > high that we have work here to do, right?
> 
> Well, that's a practical suggestion, and I agree.
> 
> It was just fun arguing with Oleg ;-)

No arguments there ;)

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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-17  9:49         ` Peter Zijlstra
@ 2009-02-17 10:56           ` Ingo Molnar
  0 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-17 10:56 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Rusty Russell, Steven Rostedt, linux-kernel, Oleg Nesterov


* Peter Zijlstra <a.p.zijlstra@chello.nl> wrote:

> On Tue, 2009-02-17 at 10:43 +0100, Ingo Molnar wrote:
> > 
> > We should not bend backwards trying to preserve that kmalloc() 
> > [and prove that it's safe and race-free] - i.e. the burden of 
> > proof is on the person insisting that it's needed, not on the 
> > person wanting to remove it.
> 
> I'm not getting it, I am removing it. The only thing I want to 
> preserve is the multi-cast ipi and not be forced into the 
> multiple single-ipi path.

i know, but i mean we are trying to achieve something hard that 
came together with the kmalloc() in essence.

So instead of trying to preserve that, how about removing it the 
obvious way and then adding back a more complex multi-cast IPI 
code?

That way we'll have a safe point to fall back to, should Oleg 
poke more holes into it, down the line ;-)

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 10:27                     ` Peter Zijlstra
  2009-02-17 10:39                       ` Nick Piggin
@ 2009-02-17 11:26                       ` Nick Piggin
  2009-02-17 11:48                         ` Peter Zijlstra
                                           ` (3 more replies)
  1 sibling, 4 replies; 103+ messages in thread
From: Nick Piggin @ 2009-02-17 11:26 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Oleg Nesterov, Jens Axboe, Suresh Siddha, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

cc linux-arch

On Tue, Feb 17, 2009 at 11:27:33AM +0100, Peter Zijlstra wrote:
> > But hmm, why even bother with all this complexity? Why not just
> > remove the outer loop completely? Do the lock and the list_replace_init
> > unconditionally. It would turn tricky lockless code into simple locked
> > code... we've already taken an interrupt anyway, so chances are pretty
> > high that we have work here to do, right?
> 
> Well, that's a practical suggestion, and I agree.

How's this?

--
Simplify the barriers in generic remote function call interrupt code.

Firstly, just unconditionally take the lock and check the list in the
generic_call_function_single_interrupt IPI handler. As we've just taken
an IPI here, the chances are fairly high that there will be work on the
list for us, so do the locking unconditionally. This removes the tricky
lockless list_empty check and dubious barriers. The change looks bigger
than it is because it is just removing an outer loop.

Secondly, clarify architecture specific IPI locking rules. Generic code
has no tools to impose any sane ordering on IPIs if they go outside
normal cache coherency, ergo the arch code must make them appear to
obey cache coherency as a "memory operation" to initiate an IPI, and
a "memory operation" to receive one. This way at least they can be
reasoned about in generic code, and smp_mb used to provide ordering. 

The combination of these two changes means that explict barriers can
be taken out of queue handling for the single case -- shared data is
explicitly locked, and ipi ordering must conform to that, so no
barriers needed. An extra barrier is needed in the many handler, so
as to ensure we load the list element after the IPI is received.

Does any architecture actually needs barriers? For the initiator I
could see it, but for the handler I would be surprised. The other
thing we could do for simplicity is just to require that a full
barrier is required before generating an IPI, and after receiving an
IPI. We can't just do that in generic code without auditing
architectures. There have been subtle hangs here on some archs in
the past.

Signed-off-by: Nick Piggin <npiggin@suse.de>

---
 kernel/smp.c |   83 +++++++++++++++++++++++++++++++----------------------------
 1 file changed, 44 insertions(+), 39 deletions(-)

Index: linux-2.6/kernel/smp.c
===================================================================
--- linux-2.6.orig/kernel/smp.c
+++ linux-2.6/kernel/smp.c
@@ -74,9 +74,16 @@ static void generic_exec_single(int cpu,
 	spin_unlock_irqrestore(&dst->lock, flags);
 
 	/*
-	 * Make the list addition visible before sending the ipi.
+	 * The list addition should be visible before sending the IPI
+	 * handler locks the list to pull the entry off it because of
+	 * normal cache coherency rules implied by spinlocks.
+	 *
+	 * If IPIs can go out of order to the cache coherency protocol
+	 * in an architecture, sufficient synchronisation should be added
+	 * to arch code to make it appear to obey cache coherency WRT
+	 * locking and barrier primitives. Generic code isn't really equipped
+	 * to do the right thing...
 	 */
-	smp_mb();
 
 	if (ipi)
 		arch_send_call_function_single_ipi(cpu);
@@ -104,6 +111,14 @@ void generic_smp_call_function_interrupt
 	int cpu = get_cpu();
 
 	/*
+	 * Ensure entry is visible on call_function_queue after we have
+	 * entered the IPI. See comment in smp_call_function_many.
+	 * If we don't have this, then we may miss an entry on the list
+	 * and never get another IPI to process it.
+	 */
+	smp_mb();
+
+	/*
 	 * It's ok to use list_for_each_rcu() here even though we may delete
 	 * 'pos', since list_del_rcu() doesn't clear ->next
 	 */
@@ -154,49 +169,37 @@ void generic_smp_call_function_single_in
 {
 	struct call_single_queue *q = &__get_cpu_var(call_single_queue);
 	LIST_HEAD(list);
+	unsigned int data_flags;
 
-	/*
-	 * Need to see other stores to list head for checking whether
-	 * list is empty without holding q->lock
-	 */
-	smp_read_barrier_depends();
-	while (!list_empty(&q->list)) {
-		unsigned int data_flags;
-
-		spin_lock(&q->lock);
-		list_replace_init(&q->list, &list);
-		spin_unlock(&q->lock);
-
-		while (!list_empty(&list)) {
-			struct call_single_data *data;
-
-			data = list_entry(list.next, struct call_single_data,
-						list);
-			list_del(&data->list);
+	spin_lock(&q->lock);
+	list_replace_init(&q->list, &list);
+	spin_unlock(&q->lock);
 
-			/*
-			 * 'data' can be invalid after this call if
-			 * flags == 0 (when called through
-			 * generic_exec_single(), so save them away before
-			 * making the call.
-			 */
-			data_flags = data->flags;
+	while (!list_empty(&list)) {
+		struct call_single_data *data;
 
-			data->func(data->info);
+		data = list_entry(list.next, struct call_single_data,
+					list);
+		list_del(&data->list);
 
-			if (data_flags & CSD_FLAG_WAIT) {
-				smp_wmb();
-				data->flags &= ~CSD_FLAG_WAIT;
-			} else if (data_flags & CSD_FLAG_LOCK) {
-				smp_wmb();
-				data->flags &= ~CSD_FLAG_LOCK;
-			} else if (data_flags & CSD_FLAG_ALLOC)
-				kfree(data);
-		}
 		/*
-		 * See comment on outer loop
+		 * 'data' can be invalid after this call if
+		 * flags == 0 (when called through
+		 * generic_exec_single(), so save them away before
+		 * making the call.
 		 */
-		smp_read_barrier_depends();
+		data_flags = data->flags;
+
+		data->func(data->info);
+
+		if (data_flags & CSD_FLAG_WAIT) {
+			smp_wmb();
+			data->flags &= ~CSD_FLAG_WAIT;
+		} else if (data_flags & CSD_FLAG_LOCK) {
+			smp_wmb();
+			data->flags &= ~CSD_FLAG_LOCK;
+		} else if (data_flags & CSD_FLAG_ALLOC)
+			kfree(data);
 	}
 }
 
@@ -375,6 +378,8 @@ void smp_call_function_many(const struct
 
 	/*
 	 * Make the list addition visible before sending the ipi.
+	 * (IPIs must obey or appear to obey normal Linux cache coherency
+	 * rules -- see comment in generic_exec_single).
 	 */
 	smp_mb();
 

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 11:26                       ` Nick Piggin
@ 2009-02-17 11:48                         ` Peter Zijlstra
  2009-02-17 15:51                         ` Paul E. McKenney
                                           ` (2 subsequent siblings)
  3 siblings, 0 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 11:48 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Oleg Nesterov, Jens Axboe, Suresh Siddha, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, 2009-02-17 at 12:26 +0100, Nick Piggin wrote:

> Simplify the barriers in generic remote function call interrupt code.
> 
> Firstly, just unconditionally take the lock and check the list in the
> generic_call_function_single_interrupt IPI handler. As we've just taken
> an IPI here, the chances are fairly high that there will be work on the
> list for us, so do the locking unconditionally. This removes the tricky
> lockless list_empty check and dubious barriers. The change looks bigger
> than it is because it is just removing an outer loop.
> 
> Secondly, clarify architecture specific IPI locking rules. Generic code
> has no tools to impose any sane ordering on IPIs if they go outside
> normal cache coherency, ergo the arch code must make them appear to
> obey cache coherency as a "memory operation" to initiate an IPI, and
> a "memory operation" to receive one. This way at least they can be
> reasoned about in generic code, and smp_mb used to provide ordering. 
> 
> The combination of these two changes means that explict barriers can
> be taken out of queue handling for the single case -- shared data is
> explicitly locked, and ipi ordering must conform to that, so no
> barriers needed. An extra barrier is needed in the many handler, so
> as to ensure we load the list element after the IPI is received.
> 
> Does any architecture actually needs barriers? For the initiator I
> could see it, but for the handler I would be surprised. The other
> thing we could do for simplicity is just to require that a full
> barrier is required before generating an IPI, and after receiving an
> IPI. We can't just do that in generic code without auditing
> architectures. There have been subtle hangs here on some archs in
> the past.
> 
> Signed-off-by: Nick Piggin <npiggin@suse.de>

Looks sane, barring any funny arch details,

Acked-by: Peter Zijlstra <a.p.zijlstra@chello.nl>

> ---
>  kernel/smp.c |   83 +++++++++++++++++++++++++++++++----------------------------
>  1 file changed, 44 insertions(+), 39 deletions(-)
> 
> Index: linux-2.6/kernel/smp.c
> ===================================================================
> --- linux-2.6.orig/kernel/smp.c
> +++ linux-2.6/kernel/smp.c
> @@ -74,9 +74,16 @@ static void generic_exec_single(int cpu,
>  	spin_unlock_irqrestore(&dst->lock, flags);
>  
>  	/*
> -	 * Make the list addition visible before sending the ipi.
> +	 * The list addition should be visible before sending the IPI
> +	 * handler locks the list to pull the entry off it because of
> +	 * normal cache coherency rules implied by spinlocks.
> +	 *
> +	 * If IPIs can go out of order to the cache coherency protocol
> +	 * in an architecture, sufficient synchronisation should be added
> +	 * to arch code to make it appear to obey cache coherency WRT
> +	 * locking and barrier primitives. Generic code isn't really equipped
> +	 * to do the right thing...
>  	 */
> -	smp_mb();
>  
>  	if (ipi)
>  		arch_send_call_function_single_ipi(cpu);
> @@ -104,6 +111,14 @@ void generic_smp_call_function_interrupt
>  	int cpu = get_cpu();
>  
>  	/*
> +	 * Ensure entry is visible on call_function_queue after we have
> +	 * entered the IPI. See comment in smp_call_function_many.
> +	 * If we don't have this, then we may miss an entry on the list
> +	 * and never get another IPI to process it.
> +	 */
> +	smp_mb();
> +
> +	/*
>  	 * It's ok to use list_for_each_rcu() here even though we may delete
>  	 * 'pos', since list_del_rcu() doesn't clear ->next
>  	 */
> @@ -154,49 +169,37 @@ void generic_smp_call_function_single_in
>  {
>  	struct call_single_queue *q = &__get_cpu_var(call_single_queue);
>  	LIST_HEAD(list);
> +	unsigned int data_flags;
>  
> -	/*
> -	 * Need to see other stores to list head for checking whether
> -	 * list is empty without holding q->lock
> -	 */
> -	smp_read_barrier_depends();
> -	while (!list_empty(&q->list)) {
> -		unsigned int data_flags;
> -
> -		spin_lock(&q->lock);
> -		list_replace_init(&q->list, &list);
> -		spin_unlock(&q->lock);
> -
> -		while (!list_empty(&list)) {
> -			struct call_single_data *data;
> -
> -			data = list_entry(list.next, struct call_single_data,
> -						list);
> -			list_del(&data->list);
> +	spin_lock(&q->lock);
> +	list_replace_init(&q->list, &list);
> +	spin_unlock(&q->lock);
>  
> -			/*
> -			 * 'data' can be invalid after this call if
> -			 * flags == 0 (when called through
> -			 * generic_exec_single(), so save them away before
> -			 * making the call.
> -			 */
> -			data_flags = data->flags;
> +	while (!list_empty(&list)) {
> +		struct call_single_data *data;
>  
> -			data->func(data->info);
> +		data = list_entry(list.next, struct call_single_data,
> +					list);
> +		list_del(&data->list);
>  
> -			if (data_flags & CSD_FLAG_WAIT) {
> -				smp_wmb();
> -				data->flags &= ~CSD_FLAG_WAIT;
> -			} else if (data_flags & CSD_FLAG_LOCK) {
> -				smp_wmb();
> -				data->flags &= ~CSD_FLAG_LOCK;
> -			} else if (data_flags & CSD_FLAG_ALLOC)
> -				kfree(data);
> -		}
>  		/*
> -		 * See comment on outer loop
> +		 * 'data' can be invalid after this call if
> +		 * flags == 0 (when called through
> +		 * generic_exec_single(), so save them away before
> +		 * making the call.
>  		 */
> -		smp_read_barrier_depends();
> +		data_flags = data->flags;
> +
> +		data->func(data->info);
> +
> +		if (data_flags & CSD_FLAG_WAIT) {
> +			smp_wmb();
> +			data->flags &= ~CSD_FLAG_WAIT;
> +		} else if (data_flags & CSD_FLAG_LOCK) {
> +			smp_wmb();
> +			data->flags &= ~CSD_FLAG_LOCK;
> +		} else if (data_flags & CSD_FLAG_ALLOC)
> +			kfree(data);
>  	}
>  }
>  
> @@ -375,6 +378,8 @@ void smp_call_function_many(const struct
>  
>  	/*
>  	 * Make the list addition visible before sending the ipi.
> +	 * (IPIs must obey or appear to obey normal Linux cache coherency
> +	 * rules -- see comment in generic_exec_single).
>  	 */
>  	smp_mb();
>  


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

* Re: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()
  2009-02-16 19:10   ` Oleg Nesterov
  2009-02-16 19:41     ` Peter Zijlstra
@ 2009-02-17 12:25     ` Oleg Nesterov
  1 sibling, 0 replies; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-17 12:25 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/16, Oleg Nesterov wrote:
>
> > +	data->refs = cpumask_weight(&data->cpumask);
> > +
> > +	spin_lock_irqsave(&call_function.lock, flags);
> > +	call_function.counter++;
> > +	list_add_tail_rcu(&data->csd.list, &call_function.queue);
> > +	spin_unlock_irqrestore(&call_function.lock, flags);
>
> What if the initialization above leaks into the critical section?
>
> I mean, generic_smp_call_function_interrupt() running on another CPU
> can see the result of list_add_tail_rcu() and cpumask_and(data->cpumask)
> but not (say) "data->refs = ...".

I was wrong, sorry.

list_add_tail_rcu() implies wmb() and list_for_each_entry_rcu() does
smp_read_barrier_depends(), so another CPU must always see the fully
initialized entry.

Oleg.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17  9:29                 ` Peter Zijlstra
  2009-02-17 10:11                   ` Nick Piggin
@ 2009-02-17 12:40                   ` Peter Zijlstra
  2009-02-17 15:43                   ` Paul E. McKenney
  2 siblings, 0 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 12:40 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Jens Axboe, Suresh Siddha, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Tue, 2009-02-17 at 10:29 +0100, Peter Zijlstra wrote:
> > It would be nice to have a comment which explains how can we miss the
> > first addition without read_barrier_depends(). And why only on alpha.
> 
> Paul, care to once again enlighten us? The best I can remember is that
> alpha has split caches, and the rmb is needed for them to become
> coherent -- no other arch is crazy in exactly that way.
> 
> But note that read_barrier_depends() is not quite a NOP for !alpha, it
> does that ACCESS_ONCE() thing, which very much makes a difference, even
> on x86.

I've been saying crazy stuff, read_barrier_depends() is a NOP, I got
confused with rcu_dereference(). My bad.


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

* [PATCH] generic-smp: remove kmalloc()
  2009-02-16 16:38 ` [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many() Peter Zijlstra
  2009-02-16 19:10   ` Oleg Nesterov
  2009-02-16 20:49   ` Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()) Oleg Nesterov
@ 2009-02-17 15:40   ` Peter Zijlstra
  2009-02-17 17:21     ` Oleg Nesterov
  2 siblings, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 15:40 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Nick Piggin, Jens Axboe, Paul E. McKenney, Ingo Molnar,
	Rusty Russell, Steven Rostedt, linux-kernel, Oleg Nesterov

Ok, so this is on top of Nick's cleanup from earlier today, and folds
everything.

No more RCU games as the storage for per-cpu entries is permanent - cpu
hotplug should be good because it does a synchronize_sched().

What we do play games with is the global list, we can extract entries
and place them to the front while its being observed. This means that
the list iteration can see some entries twice (not a problem since we
remove ourselves from the cpumask), but cannot miss entries.

Not-quite-signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
---
 smp.c |  260 ++++++++++++++++++++++++++++++++++++++++--------------------------
 1 file changed, 159 insertions(+), 101 deletions(-)

Index: linux-2.6/kernel/smp.c
===================================================================
--- linux-2.6.orig/kernel/smp.c
+++ linux-2.6/kernel/smp.c
@@ -10,23 +10,28 @@
 #include <linux/rcupdate.h>
 #include <linux/rculist.h>
 #include <linux/smp.h>
+#include <linux/cpu.h>
 
 static DEFINE_PER_CPU(struct call_single_queue, call_single_queue);
-static LIST_HEAD(call_function_queue);
-__cacheline_aligned_in_smp DEFINE_SPINLOCK(call_function_lock);
+
+static struct {
+	struct list_head	queue;
+	spinlock_t		lock;
+} call_function __cacheline_aligned_in_smp = {
+	.queue = LIST_HEAD_INIT(call_function.queue),
+	.lock  = __SPIN_LOCK_UNLOCKED(call_function.lock),
+};
 
 enum {
 	CSD_FLAG_WAIT		= 0x01,
-	CSD_FLAG_ALLOC		= 0x02,
-	CSD_FLAG_LOCK		= 0x04,
+	CSD_FLAG_LOCK		= 0x02,
 };
 
 struct call_function_data {
 	struct call_single_data csd;
 	spinlock_t lock;
 	unsigned int refs;
-	struct rcu_head rcu_head;
-	unsigned long cpumask_bits[];
+	cpumask_var_t cpumask;
 };
 
 struct call_single_queue {
@@ -34,8 +39,45 @@ struct call_single_queue {
 	spinlock_t lock;
 };
 
+static DEFINE_PER_CPU(struct call_function_data, cfd_data) = {
+	.lock = __SPIN_LOCK_UNLOCKED(cfd_data.lock),
+};
+
+static int
+hotplug_cfd(struct notifier_block *nfb, unsigned long action, void *hcpu)
+{
+	long cpu = (long)hcpu;
+	struct call_function_data *cfd = &per_cpu(cfd_data, cpu);
+
+	switch (action) {
+	case CPU_UP_PREPARE:
+	case CPU_UP_PREPARE_FROZEN:
+		if (!alloc_cpumask_var_node(&cfd->cpumask, GFP_KERNEL,
+				cpu_to_node(cpu)))
+			return NOTIFY_BAD;
+		break;
+
+#ifdef CONFIG_CPU_HOTPLUG
+	case CPU_UP_CANCELED:
+	case CPU_UP_CANCELED_FROZEN:
+
+	case CPU_DEAD:
+	case CPU_DEAD_FROZEN:
+		free_cpumask_var(cfd->cpumask);
+		break;
+#endif
+	};
+
+	return NOTIFY_OK;
+}
+
+static struct notifier_block __cpuinitdata hotplug_cfd_notifier = {
+	.notifier_call = hotplug_cfd,
+};
+
 static int __cpuinit init_call_single_data(void)
 {
+	void *cpu = (void *)(long)smp_processor_id();
 	int i;
 
 	for_each_possible_cpu(i) {
@@ -44,18 +86,61 @@ static int __cpuinit init_call_single_da
 		spin_lock_init(&q->lock);
 		INIT_LIST_HEAD(&q->list);
 	}
+
+	hotplug_cfd(&hotplug_cfd_notifier, CPU_UP_PREPARE, cpu);
+	register_cpu_notifier(&hotplug_cfd_notifier);
+
 	return 0;
 }
 early_initcall(init_call_single_data);
 
-static void csd_flag_wait(struct call_single_data *data)
+/*
+ * csd_wait/csd_complete are used for synchronous ipi calls
+ */
+static void csd_wait_prepare(struct call_single_data *data)
 {
-	/* Wait for response */
-	do {
-		if (!(data->flags & CSD_FLAG_WAIT))
-			break;
+	data->flags |= CSD_FLAG_WAIT;
+}
+
+static void csd_complete(struct call_single_data *data)
+{
+	if (data->flags & CSD_FLAG_WAIT) {
+		/*
+		 * Serialize stores to data with the flag clear and wakeup.
+		 */
+		smp_wmb();
+		data->flags &= ~CSD_FLAG_WAIT;
+	}
+}
+
+static void csd_wait(struct call_single_data *data)
+{
+	while (data->flags & CSD_FLAG_WAIT)
 		cpu_relax();
-	} while (1);
+}
+
+/*
+ * csd_lock/csd_unlock used to serialize access to per-cpu csd resources
+ *
+ * For non-synchronous ipi calls the csd can still be in use by the previous
+ * function call. For multi-cpu calls its even more interesting as we'll have
+ * to ensure no other cpu is observing our csd.
+ */
+static void csd_lock(struct call_single_data *data)
+{
+	while (data->flags & CSD_FLAG_LOCK)
+		cpu_relax();
+	data->flags = CSD_FLAG_LOCK;
+}
+
+static void csd_unlock(struct call_single_data *data)
+{
+	WARN_ON(!(data->flags & CSD_FLAG_LOCK));
+	/*
+	 * Serialize stores to data with the flags clear.
+	 */
+	smp_wmb();
+	data->flags &= ~CSD_FLAG_LOCK;
 }
 
 /*
@@ -89,16 +174,7 @@ static void generic_exec_single(int cpu,
 		arch_send_call_function_single_ipi(cpu);
 
 	if (wait)
-		csd_flag_wait(data);
-}
-
-static void rcu_free_call_data(struct rcu_head *head)
-{
-	struct call_function_data *data;
-
-	data = container_of(head, struct call_function_data, rcu_head);
-
-	kfree(data);
+		csd_wait(data);
 }
 
 /*
@@ -122,41 +198,32 @@ void generic_smp_call_function_interrupt
 	 * It's ok to use list_for_each_rcu() here even though we may delete
 	 * 'pos', since list_del_rcu() doesn't clear ->next
 	 */
-	rcu_read_lock();
-	list_for_each_entry_rcu(data, &call_function_queue, csd.list) {
+	list_for_each_entry_rcu(data, &call_function.queue, csd.list) {
 		int refs;
 
-		if (!cpumask_test_cpu(cpu, to_cpumask(data->cpumask_bits)))
-			continue;
-
-		data->csd.func(data->csd.info);
-
 		spin_lock(&data->lock);
-		cpumask_clear_cpu(cpu, to_cpumask(data->cpumask_bits));
+		if (!cpumask_test_cpu(cpu, data->cpumask)) {
+			spin_unlock(&data->lock);
+			continue;
+		}
+		cpumask_clear_cpu(cpu, data->cpumask);
 		WARN_ON(data->refs == 0);
 		data->refs--;
 		refs = data->refs;
 		spin_unlock(&data->lock);
 
+		data->csd.func(data->csd.info);
+
 		if (refs)
 			continue;
 
-		spin_lock(&call_function_lock);
+		spin_lock(&call_function.lock);
 		list_del_rcu(&data->csd.list);
-		spin_unlock(&call_function_lock);
+		spin_unlock(&call_function.lock);
 
-		if (data->csd.flags & CSD_FLAG_WAIT) {
-			/*
-			 * serialize stores to data with the flag clear
-			 * and wakeup
-			 */
-			smp_wmb();
-			data->csd.flags &= ~CSD_FLAG_WAIT;
-		}
-		if (data->csd.flags & CSD_FLAG_ALLOC)
-			call_rcu(&data->rcu_head, rcu_free_call_data);
+		csd_complete(&data->csd);
+		csd_unlock(&data->csd);
 	}
-	rcu_read_unlock();
 
 	put_cpu();
 }
@@ -192,14 +259,14 @@ void generic_smp_call_function_single_in
 
 		data->func(data->info);
 
-		if (data_flags & CSD_FLAG_WAIT) {
-			smp_wmb();
-			data->flags &= ~CSD_FLAG_WAIT;
-		} else if (data_flags & CSD_FLAG_LOCK) {
-			smp_wmb();
-			data->flags &= ~CSD_FLAG_LOCK;
-		} else if (data_flags & CSD_FLAG_ALLOC)
-			kfree(data);
+		if (data_flags & CSD_FLAG_WAIT)
+			csd_complete(data);
+
+		/*
+		 * Unlocked CSDs are valid through generic_exec_single()
+		 */
+		if (data_flags & CSD_FLAG_LOCK)
+			csd_unlock(data);
 	}
 }
 
@@ -233,19 +300,19 @@ int smp_call_function_single(int cpu, vo
 		func(info);
 		local_irq_restore(flags);
 	} else if ((unsigned)cpu < nr_cpu_ids && cpu_online(cpu)) {
-		struct call_single_data *data;
+		struct call_single_data *data = {
+			.flags = 0,
+		};
 
 		if (!wait) {
 			/*
 			 * We are calling a function on a single CPU
 			 * and we are not going to wait for it to finish.
-			 * We first try to allocate the data, but if we
-			 * fail, we fall back to use a per cpu data to pass
-			 * the information to that CPU. Since all callers
-			 * of this code will use the same data, we must
-			 * synchronize the callers to prevent a new caller
-			 * from corrupting the data before the callee
-			 * can access it.
+			 * We use a per cpu data to pass the information to
+			 * that CPU. Since all callers of this code will
+			 * use the same data, we must synchronize the
+			 * callers to prevent a new caller from corrupting
+			 * the data before the callee can access it.
 			 *
 			 * The CSD_FLAG_LOCK is used to let us know when
 			 * the IPI handler is done with the data.
@@ -255,18 +322,11 @@ int smp_call_function_single(int cpu, vo
 			 * will make sure the callee is done with the
 			 * data before a new caller will use it.
 			 */
-			data = kmalloc(sizeof(*data), GFP_ATOMIC);
-			if (data)
-				data->flags = CSD_FLAG_ALLOC;
-			else {
-				data = &per_cpu(csd_data, me);
-				while (data->flags & CSD_FLAG_LOCK)
-					cpu_relax();
-				data->flags = CSD_FLAG_LOCK;
-			}
+			data = &per_cpu(csd_data, me);
+			csd_lock(data);
 		} else {
 			data = &d;
-			data->flags = CSD_FLAG_WAIT;
+			csd_wait_prepare(data);
 		}
 
 		data->func = func;
@@ -326,14 +386,14 @@ void smp_call_function_many(const struct
 {
 	struct call_function_data *data;
 	unsigned long flags;
-	int cpu, next_cpu;
+	int cpu, next_cpu, me = smp_processor_id();
 
 	/* Can deadlock when called with interrupts disabled */
 	WARN_ON(irqs_disabled());
 
 	/* So, what's a CPU they want?  Ignoring this one. */
 	cpu = cpumask_first_and(mask, cpu_online_mask);
-	if (cpu == smp_processor_id())
+	if (cpu == me)
 		cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
 	/* No online cpus?  We're done. */
 	if (cpu >= nr_cpu_ids)
@@ -341,7 +401,7 @@ void smp_call_function_many(const struct
 
 	/* Do we have another CPU which isn't us? */
 	next_cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
-	if (next_cpu == smp_processor_id())
+	if (next_cpu == me)
 		next_cpu = cpumask_next_and(next_cpu, mask, cpu_online_mask);
 
 	/* Fastpath: do that cpu by itself. */
@@ -350,31 +410,28 @@ void smp_call_function_many(const struct
 		return;
 	}
 
-	data = kmalloc(sizeof(*data) + cpumask_size(), GFP_ATOMIC);
-	if (unlikely(!data)) {
-		/* Slow path. */
-		for_each_online_cpu(cpu) {
-			if (cpu == smp_processor_id())
-				continue;
-			if (cpumask_test_cpu(cpu, mask))
-				smp_call_function_single(cpu, func, info, wait);
-		}
-		return;
-	}
+	data = &per_cpu(cfd_data, me);
+	csd_lock(&data->csd);
 
-	spin_lock_init(&data->lock);
-	data->csd.flags = CSD_FLAG_ALLOC;
+	spin_lock_irqsave(&data->lock, flags);
 	if (wait)
-		data->csd.flags |= CSD_FLAG_WAIT;
+		csd_wait_prepare(&data->csd);
+
 	data->csd.func = func;
 	data->csd.info = info;
-	cpumask_and(to_cpumask(data->cpumask_bits), mask, cpu_online_mask);
-	cpumask_clear_cpu(smp_processor_id(), to_cpumask(data->cpumask_bits));
-	data->refs = cpumask_weight(to_cpumask(data->cpumask_bits));
-
-	spin_lock_irqsave(&call_function_lock, flags);
-	list_add_tail_rcu(&data->csd.list, &call_function_queue);
-	spin_unlock_irqrestore(&call_function_lock, flags);
+	cpumask_and(data->cpumask, mask, cpu_online_mask);
+	cpumask_clear_cpu(me, data->cpumask);
+	data->refs = cpumask_weight(data->cpumask);
+
+	spin_lock(&call_function.lock);
+	/*
+	 * Place entry at the _HEAD_ of the list, so that any cpu still
+	 * observing the entry in generic_smp_call_function_interrupt() will
+	 * not miss any other list entries.
+	 */
+	list_add_rcu(&data->csd.list, &call_function.queue);
+	spin_unlock(&call_function.lock);
+	spin_unlock_irqrestore(&data->lock, flags);
 
 	/*
 	 * Make the list addition visible before sending the ipi.
@@ -384,11 +441,11 @@ void smp_call_function_many(const struct
 	smp_mb();
 
 	/* Send a message to all CPUs in the map */
-	arch_send_call_function_ipi_mask(to_cpumask(data->cpumask_bits));
+	arch_send_call_function_ipi_mask(data->cpumask);
 
 	/* optionally wait for the CPUs to complete */
 	if (wait)
-		csd_flag_wait(&data->csd);
+		csd_wait(&data->csd);
 }
 EXPORT_SYMBOL(smp_call_function_many);
 
@@ -418,20 +475,20 @@ EXPORT_SYMBOL(smp_call_function);
 
 void ipi_call_lock(void)
 {
-	spin_lock(&call_function_lock);
+	spin_lock(&call_function.lock);
 }
 
 void ipi_call_unlock(void)
 {
-	spin_unlock(&call_function_lock);
+	spin_unlock(&call_function.lock);
 }
 
 void ipi_call_lock_irq(void)
 {
-	spin_lock_irq(&call_function_lock);
+	spin_lock_irq(&call_function.lock);
 }
 
 void ipi_call_unlock_irq(void)
 {
-	spin_unlock_irq(&call_function_lock);
+	spin_unlock_irq(&call_function.lock);
 }



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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17  9:29                 ` Peter Zijlstra
  2009-02-17 10:11                   ` Nick Piggin
  2009-02-17 12:40                   ` Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()) Peter Zijlstra
@ 2009-02-17 15:43                   ` Paul E. McKenney
  2 siblings, 0 replies; 103+ messages in thread
From: Paul E. McKenney @ 2009-02-17 15:43 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Oleg Nesterov, Jens Axboe, Suresh Siddha, Linus Torvalds,
	Nick Piggin, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel

On Tue, Feb 17, 2009 at 10:29:34AM +0100, Peter Zijlstra wrote:
> On Tue, 2009-02-17 at 00:19 +0100, Oleg Nesterov wrote:
> > On 02/16, Peter Zijlstra wrote:
> > >
> > > On Mon, 2009-02-16 at 23:02 +0100, Oleg Nesterov wrote:
> > > > On 02/16, Peter Zijlstra wrote:
> > > > >
> > > > > On Mon, 2009-02-16 at 22:32 +0100, Oleg Nesterov wrote:
> > > > > > > I was about to write a response, but found it to be a justification for
> > > > > > > the read_barrier_depends() at the end of the loop.
> > > > > >
> > > > > > I forgot to mention I don't understand the read_barrier_depends() at the
> > > > > > end of the loop as well ;)
> > > > >
> > > > > Suppose cpu0 adds to csd to cpu1:
> > > > >
> > > > >
> > > > >  cpu0:                 cpu1:
> > > > >
> > > > > add entry1
> > > > > mb();
> > > > > send ipi
> > > > >                       run ipi handler
> > > > >                       read_barrier_depends()
> > > > >                       while (!list_empty())    [A]
> > > > >                         do foo
> > > > >
> > > > > add entry2
> > > > > mb();
> > > > > [no ipi -- we still observe entry1]
> > > > >
> > > > >                         remove foo
> > > > >                         read_barrier_depends()
> > > > >                       while (!list_empty())      [B]
> > > >
> > > > Still can't understand.
> > > >
> > > > cpu1 (generic_smp_call_function_single_interrupt) does
> > > > list_replace_init(q->lock), this lock is also taken by
> > > > generic_exec_single().
> > > >
> > > > Either cpu1 sees entry2 on list, or cpu0 sees list_empty()
> > > > and sends ipi.
> > >
> > > cpu0:		cpu1:
> > >
> > > spin_lock_irqsave(&dst->lock, flags);
> > > ipi = list_empty(&dst->list);
> > > list_add_tail(&data->list, &dst->list);
> > > spin_unlock_irqrestore(&dst->lock, flags);
> > >
> > > ipi ----->
> > >
> > > 		while (!list_empty(&q->list)) {
> > >                 	unsigned int data_flags;
> > >
> > >                 	spin_lock(&q->lock);
> > >                		list_replace_init(&q->list, &list);
> > > 	                spin_unlock(&q->lock);
> > >
> > >
> > > Strictly speaking the unlock() is semi-permeable, allowing the read of
> > > q->list to enter the critical section, allowing us to observe an empty
> > > list, never getting to q->lock on cpu1.
> > 
> > Hmm. If we take &q->lock, then we alread saw !list_empty() ?
> 
> That's how I read the above code.
> 
> > And the question is, how can we miss list_empty() == F before spin_lock().
> 
> Confusion... my explanation above covers exactly this case. The reads
> determining list_empty() can slip into the q->lock section on the other
> cpu, and observe an empty list.
> 
> > > > Even if I missed something (very possible), then I can't understand
> > > > why we need rmb() only on alpha.
> > >
> > > Because only alpha is insane enough to do speculative reads? Dunno
> > > really :-)
> > 
> > Perhaps...
> > 
> > It would be nice to have a comment which explains how can we miss the
> > first addition without read_barrier_depends(). And why only on alpha.
> 
> Paul, care to once again enlighten us? The best I can remember is that
> alpha has split caches, and the rmb is needed for them to become
> coherent -- no other arch is crazy in exactly that way.

Many architectures use split caches, but Alpha made them independent.  :-/

Suppose that an Alpha system has a cache for each CPU, and that each CPU's
cache is split into banks so that even-numbered cache lines are placed
in one bank and odd-numbered cache lines in the other.  Then suppose
that CPU 0 executes the following code:

	p = malloc(sizeof(*p));
	if (p == NULL)
		deal_with_it();
	p->a = 42;
	smp_wmb(); /* this line and next same as rcu_assign_pointer().  */
	global_p = p;

This code will ensure that CPU 0 will commit the assignment to p->a to
coherent memory before commiting the assignment to global_p.

Suppose further that global_p is located in an even-numbered cache line
and that the newly allocated structure pointed to by p is in an
odd-numbered cache line.  Then suppose that CPU 1 executes the following
code:

	q = global_p;
	t = q->a;

Now, CPU 0 "published" the assignment to ->a before that to global_p,
but suppose that CPU 1's odd-numbered cache bank is very busy, so that
it has not yet processed the invalidation request corresponding to
CPU 0's assignment to p->a.

In this case, CPU 1 will see the new value of global_p, but the old
value of q->a.

This same result can be caused by certain types of value-speculation
compiler optimizations.

For more information, see:

http://www.rdrop.com/users/paulmck/scalability/paper/ordering.2007.09.19a.pdf

> But note that read_barrier_depends() is not quite a NOP for !alpha, it
> does that ACCESS_ONCE() thing, which very much makes a difference, even
> on x86.

You are thinking of rcu_dereference() rather than read_barrier_depends(),
right?

> > And arch/alpha/kernel/smp.c:handle_ipi() does mb() itself...
> 
> Right, but arguing by our memory model, we cannot assume that.

I assert that things like smp_call_function() need to perform whatever
memory barriers are required to ensure that the called function sees
any memory references performed on the originating CPU prior to the
smp_call_function().

						Thanx, Paul

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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-17  8:24     ` Peter Zijlstra
  2009-02-17  9:43       ` Ingo Molnar
@ 2009-02-17 15:44       ` Linus Torvalds
  1 sibling, 0 replies; 103+ messages in thread
From: Linus Torvalds @ 2009-02-17 15:44 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Nick Piggin, Jens Axboe, Paul E. McKenney, Ingo Molnar,
	Rusty Russell, Steven Rostedt, linux-kernel, Oleg Nesterov



On Tue, 17 Feb 2009, Peter Zijlstra wrote:
> 
> Ah, no, I don't actually. I remove the kmalloc+call_rcu stuff in 2, not
> the newly cfd mini rcu thing.

I'm not talking about the "mini-rcu", I'm talking about the _existing_ rcu 
code that you play games with, just before you then remove it.

See here:

 - patch 1/4:

	@@ -25,8 +40,11 @@ struct call_function_data {
	        struct call_single_data csd;
	        spinlock_t lock;
	        unsigned int refs;
	-       struct rcu_head rcu_head;
	-       unsigned long cpumask_bits[];
	+       union {
	+               struct rcu_head rcu_head;
	+               struct list_head free_list;
	+       };
	+       struct cpumask cpumask;
	 };

 - patch 2/4:

	 struct call_function_data {
	        struct call_single_data csd;
	        spinlock_t lock;
	        unsigned int refs;
	-       union {
	-               struct rcu_head rcu_head;
	-               struct list_head free_list;
	-       };
	+       struct list_head free_list;
	        struct cpumask cpumask;
	 };

ie that ugly/complex/subtle union of rcu_head that you added in 1/4 you 
then remove immediately in 2/4. Making the patches just harder to read. 

Just remove it. I'd say fold 1+2 into one patch, instead of playing games 
with that thing. Just make the patch remove the kmalloc.

Or at least get rid of the _games_ you play with that union. Now it just 
makes the patch unreadable.

		Linus

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 11:26                       ` Nick Piggin
  2009-02-17 11:48                         ` Peter Zijlstra
@ 2009-02-17 15:51                         ` Paul E. McKenney
  2009-02-18  2:15                           ` Suresh Siddha
  2009-02-17 19:28                         ` Q: " Oleg Nesterov
  2009-02-18  2:21                         ` Suresh Siddha
  3 siblings, 1 reply; 103+ messages in thread
From: Paul E. McKenney @ 2009-02-17 15:51 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Peter Zijlstra, Oleg Nesterov, Jens Axboe, Suresh Siddha,
	Linus Torvalds, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, Feb 17, 2009 at 12:26:57PM +0100, Nick Piggin wrote:
> cc linux-arch
> 
> On Tue, Feb 17, 2009 at 11:27:33AM +0100, Peter Zijlstra wrote:
> > > But hmm, why even bother with all this complexity? Why not just
> > > remove the outer loop completely? Do the lock and the list_replace_init
> > > unconditionally. It would turn tricky lockless code into simple locked
> > > code... we've already taken an interrupt anyway, so chances are pretty
> > > high that we have work here to do, right?
> > 
> > Well, that's a practical suggestion, and I agree.
> 
> How's this?
> 
> --
> Simplify the barriers in generic remote function call interrupt code.
> 
> Firstly, just unconditionally take the lock and check the list in the
> generic_call_function_single_interrupt IPI handler. As we've just taken
> an IPI here, the chances are fairly high that there will be work on the
> list for us, so do the locking unconditionally. This removes the tricky
> lockless list_empty check and dubious barriers. The change looks bigger
> than it is because it is just removing an outer loop.
> 
> Secondly, clarify architecture specific IPI locking rules. Generic code
> has no tools to impose any sane ordering on IPIs if they go outside
> normal cache coherency, ergo the arch code must make them appear to
> obey cache coherency as a "memory operation" to initiate an IPI, and
> a "memory operation" to receive one. This way at least they can be
> reasoned about in generic code, and smp_mb used to provide ordering. 
> 
> The combination of these two changes means that explict barriers can
> be taken out of queue handling for the single case -- shared data is
> explicitly locked, and ipi ordering must conform to that, so no
> barriers needed. An extra barrier is needed in the many handler, so
> as to ensure we load the list element after the IPI is received.
> 
> Does any architecture actually needs barriers? For the initiator I
> could see it, but for the handler I would be surprised. The other
> thing we could do for simplicity is just to require that a full
> barrier is required before generating an IPI, and after receiving an
> IPI. We can't just do that in generic code without auditing
> architectures. There have been subtle hangs here on some archs in
> the past.
> 
> Signed-off-by: Nick Piggin <npiggin@suse.de>
> 
> ---
>  kernel/smp.c |   83 +++++++++++++++++++++++++++++++----------------------------
>  1 file changed, 44 insertions(+), 39 deletions(-)
> 
> Index: linux-2.6/kernel/smp.c
> ===================================================================
> --- linux-2.6.orig/kernel/smp.c
> +++ linux-2.6/kernel/smp.c
> @@ -74,9 +74,16 @@ static void generic_exec_single(int cpu,
>  	spin_unlock_irqrestore(&dst->lock, flags);
> 
>  	/*
> -	 * Make the list addition visible before sending the ipi.
> +	 * The list addition should be visible before sending the IPI
> +	 * handler locks the list to pull the entry off it because of
> +	 * normal cache coherency rules implied by spinlocks.
> +	 *
> +	 * If IPIs can go out of order to the cache coherency protocol
> +	 * in an architecture, sufficient synchronisation should be added
> +	 * to arch code to make it appear to obey cache coherency WRT
> +	 * locking and barrier primitives. Generic code isn't really equipped
> +	 * to do the right thing...
>  	 */
> -	smp_mb();
> 
>  	if (ipi)
>  		arch_send_call_function_single_ipi(cpu);
> @@ -104,6 +111,14 @@ void generic_smp_call_function_interrupt
>  	int cpu = get_cpu();
> 
>  	/*
> +	 * Ensure entry is visible on call_function_queue after we have
> +	 * entered the IPI. See comment in smp_call_function_many.
> +	 * If we don't have this, then we may miss an entry on the list
> +	 * and never get another IPI to process it.
> +	 */
> +	smp_mb();
> +
> +	/*
>  	 * It's ok to use list_for_each_rcu() here even though we may delete
>  	 * 'pos', since list_del_rcu() doesn't clear ->next
>  	 */
> @@ -154,49 +169,37 @@ void generic_smp_call_function_single_in
>  {
>  	struct call_single_queue *q = &__get_cpu_var(call_single_queue);
>  	LIST_HEAD(list);
> +	unsigned int data_flags;
> 
> -	/*
> -	 * Need to see other stores to list head for checking whether
> -	 * list is empty without holding q->lock
> -	 */
> -	smp_read_barrier_depends();
> -	while (!list_empty(&q->list)) {
> -		unsigned int data_flags;
> -
> -		spin_lock(&q->lock);
> -		list_replace_init(&q->list, &list);
> -		spin_unlock(&q->lock);
> -
> -		while (!list_empty(&list)) {
> -			struct call_single_data *data;
> -
> -			data = list_entry(list.next, struct call_single_data,
> -						list);
> -			list_del(&data->list);
> +	spin_lock(&q->lock);
> +	list_replace_init(&q->list, &list);
> +	spin_unlock(&q->lock);

OK, I'll bite...

How do we avoid deadlock in the case where a pair of CPUs send to each
other concurrently?

							Thanx, Paul

> 
> -			/*
> -			 * 'data' can be invalid after this call if
> -			 * flags == 0 (when called through
> -			 * generic_exec_single(), so save them away before
> -			 * making the call.
> -			 */
> -			data_flags = data->flags;
> +	while (!list_empty(&list)) {
> +		struct call_single_data *data;
> 
> -			data->func(data->info);
> +		data = list_entry(list.next, struct call_single_data,
> +					list);
> +		list_del(&data->list);
> 
> -			if (data_flags & CSD_FLAG_WAIT) {
> -				smp_wmb();
> -				data->flags &= ~CSD_FLAG_WAIT;
> -			} else if (data_flags & CSD_FLAG_LOCK) {
> -				smp_wmb();
> -				data->flags &= ~CSD_FLAG_LOCK;
> -			} else if (data_flags & CSD_FLAG_ALLOC)
> -				kfree(data);
> -		}
>  		/*
> -		 * See comment on outer loop
> +		 * 'data' can be invalid after this call if
> +		 * flags == 0 (when called through
> +		 * generic_exec_single(), so save them away before
> +		 * making the call.
>  		 */
> -		smp_read_barrier_depends();
> +		data_flags = data->flags;
> +
> +		data->func(data->info);
> +
> +		if (data_flags & CSD_FLAG_WAIT) {
> +			smp_wmb();
> +			data->flags &= ~CSD_FLAG_WAIT;
> +		} else if (data_flags & CSD_FLAG_LOCK) {
> +			smp_wmb();
> +			data->flags &= ~CSD_FLAG_LOCK;
> +		} else if (data_flags & CSD_FLAG_ALLOC)
> +			kfree(data);
>  	}
>  }
> 
> @@ -375,6 +378,8 @@ void smp_call_function_many(const struct
> 
>  	/*
>  	 * Make the list addition visible before sending the ipi.
> +	 * (IPIs must obey or appear to obey normal Linux cache coherency
> +	 * rules -- see comment in generic_exec_single).
>  	 */
>  	smp_mb();
> 

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

* Re: [PATCH] generic-smp: remove kmalloc()
  2009-02-17 15:40   ` [PATCH] generic-smp: remove kmalloc() Peter Zijlstra
@ 2009-02-17 17:21     ` Oleg Nesterov
  2009-02-17 17:40       ` Peter Zijlstra
  0 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-17 17:21 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/17, Peter Zijlstra wrote:
>
> Ok, so this is on top of Nick's cleanup from earlier today, and folds
> everything.
>
> No more RCU games as the storage for per-cpu entries is permanent - cpu
> hotplug should be good because it does a synchronize_sched().
>
> What we do play games with is the global list, we can extract entries
> and place them to the front while its being observed. This means that
> the list iteration can see some entries twice (not a problem since we
> remove ourselves from the cpumask), but cannot miss entries.

I think this all is correct.

But I am wondering, don't we have another problem. Before this patch,
smp_call_function_many(wait => 0) always succeeds, no matter which
locks the caller holds.

After this patch we can deadlock, csd_lock() can spin forever if the
caller shares the lock with another func in flight.

IOW,
	void func(void *arg)
	{
		lock(LOCK);
		unlock(LOCK);
	}

CPU 0 does:

	smp_call_function(func, NULL, 0);
	lock(LOCK);
	smp_call_function(another_func, NULL, 0);
	unlock(LOCK);

If CPU 0 takes LOCK before CPU 1 calls func, the 2nd smp_call_function()
hangs in csd_lock().

I am not sure this is the real problem (even if I am right), perhaps
the answer is "don't do that".

But, otoh, afaics we can tweak generic_smp_call_function_interrupt()
a bit to avoid this problem. Something like

	list_for_each_entry_rcu(data, &call_function.queue, csd.list) {
		void (*func)(void *);
		void *info;
		int refs;

		spin_lock(&data->lock);
		if (!cpumask_test_cpu(cpu, data->cpumask)) {
			spin_unlock(&data->lock);
			continue;
		}
		cpumask_clear_cpu(cpu, data->cpumask);
		WARN_ON(data->refs == 0);
		refs = --data->refs;
		func = data->csd.func;
		info = data->csd.info;
		wait = (data->flags & CSD_FLAG_WAIT);
		spin_unlock(&data->lock);

		if (!refs) {
			spin_lock(&call_function.lock);
			list_del_rcu(&data->csd.list);
			spin_unlock(&call_function.lock);
			csd_unlock(&data->csd);
		}
		
		func(info);
		if (!refs && wait)
			csd_complete(&data->csd);
	}

I am afraid I missed something, and the code above looks wrong
because it does csd_unlock() first, then csd_complete().

But if wait == T, then nobody can reuse this per-cpu entry, the
caller of smp_call_function_many() must spin in csd_wait() on
the same CPU.

What do you think?

Oleg.


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

* Re: [PATCH] generic-smp: remove kmalloc()
  2009-02-17 17:21     ` Oleg Nesterov
@ 2009-02-17 17:40       ` Peter Zijlstra
  2009-02-17 17:46         ` Peter Zijlstra
  2009-02-17 19:29         ` [PATCH -v4] generic-ipi: " Peter Zijlstra
  0 siblings, 2 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 17:40 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, 2009-02-17 at 18:21 +0100, Oleg Nesterov wrote:
> On 02/17, Peter Zijlstra wrote:
> >
> > Ok, so this is on top of Nick's cleanup from earlier today, and folds
> > everything.
> >
> > No more RCU games as the storage for per-cpu entries is permanent - cpu
> > hotplug should be good because it does a synchronize_sched().
> >
> > What we do play games with is the global list, we can extract entries
> > and place them to the front while its being observed. This means that
> > the list iteration can see some entries twice (not a problem since we
> > remove ourselves from the cpumask), but cannot miss entries.
> 
> I think this all is correct.

*phew* :-)

> But I am wondering, don't we have another problem. Before this patch,
> smp_call_function_many(wait => 0) always succeeds, no matter which
> locks the caller holds.
> 
> After this patch we can deadlock, csd_lock() can spin forever if the
> caller shares the lock with another func in flight.
> 
> IOW,
> 	void func(void *arg)
> 	{
> 		lock(LOCK);
> 		unlock(LOCK);
> 	}
> 
> CPU 0 does:
> 
> 	smp_call_function(func, NULL, 0);
> 	lock(LOCK);
> 	smp_call_function(another_func, NULL, 0);
> 	unlock(LOCK);
> 
> If CPU 0 takes LOCK before CPU 1 calls func, the 2nd smp_call_function()
> hangs in csd_lock().
> 
> I am not sure this is the real problem (even if I am right), perhaps
> the answer is "don't do that".
> 
> But, otoh, afaics we can tweak generic_smp_call_function_interrupt()
> a bit to avoid this problem. Something like
> 
> 	list_for_each_entry_rcu(data, &call_function.queue, csd.list) {
> 		void (*func)(void *);
> 		void *info;
> 		int refs;
> 
> 		spin_lock(&data->lock);
> 		if (!cpumask_test_cpu(cpu, data->cpumask)) {
> 			spin_unlock(&data->lock);
> 			continue;
> 		}
> 		cpumask_clear_cpu(cpu, data->cpumask);
> 		WARN_ON(data->refs == 0);
> 		refs = --data->refs;
> 		func = data->csd.func;
> 		info = data->csd.info;
> 		wait = (data->flags & CSD_FLAG_WAIT);
> 		spin_unlock(&data->lock);
> 
> 		if (!refs) {
> 			spin_lock(&call_function.lock);
> 			list_del_rcu(&data->csd.list);
> 			spin_unlock(&call_function.lock);
> 			csd_unlock(&data->csd);
> 		}
> 		
> 		func(info);
> 		if (!refs && wait)
> 			csd_complete(&data->csd);
> 	}
> 
> I am afraid I missed something, and the code above looks wrong
> because it does csd_unlock() first, then csd_complete().

That does look a bit weird, but

> But if wait == T, then nobody can reuse this per-cpu entry, the
> caller of smp_call_function_many() must spin in csd_wait() on
> the same CPU.

is indeed correct.

> What do you think?

While I would say, don't do that to your deadlock scenario, I do like
the extra freedom this provides, so I'm inclined to go with this. Let me
spin a new patch and build a kernel with it ;-)


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

* Re: [PATCH] generic-smp: remove kmalloc()
  2009-02-17 17:40       ` Peter Zijlstra
@ 2009-02-17 17:46         ` Peter Zijlstra
  2009-02-17 18:30           ` Oleg Nesterov
  2009-02-17 19:29         ` [PATCH -v4] generic-ipi: " Peter Zijlstra
  1 sibling, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 17:46 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, 2009-02-17 at 18:40 +0100, Peter Zijlstra wrote:

> > What do you think?
> 
> While I would say, don't do that to your deadlock scenario, I do like
> the extra freedom this provides, so I'm inclined to go with this. Let me
> spin a new patch and build a kernel with it ;-)

Should we do the same for the _single case? It too can copy the csd data
into the stack and return before calling func.


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

* Re: [PATCH] generic-smp: remove kmalloc()
  2009-02-17 17:46         ` Peter Zijlstra
@ 2009-02-17 18:30           ` Oleg Nesterov
  0 siblings, 0 replies; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-17 18:30 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/17, Peter Zijlstra wrote:
>
> On Tue, 2009-02-17 at 18:40 +0100, Peter Zijlstra wrote:
>
> > > What do you think?
> >
> > While I would say, don't do that to your deadlock scenario, I do like
> > the extra freedom this provides, so I'm inclined to go with this. Let me
> > spin a new patch and build a kernel with it ;-)
>
> Should we do the same for the _single case? It too can copy the csd data
> into the stack and return before calling func.

If you are going to change _many(), then I'd say it makes sense to change
_single() too...

But, Peter, please think twice ;) I am not really sure we need this change.

Let's look at the deadlock scenario again,

        void func(void *arg)
        {
                lock(LOCK);
                unlock(LOCK);
        }

        smp_call_function(func, NULL, 0);

        lock(LOCK);
        smp_call_function(another_func, NULL, 0);
        unlock(LOCK);

First of all, if this LOCK can be locked from irq, then we need
lock_irq(LOCK) before the 2nd smp_call_function(). Yes, smp_call_function()
removes smp_processor_id() from ->mask, but still the code above is hardly
correct.

Otoh, perhaps we can allow to call smp_call_function() with irqs disabled
(unless wait of course), in that case this change makes a bit more sense.
Perhaps.

So this all is up to you. If you think this change does not complicate
the code too much, then we can count it as minor optimization, because it
can speedup smp_call_function() sometimes.

Oleg.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 11:26                       ` Nick Piggin
  2009-02-17 11:48                         ` Peter Zijlstra
  2009-02-17 15:51                         ` Paul E. McKenney
@ 2009-02-17 19:28                         ` Oleg Nesterov
  2009-02-17 21:32                           ` Paul E. McKenney
  2009-02-18  2:21                         ` Suresh Siddha
  3 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-17 19:28 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Peter Zijlstra, Jens Axboe, Suresh Siddha, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On 02/17, Nick Piggin wrote:
>
> How's this?

To me, this patch makes the code much more clean/understandable.

And imho it is very good it removes smp_read_barrier_depends()s
which (I think) were just wrong.


But I still have the question,

> Does any architecture actually needs barriers? For the initiator I
> could see it, but for the handler I would be surprised. The other
> thing we could do for simplicity is just to require that a full
> barrier is required before generating an IPI, and after receiving an
> IPI. We can't just do that in generic code without auditing
> architectures. There have been subtle hangs here on some archs in
> the past.

OK, so we add the barrier here:

> @@ -104,6 +111,14 @@ void generic_smp_call_function_interrupt
>  	int cpu = get_cpu();
>
>  	/*
> +	 * Ensure entry is visible on call_function_queue after we have
> +	 * entered the IPI. See comment in smp_call_function_many.
> +	 * If we don't have this, then we may miss an entry on the list
> +	 * and never get another IPI to process it.
> +	 */
> +	smp_mb();

But, any arch which needs this barrier should also call mb() in, say,
smp_reschedule_interrupt() path. Otherwise we can miss TIF_NEED_RESCHED
after return from the handler.

So the question is: is there any arch which surely needs this barrier?

IOW,
	int COND;

	void smp_xxx_interrupt(regs)
	{
		BUG_ON(!COND);
	}

	COND = 1;
	mb();
	smp_send_xxx(cpu);

can we really hit the BUG_ON() above on some arch?


(but in any case I agree, it is better to be safe and add the barrier
 like this patch does).

Oleg.


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

* [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 17:40       ` Peter Zijlstra
  2009-02-17 17:46         ` Peter Zijlstra
@ 2009-02-17 19:29         ` Peter Zijlstra
  2009-02-17 20:02           ` Oleg Nesterov
  2009-02-17 21:30           ` Paul E. McKenney
  1 sibling, 2 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 19:29 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, 2009-02-17 at 18:40 +0100, Peter Zijlstra wrote:
> Let me spin a new patch and build a kernel with it ;-)

On top of Nick's patch.

My quad has been happily building kernels with this the past 30 minutes
or so.

---
Subject: generic-ipi: remove kmalloc()

Remove the use of kmalloc() from the smp_call_function_*() calls.

Steven's generic-ipi patch (d7240b98: generic-ipi: use per cpu data for
single cpu ipi calls) started the discussion on the use of kmalloc() in
this code and fixed the smp_call_function_single(.wait=0) fallback case.

In this patch we complete this by also providing means for the _many()
call, which fully removes the need for kmalloc() in this code.

The problem with the _many() call is that other cpus might still be
observing our entry when we're done with it. It solved this by
dynamically allocating data elements and RCU-freeing it.

We solve it by using a single per-cpu entry which provides static
storage and solves one half of the problem (avoiding referencing freed
data).

The other half, ensuring the queue iteration it still possible, is done
by placing re-used entries at the head of the list. This means that if
someone was still iterating that entry when it got moved will now
re-visit the entries on the list it had already seen, but avoids
skipping over entries like would have happened had we placed the new
entry at the end.

Furthermore, visiting entries twice is not a problem, since we remove
our cpu from the entry's cpumask once its called.

We also optimize both the _single() and _many() interrupt handler to
copy the entry to their local stack and freeing it for re-use before we
call the function.

Many thanks to Oleg for his suggestions and poking him holes in my
earlier attempts.

Signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
---
 smp.c |  285 +++++++++++++++++++++++++++++++++++++++---------------------------
 1 file changed, 171 insertions(+), 114 deletions(-)

Index: linux-2.6/kernel/smp.c
===================================================================
--- linux-2.6.orig/kernel/smp.c
+++ linux-2.6/kernel/smp.c
@@ -10,23 +10,28 @@
 #include <linux/rcupdate.h>
 #include <linux/rculist.h>
 #include <linux/smp.h>
+#include <linux/cpu.h>
 
 static DEFINE_PER_CPU(struct call_single_queue, call_single_queue);
-static LIST_HEAD(call_function_queue);
-__cacheline_aligned_in_smp DEFINE_SPINLOCK(call_function_lock);
+
+static struct {
+	struct list_head	queue;
+	spinlock_t		lock;
+} call_function __cacheline_aligned_in_smp = {
+	.queue = LIST_HEAD_INIT(call_function.queue),
+	.lock  = __SPIN_LOCK_UNLOCKED(call_function.lock),
+};
 
 enum {
 	CSD_FLAG_WAIT		= 0x01,
-	CSD_FLAG_ALLOC		= 0x02,
-	CSD_FLAG_LOCK		= 0x04,
+	CSD_FLAG_LOCK		= 0x02,
 };
 
 struct call_function_data {
 	struct call_single_data csd;
 	spinlock_t lock;
 	unsigned int refs;
-	struct rcu_head rcu_head;
-	unsigned long cpumask_bits[];
+	cpumask_var_t cpumask;
 };
 
 struct call_single_queue {
@@ -34,8 +39,45 @@ struct call_single_queue {
 	spinlock_t lock;
 };
 
+static DEFINE_PER_CPU(struct call_function_data, cfd_data) = {
+	.lock = __SPIN_LOCK_UNLOCKED(cfd_data.lock),
+};
+
+static int
+hotplug_cfd(struct notifier_block *nfb, unsigned long action, void *hcpu)
+{
+	long cpu = (long)hcpu;
+	struct call_function_data *cfd = &per_cpu(cfd_data, cpu);
+
+	switch (action) {
+	case CPU_UP_PREPARE:
+	case CPU_UP_PREPARE_FROZEN:
+		if (!alloc_cpumask_var_node(&cfd->cpumask, GFP_KERNEL,
+				cpu_to_node(cpu)))
+			return NOTIFY_BAD;
+		break;
+
+#ifdef CONFIG_CPU_HOTPLUG
+	case CPU_UP_CANCELED:
+	case CPU_UP_CANCELED_FROZEN:
+
+	case CPU_DEAD:
+	case CPU_DEAD_FROZEN:
+		free_cpumask_var(cfd->cpumask);
+		break;
+#endif
+	};
+
+	return NOTIFY_OK;
+}
+
+static struct notifier_block __cpuinitdata hotplug_cfd_notifier = {
+	.notifier_call = hotplug_cfd,
+};
+
 static int __cpuinit init_call_single_data(void)
 {
+	void *cpu = (void *)(long)smp_processor_id();
 	int i;
 
 	for_each_possible_cpu(i) {
@@ -44,18 +86,59 @@ static int __cpuinit init_call_single_da
 		spin_lock_init(&q->lock);
 		INIT_LIST_HEAD(&q->list);
 	}
+
+	hotplug_cfd(&hotplug_cfd_notifier, CPU_UP_PREPARE, cpu);
+	register_cpu_notifier(&hotplug_cfd_notifier);
+
 	return 0;
 }
 early_initcall(init_call_single_data);
 
-static void csd_flag_wait(struct call_single_data *data)
+/*
+ * csd_wait/csd_complete are used for synchronous ipi calls
+ */
+static void csd_wait_prepare(struct call_single_data *data)
+{
+	data->flags |= CSD_FLAG_WAIT;
+}
+
+static void csd_complete(struct call_single_data *data)
+{
+	/*
+	 * Serialize stores to data with the flag clear and wakeup.
+	 */
+	smp_wmb();
+	data->flags &= ~CSD_FLAG_WAIT;
+}
+
+static void csd_wait(struct call_single_data *data)
+{
+	while (data->flags & CSD_FLAG_WAIT)
+		cpu_relax();
+}
+
+/*
+ * csd_lock/csd_unlock used to serialize access to per-cpu csd resources
+ *
+ * For non-synchronous ipi calls the csd can still be in use by the previous
+ * function call. For multi-cpu calls its even more interesting as we'll have
+ * to ensure no other cpu is observing our csd.
+ */
+static void csd_lock(struct call_single_data *data)
 {
-	/* Wait for response */
-	do {
-		if (!(data->flags & CSD_FLAG_WAIT))
-			break;
+	while (data->flags & CSD_FLAG_LOCK)
 		cpu_relax();
-	} while (1);
+	data->flags = CSD_FLAG_LOCK;
+}
+
+static void csd_unlock(struct call_single_data *data)
+{
+	WARN_ON(!(data->flags & CSD_FLAG_LOCK));
+	/*
+	 * Serialize stores to data with the flags clear.
+	 */
+	smp_wmb();
+	data->flags &= ~CSD_FLAG_LOCK;
 }
 
 /*
@@ -89,16 +172,7 @@ static void generic_exec_single(int cpu,
 		arch_send_call_function_single_ipi(cpu);
 
 	if (wait)
-		csd_flag_wait(data);
-}
-
-static void rcu_free_call_data(struct rcu_head *head)
-{
-	struct call_function_data *data;
-
-	data = container_of(head, struct call_function_data, rcu_head);
-
-	kfree(data);
+		csd_wait(data);
 }
 
 /*
@@ -122,41 +196,36 @@ void generic_smp_call_function_interrupt
 	 * It's ok to use list_for_each_rcu() here even though we may delete
 	 * 'pos', since list_del_rcu() doesn't clear ->next
 	 */
-	rcu_read_lock();
-	list_for_each_entry_rcu(data, &call_function_queue, csd.list) {
-		int refs;
-
-		if (!cpumask_test_cpu(cpu, to_cpumask(data->cpumask_bits)))
-			continue;
-
-		data->csd.func(data->csd.info);
+	list_for_each_entry_rcu(data, &call_function.queue, csd.list) {
+		void (*func)(void *);
+		void *info;
+		int refs, wait;
 
 		spin_lock(&data->lock);
-		cpumask_clear_cpu(cpu, to_cpumask(data->cpumask_bits));
+		if (!cpumask_test_cpu(cpu, data->cpumask)) {
+			spin_unlock(&data->lock);
+			continue;
+		}
+		cpumask_clear_cpu(cpu, data->cpumask);
 		WARN_ON(data->refs == 0);
-		data->refs--;
-		refs = data->refs;
+		refs = --data->refs;
+		func = data->csd.func;
+		info = data->csd.info;
+		wait = (data->csd.flags & CSD_FLAG_WAIT);
 		spin_unlock(&data->lock);
 
-		if (refs)
-			continue;
+		if (!refs) {
+			spin_lock(&call_function.lock);
+			list_del_rcu(&data->csd.list);
+			spin_unlock(&call_function.lock);
+			csd_unlock(&data->csd);
+		}
 
-		spin_lock(&call_function_lock);
-		list_del_rcu(&data->csd.list);
-		spin_unlock(&call_function_lock);
+		func(info);
 
-		if (data->csd.flags & CSD_FLAG_WAIT) {
-			/*
-			 * serialize stores to data with the flag clear
-			 * and wakeup
-			 */
-			smp_wmb();
-			data->csd.flags &= ~CSD_FLAG_WAIT;
-		}
-		if (data->csd.flags & CSD_FLAG_ALLOC)
-			call_rcu(&data->rcu_head, rcu_free_call_data);
+		if (!refs && wait)
+			csd_complete(&data->csd);
 	}
-	rcu_read_unlock();
 
 	put_cpu();
 }
@@ -169,7 +238,6 @@ void generic_smp_call_function_single_in
 {
 	struct call_single_queue *q = &__get_cpu_var(call_single_queue);
 	LIST_HEAD(list);
-	unsigned int data_flags;
 
 	spin_lock(&q->lock);
 	list_replace_init(&q->list, &list);
@@ -177,29 +245,28 @@ void generic_smp_call_function_single_in
 
 	while (!list_empty(&list)) {
 		struct call_single_data *data;
+		void (*func)(void *);
+		void *info;
+		int wait;
 
-		data = list_entry(list.next, struct call_single_data,
-					list);
+		data = list_first_entry(&list, struct call_single_data, list);
 		list_del(&data->list);
+		func = data->func;
+		info = data->info;
 
 		/*
-		 * 'data' can be invalid after this call if
-		 * flags == 0 (when called through
-		 * generic_exec_single(), so save them away before
-		 * making the call.
+		 * 'data' can be invalid after this if flags == 0 
+		 * when called through generic_exec_single()
 		 */
-		data_flags = data->flags;
+		wait = (data->flags & CSD_FLAG_WAIT);
 
-		data->func(data->info);
+		if (data->flags & CSD_FLAG_LOCK)
+			csd_unlock(data);
+
+		func(info);
 
-		if (data_flags & CSD_FLAG_WAIT) {
-			smp_wmb();
-			data->flags &= ~CSD_FLAG_WAIT;
-		} else if (data_flags & CSD_FLAG_LOCK) {
-			smp_wmb();
-			data->flags &= ~CSD_FLAG_LOCK;
-		} else if (data_flags & CSD_FLAG_ALLOC)
-			kfree(data);
+		if (wait)
+			csd_complete(data);
 	}
 }
 
@@ -218,7 +285,9 @@ static DEFINE_PER_CPU(struct call_single
 int smp_call_function_single(int cpu, void (*func) (void *info), void *info,
 			     int wait)
 {
-	struct call_single_data d;
+	struct call_single_data d = {
+		.flags = 0,
+	};
 	unsigned long flags;
 	/* prevent preemption and reschedule on another processor,
 	   as well as CPU removal */
@@ -239,13 +308,11 @@ int smp_call_function_single(int cpu, vo
 			/*
 			 * We are calling a function on a single CPU
 			 * and we are not going to wait for it to finish.
-			 * We first try to allocate the data, but if we
-			 * fail, we fall back to use a per cpu data to pass
-			 * the information to that CPU. Since all callers
-			 * of this code will use the same data, we must
-			 * synchronize the callers to prevent a new caller
-			 * from corrupting the data before the callee
-			 * can access it.
+			 * We use a per cpu data to pass the information to
+			 * that CPU. Since all callers of this code will
+			 * use the same data, we must synchronize the
+			 * callers to prevent a new caller from corrupting
+			 * the data before the callee can access it.
 			 *
 			 * The CSD_FLAG_LOCK is used to let us know when
 			 * the IPI handler is done with the data.
@@ -255,18 +322,11 @@ int smp_call_function_single(int cpu, vo
 			 * will make sure the callee is done with the
 			 * data before a new caller will use it.
 			 */
-			data = kmalloc(sizeof(*data), GFP_ATOMIC);
-			if (data)
-				data->flags = CSD_FLAG_ALLOC;
-			else {
-				data = &per_cpu(csd_data, me);
-				while (data->flags & CSD_FLAG_LOCK)
-					cpu_relax();
-				data->flags = CSD_FLAG_LOCK;
-			}
+			data = &per_cpu(csd_data, me);
+			csd_lock(data);
 		} else {
 			data = &d;
-			data->flags = CSD_FLAG_WAIT;
+			csd_wait_prepare(data);
 		}
 
 		data->func = func;
@@ -326,14 +386,14 @@ void smp_call_function_many(const struct
 {
 	struct call_function_data *data;
 	unsigned long flags;
-	int cpu, next_cpu;
+	int cpu, next_cpu, me = smp_processor_id();
 
 	/* Can deadlock when called with interrupts disabled */
 	WARN_ON(irqs_disabled());
 
 	/* So, what's a CPU they want?  Ignoring this one. */
 	cpu = cpumask_first_and(mask, cpu_online_mask);
-	if (cpu == smp_processor_id())
+	if (cpu == me)
 		cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
 	/* No online cpus?  We're done. */
 	if (cpu >= nr_cpu_ids)
@@ -341,7 +401,7 @@ void smp_call_function_many(const struct
 
 	/* Do we have another CPU which isn't us? */
 	next_cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
-	if (next_cpu == smp_processor_id())
+	if (next_cpu == me)
 		next_cpu = cpumask_next_and(next_cpu, mask, cpu_online_mask);
 
 	/* Fastpath: do that cpu by itself. */
@@ -350,31 +410,28 @@ void smp_call_function_many(const struct
 		return;
 	}
 
-	data = kmalloc(sizeof(*data) + cpumask_size(), GFP_ATOMIC);
-	if (unlikely(!data)) {
-		/* Slow path. */
-		for_each_online_cpu(cpu) {
-			if (cpu == smp_processor_id())
-				continue;
-			if (cpumask_test_cpu(cpu, mask))
-				smp_call_function_single(cpu, func, info, wait);
-		}
-		return;
-	}
+	data = &per_cpu(cfd_data, me);
+	csd_lock(&data->csd);
 
-	spin_lock_init(&data->lock);
-	data->csd.flags = CSD_FLAG_ALLOC;
+	spin_lock_irqsave(&data->lock, flags);
 	if (wait)
-		data->csd.flags |= CSD_FLAG_WAIT;
+		csd_wait_prepare(&data->csd);
+
 	data->csd.func = func;
 	data->csd.info = info;
-	cpumask_and(to_cpumask(data->cpumask_bits), mask, cpu_online_mask);
-	cpumask_clear_cpu(smp_processor_id(), to_cpumask(data->cpumask_bits));
-	data->refs = cpumask_weight(to_cpumask(data->cpumask_bits));
-
-	spin_lock_irqsave(&call_function_lock, flags);
-	list_add_tail_rcu(&data->csd.list, &call_function_queue);
-	spin_unlock_irqrestore(&call_function_lock, flags);
+	cpumask_and(data->cpumask, mask, cpu_online_mask);
+	cpumask_clear_cpu(me, data->cpumask);
+	data->refs = cpumask_weight(data->cpumask);
+
+	spin_lock(&call_function.lock);
+	/*
+	 * Place then entry at the _HEAD_ of the list, so that any cpu still
+	 * observing the entry in generic_smp_call_function_interrupt() will
+	 * not miss any other list entries (instead it can see some twice).
+	 */
+	list_add_rcu(&data->csd.list, &call_function.queue);
+	spin_unlock(&call_function.lock);
+	spin_unlock_irqrestore(&data->lock, flags);
 
 	/*
 	 * Make the list addition visible before sending the ipi.
@@ -384,11 +441,11 @@ void smp_call_function_many(const struct
 	smp_mb();
 
 	/* Send a message to all CPUs in the map */
-	arch_send_call_function_ipi_mask(to_cpumask(data->cpumask_bits));
+	arch_send_call_function_ipi_mask(data->cpumask);
 
 	/* optionally wait for the CPUs to complete */
 	if (wait)
-		csd_flag_wait(&data->csd);
+		csd_wait(&data->csd);
 }
 EXPORT_SYMBOL(smp_call_function_many);
 
@@ -418,20 +475,20 @@ EXPORT_SYMBOL(smp_call_function);
 
 void ipi_call_lock(void)
 {
-	spin_lock(&call_function_lock);
+	spin_lock(&call_function.lock);
 }
 
 void ipi_call_unlock(void)
 {
-	spin_unlock(&call_function_lock);
+	spin_unlock(&call_function.lock);
 }
 
 void ipi_call_lock_irq(void)
 {
-	spin_lock_irq(&call_function_lock);
+	spin_lock_irq(&call_function.lock);
 }
 
 void ipi_call_unlock_irq(void)
 {
-	spin_unlock_irq(&call_function_lock);
+	spin_unlock_irq(&call_function.lock);
 }



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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 19:29         ` [PATCH -v4] generic-ipi: " Peter Zijlstra
@ 2009-02-17 20:02           ` Oleg Nesterov
  2009-02-17 20:11             ` Peter Zijlstra
  2009-02-17 21:30           ` Paul E. McKenney
  1 sibling, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-17 20:02 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/17, Peter Zijlstra wrote:
>
> +	list_for_each_entry_rcu(data, &call_function.queue, csd.list) {
> +		void (*func)(void *);
> +		void *info;
> +		int refs, wait;
>  
>  		spin_lock(&data->lock);
> -		cpumask_clear_cpu(cpu, to_cpumask(data->cpumask_bits));
> +		if (!cpumask_test_cpu(cpu, data->cpumask)) {
> +			spin_unlock(&data->lock);
> +			continue;
> +		}
> +		cpumask_clear_cpu(cpu, data->cpumask);
>  		WARN_ON(data->refs == 0);
> -		data->refs--;
> -		refs = data->refs;
> +		refs = --data->refs;
> +		func = data->csd.func;
> +		info = data->csd.info;
> +		wait = (data->csd.flags & CSD_FLAG_WAIT);
>  		spin_unlock(&data->lock);
>  
> -		if (refs)
> -			continue;
> +		if (!refs) {
> +			spin_lock(&call_function.lock);
> +			list_del_rcu(&data->csd.list);
> +			spin_unlock(&call_function.lock);
> +			csd_unlock(&data->csd);
> +		}
>  
> -		spin_lock(&call_function_lock);
> -		list_del_rcu(&data->csd.list);
> -		spin_unlock(&call_function_lock);
> +		func(info);
>  
> -		if (data->csd.flags & CSD_FLAG_WAIT) {
> -			/*
> -			 * serialize stores to data with the flag clear
> -			 * and wakeup
> -			 */
> -			smp_wmb();
> -			data->csd.flags &= ~CSD_FLAG_WAIT;
> -		}
> -		if (data->csd.flags & CSD_FLAG_ALLOC)
> -			call_rcu(&data->rcu_head, rcu_free_call_data);
> +		if (!refs && wait)
> +			csd_complete(&data->csd);

Argh! Sorry-sorry-sorry. I managed to confused myself and you.

Please return to the previous version. With this patch we do:

		func(info);

		if (!refs && wait)
			csd_complete();

but we can't do csd_complete(), we can not be sure other CPUs
has already passed this function call.

Of course we can add another counter, but I don't think this
worth the trouble.


As for the previous version. I am almost sure I missed something
again, but do we really need both CSD_FLAG_WAIT and CSD_FLAG_LOCK
flags? smp_call_function_many(wait => 1) can just wait for
!CSD_FLAG_LOCK.

Oleg.


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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 20:02           ` Oleg Nesterov
@ 2009-02-17 20:11             ` Peter Zijlstra
  2009-02-17 20:16               ` Peter Zijlstra
  0 siblings, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 20:11 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, 2009-02-17 at 21:02 +0100, Oleg Nesterov wrote:

> Argh! Sorry-sorry-sorry. I managed to confused myself and you.
> 
> Please return to the previous version. With this patch we do:
> 
> 		func(info);
> 
> 		if (!refs && wait)
> 			csd_complete();
> 
> but we can't do csd_complete(), we can not be sure other CPUs
> has already passed this function call.
> 
> Of course we can add another counter, but I don't think this
> worth the trouble.

Eek, you're right.

> As for the previous version. I am almost sure I missed something
> again, but do we really need both CSD_FLAG_WAIT and CSD_FLAG_LOCK
> flags? smp_call_function_many(wait => 1) can just wait for
> !CSD_FLAG_LOCK.

I suppose we can.


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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 20:11             ` Peter Zijlstra
@ 2009-02-17 20:16               ` Peter Zijlstra
  2009-02-17 20:44                 ` Oleg Nesterov
  2009-02-17 20:49                 ` Peter Zijlstra
  0 siblings, 2 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 20:16 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, 2009-02-17 at 21:11 +0100, Peter Zijlstra wrote:
> 
> > As for the previous version. I am almost sure I missed something
> > again, but do we really need both CSD_FLAG_WAIT and CSD_FLAG_LOCK
> > flags? smp_call_function_many(wait => 1) can just wait for
> > !CSD_FLAG_LOCK.
> 
> I suppose we can.

Ah, no, that would mean we cannot distinguish between the sync and async
case in generic_exec_single().


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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 20:16               ` Peter Zijlstra
@ 2009-02-17 20:44                 ` Oleg Nesterov
  2009-02-17 20:49                 ` Peter Zijlstra
  1 sibling, 0 replies; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-17 20:44 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/17, Peter Zijlstra wrote:
>
> On Tue, 2009-02-17 at 21:11 +0100, Peter Zijlstra wrote:
> >
> > > As for the previous version. I am almost sure I missed something
> > > again, but do we really need both CSD_FLAG_WAIT and CSD_FLAG_LOCK
> > > flags? smp_call_function_many(wait => 1) can just wait for
> > > !CSD_FLAG_LOCK.
> >
> > I suppose we can.
>
> Ah, no, that would mean we cannot distinguish between the sync and async
> case in generic_exec_single().

Yes, generic_exec_single() needs the "bool wait" argument, it can't
use (data->flags & CSD_SINGLE_FLAG) as "should we wait or not".

In any case, even if this change is worthwhile it needs a separate
patch, please forget for now.

Oleg.


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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 20:16               ` Peter Zijlstra
  2009-02-17 20:44                 ` Oleg Nesterov
@ 2009-02-17 20:49                 ` Peter Zijlstra
  2009-02-17 22:09                   ` Oleg Nesterov
  1 sibling, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 20:49 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, 2009-02-17 at 21:16 +0100, Peter Zijlstra wrote:
> On Tue, 2009-02-17 at 21:11 +0100, Peter Zijlstra wrote:
> > 
> > > As for the previous version. I am almost sure I missed something
> > > again, but do we really need both CSD_FLAG_WAIT and CSD_FLAG_LOCK
> > > flags? smp_call_function_many(wait => 1) can just wait for
> > > !CSD_FLAG_LOCK.
> > 
> > I suppose we can.
> 
> Ah, no, that would mean we cannot distinguish between the sync and async
> case in generic_exec_single().

Which can of course be solved like so, we could even drop the new .wait
argument to __smp_call_function_single() as CSD_FLAG_WAIT wasn't exposed
before and non of the current users use it.

---
Index: linux-2.6/kernel/smp.c
===================================================================
--- linux-2.6.orig/kernel/smp.c
+++ linux-2.6/kernel/smp.c
@@ -23,8 +23,7 @@ static struct {
 };
 
 enum {
-	CSD_FLAG_WAIT		= 0x01,
-	CSD_FLAG_LOCK		= 0x02,
+	CSD_FLAG_LOCK		= 0x01,
 };
 
 struct call_function_data {
@@ -95,41 +94,21 @@ static int __cpuinit init_call_single_da
 early_initcall(init_call_single_data);
 
 /*
- * csd_wait/csd_complete are used for synchronous ipi calls
- */
-static void csd_wait_prepare(struct call_single_data *data)
-{
-	data->flags |= CSD_FLAG_WAIT;
-}
-
-static void csd_complete(struct call_single_data *data)
-{
-	if (data->flags & CSD_FLAG_WAIT) {
-		/*
-		 * Serialize stores to data with the flag clear and wakeup.
-		 */
-		smp_wmb();
-		data->flags &= ~CSD_FLAG_WAIT;
-	}
-}
-
-static void csd_wait(struct call_single_data *data)
-{
-	while (data->flags & CSD_FLAG_WAIT)
-		cpu_relax();
-}
-
-/*
  * csd_lock/csd_unlock used to serialize access to per-cpu csd resources
  *
  * For non-synchronous ipi calls the csd can still be in use by the previous
  * function call. For multi-cpu calls its even more interesting as we'll have
  * to ensure no other cpu is observing our csd.
  */
-static void csd_lock(struct call_single_data *data)
+static void csd_lock_wait(struct call_single_data *data)
 {
 	while (data->flags & CSD_FLAG_LOCK)
 		cpu_relax();
+}
+
+static void csd_lock(struct call_single_data *data)
+{
+	csd_lock_wait(data);
 	data->flags = CSD_FLAG_LOCK;
 }
 
@@ -147,11 +126,12 @@ static void csd_unlock(struct call_singl
  * Insert a previously allocated call_single_data element for execution
  * on the given CPU. data must already have ->func, ->info, and ->flags set.
  */
-static void generic_exec_single(int cpu, struct call_single_data *data)
+static
+void generic_exec_single(int cpu, struct call_single_data *data, int wait)
 {
 	struct call_single_queue *dst = &per_cpu(call_single_queue, cpu);
-	int wait = data->flags & CSD_FLAG_WAIT, ipi;
 	unsigned long flags;
+	int ipi;
 
 	spin_lock_irqsave(&dst->lock, flags);
 	ipi = list_empty(&dst->list);
@@ -174,7 +154,7 @@ static void generic_exec_single(int cpu,
 		arch_send_call_function_single_ipi(cpu);
 
 	if (wait)
-		csd_wait(data);
+		csd_lock_wait(data);
 }
 
 /*
@@ -222,7 +202,6 @@ void generic_smp_call_function_interrupt
 		if (refs)
 			continue;
 
-		csd_complete(&data->csd);
 		csd_unlock(&data->csd);
 	}
 
@@ -260,9 +239,6 @@ void generic_smp_call_function_single_in
 
 		data->func(data->info);
 
-		if (data_flags & CSD_FLAG_WAIT)
-			csd_complete(data);
-
 		/*
 		 * Unlocked CSDs are valid through generic_exec_single()
 		 */
@@ -303,36 +279,16 @@ int smp_call_function_single(int cpu, vo
 		func(info);
 		local_irq_restore(flags);
 	} else if ((unsigned)cpu < nr_cpu_ids && cpu_online(cpu)) {
-		struct call_single_data *data;
+		struct call_single_data *data = &d;
 
-		if (!wait) {
-			/*
-			 * We are calling a function on a single CPU
-			 * and we are not going to wait for it to finish.
-			 * We use a per cpu data to pass the information to
-			 * that CPU. Since all callers of this code will
-			 * use the same data, we must synchronize the
-			 * callers to prevent a new caller from corrupting
-			 * the data before the callee can access it.
-			 *
-			 * The CSD_FLAG_LOCK is used to let us know when
-			 * the IPI handler is done with the data.
-			 * The first caller will set it, and the callee
-			 * will clear it. The next caller must wait for
-			 * it to clear before we set it again. This
-			 * will make sure the callee is done with the
-			 * data before a new caller will use it.
-			 */
+		if (!wait)
 			data = &per_cpu(csd_data, me);
-			csd_lock(data);
-		} else {
-			data = &d;
-			csd_wait_prepare(data);
-		}
+
+		csd_lock(data);
 
 		data->func = func;
 		data->info = info;
-		generic_exec_single(cpu, data);
+		generic_exec_single(cpu, data, wait);
 	} else {
 		err = -ENXIO;	/* CPU not online */
 	}
@@ -352,12 +308,16 @@ EXPORT_SYMBOL(smp_call_function_single);
  * instance.
  *
  */
-void __smp_call_function_single(int cpu, struct call_single_data *data)
+void __smp_call_function_single(int cpu, struct call_single_data *data,
+				int wait)
 {
+	if (wait)
+		csd_lock(data);
+
 	/* Can deadlock when called with interrupts disabled */
-	WARN_ON((data->flags & CSD_FLAG_WAIT) && irqs_disabled());
+	WARN_ON(wait && irqs_disabled());
 
-	generic_exec_single(cpu, data);
+	generic_exec_single(cpu, data, wait);
 }
 
 /* FIXME: Shim for archs using old arch_send_call_function_ipi API. */
@@ -415,9 +375,6 @@ void smp_call_function_many(const struct
 	csd_lock(&data->csd);
 
 	spin_lock_irqsave(&data->lock, flags);
-	if (wait)
-		csd_wait_prepare(&data->csd);
-
 	data->csd.func = func;
 	data->csd.info = info;
 	cpumask_and(data->cpumask, mask, cpu_online_mask);
@@ -446,7 +403,7 @@ void smp_call_function_many(const struct
 
 	/* optionally wait for the CPUs to complete */
 	if (wait)
-		csd_wait(&data->csd);
+		csd_lock_wait(&data->csd);
 }
 EXPORT_SYMBOL(smp_call_function_many);
 
Index: linux-2.6/block/blk-softirq.c
===================================================================
--- linux-2.6.orig/block/blk-softirq.c
+++ linux-2.6/block/blk-softirq.c
@@ -64,7 +64,7 @@ static int raise_blk_irq(int cpu, struct
 		data->info = rq;
 		data->flags = 0;
 
-		__smp_call_function_single(cpu, data);
+		__smp_call_function_single(cpu, data, 0);
 		return 0;
 	}
 
Index: linux-2.6/kernel/sched.c
===================================================================
--- linux-2.6.orig/kernel/sched.c
+++ linux-2.6/kernel/sched.c
@@ -1120,7 +1120,7 @@ static void hrtick_start(struct rq *rq, 
 	if (rq == this_rq()) {
 		hrtimer_restart(timer);
 	} else if (!rq->hrtick_csd_pending) {
-		__smp_call_function_single(cpu_of(rq), &rq->hrtick_csd);
+		__smp_call_function_single(cpu_of(rq), &rq->hrtick_csd, 0);
 		rq->hrtick_csd_pending = 1;
 	}
 }
Index: linux-2.6/kernel/softirq.c
===================================================================
--- linux-2.6.orig/kernel/softirq.c
+++ linux-2.6/kernel/softirq.c
@@ -518,7 +518,7 @@ static int __try_remote_softirq(struct c
 		cp->flags = 0;
 		cp->priv = softirq;
 
-		__smp_call_function_single(cpu, cp);
+		__smp_call_function_single(cpu, cp, 0);
 		return 0;
 	}
 	return 1;
Index: linux-2.6/include/linux/smp.h
===================================================================
--- linux-2.6.orig/include/linux/smp.h
+++ linux-2.6/include/linux/smp.h
@@ -82,7 +82,8 @@ smp_call_function_mask(cpumask_t mask, v
 	return 0;
 }
 
-void __smp_call_function_single(int cpuid, struct call_single_data *data);
+void __smp_call_function_single(int cpuid, struct call_single_data *data,
+				int wait);
 
 /*
  * Generic and arch helpers



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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 19:29         ` [PATCH -v4] generic-ipi: " Peter Zijlstra
  2009-02-17 20:02           ` Oleg Nesterov
@ 2009-02-17 21:30           ` Paul E. McKenney
  2009-02-17 21:38             ` Peter Zijlstra
  1 sibling, 1 reply; 103+ messages in thread
From: Paul E. McKenney @ 2009-02-17 21:30 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Oleg Nesterov, Linus Torvalds, Nick Piggin, Jens Axboe,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, Feb 17, 2009 at 08:29:18PM +0100, Peter Zijlstra wrote:
> On Tue, 2009-02-17 at 18:40 +0100, Peter Zijlstra wrote:
> > Let me spin a new patch and build a kernel with it ;-)
> 
> On top of Nick's patch.
> 
> My quad has been happily building kernels with this the past 30 minutes
> or so.
> 
> ---
> Subject: generic-ipi: remove kmalloc()
> 
> Remove the use of kmalloc() from the smp_call_function_*() calls.
> 
> Steven's generic-ipi patch (d7240b98: generic-ipi: use per cpu data for
> single cpu ipi calls) started the discussion on the use of kmalloc() in
> this code and fixed the smp_call_function_single(.wait=0) fallback case.
> 
> In this patch we complete this by also providing means for the _many()
> call, which fully removes the need for kmalloc() in this code.
> 
> The problem with the _many() call is that other cpus might still be
> observing our entry when we're done with it. It solved this by
> dynamically allocating data elements and RCU-freeing it.
> 
> We solve it by using a single per-cpu entry which provides static
> storage and solves one half of the problem (avoiding referencing freed
> data).
> 
> The other half, ensuring the queue iteration it still possible, is done
> by placing re-used entries at the head of the list. This means that if
> someone was still iterating that entry when it got moved will now
> re-visit the entries on the list it had already seen, but avoids
> skipping over entries like would have happened had we placed the new
> entry at the end.
> 
> Furthermore, visiting entries twice is not a problem, since we remove
> our cpu from the entry's cpumask once its called.
> 
> We also optimize both the _single() and _many() interrupt handler to
> copy the entry to their local stack and freeing it for re-use before we
> call the function.
> 
> Many thanks to Oleg for his suggestions and poking him holes in my
> earlier attempts.
> 
> Signed-off-by: Peter Zijlstra <a.p.zijlstra@chello.nl>
> ---
>  smp.c |  285 +++++++++++++++++++++++++++++++++++++++---------------------------
>  1 file changed, 171 insertions(+), 114 deletions(-)
> 
> Index: linux-2.6/kernel/smp.c
> ===================================================================
> --- linux-2.6.orig/kernel/smp.c
> +++ linux-2.6/kernel/smp.c
> @@ -10,23 +10,28 @@
>  #include <linux/rcupdate.h>
>  #include <linux/rculist.h>
>  #include <linux/smp.h>
> +#include <linux/cpu.h>
> 
>  static DEFINE_PER_CPU(struct call_single_queue, call_single_queue);
> -static LIST_HEAD(call_function_queue);
> -__cacheline_aligned_in_smp DEFINE_SPINLOCK(call_function_lock);
> +
> +static struct {
> +	struct list_head	queue;
> +	spinlock_t		lock;
> +} call_function __cacheline_aligned_in_smp = {
> +	.queue = LIST_HEAD_INIT(call_function.queue),
> +	.lock  = __SPIN_LOCK_UNLOCKED(call_function.lock),
> +};
> 
>  enum {
>  	CSD_FLAG_WAIT		= 0x01,
> -	CSD_FLAG_ALLOC		= 0x02,
> -	CSD_FLAG_LOCK		= 0x04,
> +	CSD_FLAG_LOCK		= 0x02,
>  };
> 
>  struct call_function_data {
>  	struct call_single_data csd;
>  	spinlock_t lock;
>  	unsigned int refs;
> -	struct rcu_head rcu_head;
> -	unsigned long cpumask_bits[];
> +	cpumask_var_t cpumask;
>  };
> 
>  struct call_single_queue {
> @@ -34,8 +39,45 @@ struct call_single_queue {
>  	spinlock_t lock;
>  };
> 
> +static DEFINE_PER_CPU(struct call_function_data, cfd_data) = {
> +	.lock = __SPIN_LOCK_UNLOCKED(cfd_data.lock),
> +};
> +
> +static int
> +hotplug_cfd(struct notifier_block *nfb, unsigned long action, void *hcpu)
> +{
> +	long cpu = (long)hcpu;
> +	struct call_function_data *cfd = &per_cpu(cfd_data, cpu);
> +
> +	switch (action) {
> +	case CPU_UP_PREPARE:
> +	case CPU_UP_PREPARE_FROZEN:
> +		if (!alloc_cpumask_var_node(&cfd->cpumask, GFP_KERNEL,
> +				cpu_to_node(cpu)))
> +			return NOTIFY_BAD;
> +		break;
> +
> +#ifdef CONFIG_CPU_HOTPLUG
> +	case CPU_UP_CANCELED:
> +	case CPU_UP_CANCELED_FROZEN:
> +
> +	case CPU_DEAD:
> +	case CPU_DEAD_FROZEN:
> +		free_cpumask_var(cfd->cpumask);
> +		break;
> +#endif
> +	};
> +
> +	return NOTIFY_OK;
> +}
> +
> +static struct notifier_block __cpuinitdata hotplug_cfd_notifier = {
> +	.notifier_call = hotplug_cfd,
> +};
> +
>  static int __cpuinit init_call_single_data(void)
>  {
> +	void *cpu = (void *)(long)smp_processor_id();
>  	int i;
> 
>  	for_each_possible_cpu(i) {
> @@ -44,18 +86,59 @@ static int __cpuinit init_call_single_da
>  		spin_lock_init(&q->lock);
>  		INIT_LIST_HEAD(&q->list);
>  	}
> +
> +	hotplug_cfd(&hotplug_cfd_notifier, CPU_UP_PREPARE, cpu);
> +	register_cpu_notifier(&hotplug_cfd_notifier);
> +
>  	return 0;
>  }
>  early_initcall(init_call_single_data);
> 
> -static void csd_flag_wait(struct call_single_data *data)
> +/*
> + * csd_wait/csd_complete are used for synchronous ipi calls
> + */
> +static void csd_wait_prepare(struct call_single_data *data)
> +{
> +	data->flags |= CSD_FLAG_WAIT;
> +}
> +
> +static void csd_complete(struct call_single_data *data)
> +{
> +	/*
> +	 * Serialize stores to data with the flag clear and wakeup.
> +	 */
> +	smp_wmb();

Shouldn't the above be an smp_mb()?  There are reads preceding the calls
to csd_complete() that look to me like they need to remain ordered
before the flag-clearing below -- just in case of a quick reuse of this
call_single_data structure.

> +	data->flags &= ~CSD_FLAG_WAIT;
> +}
> +
> +static void csd_wait(struct call_single_data *data)
> +{
> +	while (data->flags & CSD_FLAG_WAIT)
> +		cpu_relax();
> +}
> +
> +/*
> + * csd_lock/csd_unlock used to serialize access to per-cpu csd resources
> + *
> + * For non-synchronous ipi calls the csd can still be in use by the previous
> + * function call. For multi-cpu calls its even more interesting as we'll have
> + * to ensure no other cpu is observing our csd.
> + */
> +static void csd_lock(struct call_single_data *data)
>  {
> -	/* Wait for response */
> -	do {
> -		if (!(data->flags & CSD_FLAG_WAIT))
> -			break;
> +	while (data->flags & CSD_FLAG_LOCK)
>  		cpu_relax();
> -	} while (1);
> +	data->flags = CSD_FLAG_LOCK;

OK, I'll bite...  Why don't we need a memory barrier here?

> +}
> +
> +static void csd_unlock(struct call_single_data *data)
> +{
> +	WARN_ON(!(data->flags & CSD_FLAG_LOCK));
> +	/*
> +	 * Serialize stores to data with the flags clear.
> +	 */
> +	smp_wmb();

I am a bit worried about this being smp_wmb() rather than smp_mb(),
but don't have a smoking gun.

> +	data->flags &= ~CSD_FLAG_LOCK;
>  }
> 
>  /*
> @@ -89,16 +172,7 @@ static void generic_exec_single(int cpu,
>  		arch_send_call_function_single_ipi(cpu);
> 
>  	if (wait)
> -		csd_flag_wait(data);
> -}
> -
> -static void rcu_free_call_data(struct rcu_head *head)
> -{
> -	struct call_function_data *data;
> -
> -	data = container_of(head, struct call_function_data, rcu_head);
> -
> -	kfree(data);
> +		csd_wait(data);
>  }
> 
>  /*
> @@ -122,41 +196,36 @@ void generic_smp_call_function_interrupt
>  	 * It's ok to use list_for_each_rcu() here even though we may delete
>  	 * 'pos', since list_del_rcu() doesn't clear ->next
>  	 */
> -	rcu_read_lock();
> -	list_for_each_entry_rcu(data, &call_function_queue, csd.list) {
> -		int refs;
> -
> -		if (!cpumask_test_cpu(cpu, to_cpumask(data->cpumask_bits)))
> -			continue;
> -
> -		data->csd.func(data->csd.info);
> +	list_for_each_entry_rcu(data, &call_function.queue, csd.list) {
> +		void (*func)(void *);
> +		void *info;
> +		int refs, wait;
> 
>  		spin_lock(&data->lock);
> -		cpumask_clear_cpu(cpu, to_cpumask(data->cpumask_bits));
> +		if (!cpumask_test_cpu(cpu, data->cpumask)) {
> +			spin_unlock(&data->lock);
> +			continue;
> +		}
> +		cpumask_clear_cpu(cpu, data->cpumask);
>  		WARN_ON(data->refs == 0);
> -		data->refs--;
> -		refs = data->refs;
> +		refs = --data->refs;
> +		func = data->csd.func;
> +		info = data->csd.info;
> +		wait = (data->csd.flags & CSD_FLAG_WAIT);
>  		spin_unlock(&data->lock);
> 
> -		if (refs)
> -			continue;
> +		if (!refs) {
> +			spin_lock(&call_function.lock);
> +			list_del_rcu(&data->csd.list);
> +			spin_unlock(&call_function.lock);
> +			csd_unlock(&data->csd);
> +		}
> 
> -		spin_lock(&call_function_lock);
> -		list_del_rcu(&data->csd.list);
> -		spin_unlock(&call_function_lock);
> +		func(info);
> 
> -		if (data->csd.flags & CSD_FLAG_WAIT) {
> -			/*
> -			 * serialize stores to data with the flag clear
> -			 * and wakeup
> -			 */
> -			smp_wmb();
> -			data->csd.flags &= ~CSD_FLAG_WAIT;
> -		}
> -		if (data->csd.flags & CSD_FLAG_ALLOC)
> -			call_rcu(&data->rcu_head, rcu_free_call_data);
> +		if (!refs && wait)
> +			csd_complete(&data->csd);
>  	}
> -	rcu_read_unlock();
> 
>  	put_cpu();
>  }
> @@ -169,7 +238,6 @@ void generic_smp_call_function_single_in
>  {
>  	struct call_single_queue *q = &__get_cpu_var(call_single_queue);
>  	LIST_HEAD(list);
> -	unsigned int data_flags;
> 
>  	spin_lock(&q->lock);
>  	list_replace_init(&q->list, &list);
> @@ -177,29 +245,28 @@ void generic_smp_call_function_single_in
> 
>  	while (!list_empty(&list)) {
>  		struct call_single_data *data;
> +		void (*func)(void *);
> +		void *info;
> +		int wait;
> 
> -		data = list_entry(list.next, struct call_single_data,
> -					list);
> +		data = list_first_entry(&list, struct call_single_data, list);
>  		list_del(&data->list);
> +		func = data->func;
> +		info = data->info;
> 
>  		/*
> -		 * 'data' can be invalid after this call if
> -		 * flags == 0 (when called through
> -		 * generic_exec_single(), so save them away before
> -		 * making the call.
> +		 * 'data' can be invalid after this if flags == 0 
> +		 * when called through generic_exec_single()
>  		 */
> -		data_flags = data->flags;
> +		wait = (data->flags & CSD_FLAG_WAIT);
> 
> -		data->func(data->info);
> +		if (data->flags & CSD_FLAG_LOCK)
> +			csd_unlock(data);
> +
> +		func(info);
> 
> -		if (data_flags & CSD_FLAG_WAIT) {
> -			smp_wmb();
> -			data->flags &= ~CSD_FLAG_WAIT;
> -		} else if (data_flags & CSD_FLAG_LOCK) {
> -			smp_wmb();
> -			data->flags &= ~CSD_FLAG_LOCK;
> -		} else if (data_flags & CSD_FLAG_ALLOC)
> -			kfree(data);
> +		if (wait)
> +			csd_complete(data);
>  	}
>  }
> 
> @@ -218,7 +285,9 @@ static DEFINE_PER_CPU(struct call_single
>  int smp_call_function_single(int cpu, void (*func) (void *info), void *info,
>  			     int wait)
>  {
> -	struct call_single_data d;
> +	struct call_single_data d = {
> +		.flags = 0,
> +	};

And about here I get lost -- trying to find what the heck this patch
applies to...  :-/

						Thanx, Paul

>  	unsigned long flags;
>  	/* prevent preemption and reschedule on another processor,
>  	   as well as CPU removal */
> @@ -239,13 +308,11 @@ int smp_call_function_single(int cpu, vo
>  			/*
>  			 * We are calling a function on a single CPU
>  			 * and we are not going to wait for it to finish.
> -			 * We first try to allocate the data, but if we
> -			 * fail, we fall back to use a per cpu data to pass
> -			 * the information to that CPU. Since all callers
> -			 * of this code will use the same data, we must
> -			 * synchronize the callers to prevent a new caller
> -			 * from corrupting the data before the callee
> -			 * can access it.
> +			 * We use a per cpu data to pass the information to
> +			 * that CPU. Since all callers of this code will
> +			 * use the same data, we must synchronize the
> +			 * callers to prevent a new caller from corrupting
> +			 * the data before the callee can access it.
>  			 *
>  			 * The CSD_FLAG_LOCK is used to let us know when
>  			 * the IPI handler is done with the data.
> @@ -255,18 +322,11 @@ int smp_call_function_single(int cpu, vo
>  			 * will make sure the callee is done with the
>  			 * data before a new caller will use it.
>  			 */
> -			data = kmalloc(sizeof(*data), GFP_ATOMIC);
> -			if (data)
> -				data->flags = CSD_FLAG_ALLOC;
> -			else {
> -				data = &per_cpu(csd_data, me);
> -				while (data->flags & CSD_FLAG_LOCK)
> -					cpu_relax();
> -				data->flags = CSD_FLAG_LOCK;
> -			}
> +			data = &per_cpu(csd_data, me);
> +			csd_lock(data);
>  		} else {
>  			data = &d;
> -			data->flags = CSD_FLAG_WAIT;
> +			csd_wait_prepare(data);
>  		}
> 
>  		data->func = func;
> @@ -326,14 +386,14 @@ void smp_call_function_many(const struct
>  {
>  	struct call_function_data *data;
>  	unsigned long flags;
> -	int cpu, next_cpu;
> +	int cpu, next_cpu, me = smp_processor_id();
> 
>  	/* Can deadlock when called with interrupts disabled */
>  	WARN_ON(irqs_disabled());
> 
>  	/* So, what's a CPU they want?  Ignoring this one. */
>  	cpu = cpumask_first_and(mask, cpu_online_mask);
> -	if (cpu == smp_processor_id())
> +	if (cpu == me)
>  		cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
>  	/* No online cpus?  We're done. */
>  	if (cpu >= nr_cpu_ids)
> @@ -341,7 +401,7 @@ void smp_call_function_many(const struct
> 
>  	/* Do we have another CPU which isn't us? */
>  	next_cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
> -	if (next_cpu == smp_processor_id())
> +	if (next_cpu == me)
>  		next_cpu = cpumask_next_and(next_cpu, mask, cpu_online_mask);
> 
>  	/* Fastpath: do that cpu by itself. */
> @@ -350,31 +410,28 @@ void smp_call_function_many(const struct
>  		return;
>  	}
> 
> -	data = kmalloc(sizeof(*data) + cpumask_size(), GFP_ATOMIC);
> -	if (unlikely(!data)) {
> -		/* Slow path. */
> -		for_each_online_cpu(cpu) {
> -			if (cpu == smp_processor_id())
> -				continue;
> -			if (cpumask_test_cpu(cpu, mask))
> -				smp_call_function_single(cpu, func, info, wait);
> -		}
> -		return;
> -	}
> +	data = &per_cpu(cfd_data, me);
> +	csd_lock(&data->csd);
> 
> -	spin_lock_init(&data->lock);
> -	data->csd.flags = CSD_FLAG_ALLOC;
> +	spin_lock_irqsave(&data->lock, flags);
>  	if (wait)
> -		data->csd.flags |= CSD_FLAG_WAIT;
> +		csd_wait_prepare(&data->csd);
> +
>  	data->csd.func = func;
>  	data->csd.info = info;
> -	cpumask_and(to_cpumask(data->cpumask_bits), mask, cpu_online_mask);
> -	cpumask_clear_cpu(smp_processor_id(), to_cpumask(data->cpumask_bits));
> -	data->refs = cpumask_weight(to_cpumask(data->cpumask_bits));
> -
> -	spin_lock_irqsave(&call_function_lock, flags);
> -	list_add_tail_rcu(&data->csd.list, &call_function_queue);
> -	spin_unlock_irqrestore(&call_function_lock, flags);
> +	cpumask_and(data->cpumask, mask, cpu_online_mask);
> +	cpumask_clear_cpu(me, data->cpumask);
> +	data->refs = cpumask_weight(data->cpumask);
> +
> +	spin_lock(&call_function.lock);
> +	/*
> +	 * Place then entry at the _HEAD_ of the list, so that any cpu still
> +	 * observing the entry in generic_smp_call_function_interrupt() will
> +	 * not miss any other list entries (instead it can see some twice).
> +	 */
> +	list_add_rcu(&data->csd.list, &call_function.queue);
> +	spin_unlock(&call_function.lock);
> +	spin_unlock_irqrestore(&data->lock, flags);
> 
>  	/*
>  	 * Make the list addition visible before sending the ipi.
> @@ -384,11 +441,11 @@ void smp_call_function_many(const struct
>  	smp_mb();
> 
>  	/* Send a message to all CPUs in the map */
> -	arch_send_call_function_ipi_mask(to_cpumask(data->cpumask_bits));
> +	arch_send_call_function_ipi_mask(data->cpumask);
> 
>  	/* optionally wait for the CPUs to complete */
>  	if (wait)
> -		csd_flag_wait(&data->csd);
> +		csd_wait(&data->csd);
>  }
>  EXPORT_SYMBOL(smp_call_function_many);
> 
> @@ -418,20 +475,20 @@ EXPORT_SYMBOL(smp_call_function);
> 
>  void ipi_call_lock(void)
>  {
> -	spin_lock(&call_function_lock);
> +	spin_lock(&call_function.lock);
>  }
> 
>  void ipi_call_unlock(void)
>  {
> -	spin_unlock(&call_function_lock);
> +	spin_unlock(&call_function.lock);
>  }
> 
>  void ipi_call_lock_irq(void)
>  {
> -	spin_lock_irq(&call_function_lock);
> +	spin_lock_irq(&call_function.lock);
>  }
> 
>  void ipi_call_unlock_irq(void)
>  {
> -	spin_unlock_irq(&call_function_lock);
> +	spin_unlock_irq(&call_function.lock);
>  }
> 
> 

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 19:28                         ` Q: " Oleg Nesterov
@ 2009-02-17 21:32                           ` Paul E. McKenney
  2009-02-17 21:45                             ` Oleg Nesterov
  0 siblings, 1 reply; 103+ messages in thread
From: Paul E. McKenney @ 2009-02-17 21:32 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Nick Piggin, Peter Zijlstra, Jens Axboe, Suresh Siddha,
	Linus Torvalds, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, Feb 17, 2009 at 08:28:10PM +0100, Oleg Nesterov wrote:
> On 02/17, Nick Piggin wrote:
> >
> > How's this?
> 
> To me, this patch makes the code much more clean/understandable.
> 
> And imho it is very good it removes smp_read_barrier_depends()s
> which (I think) were just wrong.
> 
> 
> But I still have the question,
> 
> > Does any architecture actually needs barriers? For the initiator I
> > could see it, but for the handler I would be surprised. The other
> > thing we could do for simplicity is just to require that a full
> > barrier is required before generating an IPI, and after receiving an
> > IPI. We can't just do that in generic code without auditing
> > architectures. There have been subtle hangs here on some archs in
> > the past.
> 
> OK, so we add the barrier here:
> 
> > @@ -104,6 +111,14 @@ void generic_smp_call_function_interrupt
> >  	int cpu = get_cpu();
> >
> >  	/*
> > +	 * Ensure entry is visible on call_function_queue after we have
> > +	 * entered the IPI. See comment in smp_call_function_many.
> > +	 * If we don't have this, then we may miss an entry on the list
> > +	 * and never get another IPI to process it.
> > +	 */
> > +	smp_mb();
> 
> But, any arch which needs this barrier should also call mb() in, say,
> smp_reschedule_interrupt() path. Otherwise we can miss TIF_NEED_RESCHED
> after return from the handler.
> 
> So the question is: is there any arch which surely needs this barrier?
> 
> IOW,
> 	int COND;
> 
> 	void smp_xxx_interrupt(regs)
> 	{
> 		BUG_ON(!COND);
> 	}
> 
> 	COND = 1;
> 	mb();
> 	smp_send_xxx(cpu);
> 
> can we really hit the BUG_ON() above on some arch?

If all of the above is executed by the same task, tripping the BUG_ON()
means either a compiler or CPU bug.

							Thanx, Paul

> (but in any case I agree, it is better to be safe and add the barrier
>  like this patch does).
> 
> Oleg.
> 

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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 21:30           ` Paul E. McKenney
@ 2009-02-17 21:38             ` Peter Zijlstra
  0 siblings, 0 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 21:38 UTC (permalink / raw)
  To: paulmck
  Cc: Oleg Nesterov, Linus Torvalds, Nick Piggin, Jens Axboe,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, 2009-02-17 at 13:30 -0800, Paul E. McKenney wrote:

> > +static void csd_complete(struct call_single_data *data)
> > +{
> > +	/*
> > +	 * Serialize stores to data with the flag clear and wakeup.
> > +	 */
> > +	smp_wmb();
> 
> Shouldn't the above be an smp_mb()?  There are reads preceding the calls
> to csd_complete() that look to me like they need to remain ordered
> before the flag-clearing below -- just in case of a quick reuse of this
> call_single_data structure.

Good point, however I just did a patch that made CSD_FLAG_WAIT go
away :-)

> > +	data->flags &= ~CSD_FLAG_WAIT;
> > +}
> > +
> > +static void csd_wait(struct call_single_data *data)
> > +{
> > +	while (data->flags & CSD_FLAG_WAIT)
> > +		cpu_relax();
> > +}
> > +
> > +/*
> > + * csd_lock/csd_unlock used to serialize access to per-cpu csd resources
> > + *
> > + * For non-synchronous ipi calls the csd can still be in use by the previous
> > + * function call. For multi-cpu calls its even more interesting as we'll have
> > + * to ensure no other cpu is observing our csd.
> > + */
> > +static void csd_lock(struct call_single_data *data)
> >  {
> > -	/* Wait for response */
> > -	do {
> > -		if (!(data->flags & CSD_FLAG_WAIT))
> > -			break;
> > +	while (data->flags & CSD_FLAG_LOCK)
> >  		cpu_relax();
> > -	} while (1);
> > +	data->flags = CSD_FLAG_LOCK;
> 
> OK, I'll bite...  Why don't we need a memory barrier here?

cpu_relax() is a compiler barrier, missing a memory barrier will just
make us spin this little while extra until the cacheline does hit us.

> > +}
> > +
> > +static void csd_unlock(struct call_single_data *data)
> > +{
> > +	WARN_ON(!(data->flags & CSD_FLAG_LOCK));
> > +	/*
> > +	 * Serialize stores to data with the flags clear.
> > +	 */
> > +	smp_wmb();
> 
> I am a bit worried about this being smp_wmb() rather than smp_mb(),
> but don't have a smoking gun.


		data->func(data->info);

		/*
		 * Unlocked CSDs are valid through generic_exec_single()
		 */
		if (data_flags & CSD_FLAG_LOCK)
			csd_unlock(data);

could the data->info read be delayed until after csd_unlock() ?

I'll make it an mb().


> And about here I get lost -- trying to find what the heck this patch
> applies to...  :-/

Right, I was in the process of sending out a full patch-set again.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 21:32                           ` Paul E. McKenney
@ 2009-02-17 21:45                             ` Oleg Nesterov
  2009-02-17 22:39                               ` Paul E. McKenney
  0 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-17 21:45 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: Nick Piggin, Peter Zijlstra, Jens Axboe, Suresh Siddha,
	Linus Torvalds, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On 02/17, Paul E. McKenney wrote:
>
> On Tue, Feb 17, 2009 at 08:28:10PM +0100, Oleg Nesterov wrote:
> >
> > So the question is: is there any arch which surely needs this barrier?
> >
> > IOW,
> > 	int COND;
> >
> > 	void smp_xxx_interrupt(regs)
> > 	{
> > 		BUG_ON(!COND);
> > 	}
> >
> > 	COND = 1;
> > 	mb();
> > 	smp_send_xxx(cpu);
> >
> > can we really hit the BUG_ON() above on some arch?
>
> If all of the above is executed by the same task, tripping the BUG_ON()
> means either a compiler or CPU bug.

I think you misunderstood...

smp_send_xxx() sends the ipi to another CPU, and smp_xxx_interrupt() is
the handler.

Oleg.


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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 20:49                 ` Peter Zijlstra
@ 2009-02-17 22:09                   ` Oleg Nesterov
  2009-02-17 22:15                     ` Peter Zijlstra
  0 siblings, 1 reply; 103+ messages in thread
From: Oleg Nesterov @ 2009-02-17 22:09 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On 02/17, Peter Zijlstra wrote:
>
> On Tue, 2009-02-17 at 21:16 +0100, Peter Zijlstra wrote:
> > On Tue, 2009-02-17 at 21:11 +0100, Peter Zijlstra wrote:
> > >
> > > > As for the previous version. I am almost sure I missed something
> > > > again, but do we really need both CSD_FLAG_WAIT and CSD_FLAG_LOCK
> > > > flags? smp_call_function_many(wait => 1) can just wait for
> > > > !CSD_FLAG_LOCK.
> > >
> > > I suppose we can.
> >
> > Ah, no, that would mean we cannot distinguish between the sync and async
> > case in generic_exec_single().
>
> Which can of course be solved like so, we could even drop the new .wait
> argument to __smp_call_function_single() as CSD_FLAG_WAIT wasn't exposed
> before and non of the current users use it.

Yes, I think this should work.

Or, if you prefer to stay with with 2 flags for now, then perhaps it makes
sense to at least clear both flags in xxx_interrupt() at once.

I guess you should re-send your previous patch in any case,

		} else if ((unsigned)cpu < nr_cpu_ids && cpu_online(cpu)) {
	-               struct call_single_data *data;
	+               struct call_single_data *data = {
	+                       .flags = 0,
	+               };

this looks a bit suspicious ;)

Oleg.


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

* Re: [PATCH -v4] generic-ipi: remove kmalloc()
  2009-02-17 22:09                   ` Oleg Nesterov
@ 2009-02-17 22:15                     ` Peter Zijlstra
  0 siblings, 0 replies; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-17 22:15 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Linus Torvalds, Nick Piggin, Jens Axboe, Paul E. McKenney,
	Ingo Molnar, Rusty Russell, Steven Rostedt, linux-kernel

On Tue, 2009-02-17 at 23:09 +0100, Oleg Nesterov wrote:

> I guess you should re-send your previous patch in any case,

Just send -v5 with 3 patches that form a full series against -tip.

> 		} else if ((unsigned)cpu < nr_cpu_ids && cpu_online(cpu)) {
> 	-               struct call_single_data *data;
> 	+               struct call_single_data *data = {
> 	+                       .flags = 0,
> 	+               };
> 
> this looks a bit suspicious ;)

Hehe, yes, I lost a refresh thereabouts. Should be fixed in the latest.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 21:45                             ` Oleg Nesterov
@ 2009-02-17 22:39                               ` Paul E. McKenney
  2009-02-18 13:52                                 ` Nick Piggin
  0 siblings, 1 reply; 103+ messages in thread
From: Paul E. McKenney @ 2009-02-17 22:39 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Nick Piggin, Peter Zijlstra, Jens Axboe, Suresh Siddha,
	Linus Torvalds, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, Feb 17, 2009 at 10:45:18PM +0100, Oleg Nesterov wrote:
> On 02/17, Paul E. McKenney wrote:
> >
> > On Tue, Feb 17, 2009 at 08:28:10PM +0100, Oleg Nesterov wrote:
> > >
> > > So the question is: is there any arch which surely needs this barrier?
> > >
> > > IOW,
> > > 	int COND;
> > >
> > > 	void smp_xxx_interrupt(regs)
> > > 	{
> > > 		BUG_ON(!COND);
> > > 	}
> > >
> > > 	COND = 1;
> > > 	mb();
> > > 	smp_send_xxx(cpu);
> > >
> > > can we really hit the BUG_ON() above on some arch?
> >
> > If all of the above is executed by the same task, tripping the BUG_ON()
> > means either a compiler or CPU bug.
> 
> I think you misunderstood...
> 
> smp_send_xxx() sends the ipi to another CPU, and smp_xxx_interrupt() is
> the handler.

You are right, I did miss that completely.  :-/

I have seen hardware in which the IPI could beat the cache invalidation
from the sending CPU to the interrupted CPU, and in which one or both of
the CPUs would have to execute special cache-flush/invalidation/whatever
instructions for the interrupted CPU to have a consistent view of the
data (in your example, "COND").

But we had a little chat with the hardware designers, and in subsequent
hardware, the IPI interacted with the cache-coherence protocol so as to
prevent the above bug from firing.  However, this was x86-based hardware,
which orders writes.  Weakly ordered systems would likely need a memory
barrier somewhere, whether as shown above or buried in the smp_send_xxx()
primitive.

							Thanx, Paul

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 15:51                         ` Paul E. McKenney
@ 2009-02-18  2:15                           ` Suresh Siddha
  2009-02-18  2:40                             ` Paul E. McKenney
  0 siblings, 1 reply; 103+ messages in thread
From: Suresh Siddha @ 2009-02-18  2:15 UTC (permalink / raw)
  To: paulmck
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, 2009-02-17 at 07:51 -0800, Paul E. McKenney wrote:
> On Tue, Feb 17, 2009 at 12:26:57PM +0100, Nick Piggin wrote:
> > +	spin_lock(&q->lock);
> > +	list_replace_init(&q->list, &list);
> > +	spin_unlock(&q->lock);
> 
> OK, I'll bite...
> 
> How do we avoid deadlock in the case where a pair of CPUs send to each
> other concurrently?

Sender takes the lock with interrupts-disabled. That should prevent any
deadlock, right?


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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 11:26                       ` Nick Piggin
                                           ` (2 preceding siblings ...)
  2009-02-17 19:28                         ` Q: " Oleg Nesterov
@ 2009-02-18  2:21                         ` Suresh Siddha
  2009-02-18 13:59                           ` Nick Piggin
  3 siblings, 1 reply; 103+ messages in thread
From: Suresh Siddha @ 2009-02-18  2:21 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Peter Zijlstra, Oleg Nesterov, Jens Axboe, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, 2009-02-17 at 03:26 -0800, Nick Piggin wrote:
> --
> Simplify the barriers in generic remote function call interrupt code.
> 
> Firstly, just unconditionally take the lock and check the list in the
> generic_call_function_single_interrupt IPI handler. As we've just taken
> an IPI here, the chances are fairly high that there will be work on the
> list for us, so do the locking unconditionally. This removes the tricky
> lockless list_empty check and dubious barriers. The change looks bigger
> than it is because it is just removing an outer loop.
> 
> Secondly, clarify architecture specific IPI locking rules. Generic code
> has no tools to impose any sane ordering on IPIs if they go outside
> normal cache coherency, ergo the arch code must make them appear to
> obey cache coherency as a "memory operation" to initiate an IPI, and
> a "memory operation" to receive one. This way at least they can be
> reasoned about in generic code, and smp_mb used to provide ordering. 
> 
> The combination of these two changes means that explict barriers can
> be taken out of queue handling for the single case -- shared data is
> explicitly locked, and ipi ordering must conform to that, so no
> barriers needed. An extra barrier is needed in the many handler, so
> as to ensure we load the list element after the IPI is received.
> 
> Does any architecture actually needs barriers? For the initiator I
> could see it, but for the handler I would be surprised. The other
> thing we could do for simplicity is just to require that a full
> barrier is required before generating an IPI, and after receiving an
> IPI. We can't just do that in generic code without auditing
> architectures. There have been subtle hangs here on some archs in
> the past.

x2apic register reads/writes don't have serializing semantics, as
opposed to uncached xapic accesses, which are inherently serializing.

With this patch, we need to fix the corresponding x2apic IPI operations.
I will take a look at it.

thanks,
suresh


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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18  2:15                           ` Suresh Siddha
@ 2009-02-18  2:40                             ` Paul E. McKenney
  0 siblings, 0 replies; 103+ messages in thread
From: Paul E. McKenney @ 2009-02-18  2:40 UTC (permalink / raw)
  To: Suresh Siddha
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, Feb 17, 2009 at 06:15:31PM -0800, Suresh Siddha wrote:
> On Tue, 2009-02-17 at 07:51 -0800, Paul E. McKenney wrote:
> > On Tue, Feb 17, 2009 at 12:26:57PM +0100, Nick Piggin wrote:
> > > +	spin_lock(&q->lock);
> > > +	list_replace_init(&q->list, &list);
> > > +	spin_unlock(&q->lock);
> > 
> > OK, I'll bite...
> > 
> > How do we avoid deadlock in the case where a pair of CPUs send to each
> > other concurrently?
> 
> Sender takes the lock with interrupts-disabled. That should prevent any
> deadlock, right?

You are of course correct!  Apologies for my confusion!!!

							Thanx, Paul

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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-17  9:43       ` Ingo Molnar
  2009-02-17  9:49         ` Peter Zijlstra
@ 2009-02-18  4:50         ` Rusty Russell
  2009-02-18 16:05           ` Ingo Molnar
  1 sibling, 1 reply; 103+ messages in thread
From: Rusty Russell @ 2009-02-18  4:50 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Peter Zijlstra, Linus Torvalds, Nick Piggin, Jens Axboe,
	Paul E. McKenney, Steven Rostedt, linux-kernel, Oleg Nesterov

On Tuesday 17 February 2009 20:13:59 Ingo Molnar wrote:
> We should not bend backwards trying to preserve that kmalloc() 
> [and prove that it's safe and race-free] - i.e. the burden of 
> proof is on the person insisting that it's needed, not on the 
> person wanting to remove it.

Respectfully disagree.  The kmalloc has been there for a very long time,
and doing fine AFAICT.

If the replacement is going to be subtle and tricksy, it had damn well
better be faster not just some geek code wank!

Cheers,
Rusty.

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-17 22:39                               ` Paul E. McKenney
@ 2009-02-18 13:52                                 ` Nick Piggin
  2009-02-18 16:09                                   ` Linus Torvalds
  0 siblings, 1 reply; 103+ messages in thread
From: Nick Piggin @ 2009-02-18 13:52 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: Oleg Nesterov, Peter Zijlstra, Jens Axboe, Suresh Siddha,
	Linus Torvalds, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, Feb 17, 2009 at 02:39:10PM -0800, Paul E. McKenney wrote:
> On Tue, Feb 17, 2009 at 10:45:18PM +0100, Oleg Nesterov wrote:
> > > If all of the above is executed by the same task, tripping the BUG_ON()
> > > means either a compiler or CPU bug.
> > 
> > I think you misunderstood...
> > 
> > smp_send_xxx() sends the ipi to another CPU, and smp_xxx_interrupt() is
> > the handler.
> 
> You are right, I did miss that completely.  :-/
> 
> I have seen hardware in which the IPI could beat the cache invalidation
> from the sending CPU to the interrupted CPU, and in which one or both of
> the CPUs would have to execute special cache-flush/invalidation/whatever
> instructions for the interrupted CPU to have a consistent view of the
> data (in your example, "COND").
> 
> But we had a little chat with the hardware designers, and in subsequent
> hardware, the IPI interacted with the cache-coherence protocol so as to
> prevent the above bug from firing.  However, this was x86-based hardware,
> which orders writes.  Weakly ordered systems would likely need a memory
> barrier somewhere, whether as shown above or buried in the smp_send_xxx()
> primitive.

I agree with you both that we *should* make arch interrupt code
do the ordering, but given the subtle lockups on some architectures
in this new code, I didn't want to make it significantly weaker...

Though perhaps it appears that I have, if I have removed an smp_mb
that x86 was relying on to emit an mfence to serialise the apic.

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18  2:21                         ` Suresh Siddha
@ 2009-02-18 13:59                           ` Nick Piggin
  2009-02-18 16:19                             ` Linus Torvalds
  2009-02-18 18:43                             ` Suresh Siddha
  0 siblings, 2 replies; 103+ messages in thread
From: Nick Piggin @ 2009-02-18 13:59 UTC (permalink / raw)
  To: Suresh Siddha
  Cc: Peter Zijlstra, Oleg Nesterov, Jens Axboe, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Tue, Feb 17, 2009 at 06:21:42PM -0800, Suresh B wrote:
> On Tue, 2009-02-17 at 03:26 -0800, Nick Piggin wrote:
> > --
> > Simplify the barriers in generic remote function call interrupt code.
> > 
> > Firstly, just unconditionally take the lock and check the list in the
> > generic_call_function_single_interrupt IPI handler. As we've just taken
> > an IPI here, the chances are fairly high that there will be work on the
> > list for us, so do the locking unconditionally. This removes the tricky
> > lockless list_empty check and dubious barriers. The change looks bigger
> > than it is because it is just removing an outer loop.
> > 
> > Secondly, clarify architecture specific IPI locking rules. Generic code
> > has no tools to impose any sane ordering on IPIs if they go outside
> > normal cache coherency, ergo the arch code must make them appear to
> > obey cache coherency as a "memory operation" to initiate an IPI, and
> > a "memory operation" to receive one. This way at least they can be
> > reasoned about in generic code, and smp_mb used to provide ordering. 
> > 
> > The combination of these two changes means that explict barriers can
> > be taken out of queue handling for the single case -- shared data is
> > explicitly locked, and ipi ordering must conform to that, so no
> > barriers needed. An extra barrier is needed in the many handler, so
> > as to ensure we load the list element after the IPI is received.
> > 
> > Does any architecture actually needs barriers? For the initiator I
> > could see it, but for the handler I would be surprised. The other
> > thing we could do for simplicity is just to require that a full
> > barrier is required before generating an IPI, and after receiving an
> > IPI. We can't just do that in generic code without auditing
> > architectures. There have been subtle hangs here on some archs in
> > the past.
> 
> x2apic register reads/writes don't have serializing semantics, as
> opposed to uncached xapic accesses, which are inherently serializing.
> 
> With this patch, we need to fix the corresponding x2apic IPI operations.
> I will take a look at it.

You're saying the problem is in generic_exec_single because I've
removed the smp_mb that inadvertently also serialises memory with
the x2apic on x86?

Indeed that could cause problems on some architectures which I
had hoped to avoid. So the patch is probably better off to first
add the smp_mb() to arch_send_call_function_xxx arch code, unless
it is immediately obvious or confirmed by arch maintainer that
such barrier is not required.


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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-18  4:50         ` Rusty Russell
@ 2009-02-18 16:05           ` Ingo Molnar
  2009-02-19  0:00             ` Jeremy Fitzhardinge
  2009-02-19  4:31             ` Rusty Russell
  0 siblings, 2 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 16:05 UTC (permalink / raw)
  To: Rusty Russell
  Cc: Peter Zijlstra, Linus Torvalds, Nick Piggin, Jens Axboe,
	Paul E. McKenney, Steven Rostedt, linux-kernel, Oleg Nesterov


* Rusty Russell <rusty@rustcorp.com.au> wrote:

> On Tuesday 17 February 2009 20:13:59 Ingo Molnar wrote:
> > We should not bend backwards trying to preserve that kmalloc() 
> > [and prove that it's safe and race-free] - i.e. the burden of 
> > proof is on the person insisting that it's needed, not on the 
> > person wanting to remove it.
> 
> Respectfully disagree.  The kmalloc has been there for a very long time,
> and doing fine AFAICT.

The kmalloc(GFP_ATOMIC) has been in kernel/smp.c for about half 
a year, since:

 |  commit 3d4422332711ef48ef0f132f1fcbfcbd56c7f3d1
 |  Author: Jens Axboe <jens.axboe@oracle.com>
 |  Date:   Thu Jun 26 11:21:34 2008 +0200
 |  Subject: [PATCH] Add generic helpers for arch IPI function calls

Which introduced kernel/smp.c to begin with. It got merged on 
July 15.

The precedecessor mechanism, smp_call_function*() on x86, used 
an on-kernel-stack buffer (and a half-wait mechanism), even in 
the async case, not a kmalloc().

I'm not sure where the "kmalloc has been there for a very long 
time" statement comes from. Are we talking about the same thing?

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 13:52                                 ` Nick Piggin
@ 2009-02-18 16:09                                   ` Linus Torvalds
  2009-02-18 16:21                                       ` Ingo Molnar
                                                       ` (2 more replies)
  0 siblings, 3 replies; 103+ messages in thread
From: Linus Torvalds @ 2009-02-18 16:09 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Suresh Siddha, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch



On Wed, 18 Feb 2009, Nick Piggin wrote:
> 
> I agree with you both that we *should* make arch interrupt code
> do the ordering, but given the subtle lockups on some architectures
> in this new code, I didn't want to make it significantly weaker...
> 
> Though perhaps it appears that I have, if I have removed an smp_mb
> that x86 was relying on to emit an mfence to serialise the apic.

The thing is, if the architecture doesn't order IPI wrt cache coherency, 
then the "smp_mb()" doesn't really do so _either_. 

It might hide some architecture-specific implementation issue, of course, 
so random amounts of "smp_mb()"s sprinkled around might well make some 
architecture "work", but it's in no way guaranteed. A smp_mb() does not 
guarantee that some separate IPI network is ordered - that may well take 
some random machine-specific IO cycle.

That said, at least on x86, taking an interrupt should be a serializing 
event, so there should be no reason for anything on the receiving side. 
The _sending_ side might need to make sure that there is serialization 
when generating the IPI (so that the IPI cannot happen while the writes 
are still in some per-CPU write buffer and haven't become part of the 
cache coherency domain).

And at least on x86 it's actually pretty hard to generate out-of-order 
accesses to begin with (_regardless_ of any issues external to the CPU). 
You have to work at it, and use a WC memory area, and I'm pretty sure we 
use UC for the apic accesses.

			Linus



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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 13:59                           ` Nick Piggin
@ 2009-02-18 16:19                             ` Linus Torvalds
  2009-02-18 16:23                               ` Ingo Molnar
  2009-02-18 18:43                             ` Suresh Siddha
  1 sibling, 1 reply; 103+ messages in thread
From: Linus Torvalds @ 2009-02-18 16:19 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Suresh Siddha, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch



On Wed, 18 Feb 2009, Nick Piggin wrote:
> > 
> > x2apic register reads/writes don't have serializing semantics, as
> > opposed to uncached xapic accesses, which are inherently serializing.
> > 
> > With this patch, we need to fix the corresponding x2apic IPI operations.
> > I will take a look at it.
> 
> You're saying the problem is in generic_exec_single because I've
> removed the smp_mb that inadvertently also serialises memory with
> the x2apic on x86?

I think Suresh is wrong on this.

The x2apic is using "wrmsr" to write events, and that's a serializing 
instruction. 

I really don't know of any way to get unordered information out of a x86 
core, except for playing games with WC memory, and WC memory would not be 
appropriate for something like an interrupt controller.

Of course, it's possible that Intel made the x2apic MSR's magic, and that 
they don't serialize, but that's very much against some very explicit 
Intel documentation. wrmsr is one of the (few) instructions that is 
mentioned all ove the documentation as being serializing.

			Linus

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 16:09                                   ` Linus Torvalds
  2009-02-18 16:21                                       ` Ingo Molnar
@ 2009-02-18 16:21                                       ` Ingo Molnar
  2009-02-19  6:47                                     ` Benjamin Herrenschmidt
  2 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 16:21 UTC (permalink / raw)
  To: Linus Torvalds, Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu
  Cc: Nick Piggin, Paul E. McKenney, Oleg Nesterov, Peter Zijlstra,
	Jens Axboe, Suresh Siddha, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch


* Linus Torvalds <torvalds@linux-foundation.org> wrote:

> On Wed, 18 Feb 2009, Nick Piggin wrote:
> > 
> > I agree with you both that we *should* make arch interrupt code
> > do the ordering, but given the subtle lockups on some architectures
> > in this new code, I didn't want to make it significantly weaker...
> > 
> > Though perhaps it appears that I have, if I have removed an smp_mb
> > that x86 was relying on to emit an mfence to serialise the apic.
> 
> The thing is, if the architecture doesn't order IPI wrt cache coherency, 
> then the "smp_mb()" doesn't really do so _either_. 
> 
> It might hide some architecture-specific implementation issue, of course, 
> so random amounts of "smp_mb()"s sprinkled around might well make some 
> architecture "work", but it's in no way guaranteed. A smp_mb() does not 
> guarantee that some separate IPI network is ordered - that may well take 
> some random machine-specific IO cycle.
> 
> That said, at least on x86, taking an interrupt should be a serializing 
> event, so there should be no reason for anything on the receiving side. 
> The _sending_ side might need to make sure that there is serialization 
> when generating the IPI (so that the IPI cannot happen while the writes 
> are still in some per-CPU write buffer and haven't become part of the 
> cache coherency domain).
> 
> And at least on x86 it's actually pretty hard to generate out-of-order 
> accesses to begin with (_regardless_ of any issues external to the CPU). 
> You have to work at it, and use a WC memory area, and I'm pretty sure we 
> use UC for the apic accesses.

yeah, we do. I do remember one x2apic related memory ordering 
bug though which happened in the past 6 months or so, lemme find 
the commit.

This one is it:

  d6f0f39: x86: add smp_mb() before sending INVALIDATE_TLB_VECTOR

attached below.

The reason for that is that x2apic changes the access sequence 
from mmio (which old lapic used to be, and which was mapped UC), 
to an MSR sequence:

 static inline void native_x2apic_icr_write(u32 low, u32 id)
 {
         wrmsrl(APIC_BASE_MSR + (APIC_ICR >> 4), ((__u64) id) << 32 | low);
 }

But ... WRMSR should already be serializing - it is documented 
as a serializing instruction.

I've cc:-ed Suresh & other APIC experts - exactly what type of 
hang did that patch fix? Do certain CPUs perhaps cut 
serialization corners, to speed up x2apic accesses?

	Ingo

------------------->
>From d6f0f39b7d05e62b347c4352d070e4afb3ade4b5 Mon Sep 17 00:00:00 2001
From: Suresh Siddha <suresh.b.siddha@intel.com>
Date: Tue, 4 Nov 2008 13:53:04 -0800
Subject: [PATCH] x86: add smp_mb() before sending INVALIDATE_TLB_VECTOR

Impact: fix rare x2apic hang

On x86, x2apic mode accesses for sending IPI's don't have serializing
semantics. If the IPI receivner refers(in lock-free fashion) to some
memory setup by the sender, the need for smp_mb() before sending the
IPI becomes critical in x2apic mode.

Add the smp_mb() in native_flush_tlb_others() before sending the IPI.

Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
Signed-off-by: Ingo Molnar <mingo@elte.hu>
---
 arch/x86/kernel/tlb_32.c |    6 ++++++
 arch/x86/kernel/tlb_64.c |    5 +++++
 2 files changed, 11 insertions(+), 0 deletions(-)

diff --git a/arch/x86/kernel/tlb_32.c b/arch/x86/kernel/tlb_32.c
index e00534b..f4049f3 100644
--- a/arch/x86/kernel/tlb_32.c
+++ b/arch/x86/kernel/tlb_32.c
@@ -154,6 +154,12 @@ void native_flush_tlb_others(const cpumask_t *cpumaskp, struct mm_struct *mm,
 	flush_mm = mm;
 	flush_va = va;
 	cpus_or(flush_cpumask, cpumask, flush_cpumask);
+
+	/*
+	 * Make the above memory operations globally visible before
+	 * sending the IPI.
+	 */
+	smp_mb();
 	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
diff --git a/arch/x86/kernel/tlb_64.c b/arch/x86/kernel/tlb_64.c
index dcbf7a1..8f919ca 100644
--- a/arch/x86/kernel/tlb_64.c
+++ b/arch/x86/kernel/tlb_64.c
@@ -183,6 +183,11 @@ void native_flush_tlb_others(const cpumask_t *cpumaskp, struct mm_struct *mm,
 	cpus_or(f->flush_cpumask, cpumask, f->flush_cpumask);
 
 	/*
+	 * Make the above memory operations globally visible before
+	 * sending the IPI.
+	 */
+	smp_mb();
+	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
 	 */

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
@ 2009-02-18 16:21                                       ` Ingo Molnar
  0 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 16:21 UTC (permalink / raw)
  To: Linus Torvalds, Pallipadi, Venkatesh, Yinghai Lu
  Cc: Nick Piggin, Paul E. McKenney, Oleg Nesterov, Peter Zijlstra,
	Jens Axboe, Suresh Siddha, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch


* Linus Torvalds <torvalds@linux-foundation.org> wrote:

> On Wed, 18 Feb 2009, Nick Piggin wrote:
> > 
> > I agree with you both that we *should* make arch interrupt code
> > do the ordering, but given the subtle lockups on some architectures
> > in this new code, I didn't want to make it significantly weaker...
> > 
> > Though perhaps it appears that I have, if I have removed an smp_mb
> > that x86 was relying on to emit an mfence to serialise the apic.
> 
> The thing is, if the architecture doesn't order IPI wrt cache coherency, 
> then the "smp_mb()" doesn't really do so _either_. 
> 
> It might hide some architecture-specific implementation issue, of course, 
> so random amounts of "smp_mb()"s sprinkled around might well make some 
> architecture "work", but it's in no way guaranteed. A smp_mb() does not 
> guarantee that some separate IPI network is ordered - that may well take 
> some random machine-specific IO cycle.
> 
> That said, at least on x86, taking an interrupt should be a serializing 
> event, so there should be no reason for anything on the receiving side. 
> The _sending_ side might need to make sure that there is serialization 
> when generating the IPI (so that the IPI cannot happen while the writes 
> are still in some per-CPU write buffer and haven't become part of the 
> cache coherency domain).
> 
> And at least on x86 it's actually pretty hard to generate out-of-order 
> accesses to begin with (_regardless_ of any issues external to the CPU). 
> You have to work at it, and use a WC memory area, and I'm pretty sure we 
> use UC for the apic accesses.

yeah, we do. I do remember one x2apic related memory ordering 
bug though which happened in the past 6 months or so, lemme find 
the commit.

This one is it:

  d6f0f39: x86: add smp_mb() before sending INVALIDATE_TLB_VECTOR

attached below.

The reason for that is that x2apic changes the access sequence 
from mmio (which old lapic used to be, and which was mapped UC), 
to an MSR sequence:

 static inline void native_x2apic_icr_write(u32 low, u32 id)
 {
         wrmsrl(APIC_BASE_MSR + (APIC_ICR >> 4), ((__u64) id) << 32 | low);
 }

But ... WRMSR should already be serializing - it is documented 
as a serializing instruction.

I've cc:-ed Suresh & other APIC experts - exactly what type of 
hang did that patch fix? Do certain CPUs perhaps cut 
serialization corners, to speed up x2apic accesses?

	Ingo

------------------->
From d6f0f39b7d05e62b347c4352d070e4afb3ade4b5 Mon Sep 17 00:00:00 2001
From: Suresh Siddha <suresh.b.siddha@intel.com>
Date: Tue, 4 Nov 2008 13:53:04 -0800
Subject: [PATCH] x86: add smp_mb() before sending INVALIDATE_TLB_VECTOR

Impact: fix rare x2apic hang

On x86, x2apic mode accesses for sending IPI's don't have serializing
semantics. If the IPI receivner refers(in lock-free fashion) to some
memory setup by the sender, the need for smp_mb() before sending the
IPI becomes critical in x2apic mode.

Add the smp_mb() in native_flush_tlb_others() before sending the IPI.

Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
Signed-off-by: Ingo Molnar <mingo@elte.hu>
---
 arch/x86/kernel/tlb_32.c |    6 ++++++
 arch/x86/kernel/tlb_64.c |    5 +++++
 2 files changed, 11 insertions(+), 0 deletions(-)

diff --git a/arch/x86/kernel/tlb_32.c b/arch/x86/kernel/tlb_32.c
index e00534b..f4049f3 100644
--- a/arch/x86/kernel/tlb_32.c
+++ b/arch/x86/kernel/tlb_32.c
@@ -154,6 +154,12 @@ void native_flush_tlb_others(const cpumask_t *cpumaskp, struct mm_struct *mm,
 	flush_mm = mm;
 	flush_va = va;
 	cpus_or(flush_cpumask, cpumask, flush_cpumask);
+
+	/*
+	 * Make the above memory operations globally visible before
+	 * sending the IPI.
+	 */
+	smp_mb();
 	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
diff --git a/arch/x86/kernel/tlb_64.c b/arch/x86/kernel/tlb_64.c
index dcbf7a1..8f919ca 100644
--- a/arch/x86/kernel/tlb_64.c
+++ b/arch/x86/kernel/tlb_64.c
@@ -183,6 +183,11 @@ void native_flush_tlb_others(const cpumask_t *cpumaskp, struct mm_struct *mm,
 	cpus_or(f->flush_cpumask, cpumask, f->flush_cpumask);
 
 	/*
+	 * Make the above memory operations globally visible before
+	 * sending the IPI.
+	 */
+	smp_mb();
+	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
 	 */

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
@ 2009-02-18 16:21                                       ` Ingo Molnar
  0 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 16:21 UTC (permalink / raw)
  To: Linus Torvalds, Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu
  Cc: Nick Piggin, Paul E. McKenney, Oleg Nesterov, Peter Zijlstra,
	Jens Axboe, Rusty Russell, Steven Rostedt, linux-kernel,
	linux-arch


* Linus Torvalds <torvalds@linux-foundation.org> wrote:

> On Wed, 18 Feb 2009, Nick Piggin wrote:
> > 
> > I agree with you both that we *should* make arch interrupt code
> > do the ordering, but given the subtle lockups on some architectures
> > in this new code, I didn't want to make it significantly weaker...
> > 
> > Though perhaps it appears that I have, if I have removed an smp_mb
> > that x86 was relying on to emit an mfence to serialise the apic.
> 
> The thing is, if the architecture doesn't order IPI wrt cache coherency, 
> then the "smp_mb()" doesn't really do so _either_. 
> 
> It might hide some architecture-specific implementation issue, of course, 
> so random amounts of "smp_mb()"s sprinkled around might well make some 
> architecture "work", but it's in no way guaranteed. A smp_mb() does not 
> guarantee that some separate IPI network is ordered - that may well take 
> some random machine-specific IO cycle.
> 
> That said, at least on x86, taking an interrupt should be a serializing 
> event, so there should be no reason for anything on the receiving side. 
> The _sending_ side might need to make sure that there is serialization 
> when generating the IPI (so that the IPI cannot happen while the writes 
> are still in some per-CPU write buffer and haven't become part of the 
> cache coherency domain).
> 
> And at least on x86 it's actually pretty hard to generate out-of-order 
> accesses to begin with (_regardless_ of any issues external to the CPU). 
> You have to work at it, and use a WC memory area, and I'm pretty sure we 
> use UC for the apic accesses.

yeah, we do. I do remember one x2apic related memory ordering 
bug though which happened in the past 6 months or so, lemme find 
the commit.

This one is it:

  d6f0f39: x86: add smp_mb() before sending INVALIDATE_TLB_VECTOR

attached below.

The reason for that is that x2apic changes the access sequence 
from mmio (which old lapic used to be, and which was mapped UC), 
to an MSR sequence:

 static inline void native_x2apic_icr_write(u32 low, u32 id)
 {
         wrmsrl(APIC_BASE_MSR + (APIC_ICR >> 4), ((__u64) id) << 32 | low);
 }

But ... WRMSR should already be serializing - it is documented 
as a serializing instruction.

I've cc:-ed Suresh & other APIC experts - exactly what type of 
hang did that patch fix? Do certain CPUs perhaps cut 
serialization corners, to speed up x2apic accesses?

	Ingo

------------------->
From d6f0f39b7d05e62b347c4352d070e4afb3ade4b5 Mon Sep 17 00:00:00 2001
From: Suresh Siddha <suresh.b.siddha@intel.com>
Date: Tue, 4 Nov 2008 13:53:04 -0800
Subject: [PATCH] x86: add smp_mb() before sending INVALIDATE_TLB_VECTOR

Impact: fix rare x2apic hang

On x86, x2apic mode accesses for sending IPI's don't have serializing
semantics. If the IPI receivner refers(in lock-free fashion) to some
memory setup by the sender, the need for smp_mb() before sending the
IPI becomes critical in x2apic mode.

Add the smp_mb() in native_flush_tlb_others() before sending the IPI.

Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
Signed-off-by: Ingo Molnar <mingo@elte.hu>
---
 arch/x86/kernel/tlb_32.c |    6 ++++++
 arch/x86/kernel/tlb_64.c |    5 +++++
 2 files changed, 11 insertions(+), 0 deletions(-)

diff --git a/arch/x86/kernel/tlb_32.c b/arch/x86/kernel/tlb_32.c
index e00534b..f4049f3 100644
--- a/arch/x86/kernel/tlb_32.c
+++ b/arch/x86/kernel/tlb_32.c
@@ -154,6 +154,12 @@ void native_flush_tlb_others(const cpumask_t *cpumaskp, struct mm_struct *mm,
 	flush_mm = mm;
 	flush_va = va;
 	cpus_or(flush_cpumask, cpumask, flush_cpumask);
+
+	/*
+	 * Make the above memory operations globally visible before
+	 * sending the IPI.
+	 */
+	smp_mb();
 	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
diff --git a/arch/x86/kernel/tlb_64.c b/arch/x86/kernel/tlb_64.c
index dcbf7a1..8f919ca 100644
--- a/arch/x86/kernel/tlb_64.c
+++ b/arch/x86/kernel/tlb_64.c
@@ -183,6 +183,11 @@ void native_flush_tlb_others(const cpumask_t *cpumaskp, struct mm_struct *mm,
 	cpus_or(f->flush_cpumask, cpumask, f->flush_cpumask);
 
 	/*
+	 * Make the above memory operations globally visible before
+	 * sending the IPI.
+	 */
+	smp_mb();
+	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
 	 */

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 16:19                             ` Linus Torvalds
@ 2009-02-18 16:23                               ` Ingo Molnar
  0 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 16:23 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Nick Piggin, Suresh Siddha, Peter Zijlstra, Oleg Nesterov,
	Jens Axboe, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch


* Linus Torvalds <torvalds@linux-foundation.org> wrote:

> On Wed, 18 Feb 2009, Nick Piggin wrote:
> > > 
> > > x2apic register reads/writes don't have serializing semantics, as
> > > opposed to uncached xapic accesses, which are inherently serializing.
> > > 
> > > With this patch, we need to fix the corresponding x2apic IPI operations.
> > > I will take a look at it.
> > 
> > You're saying the problem is in generic_exec_single because I've
> > removed the smp_mb that inadvertently also serialises memory with
> > the x2apic on x86?
> 
> I think Suresh is wrong on this.
> 
> The x2apic is using "wrmsr" to write events, and that's a 
> serializing instruction.
> 
> I really don't know of any way to get unordered information 
> out of a x86 core, except for playing games with WC memory, 
> and WC memory would not be appropriate for something like an 
> interrupt controller.
> 
> Of course, it's possible that Intel made the x2apic MSR's 
> magic, and that they don't serialize, but that's very much 
> against some very explicit Intel documentation. wrmsr is one 
> of the (few) instructions that is mentioned all ove the 
> documentation as being serializing.

heh, i just went through all those codepaths to figure out the 
SMP ordering semantics. I didnt find anything but the MSR write, 
so maybe the MSR writes did get weakened on certain CPUs.

Serializing is a serious performance penalty - and it would not 
be totally out of question to optimize xAPIC MSR accesses. If 
that's the case it's not quite nice to not document it though.

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 16:21                                       ` Ingo Molnar
  (?)
  (?)
@ 2009-02-18 16:33                                       ` Linus Torvalds
  2009-02-18 16:58                                         ` Ingo Molnar
  -1 siblings, 1 reply; 103+ messages in thread
From: Linus Torvalds @ 2009-02-18 16:33 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch



On Wed, 18 Feb 2009, Ingo Molnar wrote:
> 
> But ... WRMSR should already be serializing - it is documented 
> as a serializing instruction.

Hmm. I was thinking about this some more, and I think I've come up with an 
explanation.

"wrmsr" probably serializes _after_ doing the write. After all, it's 
historically used for changing internal CPU state, so you want to do the 
write, and then wait until the effects of the write are "stable" in the 
core.

That would explain how x2apic can use both a serializing instruction 
(wrmsr) and still effectively cause the IPI to happen out of sequence: the 
IPI can reach the destination CPU before the source CPU has flushed its 
store buffers, because the IPI is actually sent before serializing the 
core.

But I would very strongly put this in the "x2apic code bug" column. If 
this is a true issue (and your TLB patch does imply it is), then we should 
just make sure that the x2apic IPI calls always do a 'sfence' before they 
happen - regardless of whether they are for TLB flushes or for generic 
kernel cross-calls, or for anything else.

			Linus

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 16:21                                       ` Ingo Molnar
                                                         ` (2 preceding siblings ...)
  (?)
@ 2009-02-18 16:37                                       ` Gleb Natapov
  -1 siblings, 0 replies; 103+ messages in thread
From: Gleb Natapov @ 2009-02-18 16:37 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Linus Torvalds, Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu,
	Nick Piggin, Paul E. McKenney, Oleg Nesterov, Peter Zijlstra,
	Jens Axboe, Rusty Russell, Steven Rostedt, linux-kernel,
	linux-arch

On Wed, Feb 18, 2009 at 05:21:16PM +0100, Ingo Molnar wrote:
> 
> * Linus Torvalds <torvalds@linux-foundation.org> wrote:
> 
> > On Wed, 18 Feb 2009, Nick Piggin wrote:
> > > 
> > > I agree with you both that we *should* make arch interrupt code
> > > do the ordering, but given the subtle lockups on some architectures
> > > in this new code, I didn't want to make it significantly weaker...
> > > 
> > > Though perhaps it appears that I have, if I have removed an smp_mb
> > > that x86 was relying on to emit an mfence to serialise the apic.
> > 
> > The thing is, if the architecture doesn't order IPI wrt cache coherency, 
> > then the "smp_mb()" doesn't really do so _either_. 
> > 
> > It might hide some architecture-specific implementation issue, of course, 
> > so random amounts of "smp_mb()"s sprinkled around might well make some 
> > architecture "work", but it's in no way guaranteed. A smp_mb() does not 
> > guarantee that some separate IPI network is ordered - that may well take 
> > some random machine-specific IO cycle.
> > 
> > That said, at least on x86, taking an interrupt should be a serializing 
> > event, so there should be no reason for anything on the receiving side. 
> > The _sending_ side might need to make sure that there is serialization 
> > when generating the IPI (so that the IPI cannot happen while the writes 
> > are still in some per-CPU write buffer and haven't become part of the 
> > cache coherency domain).
> > 
> > And at least on x86 it's actually pretty hard to generate out-of-order 
> > accesses to begin with (_regardless_ of any issues external to the CPU). 
> > You have to work at it, and use a WC memory area, and I'm pretty sure we 
> > use UC for the apic accesses.
> 
> yeah, we do. I do remember one x2apic related memory ordering 
> bug though which happened in the past 6 months or so, lemme find 
> the commit.
> 
> This one is it:
> 
>   d6f0f39: x86: add smp_mb() before sending INVALIDATE_TLB_VECTOR
> 
> attached below.
> 
> The reason for that is that x2apic changes the access sequence 
> from mmio (which old lapic used to be, and which was mapped UC), 
> to an MSR sequence:
> 
>  static inline void native_x2apic_icr_write(u32 low, u32 id)
>  {
>          wrmsrl(APIC_BASE_MSR + (APIC_ICR >> 4), ((__u64) id) << 32 | low);
>  }
> 
> But ... WRMSR should already be serializing - it is documented 
> as a serializing instruction.
> 
FWIW that is what Intel docs says:

To allow for efficient access to the APIC registers in x2APIC mode,
the serializing semantics of WRMSR are relaxed when writing to the APIC
registers. Thus, system software should not use “WRMSR to APIC registers
in x2APIC mode” as a serializing instruction. Read and write accesses
to the APIC registers will occur in program order.

> I've cc:-ed Suresh & other APIC experts - exactly what type of 
> hang did that patch fix? Do certain CPUs perhaps cut 
> serialization corners, to speed up x2apic accesses?
> 
> 	Ingo
> 
> ------------------->
> >From d6f0f39b7d05e62b347c4352d070e4afb3ade4b5 Mon Sep 17 00:00:00 2001
> From: Suresh Siddha <suresh.b.siddha@intel.com>
> Date: Tue, 4 Nov 2008 13:53:04 -0800
> Subject: [PATCH] x86: add smp_mb() before sending INVALIDATE_TLB_VECTOR
> 
> Impact: fix rare x2apic hang
> 
> On x86, x2apic mode accesses for sending IPI's don't have serializing
> semantics. If the IPI receivner refers(in lock-free fashion) to some
> memory setup by the sender, the need for smp_mb() before sending the
> IPI becomes critical in x2apic mode.
> 
> Add the smp_mb() in native_flush_tlb_others() before sending the IPI.
> 
> Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
> Signed-off-by: Ingo Molnar <mingo@elte.hu>
> ---
>  arch/x86/kernel/tlb_32.c |    6 ++++++
>  arch/x86/kernel/tlb_64.c |    5 +++++
>  2 files changed, 11 insertions(+), 0 deletions(-)
> 
> diff --git a/arch/x86/kernel/tlb_32.c b/arch/x86/kernel/tlb_32.c
> index e00534b..f4049f3 100644
> --- a/arch/x86/kernel/tlb_32.c
> +++ b/arch/x86/kernel/tlb_32.c
> @@ -154,6 +154,12 @@ void native_flush_tlb_others(const cpumask_t *cpumaskp, struct mm_struct *mm,
>  	flush_mm = mm;
>  	flush_va = va;
>  	cpus_or(flush_cpumask, cpumask, flush_cpumask);
> +
> +	/*
> +	 * Make the above memory operations globally visible before
> +	 * sending the IPI.
> +	 */
> +	smp_mb();
>  	/*
>  	 * We have to send the IPI only to
>  	 * CPUs affected.
> diff --git a/arch/x86/kernel/tlb_64.c b/arch/x86/kernel/tlb_64.c
> index dcbf7a1..8f919ca 100644
> --- a/arch/x86/kernel/tlb_64.c
> +++ b/arch/x86/kernel/tlb_64.c
> @@ -183,6 +183,11 @@ void native_flush_tlb_others(const cpumask_t *cpumaskp, struct mm_struct *mm,
>  	cpus_or(f->flush_cpumask, cpumask, f->flush_cpumask);
>  
>  	/*
> +	 * Make the above memory operations globally visible before
> +	 * sending the IPI.
> +	 */
> +	smp_mb();
> +	/*
>  	 * We have to send the IPI only to
>  	 * CPUs affected.
>  	 */
> --
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at  http://www.tux.org/lkml/

--
			Gleb.

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 16:33                                       ` Linus Torvalds
@ 2009-02-18 16:58                                         ` Ingo Molnar
  2009-02-18 17:05                                           ` Ingo Molnar
  0 siblings, 1 reply; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 16:58 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch


* Linus Torvalds <torvalds@linux-foundation.org> wrote:

> On Wed, 18 Feb 2009, Ingo Molnar wrote:
> > 
> > But ... WRMSR should already be serializing - it is documented 
> > as a serializing instruction.
> 
> Hmm. I was thinking about this some more, and I think I've 
> come up with an explanation.
> 
> "wrmsr" probably serializes _after_ doing the write. After 
> all, it's historically used for changing internal CPU state, 
> so you want to do the write, and then wait until the effects 
> of the write are "stable" in the core.
> 
> That would explain how x2apic can use both a serializing 
> instruction (wrmsr) and still effectively cause the IPI to 
> happen out of sequence: the IPI can reach the destination CPU 
> before the source CPU has flushed its store buffers, because 
> the IPI is actually sent before serializing the core.
> 
> But I would very strongly put this in the "x2apic code bug" 
> column. If this is a true issue (and your TLB patch does imply 
> it is), then we should just make sure that the x2apic IPI 
> calls always do a 'sfence' before they happen - regardless of 
> whether they are for TLB flushes or for generic kernel 
> cross-calls, or for anything else.

Yeah, that makes perfect sense. IPIs are an out of band 
signalling mechanism that do not listen to the normal cache 
coherency rules.

Moving the smp_mb() to the x2apic specific code will also speed 
up the normal mmio-mapped IPI sequence a bit. It should be an 
smp_wmb() i suspect - which turns it into an sfence.

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 16:58                                         ` Ingo Molnar
@ 2009-02-18 17:05                                           ` Ingo Molnar
  2009-02-18 17:10                                             ` Ingo Molnar
  2009-02-18 17:14                                             ` Linus Torvalds
  0 siblings, 2 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 17:05 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch


an off-list comment pointed out this piece of information as 
well:

  http://www.sandpile.org/ia32/coherent.htm

  A WRMSR to one of the x2APIC MSRs (0000_0800h...0000_0BFFh) is 
  not guaranteed to be serializing.

So i suspect we should just enclose it in smp_mb() pairs to make 
sure it's a full barrier in both directions?

Although since it's an explicitly async mechanism with no 
ordering expectations whatsoever, just doing an smp_mb() before 
it should be enough, to make sure the IPI can never arrive to 
another CPU/core faster than the event-horizon/visibility of 
preceding operations.

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 17:05                                           ` Ingo Molnar
@ 2009-02-18 17:10                                             ` Ingo Molnar
  2009-02-18 17:17                                               ` Linus Torvalds
  2009-02-18 17:14                                             ` Linus Torvalds
  1 sibling, 1 reply; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 17:10 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch


ok, it's documented:

   Intel® 64 and IA-32 Architectures
   Software Developer’s Manual
                           Volume 3A:
       System Programming Guide, Part 1

 9.5.3        MSR Access in x2APIC Mode

 To allow for efficient access to the APIC registers in x2APIC 
 mode, the serializing semantics of WRMSR are relaxed when 
 writing to the APIC registers. Thus, system software should not 
 use “WRMSR to APIC registers in x2APIC mode” as a serializing 
 instruction. Read and write accesses to the APIC registers will 
 occur in program order. A WRMSR to an APIC register may 
 complete before all preceding stores are globally visible; 
 software can prevent this by inserting a serializing 
 instruction or MFENCE before the WRMSR.

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 17:05                                           ` Ingo Molnar
  2009-02-18 17:10                                             ` Ingo Molnar
@ 2009-02-18 17:14                                             ` Linus Torvalds
  2009-02-18 17:47                                               ` Ingo Molnar
  2009-02-18 18:33                                               ` Suresh Siddha
  1 sibling, 2 replies; 103+ messages in thread
From: Linus Torvalds @ 2009-02-18 17:14 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch



On Wed, 18 Feb 2009, Ingo Molnar wrote:
>
> an off-list comment pointed out this piece of information as 
> well:
> 
>   http://www.sandpile.org/ia32/coherent.htm
> 
>   A WRMSR to one of the x2APIC MSRs (0000_0800h...0000_0BFFh) is 
>   not guaranteed to be serializing.
> 
> So i suspect we should just enclose it in smp_mb() pairs to make 
> sure it's a full barrier in both directions?

Why would we care about "both directions"?

I think putting an sfence _before_ the wrmsr (and not even all of them - 
just put it in front of the "send IPI" sequence) should be fine. Any other 
ordering sounds like just unnecessary overhead to me.

We do want this to be low-overhead, even if we probably don't care _that_ 
much.

			Linus

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 17:10                                             ` Ingo Molnar
@ 2009-02-18 17:17                                               ` Linus Torvalds
  2009-02-18 17:23                                                 ` Ingo Molnar
  0 siblings, 1 reply; 103+ messages in thread
From: Linus Torvalds @ 2009-02-18 17:17 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch



On Wed, 18 Feb 2009, Ingo Molnar wrote:
> 
> ok, it's documented:
> 
>    Intel® 64 and IA-32 Architectures
>    Software Developer’s Manual
>                            Volume 3A:
>        System Programming Guide, Part 1
> 
>  9.5.3        MSR Access in x2APIC Mode

Well, mine says

	9.5.3 Error Handling

so I guess I need to download a newer version. After all, mine is 
"ancient" in being a year old (the February-2008 version) and not covering 
x2APIC at all.

			Linus

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 17:17                                               ` Linus Torvalds
@ 2009-02-18 17:23                                                 ` Ingo Molnar
  0 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 17:23 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch


* Linus Torvalds <torvalds@linux-foundation.org> wrote:

> On Wed, 18 Feb 2009, Ingo Molnar wrote:
> > 
> > ok, it's documented:
> > 
> >    Intel® 64 and IA-32 Architectures
> >    Software Developer’s Manual
> >                            Volume 3A:
> >        System Programming Guide, Part 1
> > 
> >  9.5.3        MSR Access in x2APIC Mode
> 
> Well, mine says
> 
> 	9.5.3 Error Handling
> 
> so I guess I need to download a newer version. After all, mine 
> is "ancient" in being a year old (the February-2008 version) 
> and not covering x2APIC at all.

Mine is 253668.pdf, that's the freshest available:

  http://download.intel.com/design/processor/manuals/253668.pdf

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 17:14                                             ` Linus Torvalds
@ 2009-02-18 17:47                                               ` Ingo Molnar
  2009-02-18 18:33                                               ` Suresh Siddha
  1 sibling, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 17:47 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Suresh Siddha, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch


* Linus Torvalds <torvalds@linux-foundation.org> wrote:

> 
> 
> On Wed, 18 Feb 2009, Ingo Molnar wrote:
> >
> > an off-list comment pointed out this piece of information as 
> > well:
> > 
> >   http://www.sandpile.org/ia32/coherent.htm
> > 
> >   A WRMSR to one of the x2APIC MSRs (0000_0800h...0000_0BFFh) is 
> >   not guaranteed to be serializing.
> > 
> > So i suspect we should just enclose it in smp_mb() pairs to make 
> > sure it's a full barrier in both directions?
> 
> Why would we care about "both directions"?
> 
> I think putting an sfence _before_ the wrmsr (and not even all 
> of them - just put it in front of the "send IPI" sequence) 
> should be fine. Any other ordering sounds like just 
> unnecessary overhead to me.
> 
> We do want this to be low-overhead, even if we probably don't 
> care _that_ much.

yeah, you are right, making sure prior stores become visible 
should be the only worry here.

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 17:14                                             ` Linus Torvalds
  2009-02-18 17:47                                               ` Ingo Molnar
@ 2009-02-18 18:33                                               ` Suresh Siddha
  1 sibling, 0 replies; 103+ messages in thread
From: Suresh Siddha @ 2009-02-18 18:33 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Ingo Molnar, Pallipadi, Venkatesh, Yinghai Lu, Nick Piggin,
	Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch,
	asit.k.mallick

On Wed, 2009-02-18 at 09:14 -0800, Linus Torvalds wrote:
> 
> On Wed, 18 Feb 2009, Ingo Molnar wrote:
> >
> > an off-list comment pointed out this piece of information as 
> > well:
> > 
> >   http://www.sandpile.org/ia32/coherent.htm
> > 
> >   A WRMSR to one of the x2APIC MSRs (0000_0800h...0000_0BFFh) is 
> >   not guaranteed to be serializing.
> > 
> > So i suspect we should just enclose it in smp_mb() pairs to make 
> > sure it's a full barrier in both directions?
> 
> Why would we care about "both directions"?

on x86 we don't need in both directions.

> 
> I think putting an sfence _before_ the wrmsr (and not even all of them - 
> just put it in front of the "send IPI" sequence) should be fine. Any other 
> ordering sounds like just unnecessary overhead to me.

For x2apic ipi's, we should use a serializing instruction or a "mfence"
instruction. "sfence" will not help in this scenario.

thanks,
suresh


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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 13:59                           ` Nick Piggin
  2009-02-18 16:19                             ` Linus Torvalds
@ 2009-02-18 18:43                             ` Suresh Siddha
  2009-02-18 19:17                               ` Ingo Molnar
  1 sibling, 1 reply; 103+ messages in thread
From: Suresh Siddha @ 2009-02-18 18:43 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Peter Zijlstra, Oleg Nesterov, Jens Axboe, Linus Torvalds,
	Paul E. McKenney, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Wed, 2009-02-18 at 05:59 -0800, Nick Piggin wrote:
> You're saying the problem is in generic_exec_single because I've
> removed the smp_mb that inadvertently also serialises memory with
> the x2apic on x86?

yes.

> 
> Indeed that could cause problems on some architectures which I
> had hoped to avoid. So the patch is probably better off to first
> add the smp_mb() to arch_send_call_function_xxx arch code, unless
> it is immediately obvious or confirmed by arch maintainer that
> such barrier is not required.

For x2apic specific operations we should add the smp_mb() sequence. But
we need to make sure that we don't end up doing it twice (once in
generic code and another in arch code) for all the ipi paths.

thanks,
suresh


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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 18:43                             ` Suresh Siddha
@ 2009-02-18 19:17                               ` Ingo Molnar
  2009-02-18 23:55                                 ` Suresh Siddha
  0 siblings, 1 reply; 103+ messages in thread
From: Ingo Molnar @ 2009-02-18 19:17 UTC (permalink / raw)
  To: Suresh Siddha
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch


* Suresh Siddha <suresh.b.siddha@intel.com> wrote:

> On Wed, 2009-02-18 at 05:59 -0800, Nick Piggin wrote:
> > You're saying the problem is in generic_exec_single because I've
> > removed the smp_mb that inadvertently also serialises memory with
> > the x2apic on x86?
> 
> yes.
> 
> > 
> > Indeed that could cause problems on some architectures which I
> > had hoped to avoid. So the patch is probably better off to first
> > add the smp_mb() to arch_send_call_function_xxx arch code, unless
> > it is immediately obvious or confirmed by arch maintainer that
> > such barrier is not required.
> 
> For x2apic specific operations we should add the smp_mb() sequence. But
> we need to make sure that we don't end up doing it twice (once in
> generic code and another in arch code) for all the ipi paths.

right now we do have an smp_mb() due to your fix in November.

So what should happen is to move that smp_mb() from the x86 
generic IPI path to the x86 x2apic IPI path. (and turn it into 
an smp_wmb() - that should be enough - we dont care about future 
reads being done sooner than this point.)

	Ingo

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 19:17                               ` Ingo Molnar
@ 2009-02-18 23:55                                 ` Suresh Siddha
  2009-02-19 12:20                                   ` Ingo Molnar
  0 siblings, 1 reply; 103+ messages in thread
From: Suresh Siddha @ 2009-02-18 23:55 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Wed, 2009-02-18 at 11:17 -0800, Ingo Molnar wrote: 
> * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> 
> > > Indeed that could cause problems on some architectures which I
> > > had hoped to avoid. So the patch is probably better off to first
> > > add the smp_mb() to arch_send_call_function_xxx arch code, unless
> > > it is immediately obvious or confirmed by arch maintainer that
> > > such barrier is not required.
> > 
> > For x2apic specific operations we should add the smp_mb() sequence. But
> > we need to make sure that we don't end up doing it twice (once in
> > generic code and another in arch code) for all the ipi paths.
> 
> right now we do have an smp_mb() due to your fix in November.
> 
> So what should happen is to move that smp_mb() from the x86 
> generic IPI path to the x86 x2apic IPI path. (and turn it into 
> an smp_wmb() - that should be enough - we dont care about future 
> reads being done sooner than this point.)

Ingo, smp_wmb() won't help. x2apic register writes can still go ahead of
the sfence. According to the SDM, we need a serializing instruction or
mfence. Our internal experiments also proved this.

Appended is the x86 portion of the patch:
---

From: Suresh Siddha <suresh.b.siddha@intel.com>
Subject: x86: move smp_mb() in x86 flush tlb path to x2apic specific IPI
paths

uncached MMIO accesses for xapic are inherently serializing and hence
we don't need explicit barriers for xapic IPI paths.

x2apic MSR writes/reads don't have serializing semantics and hence need
a serializing instruction or mfence, to make all the previous memory
stores
globally visisble before the x2apic msr write for IPI.

And hence move smp_mb() in x86 flush tlb path to x2apic specific paths.

Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
---

diff --git a/arch/x86/kernel/genx2apic_cluster.c
b/arch/x86/kernel/genx2apic_cluster.c
index 7c87156..b237248 100644
--- a/arch/x86/kernel/genx2apic_cluster.c
+++ b/arch/x86/kernel/genx2apic_cluster.c
@@ -60,6 +60,13 @@ static void x2apic_send_IPI_mask(const struct cpumask
*mask, int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		__x2apic_send_IPI_dest(
@@ -76,6 +83,13 @@ static void
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		if (query_cpu == this_cpu)
@@ -93,6 +107,13 @@ static void x2apic_send_IPI_allbutself(int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_online_cpu(query_cpu) {
 		if (query_cpu == this_cpu)
diff --git a/arch/x86/kernel/genx2apic_phys.c
b/arch/x86/kernel/genx2apic_phys.c
index 5cbae8a..f48f282 100644
--- a/arch/x86/kernel/genx2apic_phys.c
+++ b/arch/x86/kernel/genx2apic_phys.c
@@ -58,6 +58,13 @@ static void x2apic_send_IPI_mask(const struct cpumask
*mask, int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		__x2apic_send_IPI_dest(per_cpu(x86_cpu_to_apicid, query_cpu),
@@ -73,6 +80,13 @@ static void
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		if (query_cpu != this_cpu)
@@ -89,6 +103,13 @@ static void x2apic_send_IPI_allbutself(int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_online_cpu(query_cpu) {
 		if (query_cpu == this_cpu)
diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index 14c5af4..de14557 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -188,11 +188,6 @@ static void flush_tlb_others_ipi(const struct
cpumask *cpumask,
 		       cpumask, cpumask_of(smp_processor_id()));
 
 	/*
-	 * Make the above memory operations globally visible before
-	 * sending the IPI.
-	 */
-	smp_mb();
-	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
 	 */



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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-18 16:05           ` Ingo Molnar
@ 2009-02-19  0:00             ` Jeremy Fitzhardinge
  2009-02-19 12:21               ` Ingo Molnar
  2009-02-19  4:31             ` Rusty Russell
  1 sibling, 1 reply; 103+ messages in thread
From: Jeremy Fitzhardinge @ 2009-02-19  0:00 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Rusty Russell, Peter Zijlstra, Linus Torvalds, Nick Piggin,
	Jens Axboe, Paul E. McKenney, Steven Rostedt, linux-kernel,
	Oleg Nesterov

Ingo Molnar wrote:
> The kmalloc(GFP_ATOMIC) has been in kernel/smp.c for about half 
> a year, since:
>
>  |  commit 3d4422332711ef48ef0f132f1fcbfcbd56c7f3d1
>  |  Author: Jens Axboe <jens.axboe@oracle.com>
>  |  Date:   Thu Jun 26 11:21:34 2008 +0200
>  |  Subject: [PATCH] Add generic helpers for arch IPI function calls
>
> Which introduced kernel/smp.c to begin with. It got merged on 
> July 15.
>   

The kmalloc to resolve the stack lifetime vs rcu bug is more recent than 
that - around August, I think.

    J

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 16:09                                   ` Linus Torvalds
  2009-02-18 16:21                                       ` Ingo Molnar
@ 2009-02-19  0:12                                     ` Nick Piggin
  2009-02-19  6:47                                     ` Benjamin Herrenschmidt
  2 siblings, 0 replies; 103+ messages in thread
From: Nick Piggin @ 2009-02-19  0:12 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Paul E. McKenney, Oleg Nesterov, Peter Zijlstra, Jens Axboe,
	Suresh Siddha, Ingo Molnar, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Wed, Feb 18, 2009 at 08:09:21AM -0800, Linus Torvalds wrote:
> 
> 
> On Wed, 18 Feb 2009, Nick Piggin wrote:
> > 
> > I agree with you both that we *should* make arch interrupt code
> > do the ordering, but given the subtle lockups on some architectures
> > in this new code, I didn't want to make it significantly weaker...
> > 
> > Though perhaps it appears that I have, if I have removed an smp_mb
> > that x86 was relying on to emit an mfence to serialise the apic.
> 
> The thing is, if the architecture doesn't order IPI wrt cache coherency, 
> then the "smp_mb()" doesn't really do so _either_. 

Oh yes agreed three, which is why I'm saying it is just a hack
and should be removed at some point.

 
> It might hide some architecture-specific implementation issue, of course, 
> so random amounts of "smp_mb()"s sprinkled around might well make some 
> architecture "work", but it's in no way guaranteed. A smp_mb() does not 
> guarantee that some separate IPI network is ordered - that may well take 
> some random machine-specific IO cycle.

Yes, but I didn't want to pull out that smp_mb() at least until
arch maintainers can ack it. Just because there might indeed be
some random issue hidden by it.


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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-18 16:05           ` Ingo Molnar
  2009-02-19  0:00             ` Jeremy Fitzhardinge
@ 2009-02-19  4:31             ` Rusty Russell
  2009-02-19  9:10               ` Peter Zijlstra
  2009-02-19 16:52               ` Linus Torvalds
  1 sibling, 2 replies; 103+ messages in thread
From: Rusty Russell @ 2009-02-19  4:31 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Peter Zijlstra, Linus Torvalds, Nick Piggin, Jens Axboe,
	Paul E. McKenney, Steven Rostedt, linux-kernel, Oleg Nesterov

On Thursday 19 February 2009 02:35:35 Ingo Molnar wrote:
> 
> * Rusty Russell <rusty@rustcorp.com.au> wrote:
> 
> > On Tuesday 17 February 2009 20:13:59 Ingo Molnar wrote:
> > > We should not bend backwards trying to preserve that kmalloc() 
> > > [and prove that it's safe and race-free] - i.e. the burden of 
> > > proof is on the person insisting that it's needed, not on the 
> > > person wanting to remove it.
> > 
> > Respectfully disagree.  The kmalloc has been there for a very long time,
> > and doing fine AFAICT.
> 
> The kmalloc(GFP_ATOMIC) has been in kernel/smp.c for about half 
> a year

Oops, yes.

So if we care about the kmalloc, why didn't we see benchmarks when we
switched from the x86 smp_call_function_mask to the generic one?  Or did
I just miss them (there's nothing in the git commit).

Now, I think the current patch is quite neat and may not been benchmarks to
justify it, but it'd still be nice if it were faster, but noone seems to know.

Rusty.

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 16:09                                   ` Linus Torvalds
  2009-02-18 16:21                                       ` Ingo Molnar
  2009-02-19  0:12                                     ` Nick Piggin
@ 2009-02-19  6:47                                     ` Benjamin Herrenschmidt
  2009-02-19 13:11                                       ` Nick Piggin
  2 siblings, 1 reply; 103+ messages in thread
From: Benjamin Herrenschmidt @ 2009-02-19  6:47 UTC (permalink / raw)
  To: Linus Torvalds
  Cc: Nick Piggin, Paul E. McKenney, Oleg Nesterov, Peter Zijlstra,
	Jens Axboe, Suresh Siddha, Ingo Molnar, Rusty Russell,
	Steven Rostedt, linux-kernel, linux-arch


> It might hide some architecture-specific implementation issue, of course, 
> so random amounts of "smp_mb()"s sprinkled around might well make some 
> architecture "work", but it's in no way guaranteed. A smp_mb() does not 
> guarantee that some separate IPI network is ordered - that may well take 
> some random machine-specific IO cycle.
> 
> That said, at least on x86, taking an interrupt should be a serializing 
> event, so there should be no reason for anything on the receiving side. 
> The _sending_ side might need to make sure that there is serialization 
> when generating the IPI (so that the IPI cannot happen while the writes 
> are still in some per-CPU write buffer and haven't become part of the 
> cache coherency domain).
> 
> And at least on x86 it's actually pretty hard to generate out-of-order 
> accesses to begin with (_regardless_ of any issues external to the CPU). 
> You have to work at it, and use a WC memory area, and I'm pretty sure we 
> use UC for the apic accesses.

On powerpc, I suspect an smp_mb() on the sender would be useful... it
mostly depends how the IPI is generated but in most case it's going to
be an MMIO write, ie non-cached write which isn't ordered vs. any
previous cached store except using a full sync (which is what smp_mb()
does).

Cheers,
Ben.



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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-19  4:31             ` Rusty Russell
@ 2009-02-19  9:10               ` Peter Zijlstra
  2009-02-19 11:04                 ` Jens Axboe
  2009-02-19 16:52               ` Linus Torvalds
  1 sibling, 1 reply; 103+ messages in thread
From: Peter Zijlstra @ 2009-02-19  9:10 UTC (permalink / raw)
  To: Rusty Russell
  Cc: Ingo Molnar, Linus Torvalds, Nick Piggin, Jens Axboe,
	Paul E. McKenney, Steven Rostedt, linux-kernel, Oleg Nesterov

On Thu, 2009-02-19 at 15:01 +1030, Rusty Russell wrote:
> On Thursday 19 February 2009 02:35:35 Ingo Molnar wrote:
> > 
> > * Rusty Russell <rusty@rustcorp.com.au> wrote:
> > 
> > > On Tuesday 17 February 2009 20:13:59 Ingo Molnar wrote:
> > > > We should not bend backwards trying to preserve that kmalloc() 
> > > > [and prove that it's safe and race-free] - i.e. the burden of 
> > > > proof is on the person insisting that it's needed, not on the 
> > > > person wanting to remove it.
> > > 
> > > Respectfully disagree.  The kmalloc has been there for a very long time,
> > > and doing fine AFAICT.
> > 
> > The kmalloc(GFP_ATOMIC) has been in kernel/smp.c for about half 
> > a year
> 
> Oops, yes.
> 
> So if we care about the kmalloc, why didn't we see benchmarks when we
> switched from the x86 smp_call_function_mask to the generic one?  Or did
> I just miss them (there's nothing in the git commit).
> 
> Now, I think the current patch is quite neat and may not been benchmarks to
> justify it, but it'd still be nice if it were faster, but noone seems to know.

I think the problem is that even on a lively machine these routines just
aren't called that often:

CAL:         74        104         93        116   Function call interrupts
make clean; make -j8 bzImage
CAL:         74        104         93        116   Function call interrupts

We could of course construct some artificial ubench to stress it...


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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-19  9:10               ` Peter Zijlstra
@ 2009-02-19 11:04                 ` Jens Axboe
  0 siblings, 0 replies; 103+ messages in thread
From: Jens Axboe @ 2009-02-19 11:04 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Rusty Russell, Ingo Molnar, Linus Torvalds, Nick Piggin,
	Paul E. McKenney, Steven Rostedt, linux-kernel, Oleg Nesterov

On Thu, Feb 19 2009, Peter Zijlstra wrote:
> On Thu, 2009-02-19 at 15:01 +1030, Rusty Russell wrote:
> > On Thursday 19 February 2009 02:35:35 Ingo Molnar wrote:
> > > 
> > > * Rusty Russell <rusty@rustcorp.com.au> wrote:
> > > 
> > > > On Tuesday 17 February 2009 20:13:59 Ingo Molnar wrote:
> > > > > We should not bend backwards trying to preserve that kmalloc() 
> > > > > [and prove that it's safe and race-free] - i.e. the burden of 
> > > > > proof is on the person insisting that it's needed, not on the 
> > > > > person wanting to remove it.
> > > > 
> > > > Respectfully disagree.  The kmalloc has been there for a very long time,
> > > > and doing fine AFAICT.
> > > 
> > > The kmalloc(GFP_ATOMIC) has been in kernel/smp.c for about half 
> > > a year
> > 
> > Oops, yes.
> > 
> > So if we care about the kmalloc, why didn't we see benchmarks when we
> > switched from the x86 smp_call_function_mask to the generic one?  Or did
> > I just miss them (there's nothing in the git commit).
> > 
> > Now, I think the current patch is quite neat and may not been benchmarks to
> > justify it, but it'd still be nice if it were faster, but noone seems to know.
> 
> I think the problem is that even on a lively machine these routines just
> aren't called that often:
> 
> CAL:         74        104         93        116   Function call interrupts
> make clean; make -j8 bzImage
> CAL:         74        104         93        116   Function call interrupts
> 
> We could of course construct some artificial ubench to stress it...

An easy way to stress it would be to enable request cpu affinity in the
block layer, that get you tons of call function single interrupts. If
sda is the drive of interest, do:

# echo 1 > /sys/block/sda/queue/rq_affinity

and then generate some IO, the block layer will use single ipi calls to
reschedule completions to the submitting CPU.

-- 
Jens Axboe


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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-18 23:55                                 ` Suresh Siddha
@ 2009-02-19 12:20                                   ` Ingo Molnar
  2009-02-19 12:29                                     ` Nick Piggin
  2009-02-19 22:00                                     ` Suresh Siddha
  0 siblings, 2 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-19 12:20 UTC (permalink / raw)
  To: Suresh Siddha
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch


* Suresh Siddha <suresh.b.siddha@intel.com> wrote:

> On Wed, 2009-02-18 at 11:17 -0800, Ingo Molnar wrote: 
> > * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> > 
> > > > Indeed that could cause problems on some architectures which I
> > > > had hoped to avoid. So the patch is probably better off to first
> > > > add the smp_mb() to arch_send_call_function_xxx arch code, unless
> > > > it is immediately obvious or confirmed by arch maintainer that
> > > > such barrier is not required.
> > > 
> > > For x2apic specific operations we should add the smp_mb() sequence. But
> > > we need to make sure that we don't end up doing it twice (once in
> > > generic code and another in arch code) for all the ipi paths.
> > 
> > right now we do have an smp_mb() due to your fix in November.
> > 
> > So what should happen is to move that smp_mb() from the x86 
> > generic IPI path to the x86 x2apic IPI path. (and turn it into 
> > an smp_wmb() - that should be enough - we dont care about future 
> > reads being done sooner than this point.)
> 
> Ingo, smp_wmb() won't help. x2apic register writes can still 
> go ahead of the sfence. According to the SDM, we need a 
> serializing instruction or mfence. Our internal experiments 
> also proved this.

ah, yes - i got confused about how an x2apic write can pass a 
_store_ fence.

The reason is that an MSR write is a register->register move 
(not a memory write), so it it not part of the generic memory 
ordering machinery. So a serializing instruction it has to be.

> Appended is the x86 portion of the patch: ---
> 
> From: Suresh Siddha <suresh.b.siddha@intel.com>
> Subject: x86: move smp_mb() in x86 flush tlb path to x2apic specific IPI
> paths

Could you please refresh this patch to latest tip:master? The 
APIC drivers moved to arch/x86/kernel/apic/.

	Ingo

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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-19  0:00             ` Jeremy Fitzhardinge
@ 2009-02-19 12:21               ` Ingo Molnar
  0 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-19 12:21 UTC (permalink / raw)
  To: Jeremy Fitzhardinge
  Cc: Rusty Russell, Peter Zijlstra, Linus Torvalds, Nick Piggin,
	Jens Axboe, Paul E. McKenney, Steven Rostedt, linux-kernel,
	Oleg Nesterov


* Jeremy Fitzhardinge <jeremy@goop.org> wrote:

> Ingo Molnar wrote:
>> The kmalloc(GFP_ATOMIC) has been in kernel/smp.c for about half a year, 
>> since:
>>
>>  |  commit 3d4422332711ef48ef0f132f1fcbfcbd56c7f3d1
>>  |  Author: Jens Axboe <jens.axboe@oracle.com>
>>  |  Date:   Thu Jun 26 11:21:34 2008 +0200
>>  |  Subject: [PATCH] Add generic helpers for arch IPI function calls
>>
>> Which introduced kernel/smp.c to begin with. It got merged on July 15.
>>   
>
> The kmalloc to resolve the stack lifetime vs rcu bug is more 
> recent than that - around August, I think.

yeah, it was in a late -rc, and solved rare crashes that you 
reported.

	Ingo

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-19 12:20                                   ` Ingo Molnar
@ 2009-02-19 12:29                                     ` Nick Piggin
  2009-02-19 12:45                                       ` Ingo Molnar
  2009-02-19 22:00                                     ` Suresh Siddha
  1 sibling, 1 reply; 103+ messages in thread
From: Nick Piggin @ 2009-02-19 12:29 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Suresh Siddha, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Thu, Feb 19, 2009 at 01:20:31PM +0100, Ingo Molnar wrote:
> 
> * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> 
> > On Wed, 2009-02-18 at 11:17 -0800, Ingo Molnar wrote: 
> > > * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> > > So what should happen is to move that smp_mb() from the x86 
> > > generic IPI path to the x86 x2apic IPI path. (and turn it into 
> > > an smp_wmb() - that should be enough - we dont care about future 
> > > reads being done sooner than this point.)
> > 
> > Ingo, smp_wmb() won't help. x2apic register writes can still 
> > go ahead of the sfence. According to the SDM, we need a 
> > serializing instruction or mfence. Our internal experiments 
> > also proved this.
> 
> ah, yes - i got confused about how an x2apic write can pass a 
> _store_ fence.

And about how smp_wmb() doesn't emit a store fence ;)


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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-19 12:29                                     ` Nick Piggin
@ 2009-02-19 12:45                                       ` Ingo Molnar
  0 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-19 12:45 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Suresh Siddha, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch


* Nick Piggin <npiggin@suse.de> wrote:

> On Thu, Feb 19, 2009 at 01:20:31PM +0100, Ingo Molnar wrote:
> > 
> > * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> > 
> > > On Wed, 2009-02-18 at 11:17 -0800, Ingo Molnar wrote: 
> > > > * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> > > > So what should happen is to move that smp_mb() from the x86 
> > > > generic IPI path to the x86 x2apic IPI path. (and turn it into 
> > > > an smp_wmb() - that should be enough - we dont care about future 
> > > > reads being done sooner than this point.)
> > > 
> > > Ingo, smp_wmb() won't help. x2apic register writes can still 
> > > go ahead of the sfence. According to the SDM, we need a 
> > > serializing instruction or mfence. Our internal experiments 
> > > also proved this.
> > 
> > ah, yes - i got confused about how an x2apic write can pass a 
> > _store_ fence.
> 
> And about how smp_wmb() doesn't emit a store fence ;)

yeah ;-) Only wmb() emits a SFENCE all the time. Writes are 
normally ordered so we map smp_wmb() to barrier().

	Ingo

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-19  6:47                                     ` Benjamin Herrenschmidt
@ 2009-02-19 13:11                                       ` Nick Piggin
  2009-02-19 15:06                                         ` Ingo Molnar
  0 siblings, 1 reply; 103+ messages in thread
From: Nick Piggin @ 2009-02-19 13:11 UTC (permalink / raw)
  To: Benjamin Herrenschmidt
  Cc: Linus Torvalds, Paul E. McKenney, Oleg Nesterov, Peter Zijlstra,
	Jens Axboe, Suresh Siddha, Ingo Molnar, Rusty Russell,
	Steven Rostedt, linux-kernel, linux-arch

On Thu, Feb 19, 2009 at 05:47:20PM +1100, Benjamin Herrenschmidt wrote:
> 
> > It might hide some architecture-specific implementation issue, of course, 
> > so random amounts of "smp_mb()"s sprinkled around might well make some 
> > architecture "work", but it's in no way guaranteed. A smp_mb() does not 
> > guarantee that some separate IPI network is ordered - that may well take 
> > some random machine-specific IO cycle.
> > 
> > That said, at least on x86, taking an interrupt should be a serializing 
> > event, so there should be no reason for anything on the receiving side. 
> > The _sending_ side might need to make sure that there is serialization 
> > when generating the IPI (so that the IPI cannot happen while the writes 
> > are still in some per-CPU write buffer and haven't become part of the 
> > cache coherency domain).
> > 
> > And at least on x86 it's actually pretty hard to generate out-of-order 
> > accesses to begin with (_regardless_ of any issues external to the CPU). 
> > You have to work at it, and use a WC memory area, and I'm pretty sure we 
> > use UC for the apic accesses.
> 
> On powerpc, I suspect an smp_mb() on the sender would be useful... it
> mostly depends how the IPI is generated but in most case it's going to
> be an MMIO write, ie non-cached write which isn't ordered vs. any
> previous cached store except using a full sync (which is what smp_mb()
> does).

So your arch_send_call_function_single_ipi etc need to ensure this,
right?  Generic code obviously has no idea about how to do it.


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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-19 13:11                                       ` Nick Piggin
@ 2009-02-19 15:06                                         ` Ingo Molnar
  2009-02-19 21:49                                           ` Benjamin Herrenschmidt
  0 siblings, 1 reply; 103+ messages in thread
From: Ingo Molnar @ 2009-02-19 15:06 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Benjamin Herrenschmidt, Linus Torvalds, Paul E. McKenney,
	Oleg Nesterov, Peter Zijlstra, Jens Axboe, Suresh Siddha,
	Rusty Russell, Steven Rostedt, linux-kernel, linux-arch


* Nick Piggin <npiggin@suse.de> wrote:

> On Thu, Feb 19, 2009 at 05:47:20PM +1100, Benjamin Herrenschmidt wrote:
> > 
> > > It might hide some architecture-specific implementation issue, of course, 
> > > so random amounts of "smp_mb()"s sprinkled around might well make some 
> > > architecture "work", but it's in no way guaranteed. A smp_mb() does not 
> > > guarantee that some separate IPI network is ordered - that may well take 
> > > some random machine-specific IO cycle.
> > > 
> > > That said, at least on x86, taking an interrupt should be a serializing 
> > > event, so there should be no reason for anything on the receiving side. 
> > > The _sending_ side might need to make sure that there is serialization 
> > > when generating the IPI (so that the IPI cannot happen while the writes 
> > > are still in some per-CPU write buffer and haven't become part of the 
> > > cache coherency domain).
> > > 
> > > And at least on x86 it's actually pretty hard to generate out-of-order 
> > > accesses to begin with (_regardless_ of any issues external to the CPU). 
> > > You have to work at it, and use a WC memory area, and I'm pretty sure we 
> > > use UC for the apic accesses.
> > 
> > On powerpc, I suspect an smp_mb() on the sender would be 
> > useful... it mostly depends how the IPI is generated but in 
> > most case it's going to be an MMIO write, ie non-cached 
> > write which isn't ordered vs. any previous cached store 
> > except using a full sync (which is what smp_mb() does).
> 
> So your arch_send_call_function_single_ipi etc need to ensure 
> this, right?  Generic code obviously has no idea about how to 
> do it.

The thing is, the most widespread way to send IPIs (x86 
non-x2apic local APIC) does not need any barriers in the generic 
code or elsewhere, because the local APIC is mapped uncached so 
it's implicitly ordered.

So the right solution is to add barriers to those IPI 
implementations that need it. This means that the generic code 
should not have a barrier for IPI sending.

	Ingo

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

* Re: [PATCH 2/4] generic-smp: remove kmalloc usage
  2009-02-19  4:31             ` Rusty Russell
  2009-02-19  9:10               ` Peter Zijlstra
@ 2009-02-19 16:52               ` Linus Torvalds
  1 sibling, 0 replies; 103+ messages in thread
From: Linus Torvalds @ 2009-02-19 16:52 UTC (permalink / raw)
  To: Rusty Russell
  Cc: Ingo Molnar, Peter Zijlstra, Nick Piggin, Jens Axboe,
	Paul E. McKenney, Steven Rostedt, linux-kernel, Oleg Nesterov



On Thu, 19 Feb 2009, Rusty Russell wrote:
> 
> So if we care about the kmalloc, why didn't we see benchmarks when we
> switched from the x86 smp_call_function_mask to the generic one?  Or did
> I just miss them (there's nothing in the git commit).

I don't think we care about kmalloc from a performance angle. Sure, it's 
nice if we can make IPI's be really low cost, and we should aim for that, 
but the reason the kmalloc() was added was never performance - nor is that 
the reason we now try to remove it.

The kmalloc() was added for correctness reasons, and we now try to remove 
it to make the code look saner and simpler (and hopefully it gets faster 
too, but I don't think that was ever a primary issue), since we ended up 
having _three_ different cases for the whole insane memory allocation 
(on-stack, per-cpu and kmalloc), and nobody sane really wants that.

			Linus

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

* Re: Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-19 15:06                                         ` Ingo Molnar
@ 2009-02-19 21:49                                           ` Benjamin Herrenschmidt
  0 siblings, 0 replies; 103+ messages in thread
From: Benjamin Herrenschmidt @ 2009-02-19 21:49 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Nick Piggin, Linus Torvalds, Paul E. McKenney, Oleg Nesterov,
	Peter Zijlstra, Jens Axboe, Suresh Siddha, Rusty Russell,
	Steven Rostedt, linux-kernel, linux-arch

On Thu, 2009-02-19 at 16:06 +0100, Ingo Molnar wrote:
> > So your arch_send_call_function_single_ipi etc need to ensure 
> > this, right?  Generic code obviously has no idea about how to 
> > do it.
> 
> The thing is, the most widespread way to send IPIs (x86 
> non-x2apic local APIC) does not need any barriers in the generic 
> code or elsewhere, because the local APIC is mapped uncached so 
> it's implicitly ordered.
> 
> So the right solution is to add barriers to those IPI 
> implementations that need it. This means that the generic code 
> should not have a barrier for IPI sending.

I agree. In fact, our current code should be fine in any case because
our writel() which will be used to generate the IPI has a sync in it
anyway for other reasons.

Cheers,
Ben.



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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-19 12:20                                   ` Ingo Molnar
  2009-02-19 12:29                                     ` Nick Piggin
@ 2009-02-19 22:00                                     ` Suresh Siddha
  2009-02-20 10:56                                       ` Ingo Molnar
  1 sibling, 1 reply; 103+ messages in thread
From: Suresh Siddha @ 2009-02-19 22:00 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch

On Thu, 2009-02-19 at 04:20 -0800, Ingo Molnar wrote:
> Could you please refresh this patch to latest tip:master? The 
> APIC drivers moved to arch/x86/kernel/apic/.

Appended the refreshed patch. Thanks.
---

From: Suresh Siddha <suresh.b.siddha@intel.com>
Subject: x86: move smp_mb() in flush tlb path to x2apic specific paths

uncached MMIO accesses for xapic are inherently serializing and hence
we don't need explicit barriers for xapic IPI paths.

x2apic MSR writes/reads don't have serializing semantics and hence need
a serializing instruction or mfence, to make all the previous memory
stores globally visisble before the x2apic msr write for IPI.

And hence move smp_mb() in x86 flush tlb path to x2apic specific paths.

Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
---

diff --git a/arch/x86/kernel/apic/x2apic_cluster.c b/arch/x86/kernel/apic/x2apic_cluster.c
index 3f7df23..c54fffe 100644
--- a/arch/x86/kernel/apic/x2apic_cluster.c
+++ b/arch/x86/kernel/apic/x2apic_cluster.c
@@ -63,6 +63,13 @@ static void x2apic_send_IPI_mask(const struct cpumask *mask, int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		__x2apic_send_IPI_dest(
@@ -79,6 +86,13 @@ static void
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		if (query_cpu == this_cpu)
@@ -96,6 +110,13 @@ static void x2apic_send_IPI_allbutself(int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_online_cpu(query_cpu) {
 		if (query_cpu == this_cpu)
diff --git a/arch/x86/kernel/apic/x2apic_phys.c b/arch/x86/kernel/apic/x2apic_phys.c
index d2d52eb..84cc2f3 100644
--- a/arch/x86/kernel/apic/x2apic_phys.c
+++ b/arch/x86/kernel/apic/x2apic_phys.c
@@ -58,6 +58,13 @@ static void x2apic_send_IPI_mask(const struct cpumask *mask, int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		__x2apic_send_IPI_dest(per_cpu(x86_cpu_to_apicid, query_cpu),
@@ -73,6 +80,13 @@ static void
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		if (query_cpu != this_cpu)
@@ -89,6 +103,13 @@ static void x2apic_send_IPI_allbutself(int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	/*
+	 * Make previous memory operations globally visible before
+	 * sending the IPI. We need a serializing instruction or mfence
+	 * for this.
+	 */
+	smp_mb();
+
 	local_irq_save(flags);
 	for_each_online_cpu(query_cpu) {
 		if (query_cpu == this_cpu)
diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index a654d59..821e970 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -187,11 +187,6 @@ static void flush_tlb_others_ipi(const struct cpumask *cpumask,
 		       cpumask, cpumask_of(smp_processor_id()));
 
 	/*
-	 * Make the above memory operations globally visible before
-	 * sending the IPI.
-	 */
-	smp_mb();
-	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
 	 */



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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-19 22:00                                     ` Suresh Siddha
@ 2009-02-20 10:56                                       ` Ingo Molnar
  2009-02-20 18:56                                         ` Suresh Siddha
  0 siblings, 1 reply; 103+ messages in thread
From: Ingo Molnar @ 2009-02-20 10:56 UTC (permalink / raw)
  To: Suresh Siddha
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch


* Suresh Siddha <suresh.b.siddha@intel.com> wrote:

> On Thu, 2009-02-19 at 04:20 -0800, Ingo Molnar wrote:
> > Could you please refresh this patch to latest tip:master? The 
> > APIC drivers moved to arch/x86/kernel/apic/.
> 
> Appended the refreshed patch. Thanks.

thanks. Two details i noticed:

Firstly:

> +++ b/arch/x86/kernel/apic/x2apic_cluster.c
> +++ b/arch/x86/kernel/apic/x2apic_phys.c

how about x2apic_uv.c? It uses uv_write_global_mmr64() in its 
IPI sending method, which uses:

static inline void uv_write_global_mmr64(int pnode, unsigned long offset,
                                unsigned long val)
{
        *uv_global_mmr64_address(pnode, offset) = val;
}

which uses ->mmr_base, which is mapped via:

        init_extra_mapping_uc(UV_LOCAL_MMR_BASE, UV_LOCAL_MMR_SIZE);

so it should be fine because uncached - but at minimum we should 
put a comment into x2apic_uv.c that the generic IPI code relies 
on the lowlevel code serializing - i.e. relies on the UC PAT 
attribute.

Secondly, you added smp_mb(), which will translate to an MFENCE. 

But in theory it should be enough to have a wmb() here. [Note, 
not an smp_wmb() that i suggested before.] That will translate 
to an SFENCE - which will serialize writes but still allows 
reads/prefetches to pass.

So the question is, is an SFENCE there enough to serialize the 
WRMSR with previous memory-writes? It's not specified in the 
x2apic docs as far as i could see.

	Ingo

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-20 10:56                                       ` Ingo Molnar
@ 2009-02-20 18:56                                         ` Suresh Siddha
  2009-02-20 19:40                                           ` Ingo Molnar
                                                             ` (2 more replies)
  0 siblings, 3 replies; 103+ messages in thread
From: Suresh Siddha @ 2009-02-20 18:56 UTC (permalink / raw)
  To: Ingo Molnar
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch, steiner

On Fri, 2009-02-20 at 02:56 -0800, Ingo Molnar wrote:
> * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> 
> > On Thu, 2009-02-19 at 04:20 -0800, Ingo Molnar wrote:
> > > Could you please refresh this patch to latest tip:master? The 
> > > APIC drivers moved to arch/x86/kernel/apic/.
> > 
> > Appended the refreshed patch. Thanks.
> 
> thanks. Two details i noticed:
> 
> Firstly:
> 
> > +++ b/arch/x86/kernel/apic/x2apic_cluster.c
> > +++ b/arch/x86/kernel/apic/x2apic_phys.c
> 
> how about x2apic_uv.c? It uses uv_write_global_mmr64() in its 
> IPI sending method, which uses:
> 
> static inline void uv_write_global_mmr64(int pnode, unsigned long offset,
>                                 unsigned long val)
> {
>         *uv_global_mmr64_address(pnode, offset) = val;
> }
> 
> which uses ->mmr_base, which is mapped via:
> 
>         init_extra_mapping_uc(UV_LOCAL_MMR_BASE, UV_LOCAL_MMR_SIZE);
>
> so it should be fine because uncached - but at minimum we should 
> put a comment into x2apic_uv.c that the generic IPI code relies 
> on the lowlevel code serializing - i.e. relies on the UC PAT 
> attribute.

Oops. I forgot to mention that and copy Jack to confirm our
understanding. I can send a followup patch adding the comments for UV.
Jack, are you ok?

> 
> Secondly, you added smp_mb(), which will translate to an MFENCE. 
> 
> But in theory it should be enough to have a wmb() here. [Note, 
> not an smp_wmb() that i suggested before.] That will translate 
> to an SFENCE - which will serialize writes but still allows 
> reads/prefetches to pass.
> 
> So the question is, is an SFENCE there enough to serialize the 
> WRMSR with previous memory-writes? It's not specified in the 
> x2apic docs as far as i could see.

No. sfence is not enough (wrmsr to x2apic regs was still passing ahead).
We have done a small experiment to demonstrate the issue and adding
mfence fixes the issue but not sfence. We need a serializing instruction
or mfence. I will try to get the SDM updated.

thanks,
suresh


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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-20 18:56                                         ` Suresh Siddha
@ 2009-02-20 19:40                                           ` Ingo Molnar
  2009-02-20 23:28                                           ` Jack Steiner
  2009-02-25  3:32                                           ` Nick Piggin
  2 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-20 19:40 UTC (permalink / raw)
  To: Suresh Siddha
  Cc: Nick Piggin, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch, steiner


* Suresh Siddha <suresh.b.siddha@intel.com> wrote:

> > So the question is, is an SFENCE there enough to serialize 
> > the WRMSR with previous memory-writes? It's not specified in 
> > the x2apic docs as far as i could see.
> 
> No. sfence is not enough (wrmsr to x2apic regs was still 
> passing ahead). We have done a small experiment to demonstrate 
> the issue and adding mfence fixes the issue but not sfence. We 
> need a serializing instruction or mfence. I will try to get 
> the SDM updated.

Fair enough!

	Ingo

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-20 18:56                                         ` Suresh Siddha
  2009-02-20 19:40                                           ` Ingo Molnar
@ 2009-02-20 23:28                                           ` Jack Steiner
  2009-02-25  3:32                                           ` Nick Piggin
  2 siblings, 0 replies; 103+ messages in thread
From: Jack Steiner @ 2009-02-20 23:28 UTC (permalink / raw)
  To: Suresh Siddha
  Cc: Ingo Molnar, Nick Piggin, Peter Zijlstra, Oleg Nesterov,
	Jens Axboe, Linus Torvalds, Paul E. McKenney, Rusty Russell,
	Steven Rostedt, linux-kernel, linux-arch

On Fri, Feb 20, 2009 at 10:56:54AM -0800, Suresh Siddha wrote:
> On Fri, 2009-02-20 at 02:56 -0800, Ingo Molnar wrote:
> > * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> > 
> > > On Thu, 2009-02-19 at 04:20 -0800, Ingo Molnar wrote:
> > > > Could you please refresh this patch to latest tip:master? The 
> > > > APIC drivers moved to arch/x86/kernel/apic/.
> > > 
> > > Appended the refreshed patch. Thanks.
> > 
> > thanks. Two details i noticed:
> > 
> > Firstly:
> > 
> > > +++ b/arch/x86/kernel/apic/x2apic_cluster.c
> > > +++ b/arch/x86/kernel/apic/x2apic_phys.c
> > 
> > how about x2apic_uv.c? It uses uv_write_global_mmr64() in its 
> > IPI sending method, which uses:
> > 
> > static inline void uv_write_global_mmr64(int pnode, unsigned long offset,
> >                                 unsigned long val)
> > {
> >         *uv_global_mmr64_address(pnode, offset) = val;
> > }
> > 
> > which uses ->mmr_base, which is mapped via:
> > 
> >         init_extra_mapping_uc(UV_LOCAL_MMR_BASE, UV_LOCAL_MMR_SIZE);
> >
> > so it should be fine because uncached - but at minimum we should 
> > put a comment into x2apic_uv.c that the generic IPI code relies 
> > on the lowlevel code serializing - i.e. relies on the UC PAT 
> > attribute.
> 
> Oops. I forgot to mention that and copy Jack to confirm our
> understanding. I can send a followup patch adding the comments for UV.
> Jack, are you ok?

Yes. Looks ok to me.


> 
> > 
> > Secondly, you added smp_mb(), which will translate to an MFENCE. 
> > 
> > But in theory it should be enough to have a wmb() here. [Note, 
> > not an smp_wmb() that i suggested before.] That will translate 
> > to an SFENCE - which will serialize writes but still allows 
> > reads/prefetches to pass.
> > 
> > So the question is, is an SFENCE there enough to serialize the 
> > WRMSR with previous memory-writes? It's not specified in the 
> > x2apic docs as far as i could see.
> 
> No. sfence is not enough (wrmsr to x2apic regs was still passing ahead).
> We have done a small experiment to demonstrate the issue and adding
> mfence fixes the issue but not sfence. We need a serializing instruction
> or mfence. I will try to get the SDM updated.
> 
> thanks,
> suresh

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-20 18:56                                         ` Suresh Siddha
  2009-02-20 19:40                                           ` Ingo Molnar
  2009-02-20 23:28                                           ` Jack Steiner
@ 2009-02-25  3:32                                           ` Nick Piggin
  2009-02-25 12:47                                             ` Ingo Molnar
                                                               ` (2 more replies)
  2 siblings, 3 replies; 103+ messages in thread
From: Nick Piggin @ 2009-02-25  3:32 UTC (permalink / raw)
  To: Suresh Siddha
  Cc: Ingo Molnar, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch, steiner

On Fri, Feb 20, 2009 at 10:56:54AM -0800, Suresh B wrote:
> On Fri, 2009-02-20 at 02:56 -0800, Ingo Molnar wrote:
> > * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> > 
> > > On Thu, 2009-02-19 at 04:20 -0800, Ingo Molnar wrote:
> > > > Could you please refresh this patch to latest tip:master? The 
> > > > APIC drivers moved to arch/x86/kernel/apic/.
> > > 
> > > Appended the refreshed patch. Thanks.
> > 
> > thanks. Two details i noticed:
> > 
> > Firstly:
> > 
> > > +++ b/arch/x86/kernel/apic/x2apic_cluster.c
> > > +++ b/arch/x86/kernel/apic/x2apic_phys.c
> > 
> > how about x2apic_uv.c? It uses uv_write_global_mmr64() in its 
> > IPI sending method, which uses:
> > 
> > static inline void uv_write_global_mmr64(int pnode, unsigned long offset,
> >                                 unsigned long val)
> > {
> >         *uv_global_mmr64_address(pnode, offset) = val;
> > }
> > 
> > which uses ->mmr_base, which is mapped via:
> > 
> >         init_extra_mapping_uc(UV_LOCAL_MMR_BASE, UV_LOCAL_MMR_SIZE);
> >
> > so it should be fine because uncached - but at minimum we should 
> > put a comment into x2apic_uv.c that the generic IPI code relies 
> > on the lowlevel code serializing - i.e. relies on the UC PAT 
> > attribute.
> 
> Oops. I forgot to mention that and copy Jack to confirm our
> understanding. I can send a followup patch adding the comments for UV.
> Jack, are you ok?
> 
> > 
> > Secondly, you added smp_mb(), which will translate to an MFENCE. 
> > 
> > But in theory it should be enough to have a wmb() here. [Note, 
> > not an smp_wmb() that i suggested before.] That will translate 
> > to an SFENCE - which will serialize writes but still allows 
> > reads/prefetches to pass.
> > 
> > So the question is, is an SFENCE there enough to serialize the 
> > WRMSR with previous memory-writes? It's not specified in the 
> > x2apic docs as far as i could see.
> 
> No. sfence is not enough (wrmsr to x2apic regs was still passing ahead).
> We have done a small experiment to demonstrate the issue and adding
> mfence fixes the issue but not sfence. We need a serializing instruction
> or mfence. I will try to get the SDM updated.

Just if I may add something -- I would probably slightly prefer if
you explicitly used an sfence or other serializing instruction rather
than smp_mb(). Maybe call it wrmsr_fence() or something. Apart from
being self documenting, and less confusing (wrmsr is not part of
normal ordering), I assume you technically also need it on UP
systems?



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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-25  3:32                                           ` Nick Piggin
@ 2009-02-25 12:47                                             ` Ingo Molnar
  2009-02-25 18:25                                             ` Luck, Tony
  2009-03-17 18:16                                             ` Suresh Siddha
  2 siblings, 0 replies; 103+ messages in thread
From: Ingo Molnar @ 2009-02-25 12:47 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Suresh Siddha, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch, steiner


* Nick Piggin <npiggin@suse.de> wrote:

> On Fri, Feb 20, 2009 at 10:56:54AM -0800, Suresh B wrote:
> > On Fri, 2009-02-20 at 02:56 -0800, Ingo Molnar wrote:
> > > * Suresh Siddha <suresh.b.siddha@intel.com> wrote:
> > > 
> > > > On Thu, 2009-02-19 at 04:20 -0800, Ingo Molnar wrote:
> > > > > Could you please refresh this patch to latest tip:master? The 
> > > > > APIC drivers moved to arch/x86/kernel/apic/.
> > > > 
> > > > Appended the refreshed patch. Thanks.
> > > 
> > > thanks. Two details i noticed:
> > > 
> > > Firstly:
> > > 
> > > > +++ b/arch/x86/kernel/apic/x2apic_cluster.c
> > > > +++ b/arch/x86/kernel/apic/x2apic_phys.c
> > > 
> > > how about x2apic_uv.c? It uses uv_write_global_mmr64() in its 
> > > IPI sending method, which uses:
> > > 
> > > static inline void uv_write_global_mmr64(int pnode, unsigned long offset,
> > >                                 unsigned long val)
> > > {
> > >         *uv_global_mmr64_address(pnode, offset) = val;
> > > }
> > > 
> > > which uses ->mmr_base, which is mapped via:
> > > 
> > >         init_extra_mapping_uc(UV_LOCAL_MMR_BASE, UV_LOCAL_MMR_SIZE);
> > >
> > > so it should be fine because uncached - but at minimum we should 
> > > put a comment into x2apic_uv.c that the generic IPI code relies 
> > > on the lowlevel code serializing - i.e. relies on the UC PAT 
> > > attribute.
> > 
> > Oops. I forgot to mention that and copy Jack to confirm our
> > understanding. I can send a followup patch adding the comments for UV.
> > Jack, are you ok?
> > 
> > > 
> > > Secondly, you added smp_mb(), which will translate to an MFENCE. 
> > > 
> > > But in theory it should be enough to have a wmb() here. [Note, 
> > > not an smp_wmb() that i suggested before.] That will translate 
> > > to an SFENCE - which will serialize writes but still allows 
> > > reads/prefetches to pass.
> > > 
> > > So the question is, is an SFENCE there enough to serialize the 
> > > WRMSR with previous memory-writes? It's not specified in the 
> > > x2apic docs as far as i could see.
> > 
> > No. sfence is not enough (wrmsr to x2apic regs was still passing ahead).
> > We have done a small experiment to demonstrate the issue and adding
> > mfence fixes the issue but not sfence. We need a serializing instruction
> > or mfence. I will try to get the SDM updated.
> 
> Just if I may add something -- I would probably slightly 
> prefer if you explicitly used an sfence or other serializing 
> instruction rather than smp_mb(). Maybe call it wrmsr_fence() 
> or something. Apart from being self documenting, and less 
> confusing (wrmsr is not part of normal ordering), I assume you 
> technically also need it on UP systems?

Hm, UP systems shouldnt worry too much about sending IPIs to 
other CPUs, right? We optimize out same-CPU IPIs already so that 
bit should not matter.

Agreed on the self-documentation aspect.

	Ingo

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

* RE: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-25  3:32                                           ` Nick Piggin
  2009-02-25 12:47                                             ` Ingo Molnar
@ 2009-02-25 18:25                                             ` Luck, Tony
  2009-03-17 18:16                                             ` Suresh Siddha
  2 siblings, 0 replies; 103+ messages in thread
From: Luck, Tony @ 2009-02-25 18:25 UTC (permalink / raw)
  To: Nick Piggin, Siddha, Suresh B
  Cc: Ingo Molnar, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch, steiner

> Just if I may add something -- I would probably slightly prefer if
> you explicitly used an sfence or other serializing instruction rather
> than smp_mb(). Maybe call it wrmsr_fence() or something. Apart from
> being self documenting, and less confusing (wrmsr is not part of
> normal ordering), I assume you technically also need it on UP
> systems?

Maybe I lost track of this thread ... but isn't this code for the
"send ipi" path?  On a UP system do we use IPI to interrupt ourself?
Even if we did, presumably we can't get inconsistencies if there is
only one cpu.

-Tony

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

* Re: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many())
  2009-02-25  3:32                                           ` Nick Piggin
  2009-02-25 12:47                                             ` Ingo Molnar
  2009-02-25 18:25                                             ` Luck, Tony
@ 2009-03-17 18:16                                             ` Suresh Siddha
  2009-03-18  8:51                                               ` [tip:x86/x2apic] x86: add x2apic_wrmsr_fence() to x2apic flush tlb paths Suresh Siddha
  2 siblings, 1 reply; 103+ messages in thread
From: Suresh Siddha @ 2009-03-17 18:16 UTC (permalink / raw)
  To: Nick Piggin
  Cc: Ingo Molnar, Peter Zijlstra, Oleg Nesterov, Jens Axboe,
	Linus Torvalds, Paul E. McKenney, Rusty Russell, Steven Rostedt,
	linux-kernel, linux-arch, steiner

On Tue, 2009-02-24 at 19:32 -0800, Nick Piggin wrote:
> Just if I may add something -- I would probably slightly prefer if
> you explicitly used an sfence or other serializing instruction rather
> than smp_mb(). Maybe call it wrmsr_fence() or something.

I was planning to send a cleanup patch doing this, but looks like Ingo
never took the original patch. Here it is appended. Ingo, please
consider.

thanks,
suresh
---

From: Suresh Siddha <suresh.b.siddha@intel.com>
Subject: x86: Add x2apic_wrmsr_fence() to x2apic flush tlb paths

uncached MMIO accesses for xapic are inherently serializing and hence
we don't need explicit barriers for xapic IPI paths.

x2apic MSR writes/reads don't have serializing semantics and hence need
a serializing instruction or mfence, to make all the previous memory
stores globally visisble before the x2apic msr write for IPI.

Add x2apic_wrmsr_fence() in flush tlb path to x2apic specific paths.

Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
---

diff --git a/arch/x86/include/asm/apic.h b/arch/x86/include/asm/apic.h
index 394d177..cfc0871 100644
--- a/arch/x86/include/asm/apic.h
+++ b/arch/x86/include/asm/apic.h
@@ -108,6 +108,16 @@ extern void native_apic_icr_write(u32 low, u32 id);
 extern u64 native_apic_icr_read(void);
 
 #ifdef CONFIG_X86_X2APIC
+/*
+ * Make previous memory operations globally visible before
+ * sending the IPI through x2apic wrmsr. We need a serializing instruction or
+ * mfence for this.
+ */
+static inline void x2apic_wrmsr_fence(void)
+{
+	asm volatile("mfence":::"memory");
+}
+
 static inline void native_apic_msr_write(u32 reg, u32 v)
 {
 	if (reg == APIC_DFR || reg == APIC_ID || reg == APIC_LDR ||
diff --git a/arch/x86/kernel/apic/x2apic_cluster.c b/arch/x86/kernel/apic/x2apic_cluster.c
index 8fb87b6..4a903e2 100644
--- a/arch/x86/kernel/apic/x2apic_cluster.c
+++ b/arch/x86/kernel/apic/x2apic_cluster.c
@@ -57,6 +57,8 @@ static void x2apic_send_IPI_mask(const struct cpumask *mask, int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		__x2apic_send_IPI_dest(
@@ -73,6 +75,8 @@ static void
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		if (query_cpu == this_cpu)
@@ -90,6 +94,8 @@ static void x2apic_send_IPI_allbutself(int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_online_cpu(query_cpu) {
 		if (query_cpu == this_cpu)
diff --git a/arch/x86/kernel/apic/x2apic_phys.c b/arch/x86/kernel/apic/x2apic_phys.c
index 23625b9..a284359 100644
--- a/arch/x86/kernel/apic/x2apic_phys.c
+++ b/arch/x86/kernel/apic/x2apic_phys.c
@@ -58,6 +58,8 @@ static void x2apic_send_IPI_mask(const struct cpumask *mask, int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		__x2apic_send_IPI_dest(per_cpu(x86_cpu_to_apicid, query_cpu),
@@ -73,6 +75,8 @@ static void
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		if (query_cpu != this_cpu)
@@ -89,6 +93,8 @@ static void x2apic_send_IPI_allbutself(int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_online_cpu(query_cpu) {
 		if (query_cpu == this_cpu)
diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index a654d59..821e970 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -187,11 +187,6 @@ static void flush_tlb_others_ipi(const struct cpumask *cpumask,
 		       cpumask, cpumask_of(smp_processor_id()));
 
 	/*
-	 * Make the above memory operations globally visible before
-	 * sending the IPI.
-	 */
-	smp_mb();
-	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
 	 */



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

* [tip:x86/x2apic] x86: add x2apic_wrmsr_fence() to x2apic flush tlb paths
  2009-03-17 18:16                                             ` Suresh Siddha
@ 2009-03-18  8:51                                               ` Suresh Siddha
  0 siblings, 0 replies; 103+ messages in thread
From: Suresh Siddha @ 2009-03-18  8:51 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: linux-kernel, hpa, mingo, rusty, torvalds, a.p.zijlstra,
	jens.axboe, npiggin, paulmck, steiner, rostedt, suresh.b.siddha,
	tglx, oleg, mingo

Commit-ID:  ce4e240c279a31096f74afa6584a62d64a1ba8c8
Gitweb:     http://git.kernel.org/tip/ce4e240c279a31096f74afa6584a62d64a1ba8c8
Author:     Suresh Siddha <suresh.b.siddha@intel.com>
AuthorDate: Tue, 17 Mar 2009 10:16:54 -0800
Commit:     Ingo Molnar <mingo@elte.hu>
CommitDate: Wed, 18 Mar 2009 09:36:14 +0100

x86: add x2apic_wrmsr_fence() to x2apic flush tlb paths

Impact: optimize APIC IPI related barriers

Uncached MMIO accesses for xapic are inherently serializing and hence
we don't need explicit barriers for xapic IPI paths.

x2apic MSR writes/reads don't have serializing semantics and hence need
a serializing instruction or mfence, to make all the previous memory
stores globally visisble before the x2apic msr write for IPI.

Add x2apic_wrmsr_fence() in flush tlb path to x2apic specific paths.

Signed-off-by: Suresh Siddha <suresh.b.siddha@intel.com>
Cc: Peter Zijlstra <a.p.zijlstra@chello.nl>
Cc: Oleg Nesterov <oleg@redhat.com>
Cc: Jens Axboe <jens.axboe@oracle.com>
Cc: Linus Torvalds <torvalds@linux-foundation.org>
Cc: "Paul E. McKenney" <paulmck@linux.vnet.ibm.com>
Cc: Rusty Russell <rusty@rustcorp.com.au>
Cc: Steven Rostedt <rostedt@goodmis.org>
Cc: "steiner@sgi.com" <steiner@sgi.com>
Cc: Nick Piggin <npiggin@suse.de>
LKML-Reference: <1237313814.27006.203.camel@localhost.localdomain>
Signed-off-by: Ingo Molnar <mingo@elte.hu>


---
 arch/x86/include/asm/apic.h           |   10 ++++++++++
 arch/x86/kernel/apic/x2apic_cluster.c |    6 ++++++
 arch/x86/kernel/apic/x2apic_phys.c    |    6 ++++++
 arch/x86/mm/tlb.c                     |    5 -----
 4 files changed, 22 insertions(+), 5 deletions(-)

diff --git a/arch/x86/include/asm/apic.h b/arch/x86/include/asm/apic.h
index 6d5b6f0..00f5962 100644
--- a/arch/x86/include/asm/apic.h
+++ b/arch/x86/include/asm/apic.h
@@ -108,6 +108,16 @@ extern void native_apic_icr_write(u32 low, u32 id);
 extern u64 native_apic_icr_read(void);
 
 #ifdef CONFIG_X86_X2APIC
+/*
+ * Make previous memory operations globally visible before
+ * sending the IPI through x2apic wrmsr. We need a serializing instruction or
+ * mfence for this.
+ */
+static inline void x2apic_wrmsr_fence(void)
+{
+	asm volatile("mfence" : : : "memory");
+}
+
 static inline void native_apic_msr_write(u32 reg, u32 v)
 {
 	if (reg == APIC_DFR || reg == APIC_ID || reg == APIC_LDR ||
diff --git a/arch/x86/kernel/apic/x2apic_cluster.c b/arch/x86/kernel/apic/x2apic_cluster.c
index 8fb87b6..4a903e2 100644
--- a/arch/x86/kernel/apic/x2apic_cluster.c
+++ b/arch/x86/kernel/apic/x2apic_cluster.c
@@ -57,6 +57,8 @@ static void x2apic_send_IPI_mask(const struct cpumask *mask, int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		__x2apic_send_IPI_dest(
@@ -73,6 +75,8 @@ static void
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		if (query_cpu == this_cpu)
@@ -90,6 +94,8 @@ static void x2apic_send_IPI_allbutself(int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_online_cpu(query_cpu) {
 		if (query_cpu == this_cpu)
diff --git a/arch/x86/kernel/apic/x2apic_phys.c b/arch/x86/kernel/apic/x2apic_phys.c
index 23625b9..a284359 100644
--- a/arch/x86/kernel/apic/x2apic_phys.c
+++ b/arch/x86/kernel/apic/x2apic_phys.c
@@ -58,6 +58,8 @@ static void x2apic_send_IPI_mask(const struct cpumask *mask, int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		__x2apic_send_IPI_dest(per_cpu(x86_cpu_to_apicid, query_cpu),
@@ -73,6 +75,8 @@ static void
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_cpu(query_cpu, mask) {
 		if (query_cpu != this_cpu)
@@ -89,6 +93,8 @@ static void x2apic_send_IPI_allbutself(int vector)
 	unsigned long query_cpu;
 	unsigned long flags;
 
+	x2apic_wrmsr_fence();
+
 	local_irq_save(flags);
 	for_each_online_cpu(query_cpu) {
 		if (query_cpu == this_cpu)
diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index a654d59..821e970 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -187,11 +187,6 @@ static void flush_tlb_others_ipi(const struct cpumask *cpumask,
 		       cpumask, cpumask_of(smp_processor_id()));
 
 	/*
-	 * Make the above memory operations globally visible before
-	 * sending the IPI.
-	 */
-	smp_mb();
-	/*
 	 * We have to send the IPI only to
 	 * CPUs affected.
 	 */

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

end of thread, other threads:[~2009-03-18  8:53 UTC | newest]

Thread overview: 103+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2009-02-16 16:38 [PATCH 0/4] generic smp helpers vs kmalloc Peter Zijlstra
2009-02-16 16:38 ` [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many() Peter Zijlstra
2009-02-16 19:10   ` Oleg Nesterov
2009-02-16 19:41     ` Peter Zijlstra
2009-02-16 20:30       ` Oleg Nesterov
2009-02-16 20:55         ` Peter Zijlstra
2009-02-16 21:22           ` Oleg Nesterov
2009-02-17 12:25     ` Oleg Nesterov
2009-02-16 20:49   ` Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()) Oleg Nesterov
2009-02-16 21:03     ` Peter Zijlstra
2009-02-16 21:32       ` Oleg Nesterov
2009-02-16 21:45         ` Peter Zijlstra
2009-02-16 22:02           ` Oleg Nesterov
2009-02-16 22:24             ` Peter Zijlstra
2009-02-16 23:19               ` Oleg Nesterov
2009-02-17  9:29                 ` Peter Zijlstra
2009-02-17 10:11                   ` Nick Piggin
2009-02-17 10:27                     ` Peter Zijlstra
2009-02-17 10:39                       ` Nick Piggin
2009-02-17 11:26                       ` Nick Piggin
2009-02-17 11:48                         ` Peter Zijlstra
2009-02-17 15:51                         ` Paul E. McKenney
2009-02-18  2:15                           ` Suresh Siddha
2009-02-18  2:40                             ` Paul E. McKenney
2009-02-17 19:28                         ` Q: " Oleg Nesterov
2009-02-17 21:32                           ` Paul E. McKenney
2009-02-17 21:45                             ` Oleg Nesterov
2009-02-17 22:39                               ` Paul E. McKenney
2009-02-18 13:52                                 ` Nick Piggin
2009-02-18 16:09                                   ` Linus Torvalds
2009-02-18 16:21                                     ` Ingo Molnar
2009-02-18 16:21                                       ` Ingo Molnar
2009-02-18 16:21                                       ` Ingo Molnar
2009-02-18 16:33                                       ` Linus Torvalds
2009-02-18 16:58                                         ` Ingo Molnar
2009-02-18 17:05                                           ` Ingo Molnar
2009-02-18 17:10                                             ` Ingo Molnar
2009-02-18 17:17                                               ` Linus Torvalds
2009-02-18 17:23                                                 ` Ingo Molnar
2009-02-18 17:14                                             ` Linus Torvalds
2009-02-18 17:47                                               ` Ingo Molnar
2009-02-18 18:33                                               ` Suresh Siddha
2009-02-18 16:37                                       ` Gleb Natapov
2009-02-19  0:12                                     ` Nick Piggin
2009-02-19  6:47                                     ` Benjamin Herrenschmidt
2009-02-19 13:11                                       ` Nick Piggin
2009-02-19 15:06                                         ` Ingo Molnar
2009-02-19 21:49                                           ` Benjamin Herrenschmidt
2009-02-18  2:21                         ` Suresh Siddha
2009-02-18 13:59                           ` Nick Piggin
2009-02-18 16:19                             ` Linus Torvalds
2009-02-18 16:23                               ` Ingo Molnar
2009-02-18 18:43                             ` Suresh Siddha
2009-02-18 19:17                               ` Ingo Molnar
2009-02-18 23:55                                 ` Suresh Siddha
2009-02-19 12:20                                   ` Ingo Molnar
2009-02-19 12:29                                     ` Nick Piggin
2009-02-19 12:45                                       ` Ingo Molnar
2009-02-19 22:00                                     ` Suresh Siddha
2009-02-20 10:56                                       ` Ingo Molnar
2009-02-20 18:56                                         ` Suresh Siddha
2009-02-20 19:40                                           ` Ingo Molnar
2009-02-20 23:28                                           ` Jack Steiner
2009-02-25  3:32                                           ` Nick Piggin
2009-02-25 12:47                                             ` Ingo Molnar
2009-02-25 18:25                                             ` Luck, Tony
2009-03-17 18:16                                             ` Suresh Siddha
2009-03-18  8:51                                               ` [tip:x86/x2apic] x86: add x2apic_wrmsr_fence() to x2apic flush tlb paths Suresh Siddha
2009-02-17 12:40                   ` Q: smp.c && barriers (Was: [PATCH 1/4] generic-smp: remove single ipi fallback for smp_call_function_many()) Peter Zijlstra
2009-02-17 15:43                   ` Paul E. McKenney
2009-02-17 15:40   ` [PATCH] generic-smp: remove kmalloc() Peter Zijlstra
2009-02-17 17:21     ` Oleg Nesterov
2009-02-17 17:40       ` Peter Zijlstra
2009-02-17 17:46         ` Peter Zijlstra
2009-02-17 18:30           ` Oleg Nesterov
2009-02-17 19:29         ` [PATCH -v4] generic-ipi: " Peter Zijlstra
2009-02-17 20:02           ` Oleg Nesterov
2009-02-17 20:11             ` Peter Zijlstra
2009-02-17 20:16               ` Peter Zijlstra
2009-02-17 20:44                 ` Oleg Nesterov
2009-02-17 20:49                 ` Peter Zijlstra
2009-02-17 22:09                   ` Oleg Nesterov
2009-02-17 22:15                     ` Peter Zijlstra
2009-02-17 21:30           ` Paul E. McKenney
2009-02-17 21:38             ` Peter Zijlstra
2009-02-16 16:38 ` [PATCH 2/4] generic-smp: remove kmalloc usage Peter Zijlstra
2009-02-17  0:40   ` Linus Torvalds
2009-02-17  8:24     ` Peter Zijlstra
2009-02-17  9:43       ` Ingo Molnar
2009-02-17  9:49         ` Peter Zijlstra
2009-02-17 10:56           ` Ingo Molnar
2009-02-18  4:50         ` Rusty Russell
2009-02-18 16:05           ` Ingo Molnar
2009-02-19  0:00             ` Jeremy Fitzhardinge
2009-02-19 12:21               ` Ingo Molnar
2009-02-19  4:31             ` Rusty Russell
2009-02-19  9:10               ` Peter Zijlstra
2009-02-19 11:04                 ` Jens Axboe
2009-02-19 16:52               ` Linus Torvalds
2009-02-17 15:44       ` Linus Torvalds
2009-02-16 16:38 ` [PATCH 3/4] generic-smp: properly allocate the cpumasks Peter Zijlstra
2009-02-16 23:17   ` Rusty Russell
2009-02-16 16:38 ` [PATCH 4/4] generic-smp: clean up some of the csd->flags fiddling 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.