All of lore.kernel.org
 help / color / mirror / Atom feed
* [RFC PATCH 0/5] IPsec parallelization
@ 2008-12-01  7:16 Steffen Klassert
  2008-12-01  7:17 ` [RFC PATCH 1/5] padata: generic interface for parallel processing Steffen Klassert
                   ` (5 more replies)
  0 siblings, 6 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01  7:16 UTC (permalink / raw)
  To: netdev; +Cc: davem, herbert, klassert

This is a first throw to try to parallelize the expensive part of xfrm by
using a generic parallelization/serialization method. This method uses the
remote softirq invocation infrastructure for parallelization and serialization.
With this method data objects can be processed in parallel, starting 
at some given point. After doing some expensive operations in parallel, 
it is possible to serialize again. The parallelized data objects return after
serialization in the order as they were before the parallelization. 
In the case of xfrm, this makes it possible to run the expensive part in
parallel without getting packet reordering.
 
To use this parallelization method for xfrm, some changes in the crypto system
were necessary. First of all, we need to force disabling async crypto transforms
in the parallelization case, because we can't guarantee the packet order if
the packets are put to a queue during the parallel processing.
A second thing was a very high contended lock in crypto_authenc_hash() if
the crypto system runs in parallel. To get rid of this, the struct aead is
moved to percpu data, what in turn means that we have percpu IV chains now.
However, I'm not that familiar with the crypto system. So I'm not sure whether
this is acceptable as I did it, this needs review.

I did forwarding tests with two quad core machines (Intel Core 2 Quad Q6600) 
used as IPsec routers (xfrm tunnel between the two quad core machines) and two
notebooks T61 used as traffic generators.
With this testing environment I'm geting a throughput up to 910 Mbit/s (ipv4)
and 880 Mbit/s (ipv6) with aes192-sha1 encryption (measured with iperf,
_one_ tcp stream). Without the parallelization I'm getting with the same
environment about 340 Mbit/s (ipv4) and 320 Mbit/s (ipv6).

If somebody wants to test it, the parallelization is switched off by default.
To enable it, do 'echo 1 > /proc/sys/net/core/xfrm_padata'.

Steffen

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

* [RFC PATCH 1/5] padata: generic interface for parallel processing
  2008-12-01  7:16 [RFC PATCH 0/5] IPsec parallelization Steffen Klassert
@ 2008-12-01  7:17 ` Steffen Klassert
  2008-12-01  7:17 ` [RFC PATCH 2/5] xfrm: add possibility " Steffen Klassert
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01  7:17 UTC (permalink / raw)
  To: netdev; +Cc: davem, herbert, klassert

From: Steffen Klassert <steffen.klassert@secunet.com>

This patch introduces an interface to process data objects
in parallel. On request it is possible to serialize again.
The parallelized objects return in the same order as they
were before the parallelization.

Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>
---
 include/linux/interrupt.h |    1 +
 include/linux/padata.h    |  115 +++++++++++
 kernel/Makefile           |    2 +-
 kernel/padata.c           |  489 +++++++++++++++++++++++++++++++++++++++++++++
 4 files changed, 606 insertions(+), 1 deletions(-)
 create mode 100644 include/linux/padata.h
 create mode 100644 kernel/padata.c

diff --git a/include/linux/interrupt.h b/include/linux/interrupt.h
index f58a0cf..4d2f4bb 100644
--- a/include/linux/interrupt.h
+++ b/include/linux/interrupt.h
@@ -254,6 +254,7 @@ enum
 #ifdef CONFIG_HIGH_RES_TIMERS
 	HRTIMER_SOFTIRQ,
 #endif
+	PADATA_SOFTIRQ,
 	RCU_SOFTIRQ, 	/* Preferable RCU should always be the last softirq */
 
 	NR_SOFTIRQS
diff --git a/include/linux/padata.h b/include/linux/padata.h
new file mode 100644
index 0000000..6447c93
--- /dev/null
+++ b/include/linux/padata.h
@@ -0,0 +1,115 @@
+/*
+ * padata.h - header for the padata parallelization interface
+ *
+ * Copyright (C) 2008 secunet Security Networks AG
+ * Copyright (C) 2008 Steffen Klassert <steffen.klassert@secunet.com>
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms and conditions of the GNU General Public License,
+ * version 2, as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along with
+ * this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin St - Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#ifndef PADATA_H
+#define PADATA_H
+
+#include <linux/interrupt.h>
+#include <linux/smp.h>
+#include <linux/list.h>
+
+enum
+{
+	NO_PADATA=0,
+	NR_PADATA
+};
+
+struct padata_priv {
+	struct list_head	list;
+	struct call_single_data	csd;
+	int			cb_cpu;
+	int			seq_nr;
+	unsigned int		nr;
+	int			info;
+};
+
+struct padata_queue {
+	struct list_head        list;
+	atomic_t                num_obj;
+	int               	cpu_index;
+	spinlock_t              lock;
+};
+
+struct parallel_data {
+	struct work_struct	work;
+	struct padata_queue	*queue;
+	atomic_t		seq_nr;
+	atomic_t		queued_objects;
+	cpumask_t		cpu_map;
+	cpumask_t		new_cpu_map;
+	u8			flags;
+#define	PADATA_INIT			1
+#define	PADATA_FLUSH_HARD		2
+#define	PADATA_RESET_IN_PROGRESS	4
+	void			(*serial)(unsigned long data);
+	spinlock_t              lock;
+};
+
+#ifdef CONFIG_USE_GENERIC_SMP_HELPERS
+extern void __init padata_init(unsigned int nr, cpumask_t cpu_map,
+		void (*serial)(unsigned long data));
+extern void padata_dont_wait(unsigned int nr, struct padata_priv *padata);
+extern int padata_do_parallel(unsigned int softirq_nr, unsigned int nr,
+		struct padata_priv *padata, int cb_cpu);
+extern int padata_do_serial(unsigned int nr, struct padata_priv *padata);
+extern cpumask_t padata_get_cpumap(unsigned int nr);
+extern void padata_set_cpumap(unsigned int nr, cpumask_t cpu_map);
+extern void padata_add_cpu(unsigned int nr, int cpu);
+extern void padata_remove_cpu(unsigned int nr, int cpu);
+extern void padata_start(unsigned int nr);
+extern void padata_stop(unsigned int nr);
+#else
+void padata_init(unsigned int nr,cpumask_t cpu_map,
+		void (*serial)(unsigned long data));
+{
+}
+void padata_dont_wait(unsigned int nr, struct padata_priv *padata)
+{
+}
+int padata_do_parallel(unsigned int softirq_nr, unsigned int nr,
+		struct padata_priv *padata, int cb_cpu)
+{
+	return 0;
+}
+int padata_do_serial(unsigned int nr, struct padata_priv *padata)
+{
+	return 0;
+}
+cpumask_t padata_get_cpumap(unsigned int nr)
+{
+	return cpu_online_map;
+}
+void padata_set_cpumap(unsigned int nr, cpumask_t cpu_map)
+{
+}
+padata_add_cpu(unsigned int nr, int cpu)
+{
+}
+padata_remove_cpu(unsigned int nr, int cpu)
+{
+}
+padata_start(unsigned int nr)
+{
+}
+padata_stop(unsigned int nr)
+{
+}
+#endif
+#endif
diff --git a/kernel/Makefile b/kernel/Makefile
index 19fad00..730a401 100644
--- a/kernel/Makefile
+++ b/kernel/Makefile
@@ -40,7 +40,7 @@ obj-$(CONFIG_RT_MUTEXES) += rtmutex.o
 obj-$(CONFIG_DEBUG_RT_MUTEXES) += rtmutex-debug.o
 obj-$(CONFIG_RT_MUTEX_TESTER) += rtmutex-tester.o
 obj-$(CONFIG_GENERIC_ISA_DMA) += dma.o
-obj-$(CONFIG_USE_GENERIC_SMP_HELPERS) += smp.o
+obj-$(CONFIG_USE_GENERIC_SMP_HELPERS) += smp.o padata.o
 obj-$(CONFIG_SMP) += spinlock.o
 obj-$(CONFIG_DEBUG_SPINLOCK) += spinlock.o
 obj-$(CONFIG_PROVE_LOCKING) += spinlock.o
diff --git a/kernel/padata.c b/kernel/padata.c
new file mode 100644
index 0000000..20eccbd
--- /dev/null
+++ b/kernel/padata.c
@@ -0,0 +1,489 @@
+/*
+ * padata.c - generic interface to process data streams in parallel
+ *
+ * Copyright (C) 2008 secunet Security Networks AG
+ * Copyright (C) 2008 Steffen Klassert <steffen.klassert@secunet.com>
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms and conditions of the GNU General Public License,
+ * version 2, as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along with
+ * this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin St - Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#include <linux/module.h>
+#include <linux/cpumask.h>
+#include <linux/err.h>
+#include <linux/padata.h>
+
+#define MAX_SEQ_NR  1000000000
+
+static struct parallel_data padata_vec[NR_PADATA];
+static struct padata_priv *padata_get_next(struct parallel_data *par_data);
+
+static void padata_flush_hard(struct parallel_data *par_data)
+{
+	int cpu;
+	struct padata_priv *padata;
+	struct padata_queue *queue;
+
+	for_each_cpu_mask(cpu, par_data->cpu_map) {
+		queue = per_cpu_ptr(par_data->queue, cpu);
+
+		while(!list_empty(&queue->list)) {
+		padata = list_entry(queue->list.next, struct padata_priv, list);
+
+		spin_lock(&queue->lock);
+		list_del_init(&padata->list);
+		spin_unlock(&queue->lock);
+
+		atomic_dec(&par_data->queued_objects);
+		par_data->serial((unsigned long ) padata);
+		}
+	}
+}
+
+static void padata_flush_order(struct parallel_data *par_data)
+{
+	struct padata_priv *padata;
+
+	while (1) {
+		padata = padata_get_next(par_data);
+
+		if (padata && !IS_ERR(padata))
+			par_data->serial((unsigned long ) padata);
+		else
+			break;
+	}
+
+	padata_flush_hard(par_data);
+}
+
+static void padata_reset_work(struct work_struct *work)
+{
+	int cpu, cpu_index;
+	struct padata_queue *queue;
+	struct parallel_data *par_data;
+
+	par_data = container_of(work, struct parallel_data, work);
+
+	if (par_data->flags & (PADATA_INIT|PADATA_RESET_IN_PROGRESS))
+		return;
+
+	spin_lock_bh(&par_data->lock);
+	par_data->flags |= PADATA_RESET_IN_PROGRESS;
+
+	if (!(par_data->flags & PADATA_FLUSH_HARD))
+		padata_flush_order(par_data);
+	else
+		padata_flush_hard(par_data);
+
+	cpu_index = 0;
+
+	par_data->cpu_map = par_data->new_cpu_map;
+
+	for_each_cpu_mask(cpu, par_data->cpu_map) {
+		queue = per_cpu_ptr(par_data->queue, cpu);
+
+		atomic_set(&queue->num_obj, 0);
+		queue->cpu_index = cpu_index;
+		cpu_index++;
+	}
+	spin_unlock_bh(&par_data->lock);
+
+	atomic_set(&par_data->seq_nr, -1);
+	par_data->flags &= ~PADATA_RESET_IN_PROGRESS;
+	par_data->flags |= PADATA_INIT;
+}
+
+static struct padata_priv *padata_get_next(struct parallel_data *par_data)
+{
+	int cpu, num_cpus, empty;
+	int seq_nr, calc_seq_nr, next_nr;
+	struct padata_queue *queue, *next_queue;
+	struct padata_priv *padata;
+
+	empty = 0;
+	next_nr = -1;
+	next_queue = NULL;
+
+	num_cpus = cpus_weight(par_data->cpu_map);
+
+	for_each_cpu_mask(cpu, par_data->cpu_map) {
+		queue = per_cpu_ptr(par_data->queue, cpu);
+
+		/*
+		 * Calculate the seq_nr of the object that should be
+		 * next in this queue.
+		 */
+		calc_seq_nr = (atomic_read(&queue->num_obj) * num_cpus)
+						+ queue->cpu_index;
+
+		if (!list_empty(&queue->list)) {
+			padata = list_entry(queue->list.next,
+					struct padata_priv, list);
+
+			seq_nr  = padata->seq_nr;
+
+			if (unlikely(calc_seq_nr != seq_nr)) {
+				par_data->flags &= ~PADATA_INIT;
+				par_data->flags |= PADATA_FLUSH_HARD;
+				padata = NULL;
+				goto out;
+			}
+		} else {
+			seq_nr = calc_seq_nr;
+			empty++;
+		}
+
+		if (next_nr < 0 || seq_nr < next_nr) {
+			next_nr = seq_nr;
+			next_queue = queue;
+		}
+	}
+
+	padata = NULL;
+
+	if (empty == num_cpus)
+		goto out;
+
+	if (!list_empty(&next_queue->list)) {
+		padata = list_entry(next_queue->list.next,
+				struct padata_priv, list);
+
+		spin_lock(&next_queue->lock);
+		list_del_init(&padata->list);
+		spin_unlock(&next_queue->lock);
+
+		atomic_dec(&par_data->queued_objects);
+		atomic_inc(&next_queue->num_obj);
+
+		goto out;
+	}
+
+	if (next_nr % num_cpus == next_queue->cpu_index) {
+		padata = ERR_PTR(-ENODATA);
+		goto out;
+	}
+
+	padata = ERR_PTR(-EINPROGRESS);
+out:
+	return padata;
+}
+
+static void padata_action(struct softirq_action *h)
+{
+	struct list_head *cpu_list, local_list;
+
+	cpu_list = &__get_cpu_var(softirq_work_list[PADATA_SOFTIRQ]);
+
+	local_irq_disable();
+	list_replace_init(cpu_list, &local_list);
+	local_irq_enable();
+
+	while (!list_empty(&local_list)) {
+		struct padata_priv *padata;
+
+		padata = list_entry(local_list.next,
+				struct padata_priv, csd.list);
+
+		list_del_init(&padata->csd.list);
+
+		padata_vec[padata->nr].serial((unsigned long ) padata);
+	}
+}
+
+static int padata_cpu_hash(unsigned int nr, struct padata_priv *padata)
+{
+	int cpu, target_cpu, this_cpu, cpu_index;
+
+	this_cpu = smp_processor_id();
+
+	if (padata->nr != 0)
+		return this_cpu;
+
+	if (!(padata_vec[nr].flags & PADATA_INIT))
+		return this_cpu;
+
+	padata->seq_nr = atomic_inc_return(&padata_vec[nr].seq_nr);
+
+	if (padata->seq_nr > MAX_SEQ_NR) {
+		padata_vec[nr].flags &= ~PADATA_INIT;
+		padata->seq_nr = 0;
+		schedule_work(&padata_vec[nr].work);
+		return this_cpu;
+	}
+
+	padata->nr = nr;
+
+	/*
+	 * Hash the sequence numbers to the cpus by taking
+	 * seq_nr mod. number of cpus in use.
+	 */
+	cpu_index =  padata->seq_nr % cpus_weight(padata_vec[nr].cpu_map);
+
+	target_cpu = first_cpu(padata_vec[nr].cpu_map);
+	for (cpu = 0; cpu < cpu_index; cpu++)
+		target_cpu = next_cpu(target_cpu, padata_vec[nr].cpu_map);
+
+	return target_cpu;
+}
+
+/*
+ * padata_dont_wait - must be called if an object that runs in parallel will
+ * not be serialized with padata_do_serial
+ *
+ * @nr: number of the padata istance
+ * @padata: object that will not be seen by padata_do_serial
+ */
+void padata_dont_wait(unsigned int nr, struct padata_priv *padata)
+{
+	struct padata_queue *queue;
+
+	if (!(padata_vec[nr].flags & PADATA_INIT))
+		return;
+
+	if (padata->nr == 0 || padata->nr != nr)
+		return;
+
+	queue = per_cpu_ptr(padata_vec[nr].queue, smp_processor_id());
+	atomic_inc(&queue->num_obj);
+
+	padata->nr = 0;
+	padata->seq_nr = 0;
+}
+EXPORT_SYMBOL(padata_dont_wait);
+
+/*
+ * padata_do_parallel - padata parallelization function
+ *
+ * @softirq_nr: number of the softirq that will do the parallelization
+ * @nr: number of the padata istance
+ * @padata: object to be paralellized
+ * @cb_cpu: cpu number on which the serialization callback function will run
+ */
+int padata_do_parallel(unsigned int softirq_nr, unsigned int nr,
+			struct padata_priv *padata, int cb_cpu)
+{
+	int target_cpu;
+
+	padata->cb_cpu = cb_cpu;
+
+	local_bh_disable();
+	target_cpu = padata_cpu_hash(nr, padata);
+	local_bh_enable();
+
+	send_remote_softirq(&padata->csd, target_cpu, softirq_nr);
+
+	return 1;
+}
+EXPORT_SYMBOL(padata_do_parallel);
+
+/*
+ * padata_do_serial - padata serialization function
+ *
+ * @nr: number of the padata istance
+ * @padata: object to be serialized
+ *
+ * returns 1 if the serialization callback function will be called
+ * from padata, 0 else
+ */
+int padata_do_serial(unsigned int nr, struct padata_priv *padata)
+{
+	int cpu;
+	struct padata_queue *reorder_queue;
+
+	if (!(padata_vec[nr].flags & PADATA_INIT))
+		return 0;
+
+	if (padata->nr != nr || padata->nr == 0) {
+		padata_vec[nr].serial((unsigned long ) padata);
+		return 1;
+	}
+
+	cpu = smp_processor_id();
+
+	reorder_queue = per_cpu_ptr(padata_vec[nr].queue, cpu);
+
+	spin_lock(&reorder_queue->lock);
+	list_add_tail(&padata->list, &reorder_queue->list);
+	spin_unlock(&reorder_queue->lock);
+
+	atomic_inc(&padata_vec[nr].queued_objects);
+
+try_again:
+	if (!spin_trylock(&padata_vec[nr].lock))
+		goto out;
+
+	while(1) {
+		padata = padata_get_next(&padata_vec[nr]);
+
+		if (!padata || PTR_ERR(padata) == -EINPROGRESS)
+			break;
+		if (PTR_ERR(padata) == -ENODATA) {
+			spin_unlock(&padata_vec[nr].lock);
+			goto out;
+		}
+
+		send_remote_softirq(&padata->csd, padata->cb_cpu,
+				PADATA_SOFTIRQ);
+	}
+
+	if (unlikely(!(padata_vec[nr].flags & PADATA_INIT))) {
+		spin_unlock(&padata_vec[nr].lock);
+		goto reset_out;
+	}
+
+	spin_unlock(&padata_vec[nr].lock);
+
+	if (atomic_read(&padata_vec[nr].queued_objects))
+		goto try_again;
+
+out:
+	return 1;
+reset_out:
+	schedule_work(&padata_vec[nr].work);
+	return 1;
+}
+EXPORT_SYMBOL(padata_do_serial);
+
+/*
+ * padata_get_cpumap - get the cpu map that is actually in use
+ *
+ * @nr: number of the padata istance
+ */
+cpumask_t padata_get_cpumap(unsigned int nr)
+{
+	return padata_vec[nr].cpu_map;
+}
+EXPORT_SYMBOL(padata_get_cpumap);
+
+/*
+ * padata_set_cpumap - set the cpu map that padata uses
+ *
+ * @nr: number of the padata istance
+ * @cpu_map: the cpu map to use
+ */
+void padata_set_cpumap(unsigned int nr, cpumask_t cpu_map)
+{
+	padata_vec[nr].new_cpu_map = cpu_map;
+	padata_vec[nr].flags &= ~PADATA_INIT;
+	padata_vec[nr].flags |= PADATA_FLUSH_HARD;
+
+	schedule_work(&padata_vec[nr].work);
+}
+EXPORT_SYMBOL(padata_set_cpumap);
+
+/*
+ * padata_add_cpu - add a cpu to the padata cpu map
+ *
+ * @nr: number of the padata istance
+ * @cpu: cpu to remove
+ */
+void padata_add_cpu(unsigned int nr, int cpu)
+{
+	cpumask_t cpu_map = padata_vec[nr].cpu_map;
+
+	cpu_set(cpu, cpu_map);
+	padata_set_cpumap(nr, cpu_map);
+}
+EXPORT_SYMBOL(padata_add_cpu);
+
+/*
+ * padata_remove_cpu - remove a cpu from the padata cpu map
+ *
+ * @nr: number of the padata istance
+ * @cpu: cpu to remove
+ */
+void padata_remove_cpu(unsigned int nr, int cpu)
+{
+	cpumask_t cpu_map = padata_vec[nr].cpu_map;
+
+	cpu_clear(cpu, cpu_map);
+	padata_set_cpumap(nr, cpu_map);
+}
+EXPORT_SYMBOL(padata_remove_cpu);
+
+/*
+ * padata_start - start the parallel processing
+ *
+ * @nr: number of the padata istance
+ */
+void padata_start(unsigned int nr)
+{
+	if (padata_vec[nr].flags & PADATA_INIT)
+		return;
+
+	schedule_work(&padata_vec[nr].work);
+}
+EXPORT_SYMBOL(padata_start);
+
+/*
+ * padata_stop - stop the parallel processing
+ *
+ * @nr: number of the padata istance
+ */
+void padata_stop(unsigned int nr)
+{
+	padata_vec[nr].flags &= ~PADATA_INIT;
+}
+EXPORT_SYMBOL(padata_stop);
+
+/*
+ * padata_init - initialize a padata instance
+ *
+ * @nr: number of the padata istance
+ * @cpu_map: map of the cpu set that padata uses for parallelization
+ * @serial: the serialization callback function
+ *
+ * The serialization callback function must be able to run in softirq context.
+ */
+void __init padata_init(unsigned int nr, cpumask_t cpu_map,
+			void (*serial)(unsigned long data))
+{
+	int cpu, cpu_index;
+	struct padata_queue *percpu_queue, *queue;
+
+	percpu_queue = alloc_percpu(struct padata_queue);
+
+	if (!percpu_queue) {
+		printk("padata_init: Failed to alloc the serialization"
+				"queues for padata nr %d, exiting!\n", nr);
+		return;
+	}
+
+	open_softirq(PADATA_SOFTIRQ, padata_action);
+
+	cpu_index = 0;
+
+	for_each_possible_cpu(cpu) {
+		queue = per_cpu_ptr(percpu_queue, cpu);
+
+		if (cpu_isset(cpu, cpu_map)) {
+			queue->cpu_index = cpu_index;
+			cpu_index++;
+		}
+
+		INIT_LIST_HEAD(&queue->list);
+		spin_lock_init(&queue->lock);
+		atomic_set(&queue->num_obj, 0);
+	}
+
+	INIT_WORK(&padata_vec[nr].work, padata_reset_work);
+
+	atomic_set(&padata_vec[nr].seq_nr, -1);
+	atomic_set(&padata_vec[nr].queued_objects, 0);
+	padata_vec[nr].cpu_map = cpu_map;
+	padata_vec[nr].new_cpu_map = cpu_map;
+	padata_vec[nr].queue = percpu_queue;
+	padata_vec[nr].serial = serial;
+	padata_vec[nr].flags = 0;
+}
+EXPORT_SYMBOL(padata_init);
-- 
1.5.4.2


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

