All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH v5 0/8] x86/tlb: Concurrent TLB flushes
@ 2021-02-09 22:16 ` Nadav Amit
  0 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen,
	Nadav Amit, Borislav Petkov, Boris Ostrovsky, Haiyang Zhang,
	Ingo Molnar, Josh Poimboeuf, Juergen Gross, K. Y. Srinivasan,
	Paolo Bonzini, Rik van Riel, Sasha Levin, Stephen Hemminger, kvm,
	linux-hyperv, virtualization, x86, xen-devel

From: Nadav Amit <namit@vmware.com>

This is a respin of a rebased version of an old series, which I did not
follow, as I was preoccupied with personal issues (sorry).

The series improve TLB shootdown by flushing the local TLB concurrently
with remote TLBs, overlapping the IPI delivery time with the local
flush. Performance numbers can be found in the previous version [1].

The patches are essentially the same, but rebasing on the last version
required some changes. I left the reviewed-by tags - if anyone considers
it inappropriate, please let me know (and you have my apology).

[1] https://lore.kernel.org/lkml/20190823224153.15223-1-namit@vmware.com/

v4 -> v5:
* Rebase on 5.11
* Move concurrent smp logic to smp_call_function_many_cond() 
* Remove SGI-UV patch which is not needed anymore

v3 -> v4:
* Merge flush_tlb_func_local and flush_tlb_func_remote() [Peter]
* Prevent preemption on_each_cpu(). It is not needed, but it prevents
  concerns. [Peter/tglx]
* Adding acked-, review-by tags

v2 -> v3:
* Open-code the remote/local-flush decision code [Andy]
* Fix hyper-v, Xen implementations [Andrew]
* Fix redundant TLB flushes.

v1 -> v2:
* Removing the patches that Thomas took [tglx]
* Adding hyper-v, Xen compile-tested implementations [Dave]
* Removing UV [Andy]
* Adding lazy optimization, removing inline keyword [Dave]
* Restructuring patch-set

RFCv2 -> v1:
* Fix comment on flush_tlb_multi [Juergen]
* Removing async invalidation optimizations [Andy]
* Adding KVM support [Paolo]

Cc: Andy Lutomirski <luto@kernel.org>
Cc: Borislav Petkov <bp@alien8.de>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: Dave Hansen <dave.hansen@linux.intel.com>
Cc: Haiyang Zhang <haiyangz@microsoft.com>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Josh Poimboeuf <jpoimboe@redhat.com>
Cc: Juergen Gross <jgross@suse.com>
Cc: "K. Y. Srinivasan" <kys@microsoft.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Rik van Riel <riel@surriel.com>
Cc: Sasha Levin <sashal@kernel.org>
Cc: Stephen Hemminger <sthemmin@microsoft.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: kvm@vger.kernel.org
Cc: linux-hyperv@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: virtualization@lists.linux-foundation.org
Cc: x86@kernel.org
Cc: xen-devel@lists.xenproject.org

Nadav Amit (8):
  smp: Run functions concurrently in smp_call_function_many_cond()
  x86/mm/tlb: Unify flush_tlb_func_local() and flush_tlb_func_remote()
  x86/mm/tlb: Open-code on_each_cpu_cond_mask() for tlb_is_not_lazy()
  x86/mm/tlb: Flush remote and local TLBs concurrently
  x86/mm/tlb: Privatize cpu_tlbstate
  x86/mm/tlb: Do not make is_lazy dirty for no reason
  cpumask: Mark functions as pure
  x86/mm/tlb: Remove unnecessary uses of the inline keyword

 arch/x86/hyperv/mmu.c                 |  10 +-
 arch/x86/include/asm/paravirt.h       |   6 +-
 arch/x86/include/asm/paravirt_types.h |   4 +-
 arch/x86/include/asm/tlbflush.h       |  48 +++----
 arch/x86/include/asm/trace/hyperv.h   |   2 +-
 arch/x86/kernel/alternative.c         |   2 +-
 arch/x86/kernel/kvm.c                 |  11 +-
 arch/x86/mm/init.c                    |   2 +-
 arch/x86/mm/tlb.c                     | 177 +++++++++++++++-----------
 arch/x86/xen/mmu_pv.c                 |  11 +-
 include/linux/cpumask.h               |   6 +-
 include/trace/events/xen.h            |   2 +-
 kernel/smp.c                          | 148 +++++++++++----------
 13 files changed, 242 insertions(+), 187 deletions(-)

-- 
2.25.1


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

* [PATCH v5 0/8] x86/tlb: Concurrent TLB flushes
@ 2021-02-09 22:16 ` Nadav Amit
  0 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Juergen Gross, Sasha Levin, linux-hyperv, x86, Stephen Hemminger,
	kvm, Paolo Bonzini, Rik van Riel, Peter Zijlstra, Haiyang Zhang,
	Dave Hansen, linux-kernel, virtualization, xen-devel,
	Ingo Molnar, Borislav Petkov, Andy Lutomirski, Josh Poimboeuf,
	Nadav Amit, Boris Ostrovsky

From: Nadav Amit <namit@vmware.com>

This is a respin of a rebased version of an old series, which I did not
follow, as I was preoccupied with personal issues (sorry).

The series improve TLB shootdown by flushing the local TLB concurrently
with remote TLBs, overlapping the IPI delivery time with the local
flush. Performance numbers can be found in the previous version [1].

The patches are essentially the same, but rebasing on the last version
required some changes. I left the reviewed-by tags - if anyone considers
it inappropriate, please let me know (and you have my apology).

[1] https://lore.kernel.org/lkml/20190823224153.15223-1-namit@vmware.com/

v4 -> v5:
* Rebase on 5.11
* Move concurrent smp logic to smp_call_function_many_cond() 
* Remove SGI-UV patch which is not needed anymore

v3 -> v4:
* Merge flush_tlb_func_local and flush_tlb_func_remote() [Peter]
* Prevent preemption on_each_cpu(). It is not needed, but it prevents
  concerns. [Peter/tglx]
* Adding acked-, review-by tags

v2 -> v3:
* Open-code the remote/local-flush decision code [Andy]
* Fix hyper-v, Xen implementations [Andrew]
* Fix redundant TLB flushes.

v1 -> v2:
* Removing the patches that Thomas took [tglx]
* Adding hyper-v, Xen compile-tested implementations [Dave]
* Removing UV [Andy]
* Adding lazy optimization, removing inline keyword [Dave]
* Restructuring patch-set

RFCv2 -> v1:
* Fix comment on flush_tlb_multi [Juergen]
* Removing async invalidation optimizations [Andy]
* Adding KVM support [Paolo]

Cc: Andy Lutomirski <luto@kernel.org>
Cc: Borislav Petkov <bp@alien8.de>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: Dave Hansen <dave.hansen@linux.intel.com>
Cc: Haiyang Zhang <haiyangz@microsoft.com>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Josh Poimboeuf <jpoimboe@redhat.com>
Cc: Juergen Gross <jgross@suse.com>
Cc: "K. Y. Srinivasan" <kys@microsoft.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Rik van Riel <riel@surriel.com>
Cc: Sasha Levin <sashal@kernel.org>
Cc: Stephen Hemminger <sthemmin@microsoft.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: kvm@vger.kernel.org
Cc: linux-hyperv@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: virtualization@lists.linux-foundation.org
Cc: x86@kernel.org
Cc: xen-devel@lists.xenproject.org

Nadav Amit (8):
  smp: Run functions concurrently in smp_call_function_many_cond()
  x86/mm/tlb: Unify flush_tlb_func_local() and flush_tlb_func_remote()
  x86/mm/tlb: Open-code on_each_cpu_cond_mask() for tlb_is_not_lazy()
  x86/mm/tlb: Flush remote and local TLBs concurrently
  x86/mm/tlb: Privatize cpu_tlbstate
  x86/mm/tlb: Do not make is_lazy dirty for no reason
  cpumask: Mark functions as pure
  x86/mm/tlb: Remove unnecessary uses of the inline keyword

 arch/x86/hyperv/mmu.c                 |  10 +-
 arch/x86/include/asm/paravirt.h       |   6 +-
 arch/x86/include/asm/paravirt_types.h |   4 +-
 arch/x86/include/asm/tlbflush.h       |  48 +++----
 arch/x86/include/asm/trace/hyperv.h   |   2 +-
 arch/x86/kernel/alternative.c         |   2 +-
 arch/x86/kernel/kvm.c                 |  11 +-
 arch/x86/mm/init.c                    |   2 +-
 arch/x86/mm/tlb.c                     | 177 +++++++++++++++-----------
 arch/x86/xen/mmu_pv.c                 |  11 +-
 include/linux/cpumask.h               |   6 +-
 include/trace/events/xen.h            |   2 +-
 kernel/smp.c                          | 148 +++++++++++----------
 13 files changed, 242 insertions(+), 187 deletions(-)

-- 
2.25.1

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

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