* [RFC PATCH 2/5] xfrm: add possibility for parallel processing
  2008-12-01  7:16 [RFC PATCH 0/5] IPsec parallelization Steffen Klassert
  2008-12-01  7:17 ` [RFC PATCH 1/5] padata: generic interface for parallel processing Steffen Klassert
@ 2008-12-01  7:17 ` Steffen Klassert
  2008-12-01  7:19 ` [RFC PATCH 3/5] crypto: add possibility to force sync transform Steffen Klassert
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01  7:17 UTC (permalink / raw)
  To: netdev; +Cc: davem, herbert, klassert

From: Steffen Klassert <steffen.klassert@secunet.com>

This patch uses the padata parallelization interface to run the expensive
parts of xfrm in parallel.

Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>
---
 include/linux/crypto.h     |    1 +
 include/linux/interrupt.h  |    2 +
 include/linux/padata.h     |    2 +
 include/linux/skbuff.h     |    5 +
 include/linux/sysctl.h     |    3 +-
 include/net/xfrm.h         |   38 ++++++
 kernel/sysctl_check.c      |    1 +
 net/core/skbuff.c          |    3 +
 net/core/sysctl_net_core.c |   11 ++
 net/xfrm/Kconfig           |    8 ++
 net/xfrm/Makefile          |    2 +-
 net/xfrm/xfrm_input.c      |    7 +-
 net/xfrm/xfrm_output.c     |    5 +
 net/xfrm/xfrm_padata.c     |  270 ++++++++++++++++++++++++++++++++++++++++++++
 net/xfrm/xfrm_policy.c     |    2 +
 15 files changed, 357 insertions(+), 3 deletions(-)
 create mode 100644 net/xfrm/xfrm_padata.c

diff --git a/include/linux/crypto.h b/include/linux/crypto.h
index 3d2317e..d5dd094 100644
--- a/include/linux/crypto.h
+++ b/include/linux/crypto.h
@@ -78,6 +78,7 @@
 #define CRYPTO_TFM_REQ_WEAK_KEY		0x00000100
 #define CRYPTO_TFM_REQ_MAY_SLEEP	0x00000200
 #define CRYPTO_TFM_REQ_MAY_BACKLOG	0x00000400
+#define CRYPTO_TFM_REQ_FORCE_SYNC	0x00000800
 #define CRYPTO_TFM_RES_WEAK_KEY		0x00100000
 #define CRYPTO_TFM_RES_BAD_KEY_LEN   	0x00200000
 #define CRYPTO_TFM_RES_BAD_KEY_SCHED 	0x00400000
diff --git a/include/linux/interrupt.h b/include/linux/interrupt.h
index 4d2f4bb..02b7fba 100644
--- a/include/linux/interrupt.h
+++ b/include/linux/interrupt.h
@@ -248,6 +248,8 @@ enum
 	TIMER_SOFTIRQ,
 	NET_TX_SOFTIRQ,
 	NET_RX_SOFTIRQ,
+	XFRM_INPUT_SOFTIRQ,
+	XFRM_OUTPUT_SOFTIRQ,
 	BLOCK_SOFTIRQ,
 	TASKLET_SOFTIRQ,
 	SCHED_SOFTIRQ,
diff --git a/include/linux/padata.h b/include/linux/padata.h
index 6447c93..786ec44 100644
--- a/include/linux/padata.h
+++ b/include/linux/padata.h
@@ -28,6 +28,8 @@
 enum
 {
 	NO_PADATA=0,
+	XFRM_INPUT_PADATA,
+	XFRM_OUTPUT_PADATA,
 	NR_PADATA
 };
 
diff --git a/include/linux/skbuff.h b/include/linux/skbuff.h
index 2725f4e..a5c9986 100644
--- a/include/linux/skbuff.h
+++ b/include/linux/skbuff.h
@@ -18,6 +18,7 @@
 #include <linux/compiler.h>
 #include <linux/time.h>
 #include <linux/cache.h>
+#include <linux/padata.h>
 
 #include <asm/atomic.h>
 #include <asm/types.h>
@@ -261,6 +262,10 @@ struct sk_buff {
 	struct sk_buff		*next;
 	struct sk_buff		*prev;
 
+#ifdef CONFIG_XFRM_PADATA
+	struct padata_priv       padata;
+#endif
+
 	struct sock		*sk;
 	ktime_t			tstamp;
 	struct net_device	*dev;
diff --git a/include/linux/sysctl.h b/include/linux/sysctl.h
index 39d471d..fd86b44 100644
--- a/include/linux/sysctl.h
+++ b/include/linux/sysctl.h
@@ -284,7 +284,8 @@ enum
 	NET_CORE_BUDGET=19,
 	NET_CORE_AEVENT_ETIME=20,
 	NET_CORE_AEVENT_RSEQTH=21,
-	NET_CORE_WARNINGS=22,
+	NET_CORE_PADATA=22,
+	NET_CORE_WARNINGS=23,
 };
 
 /* /proc/sys/net/ethernet */
diff --git a/include/net/xfrm.h b/include/net/xfrm.h
index 11c890a..ee0ae79 100644
--- a/include/net/xfrm.h
+++ b/include/net/xfrm.h
@@ -12,6 +12,7 @@
 #include <linux/in6.h>
 #include <linux/mutex.h>
 #include <linux/audit.h>
+#include <linux/crypto.h>
 
 #include <net/sock.h>
 #include <net/dst.h>
@@ -741,6 +742,43 @@ static inline void xfrm_pols_put(struct xfrm_policy **pols, int npols)
 }
 #endif
 
+#ifdef CONFIG_XFRM_PADATA
+extern u32 xfrm_padata_conf;
+extern int xfrm_do_parallel_input(struct sk_buff *skb);
+extern int xfrm_do_parallel_output(struct sk_buff *skb);
+extern void xfrm_init_padata(void);
+extern int xfrm_padata_strategy(ctl_table *ctl, void __user *oldval,
+		size_t __user *oldlenp, void __user *newval, size_t newlen);
+extern int xfrm_padata_sysctl(struct ctl_table *ctrl, int write,
+				struct file* filp, void __user *buffer,
+				size_t *lenp, loff_t *ppos);
+static inline u32 xfrm_aead_set_flags(struct sk_buff *skb, u32 flags)
+{
+	if (skb->padata.nr == XFRM_OUTPUT_PADATA ||
+			skb->padata.nr == XFRM_INPUT_PADATA)
+
+		flags |= CRYPTO_TFM_REQ_FORCE_SYNC;
+
+	return flags;
+}
+#else
+static inline int xfrm_do_parallel_input(struct sk_buff *skb)
+{
+	return 0;
+}
+static inline int xfrm_do_parallel_output(struct sk_buff *skb)
+{
+	return 0;
+}
+static inline void xfrm_init_padata(void)
+{
+}
+static inline u32 xfrm_aead_set_flags(struct sk_buff *skb, u32 flags)
+{
+	return 0;
+}
+#endif
+
 extern void __xfrm_state_destroy(struct xfrm_state *);
 
 static inline void __xfrm_state_put(struct xfrm_state *x)
diff --git a/kernel/sysctl_check.c b/kernel/sysctl_check.c
index c35da23..011f74e 100644
--- a/kernel/sysctl_check.c
+++ b/kernel/sysctl_check.c
@@ -161,6 +161,7 @@ static const struct trans_ctl_table trans_net_core_table[] = {
 	{ NET_CORE_BUDGET,		"netdev_budget" },
 	{ NET_CORE_AEVENT_ETIME,	"xfrm_aevent_etime" },
 	{ NET_CORE_AEVENT_RSEQTH,	"xfrm_aevent_rseqth" },
+	{ NET_CORE_PADATA,		"xfrm_padata" },
 	{ NET_CORE_WARNINGS,		"warnings" },
 	{},
 };
diff --git a/net/core/skbuff.c b/net/core/skbuff.c
index d49ef83..6c8c86d 100644
--- a/net/core/skbuff.c
+++ b/net/core/skbuff.c
@@ -495,6 +495,9 @@ EXPORT_SYMBOL(skb_recycle_check);
 
 static void __copy_skb_header(struct sk_buff *new, const struct sk_buff *old)
 {
+#ifdef CONFIG_XFRM_PADATA
+	memset(&new->padata, 0, sizeof(struct padata_priv));
+#endif
 	new->tstamp		= old->tstamp;
 	new->dev		= old->dev;
 	new->transport_header	= old->transport_header;
diff --git a/net/core/sysctl_net_core.c b/net/core/sysctl_net_core.c
index f686467..7688916 100644
--- a/net/core/sysctl_net_core.c
+++ b/net/core/sysctl_net_core.c
@@ -122,6 +122,17 @@ static struct ctl_table net_core_table[] = {
 		.mode		= 0644,
 		.proc_handler	= &proc_dointvec
 	},
+#ifdef CONFIG_XFRM_PADATA
+	{
+		.ctl_name	= NET_CORE_PADATA,
+		.procname	= "xfrm_padata",
+		.data		= &xfrm_padata_conf,
+		.maxlen		= sizeof(u32),
+		.mode		= 0644,
+		.proc_handler	= &xfrm_padata_sysctl,
+		.strategy       = &xfrm_padata_strategy,
+	},
+#endif /* CONFIG_XFRM_PADATA */
 #endif /* CONFIG_XFRM */
 #endif /* CONFIG_NET */
 	{
diff --git a/net/xfrm/Kconfig b/net/xfrm/Kconfig
index 6d08167..ba509e0 100644
--- a/net/xfrm/Kconfig
+++ b/net/xfrm/Kconfig
@@ -46,6 +46,14 @@ config XFRM_STATISTICS
 
 	  If unsure, say N.
 
+config XFRM_PADATA
+	bool "Transformation parallel processing (EXPERIMENTAL)"
+	depends on INET && XFRM && USE_GENERIC_SMP_HELPERS && EXPERIMENTAL
+	---help---
+	  Support parallel processing of the expencive parts of IPsec.
+
+	  If unsure, say N.
+
 config XFRM_IPCOMP
 	tristate
 	select XFRM
diff --git a/net/xfrm/Makefile b/net/xfrm/Makefile
index 0f439a7..09f3f35 100644
--- a/net/xfrm/Makefile
+++ b/net/xfrm/Makefile
@@ -7,4 +7,4 @@ obj-$(CONFIG_XFRM) := xfrm_policy.o xfrm_state.o xfrm_hash.o \
 obj-$(CONFIG_XFRM_STATISTICS) += xfrm_proc.o
 obj-$(CONFIG_XFRM_USER) += xfrm_user.o
 obj-$(CONFIG_XFRM_IPCOMP) += xfrm_ipcomp.o
-
+obj-$(CONFIG_XFRM_PADATA) += xfrm_padata.o
diff --git a/net/xfrm/xfrm_input.c b/net/xfrm/xfrm_input.c
index 7527940..28126cd 100644
--- a/net/xfrm/xfrm_input.c
+++ b/net/xfrm/xfrm_input.c
@@ -115,7 +115,8 @@ int xfrm_input(struct sk_buff *skb, int nexthdr, __be32 spi, int encap_type)
 
 	/* A negative encap_type indicates async resumption. */
 	if (encap_type < 0) {
-		async = 1;
+		if (encap_type == -1)
+			async = 1;
 		x = xfrm_input_state(skb);
 		seq = XFRM_SKB_CB(skb)->seq.input;
 		goto resume;
@@ -185,6 +186,10 @@ int xfrm_input(struct sk_buff *skb, int nexthdr, __be32 spi, int encap_type)
 
 		XFRM_SKB_CB(skb)->seq.input = seq;
 
+
+		if (xfrm_do_parallel_input(skb))
+			return 0;
+
 		nexthdr = x->type->input(x, skb);
 
 		if (nexthdr == -EINPROGRESS)
diff --git a/net/xfrm/xfrm_output.c b/net/xfrm/xfrm_output.c
index dc50f1e..1fb134b 100644
--- a/net/xfrm/xfrm_output.c
+++ b/net/xfrm/xfrm_output.c
@@ -83,6 +83,11 @@ static int xfrm_output_one(struct sk_buff *skb, int err)
 
 		spin_unlock_bh(&x->lock);
 
+		if (xfrm_do_parallel_output(skb)) {
+			err = -EINPROGRESS;
+			goto out_exit;
+		}
+
 		err = x->type->output(x, skb);
 		if (err == -EINPROGRESS)
 			goto out_exit;
diff --git a/net/xfrm/xfrm_padata.c b/net/xfrm/xfrm_padata.c
new file mode 100644
index 0000000..4cbc95c
--- /dev/null
+++ b/net/xfrm/xfrm_padata.c
@@ -0,0 +1,270 @@
+/*
+ * xfrm_padata.c - IPsec parallelization code
+ *
+ * Copyright (C) 2008 secunet Security Networks AG
+ * Copyright (C) 2008 Steffen Klassert <steffen.klassert@secunet.com>
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms and conditions of the GNU General Public License,
+ * version 2, as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along with
+ * this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin St - Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#include<linux/padata.h>
+#include <linux/skbuff.h>
+#include <linux/interrupt.h>
+#include <net/xfrm.h>
+#include <linux/cpu.h>
+
+u32 xfrm_padata_conf __read_mostly = 0;
+
+int xfrm_padata_strategy(ctl_table *ctl, void __user *oldval,
+		size_t __user *oldlenp, void __user *newval, size_t newlen)
+{
+	int new;
+
+	if (!newval || !newlen)
+		return 0;
+
+	if (newlen != sizeof(u32))
+		return -EINVAL;
+
+	if (get_user(new, (int __user *)newval))
+		return -EFAULT;
+
+	if (new < 0 || new > 1)
+		return -EINVAL;
+
+	return 1;
+}
+
+int xfrm_padata_sysctl(struct ctl_table *ctl, int write,
+				struct file* filp, void __user *buffer,
+				size_t *lenp, loff_t *ppos)
+{
+	int old_val = xfrm_padata_conf;
+	int ret = proc_dointvec(ctl, write, filp, buffer, lenp, ppos);
+
+	if (write &&  xfrm_padata_conf != old_val) {
+		if (xfrm_padata_conf == 0){
+			padata_stop(XFRM_INPUT_PADATA);
+			padata_stop(XFRM_OUTPUT_PADATA);
+
+		} else {
+			padata_start(XFRM_INPUT_PADATA);
+			padata_start(XFRM_OUTPUT_PADATA);
+		}
+	}
+	return ret;
+}
+
+static void xfrm_input_callback(unsigned long data)
+{
+	struct sk_buff *skb;
+	struct padata_priv *padata = (void *) data;
+
+	skb = container_of(padata, struct sk_buff, padata);
+
+	xfrm_input(skb, skb->padata.info, 0 , -501);
+}
+
+static void xfrm_output_callback(unsigned long data)
+{
+	struct sk_buff *skb;
+	struct padata_priv *padata = (void *) data;
+
+	skb = container_of(padata, struct sk_buff, padata);
+
+	xfrm_output_resume(skb, skb->padata.info);
+}
+
+static void xfrm_input_action(struct softirq_action *h)
+{
+	struct xfrm_state *x;
+	struct list_head *cpu_list, local_list;
+
+	cpu_list = &__get_cpu_var(softirq_work_list[XFRM_INPUT_SOFTIRQ]);
+
+	local_irq_disable();
+	list_replace_init(cpu_list, &local_list);
+	local_irq_enable();
+
+	while (!list_empty(&local_list)) {
+		struct padata_priv *padata;
+		struct sk_buff *skb;
+
+		padata = list_entry(local_list.next, struct padata_priv,
+				csd.list);
+
+		list_del_init(&padata->csd.list);
+		skb = container_of(padata, struct sk_buff, padata);
+
+		x = xfrm_input_state(skb);
+		padata->info = x->type->input(x, skb);
+		if (padata->info == -EINPROGRESS) {
+			padata_dont_wait(XFRM_INPUT_PADATA, padata);
+			continue;
+		}
+		if (padata_do_serial(XFRM_INPUT_PADATA, padata))
+			continue;
+
+		xfrm_input(skb, padata->info, 0 , -1);
+	}
+}
+
+static void xfrm_output_action(struct softirq_action *h)
+{
+	struct list_head *cpu_list, local_list;
+
+	cpu_list = &__get_cpu_var(softirq_work_list[XFRM_OUTPUT_SOFTIRQ]);
+
+	local_irq_disable();
+	list_replace_init(cpu_list, &local_list);
+	local_irq_enable();
+
+	while (!list_empty(&local_list)) {
+		struct padata_priv *padata;
+		struct sk_buff *skb;
+		struct xfrm_state *x;
+
+		padata = list_entry(local_list.next, struct padata_priv,
+				csd.list);
+
+		list_del_init(&padata->csd.list);
+		skb = container_of(padata, struct sk_buff, padata);
+
+		x = skb->dst->xfrm;
+		padata->info = x->type->output(x, skb);
+		if (padata->info == -EINPROGRESS) {
+			padata_dont_wait(XFRM_OUTPUT_PADATA, padata);
+			continue;
+		}
+		if (padata_do_serial(XFRM_OUTPUT_PADATA, padata))
+			continue;
+
+		xfrm_output_resume(skb, padata->info);
+	}
+}
+
+static u32 simple_hashrnd;
+static int simple_hashrnd_initialized = 0;
+
+/* Borrowed from simple_tx_hash() */
+u16 xfrm_state_cpu_hash(struct xfrm_state *x, __be16 protocol, int num_cpus)
+{
+	u32 daddr, spi, proto;
+	u32 hash;
+
+	if (unlikely(!simple_hashrnd_initialized)) {
+		get_random_bytes(&simple_hashrnd, 4);
+		simple_hashrnd_initialized = 1;
+	}
+
+
+	switch (protocol) {
+	case __constant_htons(ETH_P_IP):
+
+		daddr = x->id.daddr.a4;
+		spi = x->id.spi;
+		proto = x->id.proto;
+		break;
+	case __constant_htons(ETH_P_IPV6):
+
+		daddr = x->id.daddr.a6[3];
+		spi = x->id.spi;
+		proto = x->id.proto;
+		break;
+	default:
+		return 0;
+	}
+
+	hash = jhash_3words(daddr, spi, proto, simple_hashrnd);
+
+	return (u16) (((u64) hash * num_cpus) >> 32);
+}
+
+int xfrm_do_parallel_input(struct sk_buff *skb)
+{
+	unsigned int cpu, cpu_index, num_cpus, callback_cpu;
+	struct xfrm_state *x;
+	cpumask_t cpu_map;
+
+	cpu_map = padata_get_cpumap(XFRM_INPUT_PADATA);
+	num_cpus = cpus_weight(cpu_map);
+
+	x = xfrm_input_state(skb);
+	cpu_index = xfrm_state_cpu_hash(x, skb->protocol, num_cpus);
+
+	callback_cpu = first_cpu(cpu_map);
+	for (cpu = 0; cpu < cpu_index; cpu++)
+		callback_cpu = next_cpu(callback_cpu, cpu_map);
+
+	return padata_do_parallel(XFRM_INPUT_SOFTIRQ, XFRM_INPUT_PADATA,
+						&skb->padata, callback_cpu);
+}
+
+int xfrm_do_parallel_output(struct sk_buff *skb)
+{
+	int ret;
+	unsigned int cpu, cpu_index, num_cpus, callback_cpu;
+	struct xfrm_state *x;
+	cpumask_t cpu_map;
+
+	cpu_map = padata_get_cpumap(XFRM_OUTPUT_PADATA);
+	num_cpus = cpus_weight(cpu_map);
+
+	x = skb->dst->xfrm;
+	cpu_index = xfrm_state_cpu_hash(x, skb->protocol, num_cpus);
+
+	callback_cpu = first_cpu(cpu_map);
+	for (cpu = 0; cpu < cpu_index; cpu++)
+		callback_cpu = next_cpu(callback_cpu, cpu_map);
+
+	local_bh_disable();
+	ret = padata_do_parallel(XFRM_OUTPUT_SOFTIRQ, XFRM_OUTPUT_PADATA,
+						&skb->padata, callback_cpu);
+	local_bh_enable();
+
+	return ret;
+}
+
+static int __devinit xfrm_cpu_callback(struct notifier_block *nfb,
+				unsigned long action, void *hcpu)
+{
+	int cpu = (unsigned long)hcpu;
+
+	switch (action) {
+		case CPU_ONLINE:
+		case CPU_ONLINE_FROZEN:
+			padata_add_cpu(XFRM_INPUT_PADATA, cpu);
+			padata_add_cpu(XFRM_OUTPUT_PADATA, cpu);
+			break;
+
+		case CPU_DEAD:
+		case CPU_DEAD_FROZEN:
+			padata_remove_cpu(XFRM_INPUT_PADATA, cpu);
+			padata_remove_cpu(XFRM_OUTPUT_PADATA, cpu);
+			break;
+	}
+
+	return NOTIFY_OK;
+}
+
+void __init xfrm_init_padata(void)
+{
+	open_softirq(XFRM_INPUT_SOFTIRQ, xfrm_input_action);
+	open_softirq(XFRM_OUTPUT_SOFTIRQ, xfrm_output_action);
+
+	padata_init(XFRM_INPUT_PADATA, cpu_online_map, xfrm_input_callback);
+	padata_init(XFRM_OUTPUT_PADATA, cpu_online_map, xfrm_output_callback);
+
+	hotcpu_notifier(xfrm_cpu_callback, 0);
+}
diff --git a/net/xfrm/xfrm_policy.c b/net/xfrm/xfrm_policy.c
index 058f04f..41d3670 100644
--- a/net/xfrm/xfrm_policy.c
+++ b/net/xfrm/xfrm_policy.c
@@ -2433,6 +2433,8 @@ static void __init xfrm_policy_init(void)
 
 void __init xfrm_init(void)
 {
+	xfrm_init_padata();
+
 #ifdef CONFIG_XFRM_STATISTICS
 	xfrm_statistics_init();
 #endif
-- 
1.5.4.2


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

* [RFC PATCH 3/5] crypto: add possibility to force sync transform
  2008-12-01  7:16 [RFC PATCH 0/5] IPsec parallelization Steffen Klassert
  2008-12-01  7:17 ` [RFC PATCH 1/5] padata: generic interface for parallel processing Steffen Klassert
  2008-12-01  7:17 ` [RFC PATCH 2/5] xfrm: add possibility " Steffen Klassert
@ 2008-12-01  7:19 ` Steffen Klassert
  2008-12-01 11:22   ` Herbert Xu
  2008-12-01  7:19 ` [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms Steffen Klassert
                   ` (2 subsequent siblings)
  5 siblings, 1 reply; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01  7:19 UTC (permalink / raw)
  To: netdev; +Cc: davem, herbert, klassert

From: Steffen Klassert <steffen.klassert@secunet.com>

If IPsec is parallelized by xfrm_padata, async transform
is not allowed. So Add a possibility to force sync
transform in this case.

Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>
---
 crypto/chainiv.c |   14 ++++++++++++--
 net/ipv4/esp4.c  |   12 ++++++++++--
 net/ipv6/esp6.c  |   12 ++++++++++--
 3 files changed, 32 insertions(+), 6 deletions(-)

diff --git a/crypto/chainiv.c b/crypto/chainiv.c
index 7c37a49..2079fdd 100644
--- a/crypto/chainiv.c
+++ b/crypto/chainiv.c
@@ -168,8 +168,12 @@ static int async_chainiv_givencrypt_tail(struct skcipher_givcrypt_request *req)
 	memcpy(subreq->info, ctx->iv, ivsize);
 
 	ctx->err = crypto_ablkcipher_encrypt(subreq);
-	if (ctx->err)
-		goto out;
+	if (ctx->err) {
+		if (ablkcipher_request_flags(&req->creq) & CRYPTO_TFM_REQ_FORCE_SYNC)
+			return ctx->err;
+		else
+			goto out;
+	}
 
 	memcpy(ctx->iv, subreq->info, ivsize);
 
@@ -190,6 +194,11 @@ static int async_chainiv_givencrypt(struct skcipher_givcrypt_request *req)
 	ablkcipher_request_set_crypt(subreq, req->creq.src, req->creq.dst,
 				     req->creq.nbytes, req->creq.info);
 
+	 if (ablkcipher_request_flags(&req->creq) & CRYPTO_TFM_REQ_FORCE_SYNC) {
+		 clear_bit(CHAINIV_STATE_INUSE, &ctx->state);
+		 goto sync_out;
+	 }
+
 	if (test_and_set_bit(CHAINIV_STATE_INUSE, &ctx->state))
 		goto postpone;
 
@@ -198,6 +207,7 @@ static int async_chainiv_givencrypt(struct skcipher_givcrypt_request *req)
 		goto postpone;
 	}
 
+sync_out:
 	return async_chainiv_givencrypt_tail(req);
 
 postpone:
diff --git a/net/ipv4/esp4.c b/net/ipv4/esp4.c
index 21515d4..3ea3fb2 100644
--- a/net/ipv4/esp4.c
+++ b/net/ipv4/esp4.c
@@ -114,6 +114,7 @@ static int esp_output(struct xfrm_state *x, struct sk_buff *skb)
 	void *tmp;
 	u8 *iv;
 	u8 *tail;
+	u32 flags;
 	int blksize;
 	int clen;
 	int alen;
@@ -123,6 +124,8 @@ static int esp_output(struct xfrm_state *x, struct sk_buff *skb)
 
 	err = -ENOMEM;
 
+	flags = 0;
+
 	/* Round to block size */
 	clen = skb->len;
 
@@ -207,7 +210,9 @@ static int esp_output(struct xfrm_state *x, struct sk_buff *skb)
 		     clen + alen);
 	sg_init_one(asg, esph, sizeof(*esph));
 
-	aead_givcrypt_set_callback(req, 0, esp_output_done, skb);
+	xfrm_aead_set_flags(skb, flags);
+
+	aead_givcrypt_set_callback(req, flags, esp_output_done, skb);
 	aead_givcrypt_set_crypt(req, sg, sg, clen, iv);
 	aead_givcrypt_set_assoc(req, asg, sizeof(*esph));
 	aead_givcrypt_set_giv(req, esph->enc_data,
@@ -332,6 +337,7 @@ static int esp_input(struct xfrm_state *x, struct sk_buff *skb)
 	int nfrags;
 	void *tmp;
 	u8 *iv;
+	u32 flags = 0;
 	struct scatterlist *sg;
 	struct scatterlist *asg;
 	int err = -EINVAL;
@@ -368,7 +374,9 @@ static int esp_input(struct xfrm_state *x, struct sk_buff *skb)
 	skb_to_sgvec(skb, sg, sizeof(*esph) + crypto_aead_ivsize(aead), elen);
 	sg_init_one(asg, esph, sizeof(*esph));
 
-	aead_request_set_callback(req, 0, esp_input_done, skb);
+	xfrm_aead_set_flags(skb, flags);
+
+	aead_request_set_callback(req, flags, esp_input_done, skb);
 	aead_request_set_crypt(req, sg, sg, elen, iv);
 	aead_request_set_assoc(req, asg, sizeof(*esph));
 
diff --git a/net/ipv6/esp6.c b/net/ipv6/esp6.c
index b181b08..9874adf 100644
--- a/net/ipv6/esp6.c
+++ b/net/ipv6/esp6.c
@@ -143,11 +143,14 @@ static int esp6_output(struct xfrm_state *x, struct sk_buff *skb)
 	int nfrags;
 	u8 *iv;
 	u8 *tail;
+	u32 flags;
 	struct esp_data *esp = x->data;
 
 	/* skb is pure payload to encrypt */
 	err = -ENOMEM;
 
+	flags = 0;
+
 	/* Round to block size */
 	clen = skb->len;
 
@@ -196,7 +199,9 @@ static int esp6_output(struct xfrm_state *x, struct sk_buff *skb)
 		     clen + alen);
 	sg_init_one(asg, esph, sizeof(*esph));
 
-	aead_givcrypt_set_callback(req, 0, esp_output_done, skb);
+	xfrm_aead_set_flags(skb, flags);
+
+	aead_givcrypt_set_callback(req, flags, esp_output_done, skb);
 	aead_givcrypt_set_crypt(req, sg, sg, clen, iv);
 	aead_givcrypt_set_assoc(req, asg, sizeof(*esph));
 	aead_givcrypt_set_giv(req, esph->enc_data,
@@ -279,6 +284,7 @@ static int esp6_input(struct xfrm_state *x, struct sk_buff *skb)
 	int ret = 0;
 	void *tmp;
 	u8 *iv;
+	u32 flags = 0;
 	struct scatterlist *sg;
 	struct scatterlist *asg;
 
@@ -319,7 +325,9 @@ static int esp6_input(struct xfrm_state *x, struct sk_buff *skb)
 	skb_to_sgvec(skb, sg, sizeof(*esph) + crypto_aead_ivsize(aead), elen);
 	sg_init_one(asg, esph, sizeof(*esph));
 
-	aead_request_set_callback(req, 0, esp_input_done, skb);
+	xfrm_aead_set_flags(skb, flags);
+
+	aead_request_set_callback(req, flags, esp_input_done, skb);
 	aead_request_set_crypt(req, sg, sg, elen, iv);
 	aead_request_set_assoc(req, asg, sizeof(*esph));
 
-- 
1.5.4.2


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

* [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms
  2008-12-01  7:16 [RFC PATCH 0/5] IPsec parallelization Steffen Klassert
                   ` (2 preceding siblings ...)
  2008-12-01  7:19 ` [RFC PATCH 3/5] crypto: add possibility to force sync transform Steffen Klassert
@ 2008-12-01  7:19 ` Steffen Klassert
  2008-12-01 11:38   ` Herbert Xu
  2008-12-01  7:20 ` [RFC PATCH 5/5] crypto: make struct aead percpu data Steffen Klassert
  2008-12-01  8:49 ` [RFC PATCH 0/5] IPsec parallelization Herbert Xu
  5 siblings, 1 reply; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01  7:19 UTC (permalink / raw)
  To: netdev; +Cc: davem, herbert, klassert

From: Steffen Klassert <steffen.klassert@secunet.com>

This patch adds functions to alloc/free crypto transforms
as percpu data.

Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>
---
 crypto/api.c           |   86 ++++++++++++++++++++++++++++++++++++++++++-----
 crypto/internal.h      |    3 ++
 include/linux/crypto.h |    1 +
 3 files changed, 80 insertions(+), 10 deletions(-)

diff --git a/crypto/api.c b/crypto/api.c
index 0444d24..6c8fcfb 100644
--- a/crypto/api.c
+++ b/crypto/api.c
@@ -361,6 +361,55 @@ void crypto_shoot_alg(struct crypto_alg *alg)
 }
 EXPORT_SYMBOL_GPL(crypto_shoot_alg);
 
+struct crypto_tfm *__crypto_alloc_tfm_percpu(struct crypto_alg *alg, u32 type,
+				      u32 mask)
+{
+	struct crypto_tfm *tfm = NULL;
+	struct crypto_tfm *tfm_percpu = NULL;
+	unsigned int tfm_size, cpu, cpu_err;
+	int err = -ENOMEM;
+
+	tfm_size = sizeof(*tfm) + crypto_ctxsize(alg, type, mask);
+
+	tfm_percpu =  __alloc_percpu(tfm_size);
+	if (tfm_percpu == NULL)
+		goto out_err;
+
+	for_each_possible_cpu(cpu) {
+
+		tfm = per_cpu_ptr(tfm_percpu, cpu);
+
+		tfm->__crt_alg = alg;
+
+		err = crypto_init_ops(tfm, type, mask);
+		if (err)
+			goto out_free_tfm;
+
+		if (alg->cra_init && (err = alg->cra_init(tfm))) {
+			if (err == -EAGAIN)
+				crypto_shoot_alg(alg);
+			goto cra_init_failed;
+		}
+	}
+
+	goto out;
+
+cra_init_failed:
+	for_each_possible_cpu(cpu_err) {
+		tfm = per_cpu_ptr(tfm_percpu, cpu);
+		crypto_exit_ops(tfm);
+		if (cpu_err == cpu)
+			break;
+	}
+out_free_tfm:
+	free_percpu(tfm_percpu);
+out_err:
+	tfm_percpu = ERR_PTR(err);
+out:
+	return tfm_percpu;
+}
+EXPORT_SYMBOL_GPL(__crypto_alloc_tfm_percpu);
+
 struct crypto_tfm *__crypto_alloc_tfm(struct crypto_alg *alg, u32 type,
 				      u32 mask)
 {
@@ -450,15 +499,8 @@ err:
 	return ERR_PTR(err);
 }
 EXPORT_SYMBOL_GPL(crypto_alloc_base);
- 
-/*
- *	crypto_free_tfm - Free crypto transform
- *	@tfm: Transform to free
- *
- *	crypto_free_tfm() frees up the transform and any associated resources,
- *	then drops the refcount on the associated algorithm.
- */
-void crypto_free_tfm(struct crypto_tfm *tfm)
+
+void __crypto_free_tfm(struct crypto_tfm *tfm)
 {
 	struct crypto_alg *alg;
 	int size;
@@ -474,11 +516,35 @@ void crypto_free_tfm(struct crypto_tfm *tfm)
 	crypto_exit_ops(tfm);
 	crypto_mod_put(alg);
 	memset(tfm, 0, size);
-	kfree(tfm);
 }
 
+/*
+ *	crypto_free_tfm - Free crypto transform
+ *	@tfm: Transform to free
+ *
+ *	crypto_free_tfm() frees up the transform and any associated resources,
+ *	then drops the refcount on the associated algorithm.
+ */
+void crypto_free_tfm(struct crypto_tfm *tfm)
+{
+	__crypto_free_tfm(tfm);
+	kfree(tfm);
+}
 EXPORT_SYMBOL_GPL(crypto_free_tfm);
 
+void crypto_free_tfm_percpu(struct crypto_tfm *tfm_percpu)
+{
+	unsigned int cpu;
+	struct crypto_tfm *tfm;
+
+	for_each_possible_cpu(cpu) {
+		tfm = per_cpu_ptr(tfm_percpu, cpu);
+		__crypto_free_tfm(tfm);
+	}
+	free_percpu(tfm_percpu);
+}
+EXPORT_SYMBOL_GPL(crypto_free_tfm_percpu);
+
 int crypto_has_alg(const char *name, u32 type, u32 mask)
 {
 	int ret = 0;
diff --git a/crypto/internal.h b/crypto/internal.h
index 8ef72d7..8f0fa9e 100644
--- a/crypto/internal.h
+++ b/crypto/internal.h
@@ -109,6 +109,9 @@ void crypto_alg_tested(const char *name, int err);
 void crypto_shoot_alg(struct crypto_alg *alg);
 struct crypto_tfm *__crypto_alloc_tfm(struct crypto_alg *alg, u32 type,
 				      u32 mask);
+struct crypto_tfm *__crypto_alloc_tfm_percpu(struct crypto_alg *alg, u32 type,
+				      u32 mask);
+void __crypto_free_tfm(struct crypto_tfm *tfm);
 
 int crypto_register_instance(struct crypto_template *tmpl,
 			     struct crypto_instance *inst);
diff --git a/include/linux/crypto.h b/include/linux/crypto.h
index d5dd094..da07852 100644
--- a/include/linux/crypto.h
+++ b/include/linux/crypto.h
@@ -548,6 +548,7 @@ struct crypto_attr_u32 {
 struct crypto_tfm *crypto_alloc_tfm(const char *alg_name, u32 tfm_flags);
 struct crypto_tfm *crypto_alloc_base(const char *alg_name, u32 type, u32 mask);
 void crypto_free_tfm(struct crypto_tfm *tfm);
+void crypto_free_tfm_percpu(struct crypto_tfm *tfm);
 
 int alg_test(const char *driver, const char *alg, u32 type, u32 mask);
 
-- 
1.5.4.2


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

* [RFC PATCH 5/5] crypto: make struct aead percpu data
  2008-12-01  7:16 [RFC PATCH 0/5] IPsec parallelization Steffen Klassert
                   ` (3 preceding siblings ...)
  2008-12-01  7:19 ` [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms Steffen Klassert
@ 2008-12-01  7:20 ` Steffen Klassert
  2008-12-01 11:40   ` Herbert Xu
  2008-12-01  8:49 ` [RFC PATCH 0/5] IPsec parallelization Herbert Xu
  5 siblings, 1 reply; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01  7:20 UTC (permalink / raw)
  To: netdev; +Cc: davem, herbert, klassert

From: Steffen Klassert <steffen.klassert@secunet.com>

The struct aead is now allocated as percpu data to get
rid of a very high contended lock in crypto_authenc_hash()
if IPsec runs in parallel.

Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>
---
 crypto/aead.c          |    2 +-
 crypto/chainiv.c       |    3 +-
 crypto/testmgr.c       |   14 +++++---
 include/linux/crypto.h |    2 +-
 net/ipv4/esp4.c        |   79 +++++++++++++++++++++++++++-----------------
 net/ipv6/esp6.c        |   84 +++++++++++++++++++++++++++++------------------
 6 files changed, 113 insertions(+), 71 deletions(-)

diff --git a/crypto/aead.c b/crypto/aead.c
index 3a6f3f5..8f9a2a0 100644
--- a/crypto/aead.c
+++ b/crypto/aead.c
@@ -465,7 +465,7 @@ struct crypto_aead *crypto_alloc_aead(const char *alg_name, u32 type, u32 mask)
 			goto err;
 		}
 
-		tfm = __crypto_alloc_tfm(alg, type, mask);
+		tfm = __crypto_alloc_tfm_percpu(alg, type, mask);
 		if (!IS_ERR(tfm))
 			return __crypto_aead_cast(tfm);
 
diff --git a/crypto/chainiv.c b/crypto/chainiv.c
index 2079fdd..b3e2b28 100644
--- a/crypto/chainiv.c
+++ b/crypto/chainiv.c
@@ -133,7 +133,8 @@ static int async_chainiv_schedule_work(struct async_chainiv_ctx *ctx)
 			goto out;
 	}
 
-	queued = schedule_work(&ctx->postponed);
+	queued = schedule_work_on(get_cpu(), &ctx->postponed);
+	put_cpu();
 	BUG_ON(!queued);
 
 out:
diff --git a/crypto/testmgr.c b/crypto/testmgr.c
index b828c6c..96418cd 100644
--- a/crypto/testmgr.c
+++ b/crypto/testmgr.c
@@ -885,16 +885,20 @@ out:
 static int alg_test_aead(const struct alg_test_desc *desc, const char *driver,
 			 u32 type, u32 mask)
 {
-	struct crypto_aead *tfm;
+	struct crypto_aead *tfm, *tfm_percpu;
 	int err = 0;
 
-	tfm = crypto_alloc_aead(driver, type, mask);
-	if (IS_ERR(tfm)) {
+	tfm_percpu = crypto_alloc_aead(driver, type, mask);
+
+	if (IS_ERR(tfm_percpu)) {
 		printk(KERN_ERR "alg: aead: Failed to load transform for %s: "
-		       "%ld\n", driver, PTR_ERR(tfm));
-		return PTR_ERR(tfm);
+		       "%ld\n", driver, PTR_ERR(tfm_percpu));
+		return PTR_ERR(tfm_percpu);
 	}
 
+	tfm = per_cpu_ptr(tfm_percpu, get_cpu());
+	put_cpu();
+
 	if (desc->suite.aead.enc.vecs) {
 		err = test_aead(tfm, ENCRYPT, desc->suite.aead.enc.vecs,
 				desc->suite.aead.enc.count);
diff --git a/include/linux/crypto.h b/include/linux/crypto.h
index da07852..f9c4bf2 100644
--- a/include/linux/crypto.h
+++ b/include/linux/crypto.h
@@ -800,7 +800,7 @@ static inline struct crypto_tfm *crypto_aead_tfm(struct crypto_aead *tfm)
 
 static inline void crypto_free_aead(struct crypto_aead *tfm)
 {
-	crypto_free_tfm(crypto_aead_tfm(tfm));
+	crypto_free_tfm_percpu(crypto_aead_tfm(tfm));
 }
 
 static inline struct aead_tfm *crypto_aead_crt(struct crypto_aead *tfm)
diff --git a/net/ipv4/esp4.c b/net/ipv4/esp4.c
index 3ea3fb2..daa7e7d 100644
--- a/net/ipv4/esp4.c
+++ b/net/ipv4/esp4.c
@@ -130,7 +130,8 @@ static int esp_output(struct xfrm_state *x, struct sk_buff *skb)
 	clen = skb->len;
 
 	esp = x->data;
-	aead = esp->aead;
+	aead = per_cpu_ptr(esp->aead, get_cpu());
+	put_cpu();
 	alen = crypto_aead_authsize(aead);
 
 	blksize = ALIGN(crypto_aead_blocksize(aead), 4);
@@ -237,7 +238,7 @@ static int esp_input_done2(struct sk_buff *skb, int err)
 	struct iphdr *iph;
 	struct xfrm_state *x = xfrm_input_state(skb);
 	struct esp_data *esp = x->data;
-	struct crypto_aead *aead = esp->aead;
+	struct crypto_aead *aead = per_cpu_ptr(esp->aead, smp_processor_id());
 	int alen = crypto_aead_authsize(aead);
 	int hlen = sizeof(struct ip_esp_hdr) + crypto_aead_ivsize(aead);
 	int elen = skb->len - hlen;
@@ -330,7 +331,7 @@ static int esp_input(struct xfrm_state *x, struct sk_buff *skb)
 {
 	struct ip_esp_hdr *esph;
 	struct esp_data *esp = x->data;
-	struct crypto_aead *aead = esp->aead;
+	struct crypto_aead *aead = per_cpu_ptr(esp->aead, smp_processor_id());
 	struct aead_request *req;
 	struct sk_buff *trailer;
 	int elen = skb->len - sizeof(*esph) - crypto_aead_ivsize(aead);
@@ -393,11 +394,13 @@ out:
 static u32 esp4_get_mtu(struct xfrm_state *x, int mtu)
 {
 	struct esp_data *esp = x->data;
-	u32 blksize = ALIGN(crypto_aead_blocksize(esp->aead), 4);
+	struct crypto_aead *aead = per_cpu_ptr(esp->aead, get_cpu());
+	u32 blksize = ALIGN(crypto_aead_blocksize(aead), 4);
 	u32 align = max_t(u32, blksize, esp->padlen);
 	u32 rem;
 
-	mtu -= x->props.header_len + crypto_aead_authsize(esp->aead);
+	put_cpu();
+	mtu -= x->props.header_len + crypto_aead_authsize(aead);
 	rem = mtu & (align - 1);
 	mtu &= ~(align - 1);
 
@@ -451,24 +454,27 @@ static void esp_destroy(struct xfrm_state *x)
 static int esp_init_aead(struct xfrm_state *x)
 {
 	struct esp_data *esp = x->data;
-	struct crypto_aead *aead;
-	int err;
+	struct crypto_aead *aead, *aead_percpu;
+	int err, cpu;
 
-	aead = crypto_alloc_aead(x->aead->alg_name, 0, 0);
-	err = PTR_ERR(aead);
-	if (IS_ERR(aead))
+	aead_percpu = crypto_alloc_aead(x->aead->alg_name, 0, 0);
+	err = PTR_ERR(aead_percpu);
+	if (IS_ERR(aead_percpu))
 		goto error;
 
-	esp->aead = aead;
+	esp->aead = aead_percpu;
 
-	err = crypto_aead_setkey(aead, x->aead->alg_key,
-				 (x->aead->alg_key_len + 7) / 8);
-	if (err)
-		goto error;
+	for_each_possible_cpu(cpu) {
+		aead = per_cpu_ptr(aead_percpu, cpu);
+		err = crypto_aead_setkey(aead, x->aead->alg_key,
+					 (x->aead->alg_key_len + 7) / 8);
+		if (err)
+			goto error;
 
-	err = crypto_aead_setauthsize(aead, x->aead->alg_icv_len / 8);
-	if (err)
-		goto error;
+		err = crypto_aead_setauthsize(aead, x->aead->alg_icv_len / 8);
+		if (err)
+			goto error;
+	}
 
 error:
 	return err;
@@ -477,14 +483,14 @@ error:
 static int esp_init_authenc(struct xfrm_state *x)
 {
 	struct esp_data *esp = x->data;
-	struct crypto_aead *aead;
+	struct crypto_aead *aead, *aead_percpu;
 	struct crypto_authenc_key_param *param;
 	struct rtattr *rta;
 	char *key;
 	char *p;
 	char authenc_name[CRYPTO_MAX_ALG_NAME];
 	unsigned int keylen;
-	int err;
+	int err, cpu;
 
 	err = -EINVAL;
 	if (x->ealg == NULL)
@@ -496,12 +502,12 @@ static int esp_init_authenc(struct xfrm_state *x)
 		     x->ealg->alg_name) >= CRYPTO_MAX_ALG_NAME)
 		goto error;
 
-	aead = crypto_alloc_aead(authenc_name, 0, 0);
-	err = PTR_ERR(aead);
-	if (IS_ERR(aead))
+	aead_percpu = crypto_alloc_aead(authenc_name, 0, 0);
+	err = PTR_ERR(aead_percpu);
+	if (IS_ERR(aead_percpu))
 		goto error;
 
-	esp->aead = aead;
+	esp->aead = aead_percpu;
 
 	keylen = (x->aalg ? (x->aalg->alg_key_len + 7) / 8 : 0) +
 		 (x->ealg->alg_key_len + 7) / 8 + RTA_SPACE(sizeof(*param));
@@ -527,6 +533,10 @@ static int esp_init_authenc(struct xfrm_state *x)
 		BUG_ON(!aalg_desc);
 
 		err = -EINVAL;
+		/* The aead authsize is the same for all cpus,
+		 * so just read from the the local cpu. */
+		aead = per_cpu_ptr(aead_percpu, get_cpu());
+		put_cpu();
 		if (aalg_desc->uinfo.auth.icv_fullbits/8 !=
 		    crypto_aead_authsize(aead)) {
 			NETDEBUG(KERN_INFO "ESP: %s digestsize %u != %hu\n",
@@ -536,16 +546,22 @@ static int esp_init_authenc(struct xfrm_state *x)
 			goto free_key;
 		}
 
-		err = crypto_aead_setauthsize(
-			aead, aalg_desc->uinfo.auth.icv_truncbits / 8);
-		if (err)
-			goto free_key;
+		for_each_possible_cpu(cpu) {
+			aead = per_cpu_ptr(aead_percpu, cpu);
+			err = crypto_aead_setauthsize(
+				aead, aalg_desc->uinfo.auth.icv_truncbits / 8);
+			if (err)
+				goto free_key;
+		}
 	}
 
 	param->enckeylen = cpu_to_be32((x->ealg->alg_key_len + 7) / 8);
 	memcpy(p, x->ealg->alg_key, (x->ealg->alg_key_len + 7) / 8);
 
-	err = crypto_aead_setkey(aead, key, keylen);
+	for_each_possible_cpu(cpu) {
+		aead = per_cpu_ptr(aead_percpu, cpu);
+		err = crypto_aead_setkey(aead, key, keylen);
+	}
 
 free_key:
 	kfree(key);
@@ -575,7 +591,8 @@ static int esp_init_state(struct xfrm_state *x)
 	if (err)
 		goto error;
 
-	aead = esp->aead;
+	aead = per_cpu_ptr(esp->aead, get_cpu());
+	put_cpu();
 
 	esp->padlen = 0;
 
@@ -603,7 +620,7 @@ static int esp_init_state(struct xfrm_state *x)
 	align = ALIGN(crypto_aead_blocksize(aead), 4);
 	if (esp->padlen)
 		align = max_t(u32, align, esp->padlen);
-	x->props.trailer_len = align + 1 + crypto_aead_authsize(esp->aead);
+	x->props.trailer_len = align + 1 + crypto_aead_authsize(aead);
 
 error:
 	return err;
diff --git a/net/ipv6/esp6.c b/net/ipv6/esp6.c
index 9874adf..15bb91e 100644
--- a/net/ipv6/esp6.c
+++ b/net/ipv6/esp6.c
@@ -154,7 +154,8 @@ static int esp6_output(struct xfrm_state *x, struct sk_buff *skb)
 	/* Round to block size */
 	clen = skb->len;
 
-	aead = esp->aead;
+	aead = per_cpu_ptr(esp->aead, get_cpu());
+
 	alen = crypto_aead_authsize(aead);
 
 	blksize = ALIGN(crypto_aead_blocksize(aead), 4);
@@ -218,6 +219,7 @@ static int esp6_output(struct xfrm_state *x, struct sk_buff *skb)
 	kfree(tmp);
 
 error:
+	put_cpu();
 	return err;
 }
 
@@ -225,7 +227,7 @@ static int esp_input_done2(struct sk_buff *skb, int err)
 {
 	struct xfrm_state *x = xfrm_input_state(skb);
 	struct esp_data *esp = x->data;
-	struct crypto_aead *aead = esp->aead;
+	struct crypto_aead *aead = per_cpu_ptr(esp->aead, smp_processor_id());
 	int alen = crypto_aead_authsize(aead);
 	int hlen = sizeof(struct ip_esp_hdr) + crypto_aead_ivsize(aead);
 	int elen = skb->len - hlen;
@@ -276,7 +278,7 @@ static int esp6_input(struct xfrm_state *x, struct sk_buff *skb)
 {
 	struct ip_esp_hdr *esph;
 	struct esp_data *esp = x->data;
-	struct crypto_aead *aead = esp->aead;
+	struct crypto_aead *aead = per_cpu_ptr(esp->aead, smp_processor_id());
 	struct aead_request *req;
 	struct sk_buff *trailer;
 	int elen = skb->len - sizeof(*esph) - crypto_aead_ivsize(aead);
@@ -344,11 +346,12 @@ out:
 static u32 esp6_get_mtu(struct xfrm_state *x, int mtu)
 {
 	struct esp_data *esp = x->data;
-	u32 blksize = ALIGN(crypto_aead_blocksize(esp->aead), 4);
+	struct crypto_aead *aead = per_cpu_ptr(esp->aead, get_cpu());
+	u32 blksize = ALIGN(crypto_aead_blocksize(aead), 4);
 	u32 align = max_t(u32, blksize, esp->padlen);
 	u32 rem;
 
-	mtu -= x->props.header_len + crypto_aead_authsize(esp->aead);
+	mtu -= x->props.header_len + crypto_aead_authsize(aead);
 	rem = mtu & (align - 1);
 	mtu &= ~(align - 1);
 
@@ -358,6 +361,7 @@ static u32 esp6_get_mtu(struct xfrm_state *x, int mtu)
 		mtu += min_t(u32, blksize - padsize, rem);
 	}
 
+	put_cpu();
 	return mtu - 2;
 }
 
@@ -394,24 +398,27 @@ static void esp6_destroy(struct xfrm_state *x)
 static int esp_init_aead(struct xfrm_state *x)
 {
 	struct esp_data *esp = x->data;
-	struct crypto_aead *aead;
-	int err;
+	struct crypto_aead *aead, *aead_percpu;
+	int err, cpu;
 
-	aead = crypto_alloc_aead(x->aead->alg_name, 0, 0);
-	err = PTR_ERR(aead);
-	if (IS_ERR(aead))
+	aead_percpu = crypto_alloc_aead(x->aead->alg_name, 0, 0);
+	err = PTR_ERR(aead_percpu);
+	if (IS_ERR(aead_percpu))
 		goto error;
 
-	esp->aead = aead;
+	esp->aead = aead_percpu;
 
-	err = crypto_aead_setkey(aead, x->aead->alg_key,
-				 (x->aead->alg_key_len + 7) / 8);
-	if (err)
-		goto error;
+	for_each_possible_cpu(cpu) {
+		aead = per_cpu_ptr(aead_percpu, cpu);
+		err = crypto_aead_setkey(aead, x->aead->alg_key,
+					 (x->aead->alg_key_len + 7) / 8);
+		if (err)
+			goto error;
 
-	err = crypto_aead_setauthsize(aead, x->aead->alg_icv_len / 8);
-	if (err)
-		goto error;
+		err = crypto_aead_setauthsize(aead, x->aead->alg_icv_len / 8);
+		if (err)
+			goto error;
+	}
 
 error:
 	return err;
@@ -420,14 +427,14 @@ error:
 static int esp_init_authenc(struct xfrm_state *x)
 {
 	struct esp_data *esp = x->data;
-	struct crypto_aead *aead;
+	struct crypto_aead *aead, *aead_percpu;
 	struct crypto_authenc_key_param *param;
 	struct rtattr *rta;
 	char *key;
 	char *p;
 	char authenc_name[CRYPTO_MAX_ALG_NAME];
 	unsigned int keylen;
-	int err;
+	int err, cpu;
 
 	err = -EINVAL;
 	if (x->ealg == NULL)
@@ -439,12 +446,12 @@ static int esp_init_authenc(struct xfrm_state *x)
 		     x->ealg->alg_name) >= CRYPTO_MAX_ALG_NAME)
 		goto error;
 
-	aead = crypto_alloc_aead(authenc_name, 0, 0);
-	err = PTR_ERR(aead);
-	if (IS_ERR(aead))
+	aead_percpu = crypto_alloc_aead(authenc_name, 0, 0);
+	err = PTR_ERR(aead_percpu);
+	if (IS_ERR(aead_percpu))
 		goto error;
 
-	esp->aead = aead;
+	esp->aead = aead_percpu;
 
 	keylen = (x->aalg ? (x->aalg->alg_key_len + 7) / 8 : 0) +
 		 (x->ealg->alg_key_len + 7) / 8 + RTA_SPACE(sizeof(*param));
@@ -470,6 +477,10 @@ static int esp_init_authenc(struct xfrm_state *x)
 		BUG_ON(!aalg_desc);
 
 		err = -EINVAL;
+		/* The aead authsize is the same for all cpus,
+		 * so just read from the the local cpu. */
+		aead = per_cpu_ptr(aead_percpu, get_cpu());
+
 		if (aalg_desc->uinfo.auth.icv_fullbits/8 !=
 		    crypto_aead_authsize(aead)) {
 			NETDEBUG(KERN_INFO "ESP: %s digestsize %u != %hu\n",
@@ -479,18 +490,26 @@ static int esp_init_authenc(struct xfrm_state *x)
 			goto free_key;
 		}
 
-		err = crypto_aead_setauthsize(
-			aead, aalg_desc->uinfo.auth.icv_truncbits / 8);
-		if (err)
-			goto free_key;
+		for_each_possible_cpu(cpu) {
+			aead = per_cpu_ptr(aead_percpu, cpu);
+			err = crypto_aead_setauthsize(
+				aead, aalg_desc->uinfo.auth.icv_truncbits / 8);
+			if (err)
+				goto free_key;
+		}
 	}
 
 	param->enckeylen = cpu_to_be32((x->ealg->alg_key_len + 7) / 8);
 	memcpy(p, x->ealg->alg_key, (x->ealg->alg_key_len + 7) / 8);
 
-	err = crypto_aead_setkey(aead, key, keylen);
+	for_each_possible_cpu(cpu) {
+		aead = per_cpu_ptr(aead_percpu, cpu);
+		err = crypto_aead_setkey(aead, key, keylen);
+	}
 
 free_key:
+
+	put_cpu();
 	kfree(key);
 
 error:
@@ -519,9 +538,9 @@ static int esp6_init_state(struct xfrm_state *x)
 		err = esp_init_authenc(x);
 
 	if (err)
-		goto error;
+		return err;
 
-	aead = esp->aead;
+	aead = per_cpu_ptr(esp->aead, get_cpu());
 
 	esp->padlen = 0;
 
@@ -545,9 +564,10 @@ static int esp6_init_state(struct xfrm_state *x)
 	align = ALIGN(crypto_aead_blocksize(aead), 4);
 	if (esp->padlen)
 		align = max_t(u32, align, esp->padlen);
-	x->props.trailer_len = align + 1 + crypto_aead_authsize(esp->aead);
+	x->props.trailer_len = align + 1 + crypto_aead_authsize(aead);
 
 error:
+	put_cpu();
 	return err;
 }
 
-- 
1.5.4.2


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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-01  7:16 [RFC PATCH 0/5] IPsec parallelization Steffen Klassert
                   ` (4 preceding siblings ...)
  2008-12-01  7:20 ` [RFC PATCH 5/5] crypto: make struct aead percpu data Steffen Klassert
@ 2008-12-01  8:49 ` Herbert Xu
  2008-12-01 10:29   ` David Miller
                     ` (2 more replies)
  5 siblings, 3 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01  8:49 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: netdev, davem, klassert

On Mon, Dec 01, 2008 at 08:16:14AM +0100, Steffen Klassert wrote:
> This is a first throw to try to parallelize the expensive part of xfrm by
> using a generic parallelization/serialization method. This method uses the
> remote softirq invocation infrastructure for parallelization and serialization.
> With this method data objects can be processed in parallel, starting 
> at some given point. After doing some expensive operations in parallel, 
> it is possible to serialize again. The parallelized data objects return after
> serialization in the order as they were before the parallelization. 
> In the case of xfrm, this makes it possible to run the expensive part in
> parallel without getting packet reordering.

I still think that you're much better off doing this in the
crypto layer.  As it stands the only reason why this is attractive
is because crypto is slow.

Pretty soon processors will start providing crypto support natively
so this will no longer be the case.  I'd rather see this stuff
contained in a small area instead of having it spread all over the
place as this may become obsolete any day now.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-01  8:49 ` [RFC PATCH 0/5] IPsec parallelization Herbert Xu
@ 2008-12-01 10:29   ` David Miller
  2008-12-01 11:15     ` Herbert Xu
  2008-12-01 11:20   ` Andi Kleen
  2008-12-02  8:00   ` Steffen Klassert
  2 siblings, 1 reply; 37+ messages in thread
From: David Miller @ 2008-12-01 10:29 UTC (permalink / raw)
  To: herbert; +Cc: steffen.klassert, netdev, klassert

From: Herbert Xu <herbert@gondor.apana.org.au>
Date: Mon, 1 Dec 2008 16:49:02 +0800

> On Mon, Dec 01, 2008 at 08:16:14AM +0100, Steffen Klassert wrote:
> > This is a first throw to try to parallelize the expensive part of xfrm by
> > using a generic parallelization/serialization method. This method uses the
> > remote softirq invocation infrastructure for parallelization and serialization.
> > With this method data objects can be processed in parallel, starting 
> > at some given point. After doing some expensive operations in parallel, 
> > it is possible to serialize again. The parallelized data objects return after
> > serialization in the order as they were before the parallelization. 
> > In the case of xfrm, this makes it possible to run the expensive part in
> > parallel without getting packet reordering.
> 
> I still think that you're much better off doing this in the
> crypto layer.  As it stands the only reason why this is attractive
> is because crypto is slow.
> 
> Pretty soon processors will start providing crypto support natively
> so this will no longer be the case.  I'd rather see this stuff
> contained in a small area instead of having it spread all over the
> place as this may become obsolete any day now.

Herbert, I'm not completely convinced of this line of thinking :-)

Will crypto be faster than a routing cache lookup?  Because flow
seperation helps routing performance, significantly.

The problem is that we can't seperate within a flow, that's why we
need something like Steffen's work.

And the reconstitution of the seperated crypto operations into the
original properly ordered flow can be done most cleanly (IMHO) in the
networking layer as far as I have seen so far.

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-01 10:29   ` David Miller
@ 2008-12-01 11:15     ` Herbert Xu
  2008-12-02  7:58       ` Steffen Klassert
  0 siblings, 1 reply; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 11:15 UTC (permalink / raw)
  To: David Miller; +Cc: steffen.klassert, netdev, klassert

On Mon, Dec 01, 2008 at 02:29:29AM -0800, David Miller wrote:
>
> Herbert, I'm not completely convinced of this line of thinking :-)

Well you've put some doubts into mind for a second there, but
after thinking some more I'm sticking to my position :)

> Will crypto be faster than a routing cache lookup?  Because flow
> seperation helps routing performance, significantly.

No to the first part, but that's not the point.  I'm saying that
you can do flow separation within the crypto layer, well, for
output anyway.  All you need to do is to provide some key to
the crypto layer that tells it that ordering is only required
within that key.  Alternatively, just allocate plenty of crypto
tfms as these patches do, and feed the flows down to different
tfms.

For input there's nothing we can do apart from processing completed
requests in sequence so no flow information is going to help (and
in fact if it did one could argue that the purpose of IPsec has been
defeated :)

> And the reconstitution of the seperated crypto operations into the
> original properly ordered flow can be done most cleanly (IMHO) in the
> networking layer as far as I have seen so far.

Well once I've finished the GRO stuff I can give this a go.  But
I'm pretty sure we can achieve exactly the same thing with only
minimal changes to xfrm.  Really all it needs to do is allocate
more tfm objects for the output path.

All the serialisation logic can stay in the crypto layer because
AFAICS no network-specific knowledge is being used in this patch
set.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-01  8:49 ` [RFC PATCH 0/5] IPsec parallelization Herbert Xu
  2008-12-01 10:29   ` David Miller
@ 2008-12-01 11:20   ` Andi Kleen
  2008-12-01 13:39     ` Herbert Xu
  2008-12-02  8:00   ` Steffen Klassert
  2 siblings, 1 reply; 37+ messages in thread
From: Andi Kleen @ 2008-12-01 11:20 UTC (permalink / raw)
  To: Herbert Xu; +Cc: Steffen Klassert, netdev, davem, klassert

Herbert Xu <herbert@gondor.apana.org.au> writes:
>
> I still think that you're much better off doing this in the
> crypto layer.  As it stands the only reason why this is attractive
> is because crypto is slow.
>
> Pretty soon processors will start providing crypto support natively
> so this will no longer be the case. 

I'm not sure that's a useful argument.  When cryptography is not
CPU bound anymore it will be memory bandwidth bound. And in this case
you can still get a win out of parallelization if you parallelize
over multiple sockets with own memory controller or own FSB
because that will give you more bandwidth.

-Andi

-- 
ak@linux.intel.com

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

* Re: [RFC PATCH 3/5] crypto: add possibility to force sync transform
  2008-12-01  7:19 ` [RFC PATCH 3/5] crypto: add possibility to force sync transform Steffen Klassert
@ 2008-12-01 11:22   ` Herbert Xu
  2008-12-01 13:19     ` Steffen Klassert
  0 siblings, 1 reply; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 11:22 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: netdev, davem, klassert

On Mon, Dec 01, 2008 at 08:19:03AM +0100, Steffen Klassert wrote:
.
> diff --git a/crypto/chainiv.c b/crypto/chainiv.c
> index 7c37a49..2079fdd 100644
> --- a/crypto/chainiv.c
> +++ b/crypto/chainiv.c
> @@ -168,8 +168,12 @@ static int async_chainiv_givencrypt_tail(struct skcipher_givcrypt_request *req)
>  	memcpy(subreq->info, ctx->iv, ivsize);
>  
>  	ctx->err = crypto_ablkcipher_encrypt(subreq);
> -	if (ctx->err)
> -		goto out;
> +	if (ctx->err) {
> +		if (ablkcipher_request_flags(&req->creq) & CRYPTO_TFM_REQ_FORCE_SYNC)

If you want sync algorithms, you should request for that at alloc
time:

	crypto_alloc_ablkcipher("foo", 0, CRYPTO_ALG_ASYNC);

Doing so at run-time is going to end in tears.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms
  2008-12-01  7:19 ` [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms Steffen Klassert
@ 2008-12-01 11:38   ` Herbert Xu
  2008-12-01 12:25     ` David Miller
  0 siblings, 1 reply; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 11:38 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: netdev, davem, klassert

On Mon, Dec 01, 2008 at 08:19:43AM +0100, Steffen Klassert wrote:
> From: Steffen Klassert <steffen.klassert@secunet.com>
> 
> This patch adds functions to alloc/free crypto transforms
> as percpu data.
> 
> Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>

I'd prefer to just have the user call crypto_alloc_tfm for each
CPU.  This is the slow path after all so we don't need to optimise
it that much.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 5/5] crypto: make struct aead percpu data
  2008-12-01  7:20 ` [RFC PATCH 5/5] crypto: make struct aead percpu data Steffen Klassert
@ 2008-12-01 11:40   ` Herbert Xu
  2008-12-01 13:36     ` Steffen Klassert
  0 siblings, 1 reply; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 11:40 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: netdev, davem, klassert

On Mon, Dec 01, 2008 at 08:20:43AM +0100, Steffen Klassert wrote:
> From: Steffen Klassert <steffen.klassert@secunet.com>
> 
> The struct aead is now allocated as percpu data to get
> rid of a very high contended lock in crypto_authenc_hash()
> if IPsec runs in parallel.
> 
> Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>

Well you're in luck :) One of the objectives of the shash work
is to get rid of this lock.  So instead of doing this, please
push the shash work along and we can eliminate this without
allocating loads of duplicate tfm's.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms
  2008-12-01 11:38   ` Herbert Xu
@ 2008-12-01 12:25     ` David Miller
  2008-12-01 12:38       ` Herbert Xu
  0 siblings, 1 reply; 37+ messages in thread
From: David Miller @ 2008-12-01 12:25 UTC (permalink / raw)
  To: herbert; +Cc: steffen.klassert, netdev, klassert

From: Herbert Xu <herbert@gondor.apana.org.au>
Date: Mon, 1 Dec 2008 19:38:28 +0800

> On Mon, Dec 01, 2008 at 08:19:43AM +0100, Steffen Klassert wrote:
> > From: Steffen Klassert <steffen.klassert@secunet.com>
> > 
> > This patch adds functions to alloc/free crypto transforms
> > as percpu data.
> > 
> > Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>
> 
> I'd prefer to just have the user call crypto_alloc_tfm for each
> CPU.  This is the slow path after all so we don't need to optimise
> it that much.

I somewhat disagree.

If you make this num_cpus expensive, I can nearly guarentee you this
will show up in the profiles for the workloads I was using to test the
control path optimizations with all the dynamic hash tables and
streamlined lookups.

Some cell phones networks, I am to understand, can allocate 6 SAs when
they come onto the network, perhaps never use them, then go away.
Multiply that by the number of cell phones using a network in a busy
downtown area of a city and it could be very interesting.

If, in those changes, I was removing atomics to make rule insert and
delete faster, making more TFM allocs happen might be a big no-no :)

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

* Re: [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms
  2008-12-01 12:25     ` David Miller
@ 2008-12-01 12:38       ` Herbert Xu
  2008-12-01 13:21         ` Steffen Klassert
  0 siblings, 1 reply; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 12:38 UTC (permalink / raw)
  To: David Miller; +Cc: steffen.klassert, netdev, klassert

On Mon, Dec 01, 2008 at 04:25:02AM -0800, David Miller wrote:
>
> If, in those changes, I was removing atomics to make rule insert and
> delete faster, making more TFM allocs happen might be a big no-no :)

Fair enough.  But it looks like this stuff only exists because of
the spinlock on the hash which shash will get rid of anyway.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 3/5] crypto: add possibility to force sync transform
  2008-12-01 11:22   ` Herbert Xu
@ 2008-12-01 13:19     ` Steffen Klassert
  0 siblings, 0 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01 13:19 UTC (permalink / raw)
  To: Herbert Xu; +Cc: netdev, davem, klassert

On Mon, Dec 01, 2008 at 07:22:25PM +0800, Herbert Xu wrote:
> 
> If you want sync algorithms, you should request for that at alloc
> time:
> 
> 	crypto_alloc_ablkcipher("foo", 0, CRYPTO_ALG_ASYNC);

Ok, will do that.

> 
> Doing so at run-time is going to end in tears.
> 

As I wrote, I'm not that familiar with the crypto system :-)

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

* Re: [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms
  2008-12-01 12:38       ` Herbert Xu
@ 2008-12-01 13:21         ` Steffen Klassert
  0 siblings, 0 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01 13:21 UTC (permalink / raw)
  To: Herbert Xu; +Cc: David Miller, netdev, klassert

On Mon, Dec 01, 2008 at 08:38:59PM +0800, Herbert Xu wrote:
> On Mon, Dec 01, 2008 at 04:25:02AM -0800, David Miller wrote:
> >
> > If, in those changes, I was removing atomics to make rule insert and
> > delete faster, making more TFM allocs happen might be a big no-no :)
> 
> Fair enough.  But it looks like this stuff only exists because of
> the spinlock on the hash which shash will get rid of anyway.
> 

Yes exactly, it was just to get rid of the lock.

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

* Re: [RFC PATCH 5/5] crypto: make struct aead percpu data
  2008-12-01 11:40   ` Herbert Xu
@ 2008-12-01 13:36     ` Steffen Klassert
  2008-12-01 13:44       ` Herbert Xu
  0 siblings, 1 reply; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01 13:36 UTC (permalink / raw)
  To: Herbert Xu; +Cc: netdev, davem, klassert

On Mon, Dec 01, 2008 at 07:40:00PM +0800, Herbert Xu wrote:
> On Mon, Dec 01, 2008 at 08:20:43AM +0100, Steffen Klassert wrote:
> > From: Steffen Klassert <steffen.klassert@secunet.com>
> > 
> > The struct aead is now allocated as percpu data to get
> > rid of a very high contended lock in crypto_authenc_hash()
> > if IPsec runs in parallel.
> > 
> > Signed-off-by: Steffen Klassert <steffen.klassert@secunet.com>
> 
> Well you're in luck :) One of the objectives of the shash work
> is to get rid of this lock.  So instead of doing this, please
> push the shash work along and we can eliminate this without
> allocating loads of duplicate tfm's.
> 

I searched for your shash work. Is there already some work in progress
aside from crc32?

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-01 11:20   ` Andi Kleen
@ 2008-12-01 13:39     ` Herbert Xu
  0 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:39 UTC (permalink / raw)
  To: Andi Kleen; +Cc: herbert, steffen.klassert, netdev, davem, klassert

Andi Kleen <andi@firstfloor.org> wrote:
>
> I'm not sure that's a useful argument.  When cryptography is not
> CPU bound anymore it will be memory bandwidth bound. And in this case
> you can still get a win out of parallelization if you parallelize
> over multiple sockets with own memory controller or own FSB
> because that will give you more bandwidth.

That's a good point.

Of course you can achieve the same thing by doing the parallel
processing within the crypto layer, which would also benefit other
crypto users, such as disk encryption.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 5/5] crypto: make struct aead percpu data
  2008-12-01 13:36     ` Steffen Klassert
@ 2008-12-01 13:44       ` Herbert Xu
  2008-12-01 13:47         ` [PATCH 1/6] crypto: hash - Make setkey optional Herbert Xu
                           ` (7 more replies)
  0 siblings, 8 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:44 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: netdev, davem, klassert, Linux Crypto Mailing List

On Mon, Dec 01, 2008 at 02:36:54PM +0100, Steffen Klassert wrote:
>
> I searched for your shash work. Is there already some work in progress
> aside from crc32?

I started on the algorithm conversion but have only made it as far
as null and rmd*.  But it should serve as a good template for doing
the rest.

If you could finish them for me I'd be most grateful :)

I'll push what I've got out now.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* [PATCH 1/6] crypto: hash - Make setkey optional
  2008-12-01 13:44       ` Herbert Xu
@ 2008-12-01 13:47         ` Herbert Xu
  2008-12-01 13:47         ` [PATCH 2/6] crypto: null - Switch to shash Herbert Xu
                           ` (6 subsequent siblings)
  7 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:47 UTC (permalink / raw)
  To: Steffen Klassert, netdev, davem, klassert, Linux Crypto Mailing List

crypto: hash - Make setkey optional

Since most cryptographic hash algorithms have no keys, this patch
makes the setkey function optional for ahash and shash.

Signed-off-by: Herbert Xu <herbert@gondor.apana.org.au>
---

 crypto/ahash.c |    8 +++++++-
 crypto/shash.c |    3 +++
 2 files changed, 10 insertions(+), 1 deletion(-)

diff --git a/crypto/ahash.c b/crypto/ahash.c
index 9f98956..ba5292d 100644
--- a/crypto/ahash.c
+++ b/crypto/ahash.c
@@ -162,6 +162,12 @@ static int ahash_setkey(struct crypto_ahash *tfm, const u8 *key,
 	return ahash->setkey(tfm, key, keylen);
 }
 
+static int ahash_nosetkey(struct crypto_ahash *tfm, const u8 *key,
+			  unsigned int keylen)
+{
+	return -ENOSYS;
+}
+
 int crypto_ahash_import(struct ahash_request *req, const u8 *in)
 {
 	struct crypto_ahash *tfm = crypto_ahash_reqtfm(req);
@@ -194,7 +200,7 @@ static int crypto_init_ahash_ops(struct crypto_tfm *tfm, u32 type, u32 mask)
 	crt->update = alg->update;
 	crt->final  = alg->final;
 	crt->digest = alg->digest;
-	crt->setkey = ahash_setkey;
+	crt->setkey = alg->setkey ? ahash_setkey : ahash_nosetkey;
 	crt->digestsize = alg->digestsize;
 
 	return 0;
diff --git a/crypto/shash.c b/crypto/shash.c
index 50d69a4..c9df367 100644
--- a/crypto/shash.c
+++ b/crypto/shash.c
@@ -55,6 +55,9 @@ int crypto_shash_setkey(struct crypto_shash *tfm, const u8 *key,
 	struct shash_alg *shash = crypto_shash_alg(tfm);
 	unsigned long alignmask = crypto_shash_alignmask(tfm);
 
+	if (!shash->setkey)
+		return -ENOSYS;
+
 	if ((unsigned long)key & alignmask)
 		return shash_setkey_unaligned(tfm, key, keylen);
 

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

* [PATCH 2/6] crypto: null - Switch to shash
  2008-12-01 13:44       ` Herbert Xu
  2008-12-01 13:47         ` [PATCH 1/6] crypto: hash - Make setkey optional Herbert Xu
@ 2008-12-01 13:47         ` Herbert Xu
  2008-12-01 13:47         ` [PATCH 3/6] crypto: rmd128 " Herbert Xu
                           ` (5 subsequent siblings)
  7 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:47 UTC (permalink / raw)
  To: Steffen Klassert, netdev, davem, klassert, Linux Crypto Mailing List

crypto: null - Switch to shash

This patch changes digest_null to the new shash interface.

Signed-off-by: Herbert Xu <herbert@gondor.apana.org.au>
---

 crypto/crypto_null.c |   64 ++++++++++++++++++++++++++++++++-------------------
 1 file changed, 41 insertions(+), 23 deletions(-)

diff --git a/crypto/crypto_null.c b/crypto/crypto_null.c
index 1f7d530..cb71c91 100644
--- a/crypto/crypto_null.c
+++ b/crypto/crypto_null.c
@@ -17,6 +17,7 @@
  *
  */
 
+#include <crypto/internal/hash.h>
 #include <crypto/internal/skcipher.h>
 #include <linux/init.h>
 #include <linux/module.h>
@@ -38,15 +39,31 @@ static int null_compress(struct crypto_tfm *tfm, const u8 *src,
 	return 0;
 }
 
-static void null_init(struct crypto_tfm *tfm)
-{ }
+static int null_init(struct shash_desc *desc)
+{
+	return 0;
+}
 
-static void null_update(struct crypto_tfm *tfm, const u8 *data,
-			unsigned int len)
-{ }
+static int null_update(struct shash_desc *desc, const u8 *data,
+		       unsigned int len)
+{
+	return 0;
+}
 
-static void null_final(struct crypto_tfm *tfm, u8 *out)
-{ }
+static int null_final(struct shash_desc *desc, u8 *out)
+{
+	return 0;
+}
+
+static int null_digest(struct shash_desc *desc, const u8 *data,
+		       unsigned int len, u8 *out)
+{
+	return 0;
+}
+
+static int null_hash_setkey(struct crypto_shash *tfm, const u8 *key,
+			    unsigned int keylen)
+{ return 0; }
 
 static int null_setkey(struct crypto_tfm *tfm, const u8 *key,
 		       unsigned int keylen)
@@ -89,19 +106,20 @@ static struct crypto_alg compress_null = {
 	.coa_decompress		=	null_compress } }
 };
 
-static struct crypto_alg digest_null = {
-	.cra_name		=	"digest_null",
-	.cra_flags		=	CRYPTO_ALG_TYPE_DIGEST,
-	.cra_blocksize		=	NULL_BLOCK_SIZE,
-	.cra_ctxsize		=	0,
-	.cra_module		=	THIS_MODULE,
-	.cra_list		=       LIST_HEAD_INIT(digest_null.cra_list),	
-	.cra_u			=	{ .digest = {
-	.dia_digestsize		=	NULL_DIGEST_SIZE,
-	.dia_setkey   		=	null_setkey,
-	.dia_init   		=	null_init,
-	.dia_update 		=	null_update,
-	.dia_final  		=	null_final } }
+static struct shash_alg digest_null = {
+	.digestsize		=	NULL_DIGEST_SIZE,
+	.setkey   		=	null_hash_setkey,
+	.init   		=	null_init,
+	.update 		=	null_update,
+	.finup 			=	null_digest,
+	.digest 		=	null_digest,
+	.final  		=	null_final,
+	.base			=	{
+		.cra_name		=	"digest_null",
+		.cra_flags		=	CRYPTO_ALG_TYPE_SHASH,
+		.cra_blocksize		=	NULL_BLOCK_SIZE,
+		.cra_module		=	THIS_MODULE,
+	}
 };
 
 static struct crypto_alg cipher_null = {
@@ -154,7 +172,7 @@ static int __init crypto_null_mod_init(void)
 	if (ret < 0)
 		goto out_unregister_cipher;
 
-	ret = crypto_register_alg(&digest_null);
+	ret = crypto_register_shash(&digest_null);
 	if (ret < 0)
 		goto out_unregister_skcipher;
 
@@ -166,7 +184,7 @@ out:
 	return ret;
 
 out_unregister_digest:
-	crypto_unregister_alg(&digest_null);
+	crypto_unregister_shash(&digest_null);
 out_unregister_skcipher:
 	crypto_unregister_alg(&skcipher_null);
 out_unregister_cipher:
@@ -177,7 +195,7 @@ out_unregister_cipher:
 static void __exit crypto_null_mod_fini(void)
 {
 	crypto_unregister_alg(&compress_null);
-	crypto_unregister_alg(&digest_null);
+	crypto_unregister_shash(&digest_null);
 	crypto_unregister_alg(&skcipher_null);
 	crypto_unregister_alg(&cipher_null);
 }

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

* [PATCH 3/6] crypto: rmd128 - Switch to shash
  2008-12-01 13:44       ` Herbert Xu
  2008-12-01 13:47         ` [PATCH 1/6] crypto: hash - Make setkey optional Herbert Xu
  2008-12-01 13:47         ` [PATCH 2/6] crypto: null - Switch to shash Herbert Xu
@ 2008-12-01 13:47         ` Herbert Xu
  2008-12-01 13:47         ` [PATCH 4/6] crypto: rmd160 " Herbert Xu
                           ` (4 subsequent siblings)
  7 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:47 UTC (permalink / raw)
  To: Steffen Klassert, netdev, davem, klassert, Linux Crypto Mailing List

crypto: rmd128 - Switch to shash

This patch changes rmd128 to the new shash interface.

Signed-off-by: Herbert Xu <herbert@gondor.apana.org.au>
---

 crypto/rmd128.c |   61 +++++++++++++++++++++++++++++---------------------------
 1 file changed, 32 insertions(+), 29 deletions(-)

diff --git a/crypto/rmd128.c b/crypto/rmd128.c
index 5de6fa2..1ceb673 100644
--- a/crypto/rmd128.c
+++ b/crypto/rmd128.c
@@ -13,11 +13,10 @@
  * any later version.
  *
  */
+#include <crypto/internal/hash.h>
 #include <linux/init.h>
 #include <linux/module.h>
 #include <linux/mm.h>
-#include <linux/crypto.h>
-#include <linux/cryptohash.h>
 #include <linux/types.h>
 #include <asm/byteorder.h>
 
@@ -218,9 +217,9 @@ static void rmd128_transform(u32 *state, const __le32 *in)
 	return;
 }
 
-static void rmd128_init(struct crypto_tfm *tfm)
+static int rmd128_init(struct shash_desc *desc)
 {
-	struct rmd128_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd128_ctx *rctx = shash_desc_ctx(desc);
 
 	rctx->byte_count = 0;
 
@@ -230,12 +229,14 @@ static void rmd128_init(struct crypto_tfm *tfm)
 	rctx->state[3] = RMD_H3;
 
 	memset(rctx->buffer, 0, sizeof(rctx->buffer));
+
+	return 0;
 }
 
-static void rmd128_update(struct crypto_tfm *tfm, const u8 *data,
-			  unsigned int len)
+static int rmd128_update(struct shash_desc *desc, const u8 *data,
+			 unsigned int len)
 {
-	struct rmd128_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd128_ctx *rctx = shash_desc_ctx(desc);
 	const u32 avail = sizeof(rctx->buffer) - (rctx->byte_count & 0x3f);
 
 	rctx->byte_count += len;
@@ -244,7 +245,7 @@ static void rmd128_update(struct crypto_tfm *tfm, const u8 *data,
 	if (avail > len) {
 		memcpy((char *)rctx->buffer + (sizeof(rctx->buffer) - avail),
 		       data, len);
-		return;
+		goto out;
 	}
 
 	memcpy((char *)rctx->buffer + (sizeof(rctx->buffer) - avail),
@@ -262,12 +263,15 @@ static void rmd128_update(struct crypto_tfm *tfm, const u8 *data,
 	}
 
 	memcpy(rctx->buffer, data, len);
+
+out:
+	return 0;
 }
 
 /* Add padding and return the message digest. */
-static void rmd128_final(struct crypto_tfm *tfm, u8 *out)
+static int rmd128_final(struct shash_desc *desc, u8 *out)
 {
-	struct rmd128_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd128_ctx *rctx = shash_desc_ctx(desc);
 	u32 i, index, padlen;
 	__le64 bits;
 	__le32 *dst = (__le32 *)out;
@@ -278,10 +282,10 @@ static void rmd128_final(struct crypto_tfm *tfm, u8 *out)
 	/* Pad out to 56 mod 64 */
 	index = rctx->byte_count & 0x3f;
 	padlen = (index < 56) ? (56 - index) : ((64+56) - index);
-	rmd128_update(tfm, padding, padlen);
+	rmd128_update(desc, padding, padlen);
 
 	/* Append length */
-	rmd128_update(tfm, (const u8 *)&bits, sizeof(bits));
+	rmd128_update(desc, (const u8 *)&bits, sizeof(bits));
 
 	/* Store state in digest */
 	for (i = 0; i < 4; i++)
@@ -289,31 +293,32 @@ static void rmd128_final(struct crypto_tfm *tfm, u8 *out)
 
 	/* Wipe context */
 	memset(rctx, 0, sizeof(*rctx));
+
+	return 0;
 }
 
-static struct crypto_alg alg = {
-	.cra_name	 =	"rmd128",
-	.cra_driver_name =	"rmd128",
-	.cra_flags	 =	CRYPTO_ALG_TYPE_DIGEST,
-	.cra_blocksize	 =	RMD128_BLOCK_SIZE,
-	.cra_ctxsize	 =	sizeof(struct rmd128_ctx),
-	.cra_module	 =	THIS_MODULE,
-	.cra_list	 =	LIST_HEAD_INIT(alg.cra_list),
-	.cra_u		 =	{ .digest = {
-	.dia_digestsize	 =	RMD128_DIGEST_SIZE,
-	.dia_init	 =	rmd128_init,
-	.dia_update	 =	rmd128_update,
-	.dia_final	 =	rmd128_final } }
+static struct shash_alg alg = {
+	.digestsize	=	RMD128_DIGEST_SIZE,
+	.init		=	rmd128_init,
+	.update		=	rmd128_update,
+	.final		=	rmd128_final,
+	.descsize	=	sizeof(struct rmd128_ctx),
+	.base		=	{
+		.cra_name	 =	"rmd128",
+		.cra_flags	 =	CRYPTO_ALG_TYPE_SHASH,
+		.cra_blocksize	 =	RMD128_BLOCK_SIZE,
+		.cra_module	 =	THIS_MODULE,
+	}
 };
 
 static int __init rmd128_mod_init(void)
 {
-	return crypto_register_alg(&alg);
+	return crypto_register_shash(&alg);
 }
 
 static void __exit rmd128_mod_fini(void)
 {
-	crypto_unregister_alg(&alg);
+	crypto_unregister_shash(&alg);
 }
 
 module_init(rmd128_mod_init);
@@ -321,5 +326,3 @@ module_exit(rmd128_mod_fini);
 
 MODULE_LICENSE("GPL");
 MODULE_DESCRIPTION("RIPEMD-128 Message Digest");
-
-MODULE_ALIAS("rmd128");

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

* [PATCH 4/6] crypto: rmd160 - Switch to shash
  2008-12-01 13:44       ` Herbert Xu
                           ` (2 preceding siblings ...)
  2008-12-01 13:47         ` [PATCH 3/6] crypto: rmd128 " Herbert Xu
@ 2008-12-01 13:47         ` Herbert Xu
  2008-12-01 13:47         ` [PATCH 5/6] crypto: rmd256 " Herbert Xu
                           ` (3 subsequent siblings)
  7 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:47 UTC (permalink / raw)
  To: Steffen Klassert, netdev, davem, klassert, Linux Crypto Mailing List

crypto: rmd160 - Switch to shash

This patch changes rmd160 to the new shash interface.

Signed-off-by: Herbert Xu <herbert@gondor.apana.org.au>
---

 crypto/rmd160.c |   61 +++++++++++++++++++++++++++++---------------------------
 1 file changed, 32 insertions(+), 29 deletions(-)

diff --git a/crypto/rmd160.c b/crypto/rmd160.c
index f001ec7..472261f 100644
--- a/crypto/rmd160.c
+++ b/crypto/rmd160.c
@@ -13,11 +13,10 @@
  * any later version.
  *
  */
+#include <crypto/internal/hash.h>
 #include <linux/init.h>
 #include <linux/module.h>
 #include <linux/mm.h>
-#include <linux/crypto.h>
-#include <linux/cryptohash.h>
 #include <linux/types.h>
 #include <asm/byteorder.h>
 
@@ -261,9 +260,9 @@ static void rmd160_transform(u32 *state, const __le32 *in)
 	return;
 }
 
-static void rmd160_init(struct crypto_tfm *tfm)
+static int rmd160_init(struct shash_desc *desc)
 {
-	struct rmd160_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd160_ctx *rctx = shash_desc_ctx(desc);
 
 	rctx->byte_count = 0;
 
@@ -274,12 +273,14 @@ static void rmd160_init(struct crypto_tfm *tfm)
 	rctx->state[4] = RMD_H4;
 
 	memset(rctx->buffer, 0, sizeof(rctx->buffer));
+
+	return 0;
 }
 
-static void rmd160_update(struct crypto_tfm *tfm, const u8 *data,
-			  unsigned int len)
+static int rmd160_update(struct shash_desc *desc, const u8 *data,
+			 unsigned int len)
 {
-	struct rmd160_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd160_ctx *rctx = shash_desc_ctx(desc);
 	const u32 avail = sizeof(rctx->buffer) - (rctx->byte_count & 0x3f);
 
 	rctx->byte_count += len;
@@ -288,7 +289,7 @@ static void rmd160_update(struct crypto_tfm *tfm, const u8 *data,
 	if (avail > len) {
 		memcpy((char *)rctx->buffer + (sizeof(rctx->buffer) - avail),
 		       data, len);
-		return;
+		goto out;
 	}
 
 	memcpy((char *)rctx->buffer + (sizeof(rctx->buffer) - avail),
@@ -306,12 +307,15 @@ static void rmd160_update(struct crypto_tfm *tfm, const u8 *data,
 	}
 
 	memcpy(rctx->buffer, data, len);
+
+out:
+	return 0;
 }
 
 /* Add padding and return the message digest. */
-static void rmd160_final(struct crypto_tfm *tfm, u8 *out)
+static int rmd160_final(struct shash_desc *desc, u8 *out)
 {
-	struct rmd160_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd160_ctx *rctx = shash_desc_ctx(desc);
 	u32 i, index, padlen;
 	__le64 bits;
 	__le32 *dst = (__le32 *)out;
@@ -322,10 +326,10 @@ static void rmd160_final(struct crypto_tfm *tfm, u8 *out)
 	/* Pad out to 56 mod 64 */
 	index = rctx->byte_count & 0x3f;
 	padlen = (index < 56) ? (56 - index) : ((64+56) - index);
-	rmd160_update(tfm, padding, padlen);
+	rmd160_update(desc, padding, padlen);
 
 	/* Append length */
-	rmd160_update(tfm, (const u8 *)&bits, sizeof(bits));
+	rmd160_update(desc, (const u8 *)&bits, sizeof(bits));
 
 	/* Store state in digest */
 	for (i = 0; i < 5; i++)
@@ -333,31 +337,32 @@ static void rmd160_final(struct crypto_tfm *tfm, u8 *out)
 
 	/* Wipe context */
 	memset(rctx, 0, sizeof(*rctx));
+
+	return 0;
 }
 
-static struct crypto_alg alg = {
-	.cra_name	 =	"rmd160",
-	.cra_driver_name =	"rmd160",
-	.cra_flags	 =	CRYPTO_ALG_TYPE_DIGEST,
-	.cra_blocksize	 =	RMD160_BLOCK_SIZE,
-	.cra_ctxsize	 =	sizeof(struct rmd160_ctx),
-	.cra_module	 =	THIS_MODULE,
-	.cra_list	 =	LIST_HEAD_INIT(alg.cra_list),
-	.cra_u		 =	{ .digest = {
-	.dia_digestsize	 =	RMD160_DIGEST_SIZE,
-	.dia_init	 =	rmd160_init,
-	.dia_update	 =	rmd160_update,
-	.dia_final	 =	rmd160_final } }
+static struct shash_alg alg = {
+	.digestsize	=	RMD160_DIGEST_SIZE,
+	.init		=	rmd160_init,
+	.update		=	rmd160_update,
+	.final		=	rmd160_final,
+	.descsize	=	sizeof(struct rmd160_ctx),
+	.base		=	{
+		.cra_name	 =	"rmd160",
+		.cra_flags	 =	CRYPTO_ALG_TYPE_SHASH,
+		.cra_blocksize	 =	RMD160_BLOCK_SIZE,
+		.cra_module	 =	THIS_MODULE,
+	}
 };
 
 static int __init rmd160_mod_init(void)
 {
-	return crypto_register_alg(&alg);
+	return crypto_register_shash(&alg);
 }
 
 static void __exit rmd160_mod_fini(void)
 {
-	crypto_unregister_alg(&alg);
+	crypto_unregister_shash(&alg);
 }
 
 module_init(rmd160_mod_init);
@@ -365,5 +370,3 @@ module_exit(rmd160_mod_fini);
 
 MODULE_LICENSE("GPL");
 MODULE_DESCRIPTION("RIPEMD-160 Message Digest");

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

* [PATCH 5/6] crypto: rmd256 - Switch to shash
  2008-12-01 13:44       ` Herbert Xu
                           ` (3 preceding siblings ...)
  2008-12-01 13:47         ` [PATCH 4/6] crypto: rmd160 " Herbert Xu
@ 2008-12-01 13:47         ` Herbert Xu
  2008-12-01 13:47         ` [PATCH 6/6] crypto: rmd320 " Herbert Xu
                           ` (2 subsequent siblings)
  7 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:47 UTC (permalink / raw)
  To: Steffen Klassert, netdev, davem, klassert, Linux Crypto Mailing List

crypto: rmd256 - Switch to shash

This patch changes rmd256 to the new shash interface.

Signed-off-by: Herbert Xu <herbert@gondor.apana.org.au>
---

 crypto/rmd256.c |   61 +++++++++++++++++++++++++++++---------------------------
 1 file changed, 32 insertions(+), 29 deletions(-)

diff --git a/crypto/rmd256.c b/crypto/rmd256.c
index e3de5b4..72eafa8 100644
--- a/crypto/rmd256.c
+++ b/crypto/rmd256.c
@@ -13,11 +13,10 @@
  * any later version.
  *
  */
+#include <crypto/internal/hash.h>
 #include <linux/init.h>
 #include <linux/module.h>
 #include <linux/mm.h>
-#include <linux/crypto.h>
-#include <linux/cryptohash.h>
 #include <linux/types.h>
 #include <asm/byteorder.h>
 
@@ -233,9 +232,9 @@ static void rmd256_transform(u32 *state, const __le32 *in)
 	return;
 }
 
-static void rmd256_init(struct crypto_tfm *tfm)
+static int rmd256_init(struct shash_desc *desc)
 {
-	struct rmd256_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd256_ctx *rctx = shash_desc_ctx(desc);
 
 	rctx->byte_count = 0;
 
@@ -249,12 +248,14 @@ static void rmd256_init(struct crypto_tfm *tfm)
 	rctx->state[7] = RMD_H8;
 
 	memset(rctx->buffer, 0, sizeof(rctx->buffer));
+
+	return 0;
 }
 
-static void rmd256_update(struct crypto_tfm *tfm, const u8 *data,
-			  unsigned int len)
+static int rmd256_update(struct shash_desc *desc, const u8 *data,
+			 unsigned int len)
 {
-	struct rmd256_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd256_ctx *rctx = shash_desc_ctx(desc);
 	const u32 avail = sizeof(rctx->buffer) - (rctx->byte_count & 0x3f);
 
 	rctx->byte_count += len;
@@ -263,7 +264,7 @@ static void rmd256_update(struct crypto_tfm *tfm, const u8 *data,
 	if (avail > len) {
 		memcpy((char *)rctx->buffer + (sizeof(rctx->buffer) - avail),
 		       data, len);
-		return;
+		goto out;
 	}
 
 	memcpy((char *)rctx->buffer + (sizeof(rctx->buffer) - avail),
@@ -281,12 +282,15 @@ static void rmd256_update(struct crypto_tfm *tfm, const u8 *data,
 	}
 
 	memcpy(rctx->buffer, data, len);
+
+out:
+	return 0;
 }
 
 /* Add padding and return the message digest. */
-static void rmd256_final(struct crypto_tfm *tfm, u8 *out)
+static int rmd256_final(struct shash_desc *desc, u8 *out)
 {
-	struct rmd256_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd256_ctx *rctx = shash_desc_ctx(desc);
 	u32 i, index, padlen;
 	__le64 bits;
 	__le32 *dst = (__le32 *)out;
@@ -297,10 +301,10 @@ static void rmd256_final(struct crypto_tfm *tfm, u8 *out)
 	/* Pad out to 56 mod 64 */
 	index = rctx->byte_count & 0x3f;
 	padlen = (index < 56) ? (56 - index) : ((64+56) - index);
-	rmd256_update(tfm, padding, padlen);
+	rmd256_update(desc, padding, padlen);
 
 	/* Append length */
-	rmd256_update(tfm, (const u8 *)&bits, sizeof(bits));
+	rmd256_update(desc, (const u8 *)&bits, sizeof(bits));
 
 	/* Store state in digest */
 	for (i = 0; i < 8; i++)
@@ -308,31 +312,32 @@ static void rmd256_final(struct crypto_tfm *tfm, u8 *out)
 
 	/* Wipe context */
 	memset(rctx, 0, sizeof(*rctx));
+
+	return 0;
 }
 
-static struct crypto_alg alg = {
-	.cra_name	 =	"rmd256",
-	.cra_driver_name =	"rmd256",
-	.cra_flags	 =	CRYPTO_ALG_TYPE_DIGEST,
-	.cra_blocksize	 =	RMD256_BLOCK_SIZE,
-	.cra_ctxsize	 =	sizeof(struct rmd256_ctx),
-	.cra_module	 =	THIS_MODULE,
-	.cra_list	 =	LIST_HEAD_INIT(alg.cra_list),
-	.cra_u		 =	{ .digest = {
-	.dia_digestsize	 =	RMD256_DIGEST_SIZE,
-	.dia_init	 =	rmd256_init,
-	.dia_update	 =	rmd256_update,
-	.dia_final	 =	rmd256_final } }
+static struct shash_alg alg = {
+	.digestsize	=	RMD256_DIGEST_SIZE,
+	.init		=	rmd256_init,
+	.update		=	rmd256_update,
+	.final		=	rmd256_final,
+	.descsize	=	sizeof(struct rmd256_ctx),
+	.base		=	{
+		.cra_name	 =	"rmd256",
+		.cra_flags	 =	CRYPTO_ALG_TYPE_SHASH,
+		.cra_blocksize	 =	RMD256_BLOCK_SIZE,
+		.cra_module	 =	THIS_MODULE,
+	}
 };
 
 static int __init rmd256_mod_init(void)
 {
-	return crypto_register_alg(&alg);
+	return crypto_register_shash(&alg);
 }
 
 static void __exit rmd256_mod_fini(void)
 {
-	crypto_unregister_alg(&alg);
+	crypto_unregister_shash(&alg);
 }
 
 module_init(rmd256_mod_init);
@@ -340,5 +345,3 @@ module_exit(rmd256_mod_fini);
 
 MODULE_LICENSE("GPL");
 MODULE_DESCRIPTION("RIPEMD-256 Message Digest");

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

* [PATCH 6/6] crypto: rmd320 - Switch to shash
  2008-12-01 13:44       ` Herbert Xu
                           ` (4 preceding siblings ...)
  2008-12-01 13:47         ` [PATCH 5/6] crypto: rmd256 " Herbert Xu
@ 2008-12-01 13:47         ` Herbert Xu
  2008-12-01 13:51         ` [RFC PATCH 5/5] crypto: make struct aead percpu data Herbert Xu
  2008-12-01 13:55         ` Steffen Klassert
  7 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:47 UTC (permalink / raw)
  To: Steffen Klassert, netdev, davem, klassert, Linux Crypto Mailing List

crypto: rmd320 - Switch to shash

This patch changes rmd320 to the new shash interface.

Signed-off-by: Herbert Xu <herbert@gondor.apana.org.au>
---

 crypto/rmd320.c |   61 +++++++++++++++++++++++++++++---------------------------
 1 file changed, 32 insertions(+), 29 deletions(-)

diff --git a/crypto/rmd320.c b/crypto/rmd320.c
index b143d66..86becab 100644
--- a/crypto/rmd320.c
+++ b/crypto/rmd320.c
@@ -13,11 +13,10 @@
  * any later version.
  *
  */
+#include <crypto/internal/hash.h>
 #include <linux/init.h>
 #include <linux/module.h>
 #include <linux/mm.h>
-#include <linux/crypto.h>
-#include <linux/cryptohash.h>
 #include <linux/types.h>
 #include <asm/byteorder.h>
 
@@ -280,9 +279,9 @@ static void rmd320_transform(u32 *state, const __le32 *in)
 	return;
 }
 
-static void rmd320_init(struct crypto_tfm *tfm)
+static int rmd320_init(struct shash_desc *desc)
 {
-	struct rmd320_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd320_ctx *rctx = shash_desc_ctx(desc);
 
 	rctx->byte_count = 0;
 
@@ -298,12 +297,14 @@ static void rmd320_init(struct crypto_tfm *tfm)
 	rctx->state[9] = RMD_H9;
 
 	memset(rctx->buffer, 0, sizeof(rctx->buffer));
+
+	return 0;
 }
 
-static void rmd320_update(struct crypto_tfm *tfm, const u8 *data,
-			  unsigned int len)
+static int rmd320_update(struct shash_desc *desc, const u8 *data,
+			 unsigned int len)
 {
-	struct rmd320_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd320_ctx *rctx = shash_desc_ctx(desc);
 	const u32 avail = sizeof(rctx->buffer) - (rctx->byte_count & 0x3f);
 
 	rctx->byte_count += len;
@@ -312,7 +313,7 @@ static void rmd320_update(struct crypto_tfm *tfm, const u8 *data,
 	if (avail > len) {
 		memcpy((char *)rctx->buffer + (sizeof(rctx->buffer) - avail),
 		       data, len);
-		return;
+		goto out;
 	}
 
 	memcpy((char *)rctx->buffer + (sizeof(rctx->buffer) - avail),
@@ -330,12 +331,15 @@ static void rmd320_update(struct crypto_tfm *tfm, const u8 *data,
 	}
 
 	memcpy(rctx->buffer, data, len);
+
+out:
+	return 0;
 }
 
 /* Add padding and return the message digest. */
-static void rmd320_final(struct crypto_tfm *tfm, u8 *out)
+static int rmd320_final(struct shash_desc *desc, u8 *out)
 {
-	struct rmd320_ctx *rctx = crypto_tfm_ctx(tfm);
+	struct rmd320_ctx *rctx = shash_desc_ctx(desc);
 	u32 i, index, padlen;
 	__le64 bits;
 	__le32 *dst = (__le32 *)out;
@@ -346,10 +350,10 @@ static void rmd320_final(struct crypto_tfm *tfm, u8 *out)
 	/* Pad out to 56 mod 64 */
 	index = rctx->byte_count & 0x3f;
 	padlen = (index < 56) ? (56 - index) : ((64+56) - index);
-	rmd320_update(tfm, padding, padlen);
+	rmd320_update(desc, padding, padlen);
 
 	/* Append length */
-	rmd320_update(tfm, (const u8 *)&bits, sizeof(bits));
+	rmd320_update(desc, (const u8 *)&bits, sizeof(bits));
 
 	/* Store state in digest */
 	for (i = 0; i < 10; i++)
@@ -357,31 +361,32 @@ static void rmd320_final(struct crypto_tfm *tfm, u8 *out)
 
 	/* Wipe context */
 	memset(rctx, 0, sizeof(*rctx));
+
+	return 0;
 }
 
-static struct crypto_alg alg = {
-	.cra_name	 =	"rmd320",
-	.cra_driver_name =	"rmd320",
-	.cra_flags	 =	CRYPTO_ALG_TYPE_DIGEST,
-	.cra_blocksize	 =	RMD320_BLOCK_SIZE,
-	.cra_ctxsize	 =	sizeof(struct rmd320_ctx),
-	.cra_module	 =	THIS_MODULE,
-	.cra_list	 =	LIST_HEAD_INIT(alg.cra_list),
-	.cra_u		 =	{ .digest = {
-	.dia_digestsize	 =	RMD320_DIGEST_SIZE,
-	.dia_init	 =	rmd320_init,
-	.dia_update	 =	rmd320_update,
-	.dia_final	 =	rmd320_final } }
+static struct shash_alg alg = {
+	.digestsize	=	RMD320_DIGEST_SIZE,
+	.init		=	rmd320_init,
+	.update		=	rmd320_update,
+	.final		=	rmd320_final,
+	.descsize	=	sizeof(struct rmd320_ctx),
+	.base		=	{
+		.cra_name	 =	"rmd320",
+		.cra_flags	 =	CRYPTO_ALG_TYPE_SHASH,
+		.cra_blocksize	 =	RMD320_BLOCK_SIZE,
+		.cra_module	 =	THIS_MODULE,
+	}
 };
 
 static int __init rmd320_mod_init(void)
 {
-	return crypto_register_alg(&alg);
+	return crypto_register_shash(&alg);
 }
 
 static void __exit rmd320_mod_fini(void)
 {
-	crypto_unregister_alg(&alg);
+	crypto_unregister_shash(&alg);
 }
 
 module_init(rmd320_mod_init);
@@ -389,5 +394,3 @@ module_exit(rmd320_mod_fini);
 
 MODULE_LICENSE("GPL");
 MODULE_DESCRIPTION("RIPEMD-320 Message Digest");
-
-MODULE_ALIAS("rmd320");

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

* Re: [RFC PATCH 5/5] crypto: make struct aead percpu data
  2008-12-01 13:44       ` Herbert Xu
                           ` (5 preceding siblings ...)
  2008-12-01 13:47         ` [PATCH 6/6] crypto: rmd320 " Herbert Xu
@ 2008-12-01 13:51         ` Herbert Xu
  2008-12-01 13:55         ` Steffen Klassert
  7 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-01 13:51 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: netdev, davem, klassert, Linux Crypto Mailing List

On Mon, Dec 01, 2008 at 09:44:48PM +0800, Herbert Xu wrote:
>
> If you could finish them for me I'd be most grateful :)

In case it isn't clear from the code, once we've converted all
the algorithms then we can start converting the existing users
across to the new interface.  In particular, authenc can be made
to use shash (or ahash) which would render the spinlock useless.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 5/5] crypto: make struct aead percpu data
  2008-12-01 13:44       ` Herbert Xu
                           ` (6 preceding siblings ...)
  2008-12-01 13:51         ` [RFC PATCH 5/5] crypto: make struct aead percpu data Herbert Xu
@ 2008-12-01 13:55         ` Steffen Klassert
  7 siblings, 0 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-01 13:55 UTC (permalink / raw)
  To: Herbert Xu; +Cc: netdev, davem, klassert, Linux Crypto Mailing List

On Mon, Dec 01, 2008 at 09:44:48PM +0800, Herbert Xu wrote:
> On Mon, Dec 01, 2008 at 02:36:54PM +0100, Steffen Klassert wrote:
> >
> > I searched for your shash work. Is there already some work in progress
> > aside from crc32?
> 
> I started on the algorithm conversion but have only made it as far
> as null and rmd*.  But it should serve as a good template for doing
> the rest.
> 
> If you could finish them for me I'd be most grateful :)
> 
> I'll push what I've got out now.

Thanks, I'll see what I can do :-)

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-01 11:15     ` Herbert Xu
@ 2008-12-02  7:58       ` Steffen Klassert
  2008-12-02  8:19         ` Herbert Xu
  0 siblings, 1 reply; 37+ messages in thread
From: Steffen Klassert @ 2008-12-02  7:58 UTC (permalink / raw)
  To: Herbert Xu; +Cc: David Miller, netdev, klassert

On Mon, Dec 01, 2008 at 07:15:01PM +0800, Herbert Xu wrote:
> On Mon, Dec 01, 2008 at 02:29:29AM -0800, David Miller wrote:
> >
> > Herbert, I'm not completely convinced of this line of thinking :-)
> 
> Well you've put some doubts into mind for a second there, but
> after thinking some more I'm sticking to my position :)
> 
> > Will crypto be faster than a routing cache lookup?  Because flow
> > seperation helps routing performance, significantly.
> 
> No to the first part, but that's not the point.  I'm saying that
> you can do flow separation within the crypto layer, well, for
> output anyway.  All you need to do is to provide some key to
> the crypto layer that tells it that ordering is only required
> within that key.  Alternatively, just allocate plenty of crypto
> tfms as these patches do, and feed the flows down to different
> tfms.
> 
> For input there's nothing we can do apart from processing completed
> requests in sequence so no flow information is going to help (and
> in fact if it did one could argue that the purpose of IPsec has been
> defeated :)
> 
> > And the reconstitution of the seperated crypto operations into the
> > original properly ordered flow can be done most cleanly (IMHO) in the
> > networking layer as far as I have seen so far.
> 
> Well once I've finished the GRO stuff I can give this a go.  But
> I'm pretty sure we can achieve exactly the same thing with only
> minimal changes to xfrm.  Really all it needs to do is allocate
> more tfm objects for the output path.
> 
> All the serialisation logic can stay in the crypto layer because
> AFAICS no network-specific knowledge is being used in this patch
> set.
> 

The parallelization/serialization interface is generic, so the crypto
layer can use it in the same way as xfrm.

The only network specific information that is used is the knowledge
of the xfrm_state that a packet uses. On serialization, this helps
to stay as parallel as possible. Packets that uses the same state must
go to the same cpu (because of the replay window) packets that use
different states can go to different cpus.

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-01  8:49 ` [RFC PATCH 0/5] IPsec parallelization Herbert Xu
  2008-12-01 10:29   ` David Miller
  2008-12-01 11:20   ` Andi Kleen
@ 2008-12-02  8:00   ` Steffen Klassert
  2 siblings, 0 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-02  8:00 UTC (permalink / raw)
  To: Herbert Xu; +Cc: netdev, davem, klassert

On Mon, Dec 01, 2008 at 04:49:02PM +0800, Herbert Xu wrote:
> On Mon, Dec 01, 2008 at 08:16:14AM +0100, Steffen Klassert wrote:
> > This is a first throw to try to parallelize the expensive part of xfrm by
> > using a generic parallelization/serialization method. This method uses the
> > remote softirq invocation infrastructure for parallelization and serialization.
> > With this method data objects can be processed in parallel, starting 
> > at some given point. After doing some expensive operations in parallel, 
> > it is possible to serialize again. The parallelized data objects return after
> > serialization in the order as they were before the parallelization. 
> > In the case of xfrm, this makes it possible to run the expensive part in
> > parallel without getting packet reordering.
> 
> I still think that you're much better off doing this in the
> crypto layer.  As it stands the only reason why this is attractive
> is because crypto is slow.
> 
> Pretty soon processors will start providing crypto support natively
> so this will no longer be the case.  I'd rather see this stuff
> contained in a small area instead of having it spread all over the
> place as this may become obsolete any day now.
> 

Now, that most of the crypto changes of this patchset are obsolete by
shash, 'spreading all over the place' is probaply not a reason any more
not to keep it in the network layer.

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-02  7:58       ` Steffen Klassert
@ 2008-12-02  8:19         ` Herbert Xu
  2008-12-02  8:44           ` Steffen Klassert
  0 siblings, 1 reply; 37+ messages in thread
From: Herbert Xu @ 2008-12-02  8:19 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: David Miller, netdev, klassert

On Tue, Dec 02, 2008 at 08:58:04AM +0100, Steffen Klassert wrote:
> 
> The parallelization/serialization interface is generic, so the crypto
> layer can use it in the same way as xfrm.

I don't really have an opinion on the padata stuff.  What I'm
objecting to is the way you're putting the parallelism into
xfrm.

> The only network specific information that is used is the knowledge
> of the xfrm_state that a packet uses. On serialization, this helps
> to stay as parallel as possible. Packets that uses the same state must
> go to the same cpu (because of the replay window) packets that use
> different states can go to different cpus.

Maybe I'm missing somthing, but all you're doing is paralleising
based on xfrm_state objects.  You can already do that trivially
in the crypto layer with no network-specific knowledge at all
because each xfrm_state allocates its own tfm objects.

What I'm proposing is that you create a single parallel crypto
algorithm template similar to cryptd (FWIW you could even just
extend cryptd to do this) that does the parallel processing
based on tfms.

This achieves exactly the same thing as your current patch-set
plus:

1) The uesr no longer has to make a system-wide choice of whether
to enable this, instead the control is per-SA through the usual
algorithm selection mechanism which means that this no longer
conflicts with async crypto;

2) There is no change to the xfrm code;

3) The same mechanism can benefit other crypto users such as
disk encryption.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-02  8:19         ` Herbert Xu
@ 2008-12-02  8:44           ` Steffen Klassert
  2008-12-02  8:50             ` Herbert Xu
  2008-12-02  8:53             ` Herbert Xu
  0 siblings, 2 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-02  8:44 UTC (permalink / raw)
  To: Herbert Xu; +Cc: David Miller, netdev, klassert