* [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-09 22:16 ` Nadav Amit
  (?)
@ 2021-02-09 22:16 ` Nadav Amit
  2021-02-16 12:04   ` Peter Zijlstra
                     ` (3 more replies)
  -1 siblings, 4 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen,
	Nadav Amit, Rik van Riel, Josh Poimboeuf

From: Nadav Amit <namit@vmware.com>

Currently, on_each_cpu() and similar functions do not exploit the
potential of concurrency: the function is first executed remotely and
only then it is executed locally. Functions such as TLB flush can take
considerable time, so this provides an opportunity for performance
optimization.

To do so, modify smp_call_function_many_cond(), to allows the callers to
provide a function that should be executed (remotely/locally), and run
them concurrently. Keep other smp_call_function_many() semantic as it is
today for backward compatibility: the called function is not executed in
this case locally.

smp_call_function_many_cond() does not use the optimized version for a
single remote target that smp_call_function_single() implements. For
synchronous function call, smp_call_function_single() keeps a
call_single_data (which is used for synchronization) on the stack.
Interestingly, it seems that not using this optimization provides
greater performance improvements (greater speedup with a single remote
target than with multiple ones). Presumably, holding data structures
that are intended for synchronization on the stack can introduce
overheads due to TLB misses and false-sharing when the stack is used for
other purposes.

Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Rik van Riel <riel@surriel.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Andy Lutomirski <luto@kernel.org>
Cc: Josh Poimboeuf <jpoimboe@redhat.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 kernel/smp.c | 148 ++++++++++++++++++++++++++++-----------------------
 1 file changed, 80 insertions(+), 68 deletions(-)

diff --git a/kernel/smp.c b/kernel/smp.c
index 1b6070bf97bb..c308130f3bc9 100644
--- a/kernel/smp.c
+++ b/kernel/smp.c
@@ -604,12 +604,23 @@ int smp_call_function_any(const struct cpumask *mask,
 }
 EXPORT_SYMBOL_GPL(smp_call_function_any);
 
+/*
+ * Flags to be used as scf_flags argument of smp_call_function_many_cond().
+ */
+#define SCF_WAIT	(1U << 0)	/* Wait until function execution completed */
+#define SCF_RUN_LOCAL	(1U << 1)	/* Run also locally if local cpu is set in cpumask */
+
 static void smp_call_function_many_cond(const struct cpumask *mask,
 					smp_call_func_t func, void *info,
-					bool wait, smp_cond_func_t cond_func)
+					unsigned int scf_flags,
+					smp_cond_func_t cond_func)
 {
+	int cpu, last_cpu, this_cpu = smp_processor_id();
 	struct call_function_data *cfd;
-	int cpu, next_cpu, this_cpu = smp_processor_id();
+	bool wait = scf_flags & SCF_WAIT;
+	bool run_remote = false;
+	bool run_local = false;
+	int nr_cpus = 0;
 
 	/*
 	 * Can deadlock when called with interrupts disabled.
@@ -617,8 +628,8 @@ static void smp_call_function_many_cond(const struct cpumask *mask,
 	 * send smp call function interrupt to this cpu and as such deadlocks
 	 * can't happen.
 	 */
-	WARN_ON_ONCE(cpu_online(this_cpu) && irqs_disabled()
-		     && !oops_in_progress && !early_boot_irqs_disabled);
+	if (cpu_online(this_cpu) && !oops_in_progress && !early_boot_irqs_disabled)
+		lockdep_assert_irqs_enabled();
 
 	/*
 	 * When @wait we can deadlock when we interrupt between llist_add() and
@@ -628,60 +639,65 @@ static void smp_call_function_many_cond(const struct cpumask *mask,
 	 */
 	WARN_ON_ONCE(!in_task());
 
-	/* Try to fastpath.  So, what's a CPU they want? Ignoring this one. */
+	/* Check if we need local execution. */
+	if ((scf_flags & SCF_RUN_LOCAL) && cpumask_test_cpu(this_cpu, mask))
+		run_local = true;
+
+	/* Check if we need remote execution, i.e., any CPU excluding this one. */
 	cpu = cpumask_first_and(mask, cpu_online_mask);
 	if (cpu == this_cpu)
 		cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
+	if (cpu < nr_cpu_ids)
+		run_remote = true;
 
-	/* No online cpus?  We're done. */
-	if (cpu >= nr_cpu_ids)
-		return;
-
-	/* Do we have another CPU which isn't us? */
-	next_cpu = cpumask_next_and(cpu, mask, cpu_online_mask);
-	if (next_cpu == this_cpu)
-		next_cpu = cpumask_next_and(next_cpu, mask, cpu_online_mask);
-
-	/* Fastpath: do that cpu by itself. */
-	if (next_cpu >= nr_cpu_ids) {
-		if (!cond_func || cond_func(cpu, info))
-			smp_call_function_single(cpu, func, info, wait);
-		return;
-	}
-
-	cfd = this_cpu_ptr(&cfd_data);
-
-	cpumask_and(cfd->cpumask, mask, cpu_online_mask);
-	__cpumask_clear_cpu(this_cpu, cfd->cpumask);
+	if (run_remote) {
+		cfd = this_cpu_ptr(&cfd_data);
+		cpumask_and(cfd->cpumask, mask, cpu_online_mask);
+		__cpumask_clear_cpu(this_cpu, cfd->cpumask);
 
-	/* Some callers race with other cpus changing the passed mask */
-	if (unlikely(!cpumask_weight(cfd->cpumask)))
-		return;
-
-	cpumask_clear(cfd->cpumask_ipi);
-	for_each_cpu(cpu, cfd->cpumask) {
-		call_single_data_t *csd = per_cpu_ptr(cfd->csd, cpu);
+		cpumask_clear(cfd->cpumask_ipi);
+		for_each_cpu(cpu, cfd->cpumask) {
+			call_single_data_t *csd = per_cpu_ptr(cfd->csd, cpu);
 
-		if (cond_func && !cond_func(cpu, info))
-			continue;
+			if (cond_func && !cond_func(cpu, info))
+				continue;
 
-		csd_lock(csd);
-		if (wait)
-			csd->node.u_flags |= CSD_TYPE_SYNC;
-		csd->func = func;
-		csd->info = info;
+			csd_lock(csd);
+			if (wait)
+				csd->node.u_flags |= CSD_TYPE_SYNC;
+			csd->func = func;
+			csd->info = info;
 #ifdef CONFIG_CSD_LOCK_WAIT_DEBUG
-		csd->node.src = smp_processor_id();
-		csd->node.dst = cpu;
+			csd->node.src = smp_processor_id();
+			csd->node.dst = cpu;
 #endif
-		if (llist_add(&csd->node.llist, &per_cpu(call_single_queue, cpu)))
-			__cpumask_set_cpu(cpu, cfd->cpumask_ipi);
+			if (llist_add(&csd->node.llist, &per_cpu(call_single_queue, cpu))) {
+				__cpumask_set_cpu(cpu, cfd->cpumask_ipi);
+				nr_cpus++;
+				last_cpu = cpu;
+			}
+		}
+
+		/*
+		 * Choose the most efficient way to send an IPI. Note that the
+		 * number of CPUs might be zero due to concurrent changes to the
+		 * provided mask.
+		 */
+		if (nr_cpus == 1)
+			arch_send_call_function_single_ipi(last_cpu);
+		else if (likely(nr_cpus > 1))
+			arch_send_call_function_ipi_mask(cfd->cpumask_ipi);
 	}
 
-	/* Send a message to all CPUs in the map */
-	arch_send_call_function_ipi_mask(cfd->cpumask_ipi);
+	if (run_local && (!cond_func || cond_func(this_cpu, info))) {
+		unsigned long flags;
 
-	if (wait) {
+		local_irq_save(flags);
+		func(info);
+		local_irq_restore(flags);
+	}
+
+	if (run_remote && wait) {
 		for_each_cpu(cpu, cfd->cpumask) {
 			call_single_data_t *csd;
 
@@ -692,12 +708,14 @@ static void smp_call_function_many_cond(const struct cpumask *mask,
 }
 
 /**
- * smp_call_function_many(): Run a function on a set of other CPUs.
+ * smp_call_function_many(): Run a function on a set of CPUs.
  * @mask: The set of cpus to run on (only runs on online subset).
  * @func: The function to run. This must be fast and non-blocking.
  * @info: An arbitrary pointer to pass to the function.
- * @wait: If true, wait (atomically) until function has completed
- *        on other CPUs.
+ * @flags: Bitmask that controls the operation. If %SCF_WAIT is set, wait
+ *        (atomically) until function has completed on other CPUs. If
+ *        %SCF_RUN_LOCAL is set, the function will also be run locally
+ *        if the local CPU is set in the @cpumask.
  *
  * If @wait is true, then returns once @func has returned.
  *
@@ -708,7 +726,7 @@ static void smp_call_function_many_cond(const struct cpumask *mask,
 void smp_call_function_many(const struct cpumask *mask,
 			    smp_call_func_t func, void *info, bool wait)
 {
-	smp_call_function_many_cond(mask, func, info, wait, NULL);
+	smp_call_function_many_cond(mask, func, info, wait ? SCF_WAIT : 0, NULL);
 }
 EXPORT_SYMBOL(smp_call_function_many);
 
@@ -856,16 +874,15 @@ EXPORT_SYMBOL(on_each_cpu);
 void on_each_cpu_mask(const struct cpumask *mask, smp_call_func_t func,
 			void *info, bool wait)
 {
-	int cpu = get_cpu();
+	unsigned int scf_flags;
 
-	smp_call_function_many(mask, func, info, wait);
-	if (cpumask_test_cpu(cpu, mask)) {
-		unsigned long flags;
-		local_irq_save(flags);
-		func(info);
-		local_irq_restore(flags);
-	}
-	put_cpu();
+	scf_flags = SCF_RUN_LOCAL;
+	if (wait)
+		scf_flags |= SCF_WAIT;
+
+	preempt_disable();
+	smp_call_function_many_cond(mask, func, info, scf_flags, NULL);
+	preempt_enable();
 }
 EXPORT_SYMBOL(on_each_cpu_mask);
 
@@ -894,17 +911,12 @@ EXPORT_SYMBOL(on_each_cpu_mask);
 void on_each_cpu_cond_mask(smp_cond_func_t cond_func, smp_call_func_t func,
 			   void *info, bool wait, const struct cpumask *mask)
 {
-	int cpu = get_cpu();
+	unsigned int scf_flags = SCF_RUN_LOCAL;
 
-	smp_call_function_many_cond(mask, func, info, wait, cond_func);
-	if (cpumask_test_cpu(cpu, mask) && cond_func(cpu, info)) {
-		unsigned long flags;
+	if (wait)
+		scf_flags |= SCF_WAIT;
 
-		local_irq_save(flags);
-		func(info);
-		local_irq_restore(flags);
-	}
-	put_cpu();
+	smp_call_function_many_cond(mask, func, info, scf_flags, cond_func);
 }
 EXPORT_SYMBOL(on_each_cpu_cond_mask);
 
-- 
2.25.1


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

* [PATCH v5 2/8] x86/mm/tlb: Unify flush_tlb_func_local() and flush_tlb_func_remote()
  2021-02-09 22:16 ` Nadav Amit
  (?)
  (?)
@ 2021-02-09 22:16 ` Nadav Amit
  -1 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen,
	Nadav Amit, Rik van Riel, Josh Poimboeuf

From: Nadav Amit <namit@vmware.com>

The unification of these two functions allows to use them in the updated
SMP infrastrucutre.

To do so, remove the reason argument from flush_tlb_func_local(), add
a member to struct tlb_flush_info that says which CPU initiated the
flush and act accordingly. Optimize the size of flush_tlb_info while we
are at it.

Unfortunately, this prevents us from using a constant tlb_flush_info for
arch_tlbbatch_flush(), but in a later stage we may be able to inline
tlb_flush_info into the IPI data, so it should not have an impact
eventually.

Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Rik van Riel <riel@surriel.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Andy Lutomirski <luto@kernel.org>
Cc: Josh Poimboeuf <jpoimboe@redhat.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 arch/x86/include/asm/tlbflush.h |  5 +-
 arch/x86/mm/tlb.c               | 81 +++++++++++++++------------------
 2 files changed, 39 insertions(+), 47 deletions(-)

diff --git a/arch/x86/include/asm/tlbflush.h b/arch/x86/include/asm/tlbflush.h
index 8c87a2e0b660..a7a598af116d 100644
--- a/arch/x86/include/asm/tlbflush.h
+++ b/arch/x86/include/asm/tlbflush.h
@@ -201,8 +201,9 @@ struct flush_tlb_info {
 	unsigned long		start;
 	unsigned long		end;
 	u64			new_tlb_gen;
-	unsigned int		stride_shift;
-	bool			freed_tables;
+	unsigned int		initiating_cpu;
+	u8			stride_shift;
+	u8			freed_tables;
 };
 
 void flush_tlb_local(void);
diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index 569ac1d57f55..bf12371db6c4 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -439,7 +439,7 @@ void switch_mm_irqs_off(struct mm_struct *prev, struct mm_struct *next,
 	 * NB: leave_mm() calls us with prev == NULL and tsk == NULL.
 	 */
 
-	/* We don't want flush_tlb_func_* to run concurrently with us. */
+	/* We don't want flush_tlb_func() to run concurrently with us. */
 	if (IS_ENABLED(CONFIG_PROVE_LOCKING))
 		WARN_ON_ONCE(!irqs_disabled());
 
@@ -647,14 +647,13 @@ void initialize_tlbstate_and_flush(void)
 }
 
 /*
- * flush_tlb_func_common()'s memory ordering requirement is that any
+ * flush_tlb_func()'s memory ordering requirement is that any
  * TLB fills that happen after we flush the TLB are ordered after we
  * read active_mm's tlb_gen.  We don't need any explicit barriers
  * because all x86 flush operations are serializing and the
  * atomic64_read operation won't be reordered by the compiler.
  */
-static void flush_tlb_func_common(const struct flush_tlb_info *f,
-				  bool local, enum tlb_flush_reason reason)
+static void flush_tlb_func(void *info)
 {
 	/*
 	 * We have three different tlb_gen values in here.  They are:
@@ -665,14 +664,26 @@ static void flush_tlb_func_common(const struct flush_tlb_info *f,
 	 * - f->new_tlb_gen: the generation that the requester of the flush
 	 *                   wants us to catch up to.
 	 */
+	const struct flush_tlb_info *f = info;
 	struct mm_struct *loaded_mm = this_cpu_read(cpu_tlbstate.loaded_mm);
 	u32 loaded_mm_asid = this_cpu_read(cpu_tlbstate.loaded_mm_asid);
 	u64 mm_tlb_gen = atomic64_read(&loaded_mm->context.tlb_gen);
 	u64 local_tlb_gen = this_cpu_read(cpu_tlbstate.ctxs[loaded_mm_asid].tlb_gen);
+	bool local = smp_processor_id() == f->initiating_cpu;
+	unsigned long nr_invalidate = 0;
 
 	/* This code cannot presently handle being reentered. */
 	VM_WARN_ON(!irqs_disabled());
 
+	if (!local) {
+		inc_irq_stat(irq_tlb_count);
+		count_vm_tlb_event(NR_TLB_REMOTE_FLUSH_RECEIVED);
+
+		/* Can only happen on remote CPUs */
+		if (f->mm && f->mm != loaded_mm)
+			return;
+	}
+
 	if (unlikely(loaded_mm == &init_mm))
 		return;
 
@@ -700,8 +711,7 @@ static void flush_tlb_func_common(const struct flush_tlb_info *f,
 		 * be handled can catch us all the way up, leaving no work for
 		 * the second flush.
 		 */
-		trace_tlb_flush(reason, 0);
-		return;
+		goto done;
 	}
 
 	WARN_ON_ONCE(local_tlb_gen > mm_tlb_gen);
@@ -748,46 +758,34 @@ static void flush_tlb_func_common(const struct flush_tlb_info *f,
 	    f->new_tlb_gen == local_tlb_gen + 1 &&
 	    f->new_tlb_gen == mm_tlb_gen) {
 		/* Partial flush */
-		unsigned long nr_invalidate = (f->end - f->start) >> f->stride_shift;
 		unsigned long addr = f->start;
 
+		nr_invalidate = (f->end - f->start) >> f->stride_shift;
+
 		while (addr < f->end) {
 			flush_tlb_one_user(addr);
 			addr += 1UL << f->stride_shift;
 		}
 		if (local)
 			count_vm_tlb_events(NR_TLB_LOCAL_FLUSH_ONE, nr_invalidate);
-		trace_tlb_flush(reason, nr_invalidate);
 	} else {
 		/* Full flush. */
+		nr_invalidate = TLB_FLUSH_ALL;
+
 		flush_tlb_local();
 		if (local)
 			count_vm_tlb_event(NR_TLB_LOCAL_FLUSH_ALL);
-		trace_tlb_flush(reason, TLB_FLUSH_ALL);
 	}
 
 	/* Both paths above update our state to mm_tlb_gen. */
 	this_cpu_write(cpu_tlbstate.ctxs[loaded_mm_asid].tlb_gen, mm_tlb_gen);
-}
-
-static void flush_tlb_func_local(const void *info, enum tlb_flush_reason reason)
-{
-	const struct flush_tlb_info *f = info;
-
-	flush_tlb_func_common(f, true, reason);
-}
 
-static void flush_tlb_func_remote(void *info)
-{
-	const struct flush_tlb_info *f = info;
-
-	inc_irq_stat(irq_tlb_count);
-
-	if (f->mm && f->mm != this_cpu_read(cpu_tlbstate.loaded_mm))
-		return;
-
-	count_vm_tlb_event(NR_TLB_REMOTE_FLUSH_RECEIVED);
-	flush_tlb_func_common(f, false, TLB_REMOTE_SHOOTDOWN);
+	/* Tracing is done in a unified manner to reduce the code size */
+done:
+	trace_tlb_flush(!local ? TLB_REMOTE_SHOOTDOWN :
+				(f->mm == NULL) ? TLB_LOCAL_SHOOTDOWN :
+						  TLB_LOCAL_MM_SHOOTDOWN,
+			nr_invalidate);
 }
 
 static bool tlb_is_not_lazy(int cpu, void *data)
@@ -816,10 +814,10 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
 	 * doing a speculative memory access.
 	 */
 	if (info->freed_tables)
-		smp_call_function_many(cpumask, flush_tlb_func_remote,
+		smp_call_function_many(cpumask, flush_tlb_func,
 			       (void *)info, 1);
 	else
-		on_each_cpu_cond_mask(tlb_is_not_lazy, flush_tlb_func_remote,
+		on_each_cpu_cond_mask(tlb_is_not_lazy, flush_tlb_func,
 				(void *)info, 1, cpumask);
 }
 
@@ -869,6 +867,7 @@ static inline struct flush_tlb_info *get_flush_tlb_info(struct mm_struct *mm,
 	info->stride_shift	= stride_shift;
 	info->freed_tables	= freed_tables;
 	info->new_tlb_gen	= new_tlb_gen;
+	info->initiating_cpu	= smp_processor_id();
 
 	return info;
 }
@@ -908,7 +907,7 @@ void flush_tlb_mm_range(struct mm_struct *mm, unsigned long start,
 	if (mm == this_cpu_read(cpu_tlbstate.loaded_mm)) {
 		lockdep_assert_irqs_enabled();
 		local_irq_disable();
-		flush_tlb_func_local(info, TLB_LOCAL_MM_SHOOTDOWN);
+		flush_tlb_func(info);
 		local_irq_enable();
 	}
 
@@ -1119,34 +1118,26 @@ void __flush_tlb_all(void)
 }
 EXPORT_SYMBOL_GPL(__flush_tlb_all);
 
-/*
- * arch_tlbbatch_flush() performs a full TLB flush regardless of the active mm.
- * This means that the 'struct flush_tlb_info' that describes which mappings to
- * flush is actually fixed. We therefore set a single fixed struct and use it in
- * arch_tlbbatch_flush().
- */
-static const struct flush_tlb_info full_flush_tlb_info = {
-	.mm = NULL,
-	.start = 0,
-	.end = TLB_FLUSH_ALL,
-};
-
 void arch_tlbbatch_flush(struct arch_tlbflush_unmap_batch *batch)
 {
+	struct flush_tlb_info *info;
+
 	int cpu = get_cpu();
 
+	info = get_flush_tlb_info(NULL, 0, TLB_FLUSH_ALL, 0, false, 0);
 	if (cpumask_test_cpu(cpu, &batch->cpumask)) {
 		lockdep_assert_irqs_enabled();
 		local_irq_disable();
-		flush_tlb_func_local(&full_flush_tlb_info, TLB_LOCAL_SHOOTDOWN);
+		flush_tlb_func(info);
 		local_irq_enable();
 	}
 
 	if (cpumask_any_but(&batch->cpumask, cpu) < nr_cpu_ids)
-		flush_tlb_others(&batch->cpumask, &full_flush_tlb_info);
+		flush_tlb_others(&batch->cpumask, info);
 
 	cpumask_clear(&batch->cpumask);
 
+	put_flush_tlb_info();
 	put_cpu();
 }
 
-- 
2.25.1


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

* [PATCH v5 3/8] x86/mm/tlb: Open-code on_each_cpu_cond_mask() for tlb_is_not_lazy()
  2021-02-09 22:16 ` Nadav Amit
                   ` (2 preceding siblings ...)
  (?)
@ 2021-02-09 22:16 ` Nadav Amit
  2021-02-18  8:16   ` Christoph Hellwig
  -1 siblings, 1 reply; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen,
	Nadav Amit, Rik van Riel, Josh Poimboeuf

From: Nadav Amit <namit@vmware.com>

Open-code on_each_cpu_cond_mask() in native_flush_tlb_others() to
optimize the code. Open-coding eliminates the need for the indirect branch
that is used to call is_lazy(), and in CPUs that are vulnerable to
Spectre v2, it eliminates the retpoline. In addition, it allows to use a
preallocated cpumask to compute the CPUs that should be.

This would later allow us not to adapt on_each_cpu_cond_mask() to
support local and remote functions.

Note that calling tlb_is_not_lazy() for every CPU that needs to be
flushed, as done in native_flush_tlb_multi() might look ugly, but it is
equivalent to what is currently done in on_each_cpu_cond_mask().
Actually, native_flush_tlb_multi() does it more efficiently since it
avoids using an indirect branch for the matter.

Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Rik van Riel <riel@surriel.com>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Andy Lutomirski <luto@kernel.org>
Cc: Josh Poimboeuf <jpoimboe@redhat.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 arch/x86/mm/tlb.c | 37 ++++++++++++++++++++++++++++++++-----
 1 file changed, 32 insertions(+), 5 deletions(-)

diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index bf12371db6c4..07b6701a540a 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -788,11 +788,13 @@ static void flush_tlb_func(void *info)
 			nr_invalidate);
 }
 
-static bool tlb_is_not_lazy(int cpu, void *data)
+static bool tlb_is_not_lazy(int cpu)
 {
 	return !per_cpu(cpu_tlbstate.is_lazy, cpu);
 }
 
+static DEFINE_PER_CPU(cpumask_t, flush_tlb_mask);
+
 STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
 					 const struct flush_tlb_info *info)
 {
@@ -813,12 +815,37 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
 	 * up on the new contents of what used to be page tables, while
 	 * doing a speculative memory access.
 	 */
-	if (info->freed_tables)
+	if (info->freed_tables) {
 		smp_call_function_many(cpumask, flush_tlb_func,
 			       (void *)info, 1);
-	else
-		on_each_cpu_cond_mask(tlb_is_not_lazy, flush_tlb_func,
-				(void *)info, 1, cpumask);
+	} else {
+		/*
+		 * Although we could have used on_each_cpu_cond_mask(),
+		 * open-coding it has performance advantages, as it eliminates
+		 * the need for indirect calls or retpolines. In addition, it
+		 * allows to use a designated cpumask for evaluating the
+		 * condition, instead of allocating one.
+		 *
+		 * This code works under the assumption that there are no nested
+		 * TLB flushes, an assumption that is already made in
+		 * flush_tlb_mm_range().
+		 *
+		 * cond_cpumask is logically a stack-local variable, but it is
+		 * more efficient to have it off the stack and not to allocate
+		 * it on demand. Preemption is disabled and this code is
+		 * non-reentrant.
+		 */
+		struct cpumask *cond_cpumask = this_cpu_ptr(&flush_tlb_mask);
+		int cpu;
+
+		cpumask_clear(cond_cpumask);
+
+		for_each_cpu(cpu, cpumask) {
+			if (tlb_is_not_lazy(cpu))
+				__cpumask_set_cpu(cpu, cond_cpumask);
+		}
+		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);
+	}
 }
 
 void flush_tlb_others(const struct cpumask *cpumask,
-- 
2.25.1


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

* [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently
  2021-02-09 22:16 ` Nadav Amit
@ 2021-02-09 22:16   ` Nadav Amit
  -1 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen,
	Nadav Amit, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Sasha Levin, Ingo Molnar, Borislav Petkov, x86, Juergen Gross,
	Paolo Bonzini, Boris Ostrovsky, linux-hyperv, virtualization,
	kvm, xen-devel, Michael Kelley

From: Nadav Amit <namit@vmware.com>

To improve TLB shootdown performance, flush the remote and local TLBs
concurrently. Introduce flush_tlb_multi() that does so. Introduce
paravirtual versions of flush_tlb_multi() for KVM, Xen and hyper-v (Xen
and hyper-v are only compile-tested).

While the updated smp infrastructure is capable of running a function on
a single local core, it is not optimized for this case. The multiple
function calls and the indirect branch introduce some overhead, and
might make local TLB flushes slower than they were before the recent
changes.

Before calling the SMP infrastructure, check if only a local TLB flush
is needed to restore the lost performance in this common case. This
requires to check mm_cpumask() one more time, but unless this mask is
updated very frequently, this should impact performance negatively.

Cc: "K. Y. Srinivasan" <kys@microsoft.com>
Cc: Haiyang Zhang <haiyangz@microsoft.com>
Cc: Stephen Hemminger <sthemmin@microsoft.com>
Cc: Sasha Levin <sashal@kernel.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Borislav Petkov <bp@alien8.de>
Cc: x86@kernel.org
Cc: Juergen Gross <jgross@suse.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Andy Lutomirski <luto@kernel.org>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: linux-hyperv@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: virtualization@lists.linux-foundation.org
Cc: kvm@vger.kernel.org
Cc: xen-devel@lists.xenproject.org
Reviewed-by: Michael Kelley <mikelley@microsoft.com> # Hyper-v parts
Reviewed-by: Juergen Gross <jgross@suse.com> # Xen and paravirt parts
Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 arch/x86/hyperv/mmu.c                 | 10 +++---
 arch/x86/include/asm/paravirt.h       |  6 ++--
 arch/x86/include/asm/paravirt_types.h |  4 +--
 arch/x86/include/asm/tlbflush.h       |  4 +--
 arch/x86/include/asm/trace/hyperv.h   |  2 +-
 arch/x86/kernel/kvm.c                 | 11 ++++--
 arch/x86/mm/tlb.c                     | 49 +++++++++++++++++----------
 arch/x86/xen/mmu_pv.c                 | 11 +++---
 include/trace/events/xen.h            |  2 +-
 9 files changed, 58 insertions(+), 41 deletions(-)

diff --git a/arch/x86/hyperv/mmu.c b/arch/x86/hyperv/mmu.c
index 2c87350c1fb0..681dba8de4f2 100644
--- a/arch/x86/hyperv/mmu.c
+++ b/arch/x86/hyperv/mmu.c
@@ -52,8 +52,8 @@ static inline int fill_gva_list(u64 gva_list[], int offset,
 	return gva_n - offset;
 }
 
-static void hyperv_flush_tlb_others(const struct cpumask *cpus,
-				    const struct flush_tlb_info *info)
+static void hyperv_flush_tlb_multi(const struct cpumask *cpus,
+				   const struct flush_tlb_info *info)
 {
 	int cpu, vcpu, gva_n, max_gvas;
 	struct hv_tlb_flush **flush_pcpu;
@@ -61,7 +61,7 @@ static void hyperv_flush_tlb_others(const struct cpumask *cpus,
 	u64 status = U64_MAX;
 	unsigned long flags;
 
-	trace_hyperv_mmu_flush_tlb_others(cpus, info);
+	trace_hyperv_mmu_flush_tlb_multi(cpus, info);
 
 	if (!hv_hypercall_pg)
 		goto do_native;
@@ -164,7 +164,7 @@ static void hyperv_flush_tlb_others(const struct cpumask *cpus,
 	if (!(status & HV_HYPERCALL_RESULT_MASK))
 		return;
 do_native:
-	native_flush_tlb_others(cpus, info);
+	native_flush_tlb_multi(cpus, info);
 }
 
 static u64 hyperv_flush_tlb_others_ex(const struct cpumask *cpus,
@@ -239,6 +239,6 @@ void hyperv_setup_mmu_ops(void)
 		return;
 
 	pr_info("Using hypercall for remote TLB flush\n");
-	pv_ops.mmu.flush_tlb_others = hyperv_flush_tlb_others;
+	pv_ops.mmu.flush_tlb_multi = hyperv_flush_tlb_multi;
 	pv_ops.mmu.tlb_remove_table = tlb_remove_table;
 }
diff --git a/arch/x86/include/asm/paravirt.h b/arch/x86/include/asm/paravirt.h
index f8dce11d2bc1..515e49204c8b 100644
--- a/arch/x86/include/asm/paravirt.h
+++ b/arch/x86/include/asm/paravirt.h
@@ -50,7 +50,7 @@ static inline void slow_down_io(void)
 void native_flush_tlb_local(void);
 void native_flush_tlb_global(void);
 void native_flush_tlb_one_user(unsigned long addr);
-void native_flush_tlb_others(const struct cpumask *cpumask,
+void native_flush_tlb_multi(const struct cpumask *cpumask,
 			     const struct flush_tlb_info *info);
 
 static inline void __flush_tlb_local(void)
@@ -68,10 +68,10 @@ static inline void __flush_tlb_one_user(unsigned long addr)
 	PVOP_VCALL1(mmu.flush_tlb_one_user, addr);
 }
 
-static inline void __flush_tlb_others(const struct cpumask *cpumask,
+static inline void __flush_tlb_multi(const struct cpumask *cpumask,
 				      const struct flush_tlb_info *info)
 {
-	PVOP_VCALL2(mmu.flush_tlb_others, cpumask, info);
+	PVOP_VCALL2(mmu.flush_tlb_multi, cpumask, info);
 }
 
 static inline void paravirt_tlb_remove_table(struct mmu_gather *tlb, void *table)
diff --git a/arch/x86/include/asm/paravirt_types.h b/arch/x86/include/asm/paravirt_types.h
index b6b02b7c19cc..541fe7193526 100644
--- a/arch/x86/include/asm/paravirt_types.h
+++ b/arch/x86/include/asm/paravirt_types.h
@@ -201,8 +201,8 @@ struct pv_mmu_ops {
 	void (*flush_tlb_user)(void);
 	void (*flush_tlb_kernel)(void);
 	void (*flush_tlb_one_user)(unsigned long addr);
-	void (*flush_tlb_others)(const struct cpumask *cpus,
-				 const struct flush_tlb_info *info);
+	void (*flush_tlb_multi)(const struct cpumask *cpus,
+				const struct flush_tlb_info *info);
 
 	void (*tlb_remove_table)(struct mmu_gather *tlb, void *table);
 
diff --git a/arch/x86/include/asm/tlbflush.h b/arch/x86/include/asm/tlbflush.h
index a7a598af116d..3c6681def912 100644
--- a/arch/x86/include/asm/tlbflush.h
+++ b/arch/x86/include/asm/tlbflush.h
@@ -175,7 +175,7 @@ extern void initialize_tlbstate_and_flush(void);
  *  - flush_tlb_page(vma, vmaddr) flushes one page
  *  - flush_tlb_range(vma, start, end) flushes a range of pages
  *  - flush_tlb_kernel_range(start, end) flushes a range of kernel pages
- *  - flush_tlb_others(cpumask, info) flushes TLBs on other cpus
+ *  - flush_tlb_multi(cpumask, info) flushes TLBs on multiple cpus
  *
  * ..but the i386 has somewhat limited tlb flushing capabilities,
  * and page-granular flushes are available only on i486 and up.
@@ -209,7 +209,7 @@ struct flush_tlb_info {
 void flush_tlb_local(void);
 void flush_tlb_one_user(unsigned long addr);
 void flush_tlb_one_kernel(unsigned long addr);
-void flush_tlb_others(const struct cpumask *cpumask,
+void flush_tlb_multi(const struct cpumask *cpumask,
 		      const struct flush_tlb_info *info);
 
 #ifdef CONFIG_PARAVIRT
diff --git a/arch/x86/include/asm/trace/hyperv.h b/arch/x86/include/asm/trace/hyperv.h
index 4d705cb4d63b..a8e5a7a2b460 100644
--- a/arch/x86/include/asm/trace/hyperv.h
+++ b/arch/x86/include/asm/trace/hyperv.h
@@ -8,7 +8,7 @@
 
 #if IS_ENABLED(CONFIG_HYPERV)
 
-TRACE_EVENT(hyperv_mmu_flush_tlb_others,
+TRACE_EVENT(hyperv_mmu_flush_tlb_multi,
 	    TP_PROTO(const struct cpumask *cpus,
 		     const struct flush_tlb_info *info),
 	    TP_ARGS(cpus, info),
diff --git a/arch/x86/kernel/kvm.c b/arch/x86/kernel/kvm.c
index 5e78e01ca3b4..38ea9dee2456 100644
--- a/arch/x86/kernel/kvm.c
+++ b/arch/x86/kernel/kvm.c
@@ -613,7 +613,7 @@ static int kvm_cpu_down_prepare(unsigned int cpu)
 }
 #endif
 
-static void kvm_flush_tlb_others(const struct cpumask *cpumask,
+static void kvm_flush_tlb_multi(const struct cpumask *cpumask,
 			const struct flush_tlb_info *info)
 {
 	u8 state;
@@ -627,6 +627,11 @@ static void kvm_flush_tlb_others(const struct cpumask *cpumask,
 	 * queue flush_on_enter for pre-empted vCPUs
 	 */
 	for_each_cpu(cpu, flushmask) {
+		/*
+		 * The local vCPU is never preempted, so we do not explicitly
+		 * skip check for local vCPU - it will never be cleared from
+		 * flushmask.
+		 */
 		src = &per_cpu(steal_time, cpu);
 		state = READ_ONCE(src->preempted);
 		if ((state & KVM_VCPU_PREEMPTED)) {
@@ -636,7 +641,7 @@ static void kvm_flush_tlb_others(const struct cpumask *cpumask,
 		}
 	}
 
-	native_flush_tlb_others(flushmask, info);
+	native_flush_tlb_multi(flushmask, info);
 }
 
 static void __init kvm_guest_init(void)
@@ -654,7 +659,7 @@ static void __init kvm_guest_init(void)
 	}
 
 	if (pv_tlb_flush_supported()) {
-		pv_ops.mmu.flush_tlb_others = kvm_flush_tlb_others;
+		pv_ops.mmu.flush_tlb_multi = kvm_flush_tlb_multi;
 		pv_ops.mmu.tlb_remove_table = tlb_remove_table;
 		pr_info("KVM setup pv remote TLB flush\n");
 	}
diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index 07b6701a540a..78fcbd58716e 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -24,7 +24,7 @@
 # define __flush_tlb_local		native_flush_tlb_local
 # define __flush_tlb_global		native_flush_tlb_global
 # define __flush_tlb_one_user(addr)	native_flush_tlb_one_user(addr)
-# define __flush_tlb_others(msk, info)	native_flush_tlb_others(msk, info)
+# define __flush_tlb_multi(msk, info)	native_flush_tlb_multi(msk, info)
 #endif
 
 /*
@@ -490,7 +490,7 @@ void switch_mm_irqs_off(struct mm_struct *prev, struct mm_struct *next,
 		/*
 		 * Even in lazy TLB mode, the CPU should stay set in the
 		 * mm_cpumask. The TLB shootdown code can figure out from
-		 * from cpu_tlbstate.is_lazy whether or not to send an IPI.
+		 * cpu_tlbstate.is_lazy whether or not to send an IPI.
 		 */
 		if (WARN_ON_ONCE(real_prev != &init_mm &&
 				 !cpumask_test_cpu(cpu, mm_cpumask(next))))
@@ -697,7 +697,7 @@ static void flush_tlb_func(void *info)
 		 * garbage into our TLB.  Since switching to init_mm is barely
 		 * slower than a minimal flush, just switch to init_mm.
 		 *
-		 * This should be rare, with native_flush_tlb_others skipping
+		 * This should be rare, with native_flush_tlb_multi() skipping
 		 * IPIs to lazy TLB mode CPUs.
 		 */
 		switch_mm_irqs_off(NULL, &init_mm, NULL);
@@ -795,9 +795,14 @@ static bool tlb_is_not_lazy(int cpu)
 
 static DEFINE_PER_CPU(cpumask_t, flush_tlb_mask);
 
-STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
+STATIC_NOPV void native_flush_tlb_multi(const struct cpumask *cpumask,
 					 const struct flush_tlb_info *info)
 {
+	/*
+	 * Do accounting and tracing. Note that there are (and have always been)
+	 * cases in which a remote TLB flush will be traced, but eventually
+	 * would not happen.
+	 */
 	count_vm_tlb_event(NR_TLB_REMOTE_FLUSH);
 	if (info->end == TLB_FLUSH_ALL)
 		trace_tlb_flush(TLB_REMOTE_SEND_IPI, TLB_FLUSH_ALL);
@@ -816,8 +821,8 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
 	 * doing a speculative memory access.
 	 */
 	if (info->freed_tables) {
-		smp_call_function_many(cpumask, flush_tlb_func,
-			       (void *)info, 1);
+		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
+				      cpumask);
 	} else {
 		/*
 		 * Although we could have used on_each_cpu_cond_mask(),
@@ -844,14 +849,15 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
 			if (tlb_is_not_lazy(cpu))
 				__cpumask_set_cpu(cpu, cond_cpumask);
 		}
-		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);
+		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
+				      cpumask);
 	}
 }
 
-void flush_tlb_others(const struct cpumask *cpumask,
+void flush_tlb_multi(const struct cpumask *cpumask,
 		      const struct flush_tlb_info *info)
 {
-	__flush_tlb_others(cpumask, info);
+	__flush_tlb_multi(cpumask, info);
 }
 
 /*
@@ -931,16 +937,20 @@ void flush_tlb_mm_range(struct mm_struct *mm, unsigned long start,
 	info = get_flush_tlb_info(mm, start, end, stride_shift, freed_tables,
 				  new_tlb_gen);
 
-	if (mm == this_cpu_read(cpu_tlbstate.loaded_mm)) {
+	/*
+	 * flush_tlb_multi() is not optimized for the common case in which only
+	 * a local TLB flush is needed. Optimize this use-case by calling
+	 * flush_tlb_func_local() directly in this case.
+	 */
+	if (cpumask_any_but(mm_cpumask(mm), cpu) < nr_cpu_ids) {
+		flush_tlb_multi(mm_cpumask(mm), info);
+	} else if (mm == this_cpu_read(cpu_tlbstate.loaded_mm)) {
 		lockdep_assert_irqs_enabled();
 		local_irq_disable();
 		flush_tlb_func(info);
 		local_irq_enable();
 	}
 
-	if (cpumask_any_but(mm_cpumask(mm), cpu) < nr_cpu_ids)
-		flush_tlb_others(mm_cpumask(mm), info);
-
 	put_flush_tlb_info();
 	put_cpu();
 }
@@ -1151,17 +1161,20 @@ void arch_tlbbatch_flush(struct arch_tlbflush_unmap_batch *batch)
 
 	int cpu = get_cpu();
 
-	info = get_flush_tlb_info(NULL, 0, TLB_FLUSH_ALL, 0, false, 0);
-	if (cpumask_test_cpu(cpu, &batch->cpumask)) {
+	/*
+	 * flush_tlb_multi() is not optimized for the common case in which only
+	 * a local TLB flush is needed. Optimize this use-case by calling
+	 * flush_tlb_func_local() directly in this case.
+	 */
+	if (cpumask_any_but(&batch->cpumask, cpu) < nr_cpu_ids) {
+		flush_tlb_multi(&batch->cpumask, info);
+	} else if (cpumask_test_cpu(cpu, &batch->cpumask)) {
 		lockdep_assert_irqs_enabled();
 		local_irq_disable();
 		flush_tlb_func(info);
 		local_irq_enable();
 	}
 
-	if (cpumask_any_but(&batch->cpumask, cpu) < nr_cpu_ids)
-		flush_tlb_others(&batch->cpumask, info);
-
 	cpumask_clear(&batch->cpumask);
 
 	put_flush_tlb_info();
diff --git a/arch/x86/xen/mmu_pv.c b/arch/x86/xen/mmu_pv.c
index cf2ade864c30..09b95c0e876e 100644
--- a/arch/x86/xen/mmu_pv.c
+++ b/arch/x86/xen/mmu_pv.c
@@ -1247,8 +1247,8 @@ static void xen_flush_tlb_one_user(unsigned long addr)
 	preempt_enable();
 }
 
-static void xen_flush_tlb_others(const struct cpumask *cpus,
-				 const struct flush_tlb_info *info)
+static void xen_flush_tlb_multi(const struct cpumask *cpus,
+				const struct flush_tlb_info *info)
 {
 	struct {
 		struct mmuext_op op;
@@ -1258,7 +1258,7 @@ static void xen_flush_tlb_others(const struct cpumask *cpus,
 	const size_t mc_entry_size = sizeof(args->op) +
 		sizeof(args->mask[0]) * BITS_TO_LONGS(num_possible_cpus());
 
-	trace_xen_mmu_flush_tlb_others(cpus, info->mm, info->start, info->end);
+	trace_xen_mmu_flush_tlb_multi(cpus, info->mm, info->start, info->end);
 
 	if (cpumask_empty(cpus))
 		return;		/* nothing to do */
@@ -1267,9 +1267,8 @@ static void xen_flush_tlb_others(const struct cpumask *cpus,
 	args = mcs.args;
 	args->op.arg2.vcpumask = to_cpumask(args->mask);
 
-	/* Remove us, and any offline CPUS. */
+	/* Remove any offline CPUs */
 	cpumask_and(to_cpumask(args->mask), cpus, cpu_online_mask);
-	cpumask_clear_cpu(smp_processor_id(), to_cpumask(args->mask));
 
 	args->op.cmd = MMUEXT_TLB_FLUSH_MULTI;
 	if (info->end != TLB_FLUSH_ALL &&
@@ -2086,7 +2085,7 @@ static const struct pv_mmu_ops xen_mmu_ops __initconst = {
 	.flush_tlb_user = xen_flush_tlb,
 	.flush_tlb_kernel = xen_flush_tlb,
 	.flush_tlb_one_user = xen_flush_tlb_one_user,
-	.flush_tlb_others = xen_flush_tlb_others,
+	.flush_tlb_multi = xen_flush_tlb_multi,
 	.tlb_remove_table = tlb_remove_table,
 
 	.pgd_alloc = xen_pgd_alloc,
diff --git a/include/trace/events/xen.h b/include/trace/events/xen.h
index 3b61b587e137..44a3f565264d 100644
--- a/include/trace/events/xen.h
+++ b/include/trace/events/xen.h
@@ -346,7 +346,7 @@ TRACE_EVENT(xen_mmu_flush_tlb_one_user,
 	    TP_printk("addr %lx", __entry->addr)
 	);
 
-TRACE_EVENT(xen_mmu_flush_tlb_others,
+TRACE_EVENT(xen_mmu_flush_tlb_multi,
 	    TP_PROTO(const struct cpumask *cpus, struct mm_struct *mm,
 		     unsigned long addr, unsigned long end),
 	    TP_ARGS(cpus, mm, addr, end),
-- 
2.25.1


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

* [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently
@ 2021-02-09 22:16   ` Nadav Amit
  0 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Sasha Levin, Juergen Gross, linux-hyperv, x86, Stephen Hemminger,
	xen-devel, kvm, Peter Zijlstra, Haiyang Zhang, Dave Hansen,
	linux-kernel, virtualization, Ingo Molnar, Nadav Amit,
	Andy Lutomirski, Paolo Bonzini, Borislav Petkov, Boris Ostrovsky,
	Michael Kelley

From: Nadav Amit <namit@vmware.com>

To improve TLB shootdown performance, flush the remote and local TLBs
concurrently. Introduce flush_tlb_multi() that does so. Introduce
paravirtual versions of flush_tlb_multi() for KVM, Xen and hyper-v (Xen
and hyper-v are only compile-tested).

While the updated smp infrastructure is capable of running a function on
a single local core, it is not optimized for this case. The multiple
function calls and the indirect branch introduce some overhead, and
might make local TLB flushes slower than they were before the recent
changes.

Before calling the SMP infrastructure, check if only a local TLB flush
is needed to restore the lost performance in this common case. This
requires to check mm_cpumask() one more time, but unless this mask is
updated very frequently, this should impact performance negatively.

Cc: "K. Y. Srinivasan" <kys@microsoft.com>
Cc: Haiyang Zhang <haiyangz@microsoft.com>
Cc: Stephen Hemminger <sthemmin@microsoft.com>
Cc: Sasha Levin <sashal@kernel.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Borislav Petkov <bp@alien8.de>
Cc: x86@kernel.org
Cc: Juergen Gross <jgross@suse.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Andy Lutomirski <luto@kernel.org>
Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Boris Ostrovsky <boris.ostrovsky@oracle.com>
Cc: linux-hyperv@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
Cc: virtualization@lists.linux-foundation.org
Cc: kvm@vger.kernel.org
Cc: xen-devel@lists.xenproject.org
Reviewed-by: Michael Kelley <mikelley@microsoft.com> # Hyper-v parts
Reviewed-by: Juergen Gross <jgross@suse.com> # Xen and paravirt parts
Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 arch/x86/hyperv/mmu.c                 | 10 +++---
 arch/x86/include/asm/paravirt.h       |  6 ++--
 arch/x86/include/asm/paravirt_types.h |  4 +--
 arch/x86/include/asm/tlbflush.h       |  4 +--
 arch/x86/include/asm/trace/hyperv.h   |  2 +-
 arch/x86/kernel/kvm.c                 | 11 ++++--
 arch/x86/mm/tlb.c                     | 49 +++++++++++++++++----------
 arch/x86/xen/mmu_pv.c                 | 11 +++---
 include/trace/events/xen.h            |  2 +-
 9 files changed, 58 insertions(+), 41 deletions(-)

diff --git a/arch/x86/hyperv/mmu.c b/arch/x86/hyperv/mmu.c
index 2c87350c1fb0..681dba8de4f2 100644
--- a/arch/x86/hyperv/mmu.c
+++ b/arch/x86/hyperv/mmu.c
@@ -52,8 +52,8 @@ static inline int fill_gva_list(u64 gva_list[], int offset,
 	return gva_n - offset;
 }
 
-static void hyperv_flush_tlb_others(const struct cpumask *cpus,
-				    const struct flush_tlb_info *info)
+static void hyperv_flush_tlb_multi(const struct cpumask *cpus,
+				   const struct flush_tlb_info *info)
 {
 	int cpu, vcpu, gva_n, max_gvas;
 	struct hv_tlb_flush **flush_pcpu;
@@ -61,7 +61,7 @@ static void hyperv_flush_tlb_others(const struct cpumask *cpus,
 	u64 status = U64_MAX;
 	unsigned long flags;
 
-	trace_hyperv_mmu_flush_tlb_others(cpus, info);
+	trace_hyperv_mmu_flush_tlb_multi(cpus, info);
 
 	if (!hv_hypercall_pg)
 		goto do_native;
@@ -164,7 +164,7 @@ static void hyperv_flush_tlb_others(const struct cpumask *cpus,
 	if (!(status & HV_HYPERCALL_RESULT_MASK))
 		return;
 do_native:
-	native_flush_tlb_others(cpus, info);
+	native_flush_tlb_multi(cpus, info);
 }
 
 static u64 hyperv_flush_tlb_others_ex(const struct cpumask *cpus,
@@ -239,6 +239,6 @@ void hyperv_setup_mmu_ops(void)
 		return;
 
 	pr_info("Using hypercall for remote TLB flush\n");
-	pv_ops.mmu.flush_tlb_others = hyperv_flush_tlb_others;
+	pv_ops.mmu.flush_tlb_multi = hyperv_flush_tlb_multi;
 	pv_ops.mmu.tlb_remove_table = tlb_remove_table;
 }
diff --git a/arch/x86/include/asm/paravirt.h b/arch/x86/include/asm/paravirt.h
index f8dce11d2bc1..515e49204c8b 100644
--- a/arch/x86/include/asm/paravirt.h
+++ b/arch/x86/include/asm/paravirt.h
@@ -50,7 +50,7 @@ static inline void slow_down_io(void)
 void native_flush_tlb_local(void);
 void native_flush_tlb_global(void);
 void native_flush_tlb_one_user(unsigned long addr);
-void native_flush_tlb_others(const struct cpumask *cpumask,
+void native_flush_tlb_multi(const struct cpumask *cpumask,
 			     const struct flush_tlb_info *info);
 
 static inline void __flush_tlb_local(void)
@@ -68,10 +68,10 @@ static inline void __flush_tlb_one_user(unsigned long addr)
 	PVOP_VCALL1(mmu.flush_tlb_one_user, addr);
 }
 
-static inline void __flush_tlb_others(const struct cpumask *cpumask,
+static inline void __flush_tlb_multi(const struct cpumask *cpumask,
 				      const struct flush_tlb_info *info)
 {
-	PVOP_VCALL2(mmu.flush_tlb_others, cpumask, info);
+	PVOP_VCALL2(mmu.flush_tlb_multi, cpumask, info);
 }
 
 static inline void paravirt_tlb_remove_table(struct mmu_gather *tlb, void *table)
diff --git a/arch/x86/include/asm/paravirt_types.h b/arch/x86/include/asm/paravirt_types.h
index b6b02b7c19cc..541fe7193526 100644
--- a/arch/x86/include/asm/paravirt_types.h
+++ b/arch/x86/include/asm/paravirt_types.h
@@ -201,8 +201,8 @@ struct pv_mmu_ops {
 	void (*flush_tlb_user)(void);
 	void (*flush_tlb_kernel)(void);
 	void (*flush_tlb_one_user)(unsigned long addr);
-	void (*flush_tlb_others)(const struct cpumask *cpus,
-				 const struct flush_tlb_info *info);
+	void (*flush_tlb_multi)(const struct cpumask *cpus,
+				const struct flush_tlb_info *info);
 
 	void (*tlb_remove_table)(struct mmu_gather *tlb, void *table);
 
diff --git a/arch/x86/include/asm/tlbflush.h b/arch/x86/include/asm/tlbflush.h
index a7a598af116d..3c6681def912 100644
--- a/arch/x86/include/asm/tlbflush.h
+++ b/arch/x86/include/asm/tlbflush.h
@@ -175,7 +175,7 @@ extern void initialize_tlbstate_and_flush(void);
  *  - flush_tlb_page(vma, vmaddr) flushes one page
  *  - flush_tlb_range(vma, start, end) flushes a range of pages
  *  - flush_tlb_kernel_range(start, end) flushes a range of kernel pages
- *  - flush_tlb_others(cpumask, info) flushes TLBs on other cpus
+ *  - flush_tlb_multi(cpumask, info) flushes TLBs on multiple cpus
  *
  * ..but the i386 has somewhat limited tlb flushing capabilities,
  * and page-granular flushes are available only on i486 and up.
@@ -209,7 +209,7 @@ struct flush_tlb_info {
 void flush_tlb_local(void);
 void flush_tlb_one_user(unsigned long addr);
 void flush_tlb_one_kernel(unsigned long addr);
-void flush_tlb_others(const struct cpumask *cpumask,
+void flush_tlb_multi(const struct cpumask *cpumask,
 		      const struct flush_tlb_info *info);
 
 #ifdef CONFIG_PARAVIRT
diff --git a/arch/x86/include/asm/trace/hyperv.h b/arch/x86/include/asm/trace/hyperv.h
index 4d705cb4d63b..a8e5a7a2b460 100644
--- a/arch/x86/include/asm/trace/hyperv.h
+++ b/arch/x86/include/asm/trace/hyperv.h
@@ -8,7 +8,7 @@
 
 #if IS_ENABLED(CONFIG_HYPERV)
 
-TRACE_EVENT(hyperv_mmu_flush_tlb_others,
+TRACE_EVENT(hyperv_mmu_flush_tlb_multi,
 	    TP_PROTO(const struct cpumask *cpus,
 		     const struct flush_tlb_info *info),
 	    TP_ARGS(cpus, info),
diff --git a/arch/x86/kernel/kvm.c b/arch/x86/kernel/kvm.c
index 5e78e01ca3b4..38ea9dee2456 100644
--- a/arch/x86/kernel/kvm.c
+++ b/arch/x86/kernel/kvm.c
@@ -613,7 +613,7 @@ static int kvm_cpu_down_prepare(unsigned int cpu)
 }
 #endif
 
-static void kvm_flush_tlb_others(const struct cpumask *cpumask,
+static void kvm_flush_tlb_multi(const struct cpumask *cpumask,
 			const struct flush_tlb_info *info)
 {
 	u8 state;
@@ -627,6 +627,11 @@ static void kvm_flush_tlb_others(const struct cpumask *cpumask,
 	 * queue flush_on_enter for pre-empted vCPUs
 	 */
 	for_each_cpu(cpu, flushmask) {
+		/*
+		 * The local vCPU is never preempted, so we do not explicitly
+		 * skip check for local vCPU - it will never be cleared from
+		 * flushmask.
+		 */
 		src = &per_cpu(steal_time, cpu);
 		state = READ_ONCE(src->preempted);
 		if ((state & KVM_VCPU_PREEMPTED)) {
@@ -636,7 +641,7 @@ static void kvm_flush_tlb_others(const struct cpumask *cpumask,
 		}
 	}
 
-	native_flush_tlb_others(flushmask, info);
+	native_flush_tlb_multi(flushmask, info);
 }
 
 static void __init kvm_guest_init(void)
@@ -654,7 +659,7 @@ static void __init kvm_guest_init(void)
 	}
 
 	if (pv_tlb_flush_supported()) {
-		pv_ops.mmu.flush_tlb_others = kvm_flush_tlb_others;
+		pv_ops.mmu.flush_tlb_multi = kvm_flush_tlb_multi;
 		pv_ops.mmu.tlb_remove_table = tlb_remove_table;
 		pr_info("KVM setup pv remote TLB flush\n");
 	}
diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index 07b6701a540a..78fcbd58716e 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -24,7 +24,7 @@
 # define __flush_tlb_local		native_flush_tlb_local
 # define __flush_tlb_global		native_flush_tlb_global
 # define __flush_tlb_one_user(addr)	native_flush_tlb_one_user(addr)
-# define __flush_tlb_others(msk, info)	native_flush_tlb_others(msk, info)
+# define __flush_tlb_multi(msk, info)	native_flush_tlb_multi(msk, info)
 #endif
 
 /*
@@ -490,7 +490,7 @@ void switch_mm_irqs_off(struct mm_struct *prev, struct mm_struct *next,
 		/*
 		 * Even in lazy TLB mode, the CPU should stay set in the
 		 * mm_cpumask. The TLB shootdown code can figure out from
-		 * from cpu_tlbstate.is_lazy whether or not to send an IPI.
+		 * cpu_tlbstate.is_lazy whether or not to send an IPI.
 		 */
 		if (WARN_ON_ONCE(real_prev != &init_mm &&
 				 !cpumask_test_cpu(cpu, mm_cpumask(next))))
@@ -697,7 +697,7 @@ static void flush_tlb_func(void *info)
 		 * garbage into our TLB.  Since switching to init_mm is barely
 		 * slower than a minimal flush, just switch to init_mm.
 		 *
-		 * This should be rare, with native_flush_tlb_others skipping
+		 * This should be rare, with native_flush_tlb_multi() skipping
 		 * IPIs to lazy TLB mode CPUs.
 		 */
 		switch_mm_irqs_off(NULL, &init_mm, NULL);
@@ -795,9 +795,14 @@ static bool tlb_is_not_lazy(int cpu)
 
 static DEFINE_PER_CPU(cpumask_t, flush_tlb_mask);
 
-STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
+STATIC_NOPV void native_flush_tlb_multi(const struct cpumask *cpumask,
 					 const struct flush_tlb_info *info)
 {
+	/*
+	 * Do accounting and tracing. Note that there are (and have always been)
+	 * cases in which a remote TLB flush will be traced, but eventually
+	 * would not happen.
+	 */
 	count_vm_tlb_event(NR_TLB_REMOTE_FLUSH);
 	if (info->end == TLB_FLUSH_ALL)
 		trace_tlb_flush(TLB_REMOTE_SEND_IPI, TLB_FLUSH_ALL);
@@ -816,8 +821,8 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
 	 * doing a speculative memory access.
 	 */
 	if (info->freed_tables) {
-		smp_call_function_many(cpumask, flush_tlb_func,
-			       (void *)info, 1);
+		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
+				      cpumask);
 	} else {
 		/*
 		 * Although we could have used on_each_cpu_cond_mask(),
@@ -844,14 +849,15 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
 			if (tlb_is_not_lazy(cpu))
 				__cpumask_set_cpu(cpu, cond_cpumask);
 		}
-		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);
+		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
+				      cpumask);
 	}
 }
 
-void flush_tlb_others(const struct cpumask *cpumask,
+void flush_tlb_multi(const struct cpumask *cpumask,
 		      const struct flush_tlb_info *info)
 {
-	__flush_tlb_others(cpumask, info);
+	__flush_tlb_multi(cpumask, info);
 }
 
 /*
@@ -931,16 +937,20 @@ void flush_tlb_mm_range(struct mm_struct *mm, unsigned long start,
 	info = get_flush_tlb_info(mm, start, end, stride_shift, freed_tables,
 				  new_tlb_gen);
 
-	if (mm == this_cpu_read(cpu_tlbstate.loaded_mm)) {
+	/*
+	 * flush_tlb_multi() is not optimized for the common case in which only
+	 * a local TLB flush is needed. Optimize this use-case by calling
+	 * flush_tlb_func_local() directly in this case.
+	 */
+	if (cpumask_any_but(mm_cpumask(mm), cpu) < nr_cpu_ids) {
+		flush_tlb_multi(mm_cpumask(mm), info);
+	} else if (mm == this_cpu_read(cpu_tlbstate.loaded_mm)) {
 		lockdep_assert_irqs_enabled();
 		local_irq_disable();
 		flush_tlb_func(info);
 		local_irq_enable();
 	}
 
-	if (cpumask_any_but(mm_cpumask(mm), cpu) < nr_cpu_ids)
-		flush_tlb_others(mm_cpumask(mm), info);
-
 	put_flush_tlb_info();
 	put_cpu();
 }
@@ -1151,17 +1161,20 @@ void arch_tlbbatch_flush(struct arch_tlbflush_unmap_batch *batch)
 
 	int cpu = get_cpu();
 
-	info = get_flush_tlb_info(NULL, 0, TLB_FLUSH_ALL, 0, false, 0);
-	if (cpumask_test_cpu(cpu, &batch->cpumask)) {
+	/*
+	 * flush_tlb_multi() is not optimized for the common case in which only
+	 * a local TLB flush is needed. Optimize this use-case by calling
+	 * flush_tlb_func_local() directly in this case.
+	 */
+	if (cpumask_any_but(&batch->cpumask, cpu) < nr_cpu_ids) {
+		flush_tlb_multi(&batch->cpumask, info);
+	} else if (cpumask_test_cpu(cpu, &batch->cpumask)) {
 		lockdep_assert_irqs_enabled();
 		local_irq_disable();
 		flush_tlb_func(info);
 		local_irq_enable();
 	}
 
-	if (cpumask_any_but(&batch->cpumask, cpu) < nr_cpu_ids)
-		flush_tlb_others(&batch->cpumask, info);
-
 	cpumask_clear(&batch->cpumask);
 
 	put_flush_tlb_info();
diff --git a/arch/x86/xen/mmu_pv.c b/arch/x86/xen/mmu_pv.c
index cf2ade864c30..09b95c0e876e 100644
--- a/arch/x86/xen/mmu_pv.c
+++ b/arch/x86/xen/mmu_pv.c
@@ -1247,8 +1247,8 @@ static void xen_flush_tlb_one_user(unsigned long addr)
 	preempt_enable();
 }
 
-static void xen_flush_tlb_others(const struct cpumask *cpus,
-				 const struct flush_tlb_info *info)
+static void xen_flush_tlb_multi(const struct cpumask *cpus,
+				const struct flush_tlb_info *info)
 {
 	struct {
 		struct mmuext_op op;
@@ -1258,7 +1258,7 @@ static void xen_flush_tlb_others(const struct cpumask *cpus,
 	const size_t mc_entry_size = sizeof(args->op) +
 		sizeof(args->mask[0]) * BITS_TO_LONGS(num_possible_cpus());
 
-	trace_xen_mmu_flush_tlb_others(cpus, info->mm, info->start, info->end);
+	trace_xen_mmu_flush_tlb_multi(cpus, info->mm, info->start, info->end);
 
 	if (cpumask_empty(cpus))
 		return;		/* nothing to do */
@@ -1267,9 +1267,8 @@ static void xen_flush_tlb_others(const struct cpumask *cpus,
 	args = mcs.args;
 	args->op.arg2.vcpumask = to_cpumask(args->mask);
 
-	/* Remove us, and any offline CPUS. */
+	/* Remove any offline CPUs */
 	cpumask_and(to_cpumask(args->mask), cpus, cpu_online_mask);
-	cpumask_clear_cpu(smp_processor_id(), to_cpumask(args->mask));
 
 	args->op.cmd = MMUEXT_TLB_FLUSH_MULTI;
 	if (info->end != TLB_FLUSH_ALL &&
@@ -2086,7 +2085,7 @@ static const struct pv_mmu_ops xen_mmu_ops __initconst = {
 	.flush_tlb_user = xen_flush_tlb,
 	.flush_tlb_kernel = xen_flush_tlb,
 	.flush_tlb_one_user = xen_flush_tlb_one_user,
-	.flush_tlb_others = xen_flush_tlb_others,
+	.flush_tlb_multi = xen_flush_tlb_multi,
 	.tlb_remove_table = tlb_remove_table,
 
 	.pgd_alloc = xen_pgd_alloc,
diff --git a/include/trace/events/xen.h b/include/trace/events/xen.h
index 3b61b587e137..44a3f565264d 100644
--- a/include/trace/events/xen.h
+++ b/include/trace/events/xen.h
@@ -346,7 +346,7 @@ TRACE_EVENT(xen_mmu_flush_tlb_one_user,
 	    TP_printk("addr %lx", __entry->addr)
 	);
 
-TRACE_EVENT(xen_mmu_flush_tlb_others,
+TRACE_EVENT(xen_mmu_flush_tlb_multi,
 	    TP_PROTO(const struct cpumask *cpus, struct mm_struct *mm,
 		     unsigned long addr, unsigned long end),
 	    TP_ARGS(cpus, mm, addr, end),
-- 
2.25.1

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

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

* [PATCH v5 5/8] x86/mm/tlb: Privatize cpu_tlbstate
  2021-02-09 22:16 ` Nadav Amit
                   ` (4 preceding siblings ...)
  (?)
@ 2021-02-09 22:16 ` Nadav Amit
  -1 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen, Nadav Amit

From: Nadav Amit <namit@vmware.com>

cpu_tlbstate is mostly private and only the variable is_lazy is shared.
This causes some false-sharing when TLB flushes are performed.

Break cpu_tlbstate intro cpu_tlbstate and cpu_tlbstate_shared, and mark
each one accordingly.

Cc: Andy Lutomirski <luto@kernel.org>
Cc: Peter Zijlstra <peterz@infradead.org>
Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 arch/x86/include/asm/tlbflush.h | 39 ++++++++++++++++++---------------
 arch/x86/kernel/alternative.c   |  2 +-
 arch/x86/mm/init.c              |  2 +-
 arch/x86/mm/tlb.c               | 17 ++++++++------
 4 files changed, 33 insertions(+), 27 deletions(-)

diff --git a/arch/x86/include/asm/tlbflush.h b/arch/x86/include/asm/tlbflush.h
index 3c6681def912..fa952eadbc2e 100644
--- a/arch/x86/include/asm/tlbflush.h
+++ b/arch/x86/include/asm/tlbflush.h
@@ -89,23 +89,6 @@ struct tlb_state {
 	u16 loaded_mm_asid;
 	u16 next_asid;
 
-	/*
-	 * We can be in one of several states:
-	 *
-	 *  - Actively using an mm.  Our CPU's bit will be set in
-	 *    mm_cpumask(loaded_mm) and is_lazy == false;
-	 *
-	 *  - Not using a real mm.  loaded_mm == &init_mm.  Our CPU's bit
-	 *    will not be set in mm_cpumask(&init_mm) and is_lazy == false.
-	 *
-	 *  - Lazily using a real mm.  loaded_mm != &init_mm, our bit
-	 *    is set in mm_cpumask(loaded_mm), but is_lazy == true.
-	 *    We're heuristically guessing that the CR3 load we
-	 *    skipped more than makes up for the overhead added by
-	 *    lazy mode.
-	 */
-	bool is_lazy;
-
 	/*
 	 * If set we changed the page tables in such a way that we
 	 * needed an invalidation of all contexts (aka. PCIDs / ASIDs).
@@ -151,7 +134,27 @@ struct tlb_state {
 	 */
 	struct tlb_context ctxs[TLB_NR_DYN_ASIDS];
 };
-DECLARE_PER_CPU_SHARED_ALIGNED(struct tlb_state, cpu_tlbstate);
+DECLARE_PER_CPU_ALIGNED(struct tlb_state, cpu_tlbstate);
+
+struct tlb_state_shared {
+	/*
+	 * We can be in one of several states:
+	 *
+	 *  - Actively using an mm.  Our CPU's bit will be set in
+	 *    mm_cpumask(loaded_mm) and is_lazy == false;
+	 *
+	 *  - Not using a real mm.  loaded_mm == &init_mm.  Our CPU's bit
+	 *    will not be set in mm_cpumask(&init_mm) and is_lazy == false.
+	 *
+	 *  - Lazily using a real mm.  loaded_mm != &init_mm, our bit
+	 *    is set in mm_cpumask(loaded_mm), but is_lazy == true.
+	 *    We're heuristically guessing that the CR3 load we
+	 *    skipped more than makes up for the overhead added by
+	 *    lazy mode.
+	 */
+	bool is_lazy;
+};
+DECLARE_PER_CPU_SHARED_ALIGNED(struct tlb_state_shared, cpu_tlbstate_shared);
 
 bool nmi_uaccess_okay(void);
 #define nmi_uaccess_okay nmi_uaccess_okay
diff --git a/arch/x86/kernel/alternative.c b/arch/x86/kernel/alternative.c
index 8d778e46725d..94649f86d653 100644
--- a/arch/x86/kernel/alternative.c
+++ b/arch/x86/kernel/alternative.c
@@ -813,7 +813,7 @@ static inline temp_mm_state_t use_temporary_mm(struct mm_struct *mm)
 	 * with a stale address space WITHOUT being in lazy mode after
 	 * restoring the previous mm.
 	 */
-	if (this_cpu_read(cpu_tlbstate.is_lazy))
+	if (this_cpu_read(cpu_tlbstate_shared.is_lazy))
 		leave_mm(smp_processor_id());
 
 	temp_state.mm = this_cpu_read(cpu_tlbstate.loaded_mm);
diff --git a/arch/x86/mm/init.c b/arch/x86/mm/init.c
index e26f5c5c6565..5afa8bdd2021 100644
--- a/arch/x86/mm/init.c
+++ b/arch/x86/mm/init.c
@@ -1008,7 +1008,7 @@ void __init zone_sizes_init(void)
 	free_area_init(max_zone_pfns);
 }
 
-__visible DEFINE_PER_CPU_SHARED_ALIGNED(struct tlb_state, cpu_tlbstate) = {
+__visible DEFINE_PER_CPU_ALIGNED(struct tlb_state, cpu_tlbstate) = {
 	.loaded_mm = &init_mm,
 	.next_asid = 1,
 	.cr4 = ~0UL,	/* fail hard if we screw up cr4 shadow initialization */
diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index 78fcbd58716e..e0271e0f84ea 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -300,7 +300,7 @@ void leave_mm(int cpu)
 		return;
 
 	/* Warn if we're not lazy. */
-	WARN_ON(!this_cpu_read(cpu_tlbstate.is_lazy));
+	WARN_ON(!this_cpu_read(cpu_tlbstate_shared.is_lazy));
 
 	switch_mm(NULL, &init_mm, NULL);
 }
@@ -424,7 +424,7 @@ void switch_mm_irqs_off(struct mm_struct *prev, struct mm_struct *next,
 {
 	struct mm_struct *real_prev = this_cpu_read(cpu_tlbstate.loaded_mm);
 	u16 prev_asid = this_cpu_read(cpu_tlbstate.loaded_mm_asid);
-	bool was_lazy = this_cpu_read(cpu_tlbstate.is_lazy);
+	bool was_lazy = this_cpu_read(cpu_tlbstate_shared.is_lazy);
 	unsigned cpu = smp_processor_id();
 	u64 next_tlb_gen;
 	bool need_flush;
@@ -469,7 +469,7 @@ void switch_mm_irqs_off(struct mm_struct *prev, struct mm_struct *next,
 		__flush_tlb_all();
 	}
 #endif
-	this_cpu_write(cpu_tlbstate.is_lazy, false);
+	this_cpu_write(cpu_tlbstate_shared.is_lazy, false);
 
 	/*
 	 * The membarrier system call requires a full memory barrier and
@@ -490,7 +490,7 @@ void switch_mm_irqs_off(struct mm_struct *prev, struct mm_struct *next,
 		/*
 		 * Even in lazy TLB mode, the CPU should stay set in the
 		 * mm_cpumask. The TLB shootdown code can figure out from
-		 * cpu_tlbstate.is_lazy whether or not to send an IPI.
+		 * cpu_tlbstate_shared.is_lazy whether or not to send an IPI.
 		 */
 		if (WARN_ON_ONCE(real_prev != &init_mm &&
 				 !cpumask_test_cpu(cpu, mm_cpumask(next))))
@@ -598,7 +598,7 @@ void enter_lazy_tlb(struct mm_struct *mm, struct task_struct *tsk)
 	if (this_cpu_read(cpu_tlbstate.loaded_mm) == &init_mm)
 		return;
 
-	this_cpu_write(cpu_tlbstate.is_lazy, true);
+	this_cpu_write(cpu_tlbstate_shared.is_lazy, true);
 }
 
 /*
@@ -690,7 +690,7 @@ static void flush_tlb_func(void *info)
 	VM_WARN_ON(this_cpu_read(cpu_tlbstate.ctxs[loaded_mm_asid].ctx_id) !=
 		   loaded_mm->context.ctx_id);
 
-	if (this_cpu_read(cpu_tlbstate.is_lazy)) {
+	if (this_cpu_read(cpu_tlbstate_shared.is_lazy)) {
 		/*
 		 * We're in lazy mode.  We need to at least flush our
 		 * paging-structure cache to avoid speculatively reading
@@ -790,11 +790,14 @@ static void flush_tlb_func(void *info)
 
 static bool tlb_is_not_lazy(int cpu)
 {
-	return !per_cpu(cpu_tlbstate.is_lazy, cpu);
+	return !per_cpu(cpu_tlbstate_shared.is_lazy, cpu);
 }
 
 static DEFINE_PER_CPU(cpumask_t, flush_tlb_mask);
 
+DEFINE_PER_CPU_ALIGNED(struct tlb_state_shared, cpu_tlbstate_shared);
+EXPORT_PER_CPU_SYMBOL(cpu_tlbstate_shared);
+
 STATIC_NOPV void native_flush_tlb_multi(const struct cpumask *cpumask,
 					 const struct flush_tlb_info *info)
 {
-- 
2.25.1


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

* [PATCH v5 6/8] x86/mm/tlb: Do not make is_lazy dirty for no reason
  2021-02-09 22:16 ` Nadav Amit
                   ` (5 preceding siblings ...)
  (?)
@ 2021-02-09 22:16 ` Nadav Amit
  -1 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen, Nadav Amit

From: Nadav Amit <namit@vmware.com>

Blindly writing to is_lazy for no reason, when the written value is
identical to the old value, makes the cacheline dirty for no reason.
Avoid making such writes to prevent cache coherency traffic for no
reason.

Cc: Andy Lutomirski <luto@kernel.org>
Cc: Peter Zijlstra <peterz@infradead.org>
Suggested-by: Dave Hansen <dave.hansen@linux.intel.com>
Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 arch/x86/mm/tlb.c | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index e0271e0f84ea..98d212518f67 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -469,7 +469,8 @@ void switch_mm_irqs_off(struct mm_struct *prev, struct mm_struct *next,
 		__flush_tlb_all();
 	}
 #endif
-	this_cpu_write(cpu_tlbstate_shared.is_lazy, false);
+	if (was_lazy)
+		this_cpu_write(cpu_tlbstate_shared.is_lazy, false);
 
 	/*
 	 * The membarrier system call requires a full memory barrier and
-- 
2.25.1


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

* [PATCH v5 7/8] cpumask: Mark functions as pure
  2021-02-09 22:16 ` Nadav Amit
                   ` (6 preceding siblings ...)
  (?)
@ 2021-02-09 22:16 ` Nadav Amit
  2021-02-16 12:14   ` Peter Zijlstra
  -1 siblings, 1 reply; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen, Nadav Amit

From: Nadav Amit <namit@vmware.com>

cpumask_next_and() and cpumask_any_but() are pure, and marking them as
such seems to generate different and presumably better code for
native_flush_tlb_multi().

Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 include/linux/cpumask.h | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/include/linux/cpumask.h b/include/linux/cpumask.h
index 383684e30f12..e86b7d027cfb 100644
--- a/include/linux/cpumask.h
+++ b/include/linux/cpumask.h
@@ -235,7 +235,7 @@ static inline unsigned int cpumask_last(const struct cpumask *srcp)
 	return find_last_bit(cpumask_bits(srcp), nr_cpumask_bits);
 }
 
-unsigned int cpumask_next(int n, const struct cpumask *srcp);
+unsigned int __pure cpumask_next(int n, const struct cpumask *srcp);
 
 /**
  * cpumask_next_zero - get the next unset cpu in a cpumask
@@ -252,8 +252,8 @@ static inline unsigned int cpumask_next_zero(int n, const struct cpumask *srcp)
 	return find_next_zero_bit(cpumask_bits(srcp), nr_cpumask_bits, n+1);
 }
 
-int cpumask_next_and(int n, const struct cpumask *, const struct cpumask *);
-int cpumask_any_but(const struct cpumask *mask, unsigned int cpu);
+__pure int cpumask_next_and(int n, const struct cpumask *, const struct cpumask *);
+__pure int cpumask_any_but(const struct cpumask *mask, unsigned int cpu);
 unsigned int cpumask_local_spread(unsigned int i, int node);
 int cpumask_any_and_distribute(const struct cpumask *src1p,
 			       const struct cpumask *src2p);
-- 
2.25.1


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

* [PATCH v5 8/8] x86/mm/tlb: Remove unnecessary uses of the inline keyword
  2021-02-09 22:16 ` Nadav Amit
                   ` (7 preceding siblings ...)
  (?)
@ 2021-02-09 22:16 ` Nadav Amit
  -1 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-09 22:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: linux-kernel, Andy Lutomirski, Peter Zijlstra, Dave Hansen, Nadav Amit

From: Nadav Amit <namit@vmware.com>

The compiler is smart enough without these hints.

Cc: Andy Lutomirski <luto@kernel.org>
Cc: Peter Zijlstra <peterz@infradead.org>
Suggested-by: Dave Hansen <dave.hansen@linux.intel.com>
Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
Signed-off-by: Nadav Amit <namit@vmware.com>
---
 arch/x86/mm/tlb.c | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/arch/x86/mm/tlb.c b/arch/x86/mm/tlb.c
index 98d212518f67..4cc28c624d1f 100644
--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -316,7 +316,7 @@ void switch_mm(struct mm_struct *prev, struct mm_struct *next,
 	local_irq_restore(flags);
 }
 
-static inline unsigned long mm_mangle_tif_spec_ib(struct task_struct *next)
+static unsigned long mm_mangle_tif_spec_ib(struct task_struct *next)
 {
 	unsigned long next_tif = task_thread_info(next)->flags;
 	unsigned long ibpb = (next_tif >> TIF_SPEC_IB) & LAST_USER_MM_IBPB;
@@ -882,7 +882,7 @@ static DEFINE_PER_CPU_SHARED_ALIGNED(struct flush_tlb_info, flush_tlb_info);
 static DEFINE_PER_CPU(unsigned int, flush_tlb_info_idx);
 #endif
 
-static inline struct flush_tlb_info *get_flush_tlb_info(struct mm_struct *mm,
+static struct flush_tlb_info *get_flush_tlb_info(struct mm_struct *mm,
 			unsigned long start, unsigned long end,
 			unsigned int stride_shift, bool freed_tables,
 			u64 new_tlb_gen)
@@ -909,7 +909,7 @@ static inline struct flush_tlb_info *get_flush_tlb_info(struct mm_struct *mm,
 	return info;
 }
 
-static inline void put_flush_tlb_info(void)
+static void put_flush_tlb_info(void)
 {
 #ifdef CONFIG_DEBUG_VM
 	/* Complete reentrency prevention checks */
-- 
2.25.1


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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-09 22:16 ` [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond() Nadav Amit
@ 2021-02-16 12:04   ` Peter Zijlstra
  2021-02-16 18:49     ` Nadav Amit
  2021-02-16 12:06   ` Peter Zijlstra
                     ` (2 subsequent siblings)
  3 siblings, 1 reply; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-16 12:04 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Nadav Amit, Rik van Riel, Josh Poimboeuf

On Tue, Feb 09, 2021 at 02:16:46PM -0800, Nadav Amit wrote:
> @@ -894,17 +911,12 @@ EXPORT_SYMBOL(on_each_cpu_mask);
>  void on_each_cpu_cond_mask(smp_cond_func_t cond_func, smp_call_func_t func,
>  			   void *info, bool wait, const struct cpumask *mask)
>  {
> -	int cpu = get_cpu();
> +	unsigned int scf_flags = SCF_RUN_LOCAL;
>  
> -	smp_call_function_many_cond(mask, func, info, wait, cond_func);
> -	if (cpumask_test_cpu(cpu, mask) && cond_func(cpu, info)) {
> -		unsigned long flags;
> +	if (wait)
> +		scf_flags |= SCF_WAIT;
>  
> -		local_irq_save(flags);
> -		func(info);
> -		local_irq_restore(flags);
> -	}
> -	put_cpu();
> +	smp_call_function_many_cond(mask, func, info, scf_flags, cond_func);
>  }
>  EXPORT_SYMBOL(on_each_cpu_cond_mask);

You lost the preempt_disable() there, I've added it back:

---
--- a/kernel/smp.c
+++ b/kernel/smp.c
@@ -920,7 +920,9 @@ void on_each_cpu_cond_mask(smp_cond_func
 	if (wait)
 		scf_flags |= SCF_WAIT;
 
+	preempt_disable();
 	smp_call_function_many_cond(mask, func, info, scf_flags, cond_func);
+	preempt_enable();
 }
 EXPORT_SYMBOL(on_each_cpu_cond_mask);
 

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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-09 22:16 ` [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond() Nadav Amit
  2021-02-16 12:04   ` Peter Zijlstra
@ 2021-02-16 12:06   ` Peter Zijlstra
  2021-02-16 16:32   ` Peter Zijlstra
  2021-02-18  8:09   ` Christoph Hellwig
  3 siblings, 0 replies; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-16 12:06 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Nadav Amit, Rik van Riel, Josh Poimboeuf

On Tue, Feb 09, 2021 at 02:16:46PM -0800, Nadav Amit wrote:
>  void smp_call_function_many(const struct cpumask *mask,
>  			    smp_call_func_t func, void *info, bool wait)
>  {
> +	smp_call_function_many_cond(mask, func, info, wait ? SCF_WAIT : 0, NULL);

	wait*SCF_WAIT

is shorter and sometimes generates better code

>  }
>  EXPORT_SYMBOL(smp_call_function_many);

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

* Re: [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently
  2021-02-09 22:16   ` Nadav Amit
@ 2021-02-16 12:10     ` Peter Zijlstra
  -1 siblings, 0 replies; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-16 12:10 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Nadav Amit, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Sasha Levin, Ingo Molnar, Borislav Petkov, x86, Juergen Gross,
	Paolo Bonzini, Boris Ostrovsky, linux-hyperv, virtualization,
	kvm, xen-devel, Michael Kelley

On Tue, Feb 09, 2021 at 02:16:49PM -0800, Nadav Amit wrote:
> @@ -816,8 +821,8 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
>  	 * doing a speculative memory access.
>  	 */
>  	if (info->freed_tables) {
> -		smp_call_function_many(cpumask, flush_tlb_func,
> -			       (void *)info, 1);
> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> +				      cpumask);
>  	} else {
>  		/*
>  		 * Although we could have used on_each_cpu_cond_mask(),
> @@ -844,14 +849,15 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
>  			if (tlb_is_not_lazy(cpu))
>  				__cpumask_set_cpu(cpu, cond_cpumask);
>  		}
> -		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);
> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> +				      cpumask);
>  	}
>  }

Surely on_each_cpu_mask() is more appropriate? There the compiler can do
the NULL propagation because it's on the same TU.

--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -821,8 +821,7 @@ STATIC_NOPV void native_flush_tlb_multi(
 	 * doing a speculative memory access.
 	 */
 	if (info->freed_tables) {
-		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
-				      cpumask);
+		on_each_cpu_mask(cpumask, flush_tlb_func, (void *)info, true);
 	} else {
 		/*
 		 * Although we could have used on_each_cpu_cond_mask(),
@@ -849,8 +848,7 @@ STATIC_NOPV void native_flush_tlb_multi(
 			if (tlb_is_not_lazy(cpu))
 				__cpumask_set_cpu(cpu, cond_cpumask);
 		}
-		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
-				      cpumask);
+		on_each_cpu_mask(cpumask, flush_tlb_func, (void *)info, true);
 	}
 }
 

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

* Re: [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently
@ 2021-02-16 12:10     ` Peter Zijlstra
  0 siblings, 0 replies; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-16 12:10 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Sasha Levin, Juergen Gross, linux-hyperv, x86, Stephen Hemminger,
	xen-devel, kvm, Haiyang Zhang, Dave Hansen, linux-kernel,
	virtualization, Ingo Molnar, Nadav Amit, Andy Lutomirski,
	Paolo Bonzini, Borislav Petkov, Thomas Gleixner, Boris Ostrovsky,
	Michael Kelley

On Tue, Feb 09, 2021 at 02:16:49PM -0800, Nadav Amit wrote:
> @@ -816,8 +821,8 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
>  	 * doing a speculative memory access.
>  	 */
>  	if (info->freed_tables) {
> -		smp_call_function_many(cpumask, flush_tlb_func,
> -			       (void *)info, 1);
> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> +				      cpumask);
>  	} else {
>  		/*
>  		 * Although we could have used on_each_cpu_cond_mask(),
> @@ -844,14 +849,15 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
>  			if (tlb_is_not_lazy(cpu))
>  				__cpumask_set_cpu(cpu, cond_cpumask);
>  		}
> -		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);
> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> +				      cpumask);
>  	}
>  }

Surely on_each_cpu_mask() is more appropriate? There the compiler can do
the NULL propagation because it's on the same TU.

--- a/arch/x86/mm/tlb.c
+++ b/arch/x86/mm/tlb.c
@@ -821,8 +821,7 @@ STATIC_NOPV void native_flush_tlb_multi(
 	 * doing a speculative memory access.
 	 */
 	if (info->freed_tables) {
-		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
-				      cpumask);
+		on_each_cpu_mask(cpumask, flush_tlb_func, (void *)info, true);
 	} else {
 		/*
 		 * Although we could have used on_each_cpu_cond_mask(),
@@ -849,8 +848,7 @@ STATIC_NOPV void native_flush_tlb_multi(
 			if (tlb_is_not_lazy(cpu))
 				__cpumask_set_cpu(cpu, cond_cpumask);
 		}
-		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
-				      cpumask);
+		on_each_cpu_mask(cpumask, flush_tlb_func, (void *)info, true);
 	}
 }
 
_______________________________________________
Virtualization mailing list
Virtualization@lists.linux-foundation.org
https://lists.linuxfoundation.org/mailman/listinfo/virtualization

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

* Re: [PATCH v5 7/8] cpumask: Mark functions as pure
  2021-02-09 22:16 ` [PATCH v5 7/8] cpumask: Mark functions as pure Nadav Amit
@ 2021-02-16 12:14   ` Peter Zijlstra
  0 siblings, 0 replies; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-16 12:14 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen, Nadav Amit

On Tue, Feb 09, 2021 at 02:16:52PM -0800, Nadav Amit wrote:

> @@ -235,7 +235,7 @@ static inline unsigned int cpumask_last(const struct cpumask *srcp)
>  	return find_last_bit(cpumask_bits(srcp), nr_cpumask_bits);
>  }
>  
> -unsigned int cpumask_next(int n, const struct cpumask *srcp);
> +unsigned int __pure cpumask_next(int n, const struct cpumask *srcp);
>  
>  /**
>   * cpumask_next_zero - get the next unset cpu in a cpumask

I've changed it to:

__pure unsigned int cpumask_next(...);

to be consistent with these:

> @@ -252,8 +252,8 @@ static inline unsigned int cpumask_next_zero(int n, const struct cpumask *srcp)
>  	return find_next_zero_bit(cpumask_bits(srcp), nr_cpumask_bits, n+1);
>  }
>  
> -int cpumask_next_and(int n, const struct cpumask *, const struct cpumask *);
> -int cpumask_any_but(const struct cpumask *mask, unsigned int cpu);
> +__pure int cpumask_next_and(int n, const struct cpumask *, const struct cpumask *);
> +__pure int cpumask_any_but(const struct cpumask *mask, unsigned int cpu);
>  unsigned int cpumask_local_spread(unsigned int i, int node);
>  int cpumask_any_and_distribute(const struct cpumask *src1p,
>  			       const struct cpumask *src2p);



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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-09 22:16 ` [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond() Nadav Amit
  2021-02-16 12:04   ` Peter Zijlstra
  2021-02-16 12:06   ` Peter Zijlstra
@ 2021-02-16 16:32   ` Peter Zijlstra
  2021-02-16 18:53     ` Nadav Amit
  2021-02-18  8:09   ` Christoph Hellwig
  3 siblings, 1 reply; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-16 16:32 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Nadav Amit, Rik van Riel, Josh Poimboeuf

On Tue, Feb 09, 2021 at 02:16:46PM -0800, Nadav Amit wrote:
> From: Nadav Amit <namit@vmware.com>
> 
> Currently, on_each_cpu() and similar functions do not exploit the
> potential of concurrency: the function is first executed remotely and
> only then it is executed locally. Functions such as TLB flush can take
> considerable time, so this provides an opportunity for performance
> optimization.
> 
> To do so, modify smp_call_function_many_cond(), to allows the callers to
> provide a function that should be executed (remotely/locally), and run
> them concurrently. Keep other smp_call_function_many() semantic as it is
> today for backward compatibility: the called function is not executed in
> this case locally.
> 
> smp_call_function_many_cond() does not use the optimized version for a
> single remote target that smp_call_function_single() implements. For
> synchronous function call, smp_call_function_single() keeps a
> call_single_data (which is used for synchronization) on the stack.
> Interestingly, it seems that not using this optimization provides
> greater performance improvements (greater speedup with a single remote
> target than with multiple ones). Presumably, holding data structures
> that are intended for synchronization on the stack can introduce
> overheads due to TLB misses and false-sharing when the stack is used for
> other purposes.
> 
> Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
> Cc: Peter Zijlstra <peterz@infradead.org>
> Cc: Rik van Riel <riel@surriel.com>
> Cc: Thomas Gleixner <tglx@linutronix.de>
> Cc: Andy Lutomirski <luto@kernel.org>
> Cc: Josh Poimboeuf <jpoimboe@redhat.com>
> Signed-off-by: Nadav Amit <namit@vmware.com>

Kernel-CI is giving me a regression that's most likely this patch:

  https://kernelci.org/test/case/id/602bdd621c979f83faaddcc6/

I'm not sure I can explain it yet. It did get me looking at
on_each_cpu() and it appears that wants to be converted too, something
like the below perhaps.


--- a/kernel/smp.c
+++ b/kernel/smp.c
@@ -848,14 +848,7 @@ void __init smp_init(void)
  */
 void on_each_cpu(smp_call_func_t func, void *info, int wait)
 {
-	unsigned long flags;
-
-	preempt_disable();
-	smp_call_function(func, info, wait);
-	local_irq_save(flags);
-	func(info);
-	local_irq_restore(flags);
-	preempt_enable();
+	on_each_cpu_mask(cpu_online_mask, func, info, wait);
 }
 EXPORT_SYMBOL(on_each_cpu);
 
@@ -878,15 +871,7 @@ EXPORT_SYMBOL(on_each_cpu);
 void on_each_cpu_mask(const struct cpumask *mask, smp_call_func_t func,
 			void *info, bool wait)
 {
-	unsigned int scf_flags;
-
-	scf_flags = SCF_RUN_LOCAL;
-	if (wait)
-		scf_flags |= SCF_WAIT;
-
-	preempt_disable();
-	smp_call_function_many_cond(mask, func, info, scf_flags, NULL);
-	preempt_enable();
+	on_each_cpu_cond_mask(NULL, func, info, wait, mask);
 }
 EXPORT_SYMBOL(on_each_cpu_mask);
 
@@ -912,6 +897,13 @@ EXPORT_SYMBOL(on_each_cpu_mask);
  * You must not call this function with disabled interrupts or
  * from a hardware interrupt handler or from a bottom half handler.
  */
+void on_each_cpu_cond(smp_cond_func_t cond_func, smp_call_func_t func,
+		      void *info, bool wait)
+{
+	on_each_cpu_cond_mask(cond_func, func, info, wait, cpu_online_mask);
+}
+EXPORT_SYMBOL(on_each_cpu_cond);
+
 void on_each_cpu_cond_mask(smp_cond_func_t cond_func, smp_call_func_t func,
 			   void *info, bool wait, const struct cpumask *mask)
 {
@@ -926,13 +918,6 @@ void on_each_cpu_cond_mask(smp_cond_func
 }
 EXPORT_SYMBOL(on_each_cpu_cond_mask);
 
-void on_each_cpu_cond(smp_cond_func_t cond_func, smp_call_func_t func,
-		      void *info, bool wait)
-{
-	on_each_cpu_cond_mask(cond_func, func, info, wait, cpu_online_mask);
-}
-EXPORT_SYMBOL(on_each_cpu_cond);
-
 static void do_nothing(void *unused)
 {
 }
~

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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-16 12:04   ` Peter Zijlstra
@ 2021-02-16 18:49     ` Nadav Amit
  0 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-16 18:49 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thomas Gleixner, LKML, Andy Lutomirski, Dave Hansen,
	Rik van Riel, Josh Poimboeuf

> On Feb 16, 2021, at 4:04 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Tue, Feb 09, 2021 at 02:16:46PM -0800, Nadav Amit wrote:
>> @@ -894,17 +911,12 @@ EXPORT_SYMBOL(on_each_cpu_mask);
>> void on_each_cpu_cond_mask(smp_cond_func_t cond_func, smp_call_func_t func,
>> 			   void *info, bool wait, const struct cpumask *mask)
>> {
>> -	int cpu = get_cpu();
>> +	unsigned int scf_flags = SCF_RUN_LOCAL;
>> 
>> -	smp_call_function_many_cond(mask, func, info, wait, cond_func);
>> -	if (cpumask_test_cpu(cpu, mask) && cond_func(cpu, info)) {
>> -		unsigned long flags;
>> +	if (wait)
>> +		scf_flags |= SCF_WAIT;
>> 
>> -		local_irq_save(flags);
>> -		func(info);
>> -		local_irq_restore(flags);
>> -	}
>> -	put_cpu();
>> +	smp_call_function_many_cond(mask, func, info, scf_flags, cond_func);
>> }
>> EXPORT_SYMBOL(on_each_cpu_cond_mask);
> 
> You lost the preempt_disable() there, I've added it back:
> 
> ---
> --- a/kernel/smp.c
> +++ b/kernel/smp.c
> @@ -920,7 +920,9 @@ void on_each_cpu_cond_mask(smp_cond_func
> 	if (wait)
> 		scf_flags |= SCF_WAIT;
> 
> +	preempt_disable();
> 	smp_call_function_many_cond(mask, func, info, scf_flags, cond_func);
> +	preempt_enable();
> }
> EXPORT_SYMBOL(on_each_cpu_cond_mask);

Indeed. I will add lockdep_assert_preemption_disabled() to
smp_call_function_many_cond() to prevent this mistake from reoccurring.


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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-16 16:32   ` Peter Zijlstra
@ 2021-02-16 18:53     ` Nadav Amit
  2021-02-16 18:59       ` Peter Zijlstra
  0 siblings, 1 reply; 33+ messages in thread
From: Nadav Amit @ 2021-02-16 18:53 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Rik van Riel, Josh Poimboeuf

> On Feb 16, 2021, at 8:32 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Tue, Feb 09, 2021 at 02:16:46PM -0800, Nadav Amit wrote:
>> From: Nadav Amit <namit@vmware.com>
>> 
>> Currently, on_each_cpu() and similar functions do not exploit the
>> potential of concurrency: the function is first executed remotely and
>> only then it is executed locally. Functions such as TLB flush can take
>> considerable time, so this provides an opportunity for performance
>> optimization.
>> 
>> To do so, modify smp_call_function_many_cond(), to allows the callers to
>> provide a function that should be executed (remotely/locally), and run
>> them concurrently. Keep other smp_call_function_many() semantic as it is
>> today for backward compatibility: the called function is not executed in
>> this case locally.
>> 
>> smp_call_function_many_cond() does not use the optimized version for a
>> single remote target that smp_call_function_single() implements. For
>> synchronous function call, smp_call_function_single() keeps a
>> call_single_data (which is used for synchronization) on the stack.
>> Interestingly, it seems that not using this optimization provides
>> greater performance improvements (greater speedup with a single remote
>> target than with multiple ones). Presumably, holding data structures
>> that are intended for synchronization on the stack can introduce
>> overheads due to TLB misses and false-sharing when the stack is used for
>> other purposes.
>> 
>> Reviewed-by: Dave Hansen <dave.hansen@linux.intel.com>
>> Cc: Peter Zijlstra <peterz@infradead.org>
>> Cc: Rik van Riel <riel@surriel.com>
>> Cc: Thomas Gleixner <tglx@linutronix.de>
>> Cc: Andy Lutomirski <luto@kernel.org>
>> Cc: Josh Poimboeuf <jpoimboe@redhat.com>
>> Signed-off-by: Nadav Amit <namit@vmware.com>
> 
> Kernel-CI is giving me a regression that's most likely this patch:
> 
>  https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fkernelci.org%2Ftest%2Fcase%2Fid%2F602bdd621c979f83faaddcc6%2F&amp;data=04%7C01%7Cnamit%40vmware.com%7C7dc93f3b74d8488de06f08d8d2988b0a%7Cb39138ca3cee4b4aa4d6cd83d9dd62f0%7C0%7C0%7C637490899907612612%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&amp;sdata=PFs0ydMLh6xVfAQzAxSNd108YjxKMopNwxqsm82lEog%3D&amp;reserved=0
> 
> I'm not sure I can explain it yet. It did get me looking at
> on_each_cpu() and it appears that wants to be converted too, something
> like the below perhaps.

Looks like a good cleanup, but I cannot say I understand the problem and how
it would solve it. Err...

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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-16 18:53     ` Nadav Amit
@ 2021-02-16 18:59       ` Peter Zijlstra
  2021-02-16 19:04         ` Nadav Amit
  2021-02-17  1:02         ` Nadav Amit
  0 siblings, 2 replies; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-16 18:59 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Rik van Riel, Josh Poimboeuf

On Tue, Feb 16, 2021 at 06:53:09PM +0000, Nadav Amit wrote:
> > On Feb 16, 2021, at 8:32 AM, Peter Zijlstra <peterz@infradead.org> wrote:

> > I'm not sure I can explain it yet. It did get me looking at
> > on_each_cpu() and it appears that wants to be converted too, something
> > like the below perhaps.
> 
> Looks like a good cleanup, but I cannot say I understand the problem and how
> it would solve it. Err...

Yeah, me neither. Bit of a mystery so far.

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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-16 18:59       ` Peter Zijlstra
@ 2021-02-16 19:04         ` Nadav Amit
  2021-02-17  1:02         ` Nadav Amit
  1 sibling, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-16 19:04 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Rik van Riel, Josh Poimboeuf

> On Feb 16, 2021, at 10:59 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Tue, Feb 16, 2021 at 06:53:09PM +0000, Nadav Amit wrote:
>>> On Feb 16, 2021, at 8:32 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
>>> I'm not sure I can explain it yet. It did get me looking at
>>> on_each_cpu() and it appears that wants to be converted too, something
>>> like the below perhaps.
>> 
>> Looks like a good cleanup, but I cannot say I understand the problem and how
>> it would solve it. Err...
> 
> Yeah, me neither. Bit of a mystery so far.

I’ll try to see whether I can figure out about it. Perhaps there is
somewhere an assumption of ordering between the local and remote function
invocations.

Regardless, would you want me to have on_each_cpu() as inline or to keep it
in smp.c?

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

* Re: [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently
  2021-02-16 12:10     ` Peter Zijlstra
@ 2021-02-16 19:17       ` Nadav Amit
  -1 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-16 19:17 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thomas Gleixner, LKML, Andy Lutomirski, Dave Hansen,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger, Sasha Levin,
	Ingo Molnar, Borislav Petkov, X86 ML, Juergen Gross,
	Paolo Bonzini, Boris Ostrovsky, linux-hyperv,
	Linux Virtualization, KVM, xen-devel, Michael Kelley

> On Feb 16, 2021, at 4:10 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Tue, Feb 09, 2021 at 02:16:49PM -0800, Nadav Amit wrote:
>> @@ -816,8 +821,8 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
>> 	 * doing a speculative memory access.
>> 	 */
>> 	if (info->freed_tables) {
>> -		smp_call_function_many(cpumask, flush_tlb_func,
>> -			       (void *)info, 1);
>> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
>> +				      cpumask);
>> 	} else {
>> 		/*
>> 		 * Although we could have used on_each_cpu_cond_mask(),
>> @@ -844,14 +849,15 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
>> 			if (tlb_is_not_lazy(cpu))
>> 				__cpumask_set_cpu(cpu, cond_cpumask);
>> 		}
>> -		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);
>> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
>> +				      cpumask);
>> 	}
>> }
> 
> Surely on_each_cpu_mask() is more appropriate? There the compiler can do
> the NULL propagation because it's on the same TU.
> 
> --- a/arch/x86/mm/tlb.c
> +++ b/arch/x86/mm/tlb.c
> @@ -821,8 +821,7 @@ STATIC_NOPV void native_flush_tlb_multi(
> 	 * doing a speculative memory access.
> 	 */
> 	if (info->freed_tables) {
> -		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> -				      cpumask);
> +		on_each_cpu_mask(cpumask, flush_tlb_func, (void *)info, true);
> 	} else {
> 		/*
> 		 * Although we could have used on_each_cpu_cond_mask(),
> @@ -849,8 +848,7 @@ STATIC_NOPV void native_flush_tlb_multi(
> 			if (tlb_is_not_lazy(cpu))
> 				__cpumask_set_cpu(cpu, cond_cpumask);
> 		}
> -		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> -				      cpumask);
> +		on_each_cpu_mask(cpumask, flush_tlb_func, (void *)info, true);
> 	}
> }

Indeed, and there is actually an additional bug - I used cpumask in the
second on_each_cpu_cond_mask() instead of cond_cpumask.


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

* Re: [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently
@ 2021-02-16 19:17       ` Nadav Amit
  0 siblings, 0 replies; 33+ messages in thread
From: Nadav Amit @ 2021-02-16 19:17 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Sasha Levin, Juergen Gross, linux-hyperv, X86 ML,
	Stephen Hemminger, KVM, Haiyang Zhang, Dave Hansen, LKML,
	Linux Virtualization, Ingo Molnar, Borislav Petkov,
	Andy Lutomirski, xen-devel, Paolo Bonzini, Thomas Gleixner,
	Boris Ostrovsky, Michael Kelley

> On Feb 16, 2021, at 4:10 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Tue, Feb 09, 2021 at 02:16:49PM -0800, Nadav Amit wrote:
>> @@ -816,8 +821,8 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
>> 	 * doing a speculative memory access.
>> 	 */
>> 	if (info->freed_tables) {
>> -		smp_call_function_many(cpumask, flush_tlb_func,
>> -			       (void *)info, 1);
>> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
>> +				      cpumask);
>> 	} else {
>> 		/*
>> 		 * Although we could have used on_each_cpu_cond_mask(),
>> @@ -844,14 +849,15 @@ STATIC_NOPV void native_flush_tlb_others(const struct cpumask *cpumask,
>> 			if (tlb_is_not_lazy(cpu))
>> 				__cpumask_set_cpu(cpu, cond_cpumask);
>> 		}
>> -		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);
>> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
>> +				      cpumask);
>> 	}
>> }
> 
> Surely on_each_cpu_mask() is more appropriate? There the compiler can do
> the NULL propagation because it's on the same TU.
> 
> --- a/arch/x86/mm/tlb.c
> +++ b/arch/x86/mm/tlb.c
> @@ -821,8 +821,7 @@ STATIC_NOPV void native_flush_tlb_multi(
> 	 * doing a speculative memory access.
> 	 */
> 	if (info->freed_tables) {
> -		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> -				      cpumask);
> +		on_each_cpu_mask(cpumask, flush_tlb_func, (void *)info, true);
> 	} else {
> 		/*
> 		 * Although we could have used on_each_cpu_cond_mask(),
> @@ -849,8 +848,7 @@ STATIC_NOPV void native_flush_tlb_multi(
> 			if (tlb_is_not_lazy(cpu))
> 				__cpumask_set_cpu(cpu, cond_cpumask);
> 		}
> -		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> -				      cpumask);
> +		on_each_cpu_mask(cpumask, flush_tlb_func, (void *)info, true);
> 	}
> }

Indeed, and there is actually an additional bug - I used cpumask in the
second on_each_cpu_cond_mask() instead of cond_cpumask.

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

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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-16 18:59       ` Peter Zijlstra
  2021-02-16 19:04         ` Nadav Amit
@ 2021-02-17  1:02         ` Nadav Amit
  2021-02-18 12:55           ` Peter Zijlstra
  1 sibling, 1 reply; 33+ messages in thread
From: Nadav Amit @ 2021-02-17  1:02 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Rik van Riel, Josh Poimboeuf

> On Feb 16, 2021, at 10:59 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Tue, Feb 16, 2021 at 06:53:09PM +0000, Nadav Amit wrote:
>>> On Feb 16, 2021, at 8:32 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
>>> I'm not sure I can explain it yet. It did get me looking at
>>> on_each_cpu() and it appears that wants to be converted too, something
>>> like the below perhaps.
>> 
>> Looks like a good cleanup, but I cannot say I understand the problem and how
>> it would solve it. Err...
> 
> Yeah, me neither. Bit of a mystery so far.

This stall seems to be real. Intuitively I presumed preemption was
mistakenly enabled, but it does not seem so.

Any chance you can build the kernel with “CONFIG_CSD_LOCK_WAIT_DEBUG=Y” and
rerun it? Perhaps that output will tell us more.


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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-09 22:16 ` [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond() Nadav Amit
                     ` (2 preceding siblings ...)
  2021-02-16 16:32   ` Peter Zijlstra
@ 2021-02-18  8:09   ` Christoph Hellwig
  2021-02-18  9:36     ` Nadav Amit
  3 siblings, 1 reply; 33+ messages in thread
From: Christoph Hellwig @ 2021-02-18  8:09 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Peter Zijlstra,
	Dave Hansen, Nadav Amit, Rik van Riel, Josh Poimboeuf

On Tue, Feb 09, 2021 at 02:16:46PM -0800, Nadav Amit wrote:
> +/*
> + * Flags to be used as scf_flags argument of smp_call_function_many_cond().
> + */
> +#define SCF_WAIT	(1U << 0)	/* Wait until function execution completed */
> +#define SCF_RUN_LOCAL	(1U << 1)	/* Run also locally if local cpu is set in cpumask */

Can you move the comments on top of the defines to avoid the crazy
long lines?

> +	if (cpu_online(this_cpu) && !oops_in_progress && !early_boot_irqs_disabled)

Another pointlessly overly long line, with various more following.

>  EXPORT_SYMBOL(on_each_cpu_cond_mask);

This isn't used by any modular code, so maybe throw in a patch to drop
the export?

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

* Re: [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently
  2021-02-09 22:16   ` Nadav Amit
@ 2021-02-18  8:14     ` Christoph Hellwig
  -1 siblings, 0 replies; 33+ messages in thread
From: Christoph Hellwig @ 2021-02-18  8:14 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Peter Zijlstra,
	Dave Hansen, Nadav Amit, K. Y. Srinivasan, Haiyang Zhang,
	Stephen Hemminger, Sasha Levin, Ingo Molnar, Borislav Petkov,
	x86, Juergen Gross, Paolo Bonzini, Boris Ostrovsky, linux-hyperv,
	virtualization, kvm, xen-devel, Michael Kelley

Given that the last patch killed the last previously existing
user of on_each_cpu_cond_mask there are now the only users.

>  	if (info->freed_tables) {
> -		smp_call_function_many(cpumask, flush_tlb_func,
> -			       (void *)info, 1);
> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> +				      cpumask);

.. 

> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> +				      cpumask);

Which means the cond_func is unused, and thus on_each_cpu_cond_mask can
go away entirely in favor of on_each_cpu_cond.

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

* Re: [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently
@ 2021-02-18  8:14     ` Christoph Hellwig
  0 siblings, 0 replies; 33+ messages in thread
From: Christoph Hellwig @ 2021-02-18  8:14 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Sasha Levin, Juergen Gross, linux-hyperv, x86, Stephen Hemminger,
	xen-devel, kvm, Peter Zijlstra, Haiyang Zhang, Dave Hansen,
	linux-kernel, virtualization, Ingo Molnar, Nadav Amit,
	Andy Lutomirski, Paolo Bonzini, Borislav Petkov, Thomas Gleixner,
	Boris Ostrovsky, Michael Kelley

Given that the last patch killed the last previously existing
user of on_each_cpu_cond_mask there are now the only users.

>  	if (info->freed_tables) {
> -		smp_call_function_many(cpumask, flush_tlb_func,
> -			       (void *)info, 1);
> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> +				      cpumask);

.. 

> +		on_each_cpu_cond_mask(NULL, flush_tlb_func, (void *)info, true,
> +				      cpumask);

Which means the cond_func is unused, and thus on_each_cpu_cond_mask can
go away entirely in favor of on_each_cpu_cond.
_______________________________________________
Virtualization mailing list
Virtualization@lists.linux-foundation.org
https://lists.linuxfoundation.org/mailman/listinfo/virtualization

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

* Re: [PATCH v5 3/8] x86/mm/tlb: Open-code on_each_cpu_cond_mask() for tlb_is_not_lazy()
  2021-02-09 22:16 ` [PATCH v5 3/8] x86/mm/tlb: Open-code on_each_cpu_cond_mask() for tlb_is_not_lazy() Nadav Amit
@ 2021-02-18  8:16   ` Christoph Hellwig
  2021-02-18  8:24     ` Nadav Amit
  0 siblings, 1 reply; 33+ messages in thread
From: Christoph Hellwig @ 2021-02-18  8:16 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Peter Zijlstra,
	Dave Hansen, Nadav Amit, Rik van Riel, Josh Poimboeuf

On Tue, Feb 09, 2021 at 02:16:48PM -0800, Nadav Amit wrote:
> +		/*
> +		 * Although we could have used on_each_cpu_cond_mask(),
> +		 * open-coding it has performance advantages, as it eliminates
> +		 * the need for indirect calls or retpolines. In addition, it
> +		 * allows to use a designated cpumask for evaluating the
> +		 * condition, instead of allocating one.
> +		 *
> +		 * This code works under the assumption that there are no nested
> +		 * TLB flushes, an assumption that is already made in
> +		 * flush_tlb_mm_range().
> +		 *
> +		 * cond_cpumask is logically a stack-local variable, but it is
> +		 * more efficient to have it off the stack and not to allocate
> +		 * it on demand. Preemption is disabled and this code is
> +		 * non-reentrant.
> +		 */
> +		struct cpumask *cond_cpumask = this_cpu_ptr(&flush_tlb_mask);
> +		int cpu;
> +
> +		cpumask_clear(cond_cpumask);
> +
> +		for_each_cpu(cpu, cpumask) {
> +			if (tlb_is_not_lazy(cpu))
> +				__cpumask_set_cpu(cpu, cond_cpumask);
> +		}
> +		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);

No need for the cast here, which would also avoid the pointlessly
overly long line.

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

* Re: [PATCH v5 3/8] x86/mm/tlb: Open-code on_each_cpu_cond_mask() for tlb_is_not_lazy()
  2021-02-18  8:16   ` Christoph Hellwig
@ 2021-02-18  8:24     ` Nadav Amit
  2021-02-18  8:28       ` Christoph Hellwig
  0 siblings, 1 reply; 33+ messages in thread
From: Nadav Amit @ 2021-02-18  8:24 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Peter Zijlstra,
	Dave Hansen, Rik van Riel, Josh Poimboeuf

> On Feb 18, 2021, at 12:16 AM, Christoph Hellwig <hch@infradead.org> wrote:
> 
> On Tue, Feb 09, 2021 at 02:16:48PM -0800, Nadav Amit wrote:
>> +		/*
>> +		 * Although we could have used on_each_cpu_cond_mask(),
>> +		 * open-coding it has performance advantages, as it eliminates
>> +		 * the need for indirect calls or retpolines. In addition, it
>> +		 * allows to use a designated cpumask for evaluating the
>> +		 * condition, instead of allocating one.
>> +		 *
>> +		 * This code works under the assumption that there are no nested
>> +		 * TLB flushes, an assumption that is already made in
>> +		 * flush_tlb_mm_range().
>> +		 *
>> +		 * cond_cpumask is logically a stack-local variable, but it is
>> +		 * more efficient to have it off the stack and not to allocate
>> +		 * it on demand. Preemption is disabled and this code is
>> +		 * non-reentrant.
>> +		 */
>> +		struct cpumask *cond_cpumask = this_cpu_ptr(&flush_tlb_mask);
>> +		int cpu;
>> +
>> +		cpumask_clear(cond_cpumask);
>> +
>> +		for_each_cpu(cpu, cpumask) {
>> +			if (tlb_is_not_lazy(cpu))
>> +				__cpumask_set_cpu(cpu, cond_cpumask);
>> +		}
>> +		smp_call_function_many(cond_cpumask, flush_tlb_func, (void *)info, 1);
> 
> No need for the cast here, which would also avoid the pointlessly
> overly long line.

Actually, there is - to remove the const qualifier. You might argue it is
ugly, but that’s the way it is also how it is done right now.

In general, thanks for the feedback (I will reply after I follow your
feedback). I do have a general question - I thought it was decided that
clarity should be preferred over following the 80-column limit. Please let
me know if I misunderstood.

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

* Re: [PATCH v5 3/8] x86/mm/tlb: Open-code on_each_cpu_cond_mask() for tlb_is_not_lazy()
  2021-02-18  8:24     ` Nadav Amit
@ 2021-02-18  8:28       ` Christoph Hellwig
  0 siblings, 0 replies; 33+ messages in thread
From: Christoph Hellwig @ 2021-02-18  8:28 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Christoph Hellwig, Thomas Gleixner, linux-kernel,
	Andy Lutomirski, Peter Zijlstra, Dave Hansen, Rik van Riel,
	Josh Poimboeuf

On Thu, Feb 18, 2021 at 08:24:23AM +0000, Nadav Amit wrote:
> In general, thanks for the feedback (I will reply after I follow your
> feedback). I do have a general question - I thought it was decided that
> clarity should be preferred over following the 80-column limit. Please let
> me know if I misunderstood.

To quote the coding style document:

"The preferred limit on the length of a single line is 80 columns.

Statements longer than 80 columns should be broken into sensible chunks,
unless exceeding 80 columns significantly increases readability and does
not hide information."

While a lot of the decisions are subjective, just spilling over for no
good reason does not significantly improve readability in general.

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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-18  8:09   ` Christoph Hellwig
@ 2021-02-18  9:36     ` Nadav Amit
  2021-02-18 11:12       ` Peter Zijlstra
  0 siblings, 1 reply; 33+ messages in thread
From: Nadav Amit @ 2021-02-18  9:36 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Thomas Gleixner, LKML, Andy Lutomirski, Peter Zijlstra,
	Dave Hansen, Rik van Riel, Josh Poimboeuf

> On Feb 18, 2021, at 12:09 AM, Christoph Hellwig <hch@infradead.org> wrote:
> 
> On Tue, Feb 09, 2021 at 02:16:46PM -0800, Nadav Amit wrote:
>> +/*
>> + * Flags to be used as scf_flags argument of smp_call_function_many_cond().
>> + */
>> +#define SCF_WAIT	(1U << 0)	/* Wait until function execution completed */
>> +#define SCF_RUN_LOCAL	(1U << 1)	/* Run also locally if local cpu is set in cpumask */
> 
> Can you move the comments on top of the defines to avoid the crazy
> long lines?
> 
>> +	if (cpu_online(this_cpu) && !oops_in_progress && !early_boot_irqs_disabled)
> 
> Another pointlessly overly long line, with various more following.
> 
>> EXPORT_SYMBOL(on_each_cpu_cond_mask);
> 
> This isn't used by any modular code, so maybe throw in a patch to drop
> the export?

I prefer to export on_each_cpu_cond_mask() and instead turn the users
(on_each_cpu(), on_each_cpu_mask() and on_each_cpu_cond()) into inline
functions in smp.h. Otherwise, the call-chain becomes longer for no reason.
Let me know if you object.

So I will add something like:

-- >8 --

Author: Nadav Amit <namit@vmware.com>
Date:   Tue Feb 16 11:04:30 2021 -0800

    smp: inline on_each_cpu_cond() and on_each_cpu_cond_mask()
    
    Suggested-by: Peter Zijlstra <peterz@infradead.org>
    Signed-off-by: Nadav Amit <namit@vmware.com>

diff --git a/include/linux/smp.h b/include/linux/smp.h
index 70c6f6284dcf..84a0b4828f66 100644
--- a/include/linux/smp.h
+++ b/include/linux/smp.h
@@ -50,30 +50,52 @@ extern unsigned int total_cpus;
 int smp_call_function_single(int cpuid, smp_call_func_t func, void *info,
                             int wait);
 
+void on_each_cpu_cond_mask(smp_cond_func_t cond_func, smp_call_func_t func,
+                          void *info, bool wait, const struct cpumask *mask);
+
+int smp_call_function_single_async(int cpu, call_single_data_t *csd);
+
 /*
  * Call a function on all processors
  */
-void on_each_cpu(smp_call_func_t func, void *info, int wait);
+static inline void on_each_cpu(smp_call_func_t func, void *info, int wait)
+{
+       on_each_cpu_cond_mask(NULL, func, info, wait, cpu_online_mask);
+}
 
-/*
- * Call a function on processors specified by mask, which might include
- * the local one.
+/**
+ * on_each_cpu_mask(): Run a function on processors specified by
+ * cpumask, which may include the local processor.
+ * @mask: The set of cpus to run on (only runs on online subset).
+ * @func: The function to run. This must be fast and non-blocking.
+ * @info: An arbitrary pointer to pass to the function.
+ * @wait: If true, wait (atomically) until function has completed
+ *        on other CPUs.
+ *
+ * If @wait is true, then returns once @func has returned.
+ *
+ * You must not call this function with disabled interrupts or from a
+ * hardware interrupt handler or from a bottom half handler.  The
+ * exception is that it may be used during early boot while
+ * early_boot_irqs_disabled is set.
  */
-void on_each_cpu_mask(const struct cpumask *mask, smp_call_func_t func,
-               void *info, bool wait);
+static inline void on_each_cpu_mask(const struct cpumask *mask,
+                                   smp_call_func_t func, void *info, bool wait)
+{
+       on_each_cpu_cond_mask(NULL, func, info, wait, mask);
+}
 
 /*
  * Call a function on each processor for which the supplied function
  * cond_func returns a positive value. This may include the local
- * processor.
+ * processor.  May be used during early boot while early_boot_irqs_disabled is
+ * set. Use local_irq_save/restore() instead of local_irq_disable/enable().
  */
-void on_each_cpu_cond(smp_cond_func_t cond_func, smp_call_func_t func,
-                     void *info, bool wait);
-
-void on_each_cpu_cond_mask(smp_cond_func_t cond_func, smp_call_func_t func,
-                          void *info, bool wait, const struct cpumask *mask);
-
-int smp_call_function_single_async(int cpu, call_single_data_t *csd);
+static inline void on_each_cpu_cond(smp_cond_func_t cond_func,
+                                   smp_call_func_t func, void *info, bool wait)
+{
+       on_each_cpu_cond_mask(cond_func, func, info, wait, cpu_online_mask);
+}
 
 #ifdef CONFIG_SMP
 
diff --git a/kernel/smp.c b/kernel/smp.c
index 629f1f7b80db..a75f3d1dd1b7 100644
--- a/kernel/smp.c
+++ b/kernel/smp.c
@@ -843,55 +843,6 @@ void __init smp_init(void)
        smp_cpus_done(setup_max_cpus);
 }
 
-/*
- * Call a function on all processors.  May be used during early boot while
- * early_boot_irqs_disabled is set.  Use local_irq_save/restore() instead
- * of local_irq_disable/enable().
- */
-void on_each_cpu(smp_call_func_t func, void *info, int wait)
-{
-       unsigned long flags;
-
-       preempt_disable();
-       smp_call_function(func, info, wait);
-       local_irq_save(flags);
-       func(info);
-       local_irq_restore(flags);
-       preempt_enable();
-}
-EXPORT_SYMBOL(on_each_cpu);
-
-/**
- * on_each_cpu_mask(): Run a function on processors specified by
- * cpumask, which may include the local processor.
- * @mask: The set of cpus to run on (only runs on online subset).
- * @func: The function to run. This must be fast and non-blocking.
- * @info: An arbitrary pointer to pass to the function.
- * @wait: If true, wait (atomically) until function has completed
- *        on other CPUs.
- *
- * If @wait is true, then returns once @func has returned.
- *
- * You must not call this function with disabled interrupts or from a
- * hardware interrupt handler or from a bottom half handler.  The
- * exception is that it may be used during early boot while
- * early_boot_irqs_disabled is set.
- */
-void on_each_cpu_mask(const struct cpumask *mask, smp_call_func_t func,
-                       void *info, bool wait)
-{
-       unsigned int scf_flags;
-
-       scf_flags = SCF_RUN_LOCAL;
-       if (wait)
-               scf_flags |= SCF_WAIT;
-
-       preempt_disable();
-       smp_call_function_many_cond(mask, func, info, scf_flags, NULL);
-       preempt_enable();
-}
-EXPORT_SYMBOL(on_each_cpu_mask);
-
 /*
  * on_each_cpu_cond(): Call a function on each processor for which
  * the supplied function cond_func returns true, optionally waiting
@@ -928,13 +879,6 @@ void on_each_cpu_cond_mask(smp_cond_func_t cond_func, smp_call_func_t func,
 }
 EXPORT_SYMBOL(on_each_cpu_cond_mask);
 
-void on_each_cpu_cond(smp_cond_func_t cond_func, smp_call_func_t func,
-                     void *info, bool wait)
-{
-       on_each_cpu_cond_mask(cond_func, func, info, wait, cpu_online_mask);
-}
-EXPORT_SYMBOL(on_each_cpu_cond);
-
 static void do_nothing(void *unused)
 {
 }


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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-18  9:36     ` Nadav Amit
@ 2021-02-18 11:12       ` Peter Zijlstra
  0 siblings, 0 replies; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-18 11:12 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Christoph Hellwig, Thomas Gleixner, LKML, Andy Lutomirski,
	Dave Hansen, Rik van Riel, Josh Poimboeuf

On Thu, Feb 18, 2021 at 09:36:09AM +0000, Nadav Amit wrote:
> I prefer to export on_each_cpu_cond_mask() and instead turn the users
> (on_each_cpu(), on_each_cpu_mask() and on_each_cpu_cond()) into inline
> functions in smp.h.

That makes it impossible (barring LTO) for the compiler to do constant
propagation on the cond and mask thingies.

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

* Re: [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond()
  2021-02-17  1:02         ` Nadav Amit
@ 2021-02-18 12:55           ` Peter Zijlstra
  0 siblings, 0 replies; 33+ messages in thread
From: Peter Zijlstra @ 2021-02-18 12:55 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Thomas Gleixner, linux-kernel, Andy Lutomirski, Dave Hansen,
	Rik van Riel, Josh Poimboeuf

On Wed, Feb 17, 2021 at 01:02:41AM +0000, Nadav Amit wrote:
> > On Feb 16, 2021, at 10:59 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> > 
> > On Tue, Feb 16, 2021 at 06:53:09PM +0000, Nadav Amit wrote:
> >>> On Feb 16, 2021, at 8:32 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> > 
> >>> I'm not sure I can explain it yet. It did get me looking at
> >>> on_each_cpu() and it appears that wants to be converted too, something
> >>> like the below perhaps.
> >> 
> >> Looks like a good cleanup, but I cannot say I understand the problem and how
> >> it would solve it. Err...
> > 
> > Yeah, me neither. Bit of a mystery so far.
> 
> This stall seems to be real. Intuitively I presumed preemption was
> mistakenly enabled, but it does not seem so.
> 
> Any chance you can build the kernel with “CONFIG_CSD_LOCK_WAIT_DEBUG=Y” and
> rerun it? Perhaps that output will tell us more.

Sadly not my system. It's a KernelCI box.

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

end of thread, other threads:[~2021-02-18 15:13 UTC | newest]

Thread overview: 33+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-02-09 22:16 [PATCH v5 0/8] x86/tlb: Concurrent TLB flushes Nadav Amit
2021-02-09 22:16 ` Nadav Amit
2021-02-09 22:16 ` [PATCH v5 1/8] smp: Run functions concurrently in smp_call_function_many_cond() Nadav Amit
2021-02-16 12:04   ` Peter Zijlstra
2021-02-16 18:49     ` Nadav Amit
2021-02-16 12:06   ` Peter Zijlstra
2021-02-16 16:32   ` Peter Zijlstra
2021-02-16 18:53     ` Nadav Amit
2021-02-16 18:59       ` Peter Zijlstra
2021-02-16 19:04         ` Nadav Amit
2021-02-17  1:02         ` Nadav Amit
2021-02-18 12:55           ` Peter Zijlstra
2021-02-18  8:09   ` Christoph Hellwig
2021-02-18  9:36     ` Nadav Amit
2021-02-18 11:12       ` Peter Zijlstra
2021-02-09 22:16 ` [PATCH v5 2/8] x86/mm/tlb: Unify flush_tlb_func_local() and flush_tlb_func_remote() Nadav Amit
2021-02-09 22:16 ` [PATCH v5 3/8] x86/mm/tlb: Open-code on_each_cpu_cond_mask() for tlb_is_not_lazy() Nadav Amit
2021-02-18  8:16   ` Christoph Hellwig
2021-02-18  8:24     ` Nadav Amit
2021-02-18  8:28       ` Christoph Hellwig
2021-02-09 22:16 ` [PATCH v5 4/8] x86/mm/tlb: Flush remote and local TLBs concurrently Nadav Amit
2021-02-09 22:16   ` Nadav Amit
2021-02-16 12:10   ` Peter Zijlstra
2021-02-16 12:10     ` Peter Zijlstra
2021-02-16 19:17     ` Nadav Amit
2021-02-16 19:17       ` Nadav Amit
2021-02-18  8:14   ` Christoph Hellwig
2021-02-18  8:14     ` Christoph Hellwig
2021-02-09 22:16 ` [PATCH v5 5/8] x86/mm/tlb: Privatize cpu_tlbstate Nadav Amit
2021-02-09 22:16 ` [PATCH v5 6/8] x86/mm/tlb: Do not make is_lazy dirty for no reason Nadav Amit
2021-02-09 22:16 ` [PATCH v5 7/8] cpumask: Mark functions as pure Nadav Amit
2021-02-16 12:14   ` Peter Zijlstra
2021-02-09 22:16 ` [PATCH v5 8/8] x86/mm/tlb: Remove unnecessary uses of the inline keyword Nadav Amit

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.