On Tue, Dec 02, 2008 at 04:19:48PM +0800, Herbert Xu wrote:
> On Tue, Dec 02, 2008 at 08:58:04AM +0100, Steffen Klassert wrote:
> > 
> > The parallelization/serialization interface is generic, so the crypto
> > layer can use it in the same way as xfrm.
> 
> I don't really have an opinion on the padata stuff.  What I'm
> objecting to is the way you're putting the parallelism into
> xfrm.
> 
> > The only network specific information that is used is the knowledge
> > of the xfrm_state that a packet uses. On serialization, this helps
> > to stay as parallel as possible. Packets that uses the same state must
> > go to the same cpu (because of the replay window) packets that use
> > different states can go to different cpus.
> 
> Maybe I'm missing somthing, but all you're doing is paralleising
> based on xfrm_state objects.  You can already do that trivially
> in the crypto layer with no network-specific knowledge at all
> because each xfrm_state allocates its own tfm objects.

No, it's not just parallelizing based on xfrm_states. We are running
in parallel even within the same state. That's why I'm getting a 
bandwith up to 900Mbit/s when sending one tcp stream. Parallelizing
based on state would not help that much if you are sending just one
stream.

What happens is that we are sending the packets round robin to
the different cpus on parallelization. On serialization, the
packets will be brought back to the order as they were before the 
parallelization. And to not just send all the packets back to one cpu,
I'm doing parallelization based on xfrm_state after that.

> 
> What I'm proposing is that you create a single parallel crypto
> algorithm template similar to cryptd (FWIW you could even just
> extend cryptd to do this) that does the parallel processing
> based on tfms.
> 
> This achieves exactly the same thing as your current patch-set
> plus:
> 
> 1) The uesr no longer has to make a system-wide choice of whether
> to enable this, instead the control is per-SA through the usual
> algorithm selection mechanism which means that this no longer
> conflicts with async crypto;
> 
> 2) There is no change to the xfrm code;
> 
> 3) The same mechanism can benefit other crypto users such as
> disk encryption.

The padata stuff is generic, so it can be used even for disk
encryption or for anything else that should run in parallel but
needs a certain order at a given point.

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-02  8:44           ` Steffen Klassert
@ 2008-12-02  8:50             ` Herbert Xu
  2008-12-02  9:21               ` Steffen Klassert
  2008-12-02  8:53             ` Herbert Xu
  1 sibling, 1 reply; 37+ messages in thread
From: Herbert Xu @ 2008-12-02  8:50 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: David Miller, netdev, klassert

On Tue, Dec 02, 2008 at 09:44:19AM +0100, Steffen Klassert wrote:
>
> > Maybe I'm missing somthing, but all you're doing is paralleising
> > based on xfrm_state objects.  You can already do that trivially
> > in the crypto layer with no network-specific knowledge at all
> > because each xfrm_state allocates its own tfm objects.
> 
> No, it's not just parallelizing based on xfrm_states. We are running
> in parallel even within the same state. That's why I'm getting a 
> bandwith up to 900Mbit/s when sending one tcp stream. Parallelizing
> based on state would not help that much if you are sending just one
> stream.

I understand.  What I meant was maintaining ordering within an
xfrm_state.  Since each xfrm_state has its own tfms, all you have
to do is round-robin within each crypto tfm while maintaining
ordering to achieve the same result.

> > This achieves exactly the same thing as your current patch-set
> > plus:
> > 
> > 1) The uesr no longer has to make a system-wide choice of whether
> > to enable this, instead the control is per-SA through the usual
> > algorithm selection mechanism which means that this no longer
> > conflicts with async crypto;
> > 
> > 2) There is no change to the xfrm code;
> > 
> > 3) The same mechanism can benefit other crypto users such as
> > disk encryption.
> 
> The padata stuff is generic, so it can be used even for disk
> encryption or for anything else that should run in parallel but
> needs a certain order at a given point.

What about the first issue?

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-02  8:44           ` Steffen Klassert
  2008-12-02  8:50             ` Herbert Xu
@ 2008-12-02  8:53             ` Herbert Xu
  2008-12-02  9:39               ` Steffen Klassert
  1 sibling, 1 reply; 37+ messages in thread
From: Herbert Xu @ 2008-12-02  8:53 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: David Miller, netdev, klassert

On Tue, Dec 02, 2008 at 09:44:19AM +0100, Steffen Klassert wrote:
>
> > 3) The same mechanism can benefit other crypto users such as
> > disk encryption.
> 
> The padata stuff is generic, so it can be used even for disk
> encryption or for anything else that should run in parallel but
> needs a certain order at a given point.

The padata stuff might be great, but does that mean that we really
want to reimplement exactly the same logic in two different places
when we can do it just once in the crypto layer (which could also
use padata as you suggested)?

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-02  8:50             ` Herbert Xu
@ 2008-12-02  9:21               ` Steffen Klassert
  0 siblings, 0 replies; 37+ messages in thread
From: Steffen Klassert @ 2008-12-02  9:21 UTC (permalink / raw)
  To: Herbert Xu; +Cc: David Miller, netdev, klassert

On Tue, Dec 02, 2008 at 04:50:52PM +0800, Herbert Xu wrote:
> 
> > > This achieves exactly the same thing as your current patch-set
> > > plus:
> > > 
> > > 1) The uesr no longer has to make a system-wide choice of whether
> > > to enable this, instead the control is per-SA through the usual
> > > algorithm selection mechanism which means that this no longer
> > > conflicts with async crypto;
> > > 
> > > 2) There is no change to the xfrm code;
> > > 
> > > 3) The same mechanism can benefit other crypto users such as
> > > disk encryption.
> > 
> > The padata stuff is generic, so it can be used even for disk
> > encryption or for anything else that should run in parallel but
> > needs a certain order at a given point.
> 
> What about the first issue?
> 

Yes, that's a point. If it is in the network layer we probaply need this
system-wide choice because it would bring a useless overhead to
the sk_buff if this thing is not used. So this could be a plus for
putting it to the crypto layer if we don't need this system-wide choice
there.

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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-02  8:53             ` Herbert Xu
@ 2008-12-02  9:39               ` Steffen Klassert
  2008-12-02 10:37                 ` Herbert Xu
  0 siblings, 1 reply; 37+ messages in thread
From: Steffen Klassert @ 2008-12-02  9:39 UTC (permalink / raw)
  To: Herbert Xu; +Cc: David Miller, netdev, klassert

On Tue, Dec 02, 2008 at 04:53:25PM +0800, Herbert Xu wrote:
> On Tue, Dec 02, 2008 at 09:44:19AM +0100, Steffen Klassert wrote:
> >
> > > 3) The same mechanism can benefit other crypto users such as
> > > disk encryption.
> > 
> > The padata stuff is generic, so it can be used even for disk
> > encryption or for anything else that should run in parallel but
> > needs a certain order at a given point.
> 
> The padata stuff might be great, but does that mean that we really
> want to reimplement exactly the same logic in two different places
> when we can do it just once in the crypto layer (which could also
> use padata as you suggested)?
> 

I would not mind to move the padata hooks to the crypto layer.

But what's with the interface? Should it be moved to the crypto
layer too? I'm not sure, but perhaps it could find users beyond
the crypto layer.


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

* Re: [RFC PATCH 0/5] IPsec parallelization
  2008-12-02  9:39               ` Steffen Klassert
@ 2008-12-02 10:37                 ` Herbert Xu
  0 siblings, 0 replies; 37+ messages in thread
From: Herbert Xu @ 2008-12-02 10:37 UTC (permalink / raw)
  To: Steffen Klassert; +Cc: David Miller, netdev, klassert

On Tue, Dec 02, 2008 at 10:39:11AM +0100, Steffen Klassert wrote:
>
> I would not mind to move the padata hooks to the crypto layer.
> 
> But what's with the interface? Should it be moved to the crypto
> layer too? I'm not sure, but perhaps it could find users beyond
> the crypto layer.

As I said I have no objections to the padata stuff itself so
I don't really mind either way.

Cheers,
-- 
Visit Openswan at http://www.openswan.org/
Email: Herbert Xu ~{PmV>HI~} <herbert@gondor.apana.org.au>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

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

end of thread, other threads:[~2008-12-02 10:37 UTC | newest]

Thread overview: 37+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2008-12-01  7:16 [RFC PATCH 0/5] IPsec parallelization Steffen Klassert
2008-12-01  7:17 ` [RFC PATCH 1/5] padata: generic interface for parallel processing Steffen Klassert
2008-12-01  7:17 ` [RFC PATCH 2/5] xfrm: add possibility " Steffen Klassert
2008-12-01  7:19 ` [RFC PATCH 3/5] crypto: add possibility to force sync transform Steffen Klassert
2008-12-01 11:22   ` Herbert Xu
2008-12-01 13:19     ` Steffen Klassert
2008-12-01  7:19 ` [RFC PATCH 4/5] crypto: allow allocation of percpu crypto transforms Steffen Klassert
2008-12-01 11:38   ` Herbert Xu
2008-12-01 12:25     ` David Miller
2008-12-01 12:38       ` Herbert Xu
2008-12-01 13:21         ` Steffen Klassert
2008-12-01  7:20 ` [RFC PATCH 5/5] crypto: make struct aead percpu data Steffen Klassert
2008-12-01 11:40   ` Herbert Xu
2008-12-01 13:36     ` Steffen Klassert
2008-12-01 13:44       ` Herbert Xu
2008-12-01 13:47         ` [PATCH 1/6] crypto: hash - Make setkey optional Herbert Xu
2008-12-01 13:47         ` [PATCH 2/6] crypto: null - Switch to shash Herbert Xu
2008-12-01 13:47         ` [PATCH 3/6] crypto: rmd128 " Herbert Xu
2008-12-01 13:47         ` [PATCH 4/6] crypto: rmd160 " Herbert Xu
2008-12-01 13:47         ` [PATCH 5/6] crypto: rmd256 " Herbert Xu
2008-12-01 13:47         ` [PATCH 6/6] crypto: rmd320 " Herbert Xu
2008-12-01 13:51         ` [RFC PATCH 5/5] crypto: make struct aead percpu data Herbert Xu
2008-12-01 13:55         ` Steffen Klassert
2008-12-01  8:49 ` [RFC PATCH 0/5] IPsec parallelization Herbert Xu
2008-12-01 10:29   ` David Miller
2008-12-01 11:15     ` Herbert Xu
2008-12-02  7:58       ` Steffen Klassert
2008-12-02  8:19         ` Herbert Xu
2008-12-02  8:44           ` Steffen Klassert
2008-12-02  8:50             ` Herbert Xu
2008-12-02  9:21               ` Steffen Klassert
2008-12-02  8:53             ` Herbert Xu
2008-12-02  9:39               ` Steffen Klassert
2008-12-02 10:37                 ` Herbert Xu
2008-12-01 11:20   ` Andi Kleen
2008-12-01 13:39     ` Herbert Xu
2008-12-02  8:00   ` Steffen Klassert

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.