All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH v5 0/7] crypto: SHA1 multibuffer implementation
       [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
@ 2014-07-22 22:09 ` Tim Chen
  2014-07-22 22:09 ` [PATCH v5 1/7] sched: Add function single_task_running to let a task check if it is the only task running on a cpu Tim Chen
                   ` (6 subsequent siblings)
  7 siblings, 0 replies; 15+ messages in thread
From: Tim Chen @ 2014-07-22 22:09 UTC (permalink / raw)
  To: Herbert Xu, H. Peter Anvin, David S.Miller, Peter Zijlstra, Ingo Molnar
  Cc: Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Tim Chen, Jussi Kivilinna, Thomas Gleixner,
	Tadeusz Struk, tkhai, linux-crypto, linux-kernel

Herbert,

I've updated my implementation from v4 to have the multi-buffer daemon
flush the jobs if there're no other jobs running on the cpu.  The flusher
logic is reorganized so it is contained in mcryptd per Peter's feedback.
The multi-buffer crypto daemon now directly takes advantage of the cpu
if there're no other jobs running on cpu.  The flush routine was moved
out of the notifier path to crypto daemon.  To check that there're
no other jobs running, I've added the single_task_running function to
obtain the information.

Please note also that a separate bug fix to the crypto scatter gather list walk
for the null string needs to be incorporated, which I encountered during
my testing: http://marc.info/?l=linux-crypto-vger&m=140503429412699&w=2

In this patch series, we introduce the multi-buffer crypto algorithm on
x86_64 and apply it to SHA1 hash computation.  The multi-buffer technique
takes advantage of the 8 data lanes in the AVX2 registers and allows
computation to be performed on data from multiple jobs in parallel.
This allows us to parallelize computations when data inter-dependency in
a single crypto job prevents us to fully parallelize our computations.
The algorithm can be extended to other hashing and encryption schemes
in the future.

On multi-buffer SHA1 computation with AVX2, we saw throughput increase
up to 2.2x over the existing x86_64 single buffer AVX2 algorithm.

The multi-buffer crypto algorithm is described in the following paper:
Processing Multiple Buffers in Parallel to Increase Performance on
Intel® Architecture Processors
http://www.intel.com/content/www/us/en/communications/communications-ia-multi-buffer-paper.html

The outline of the algorithm is sketched below:
Any driver requesting the crypto service will place an async crypto
request on the workqueue.  The multi-buffer crypto daemon will pull
request from work queue and put each request in an empty data lane for
multi-buffer crypto computation.  When all the empty lanes are filled,
computation will commence on the jobs in parallel and the job with the
shortest remaining buffer will get completed and be returned.  To prevent
prolonged stall when there is no new jobs arriving, we will flush a
crypto job if it has not been completed after a maximum allowable delay,
or when cpu becomes idle and cpu cycles become available.

The multi-buffer algorithm necessitates mapping multiple scatter gather
buffers to linear addresses simultaneously. The crypto daemon may need
to sleep and yield the cpu to work on something else from time to time.
We made a change to not use kmap_atomic to do scatter-gather buffer
mapping and take advantage of the fact that we can directly translate
address the buffer's address to its linear address with x86_64.
To accommodate the fragmented nature of scatter-gather, we will keep
submitting the next scatter-buffer fragment for a job for multi-buffer
computation until a job is completed and no more buffer fragments remain.
At that time we will pull a new job to fill the now empty data slot.
We call a get_completed_job function to check whether there are other
jobs that have been completed when we job when we have no new job arrival
to prevent extraneous delay in returning any completed jobs.

The multi-buffer algorithm should be used for cases where crypto jobs
submissions are at a reasonable high rate.  For low crypto job submission
rate, this algorithm will not be beneficial. The reason is at low rate,
we do not fill out the data lanes before flushing the jobs instead of
processing them with all the data lanes full.  We will miss the benefit
of parallel computation, and adding delay to the processing of the crypto
job at the same time.  Some tuning of the maximum latency parameter may
be needed to get the best performance.

Note that the tcrypt SHA1 speed test, we wait for a previous job to
be completed before submitting a new job.  Hence this is not a valid
test for multi-buffer algorithm as it requires multiple outstanding jobs
submitted to fill the all data lanes to be effective (i.e. 8 outstanding
jobs for the AVX2 case).

Feedbacks and testings will be most welcomed.

Tim Chen

Change log:

v5.
1. Flush the job from the crypto daemon if the crypto daemon has no
other jobs to process and no other tasks are running on the cpu.
2. Change implementation of the idle flush so we do not hook into
the idle notifier. 

v4
1. Move the early flush of jobs when cpu becomes idle to crypto thread.
2. Move shash_ahash_mcryptd_digest to mcryptd.c 
http://www.gossamer-threads.com/lists/linux/kernel/1964734

v3
1. Add notifier to multi-buffer algorithm to flush job when the cpu
goes to idle to take advantage of available cpu cycles. 
2. Clean up of error messages.
http://marc.info/?l=linux-crypto-vger&m=140252063401632&w=2

v2
1. Change the sha1 crypto walk to use the new crypto_ahash_walk
interface for proper kmap.
2. Drop the hack that map buffer in crypto_hash_walk without kmap_atomic
as the new crypto_ahash_walk interface is merged.
3. Reorganize some
of the mcryptd hash interface code from ahash.c to mcryptd.c
http://marc.info/?l=linux-crypto-vger&m=140088627927559&w=2

v1
refer to: http://www.spinics.net/lists/linux-crypto/msg10993.html

Tim Chen (7):
  sched: Add function single_task_running to let a task check if it is
    the only task running on a cpu
  crypto: SHA1 multibuffer crypto hash infrastructure
  crypto: SHA1 multibuffer crypto opportunistic flush
  crypto: SHA1 multibuffer algorithm data structures
  crypto: SHA1 multibuffer submit and flush routines for AVX2
  crypto: SHA1 multibuffer crypto computation (x8 AVX2)
  crypto: SHA1 multibuffer scheduler

 arch/x86/crypto/Makefile                         |   2 +
 arch/x86/crypto/sha-mb/Makefile                  |  11 +
 arch/x86/crypto/sha-mb/sha1_mb.c                 | 935 +++++++++++++++++++++++
 arch/x86/crypto/sha-mb/sha1_mb_mgr_datastruct.S  | 287 +++++++
 arch/x86/crypto/sha-mb/sha1_mb_mgr_flush_avx2.S  | 327 ++++++++
 arch/x86/crypto/sha-mb/sha1_mb_mgr_init_avx2.c   |  64 ++
 arch/x86/crypto/sha-mb/sha1_mb_mgr_submit_avx2.S | 228 ++++++
 arch/x86/crypto/sha-mb/sha1_x8_avx2.S            | 472 ++++++++++++
 arch/x86/crypto/sha-mb/sha_mb_ctx.h              | 136 ++++
 arch/x86/crypto/sha-mb/sha_mb_mgr.h              | 110 +++
 crypto/Kconfig                                   |  30 +
 crypto/Makefile                                  |   1 +
 crypto/mcryptd.c                                 | 705 +++++++++++++++++
 include/crypto/internal/hash.h                   |   9 +
 include/crypto/mcryptd.h                         | 112 +++
 include/linux/sched.h                            |   1 +
 kernel/sched/core.c                              |  12 +
 17 files changed, 3442 insertions(+)
 create mode 100644 arch/x86/crypto/sha-mb/Makefile
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb.c
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb_mgr_datastruct.S
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb_mgr_flush_avx2.S
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb_mgr_init_avx2.c
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb_mgr_submit_avx2.S
 create mode 100644 arch/x86/crypto/sha-mb/sha1_x8_avx2.S
 create mode 100644 arch/x86/crypto/sha-mb/sha_mb_ctx.h
 create mode 100644 arch/x86/crypto/sha-mb/sha_mb_mgr.h
 create mode 100644 crypto/mcryptd.c
 create mode 100644 include/crypto/mcryptd.h

-- 
1.7.11.7

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

* [PATCH v5 1/7] sched: Add function single_task_running to let a task check if it is the only task running on a cpu
       [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
  2014-07-22 22:09 ` [PATCH v5 0/7] crypto: SHA1 multibuffer implementation Tim Chen
@ 2014-07-22 22:09 ` Tim Chen
  2014-07-22 22:09 ` [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure Tim Chen
                   ` (5 subsequent siblings)
  7 siblings, 0 replies; 15+ messages in thread
From: Tim Chen @ 2014-07-22 22:09 UTC (permalink / raw)
  To: Herbert Xu, H. Peter Anvin, David S.Miller, Peter Zijlstra, Ingo Molnar
  Cc: Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Tim Chen, Jussi Kivilinna, Thomas Gleixner,
	Tadeusz Struk, tkhai, linux-crypto, linux-kernel

This function will help an async task processing batched jobs from workqueue
decide if it wants to keep processing on more chunks of batched work that
can be delayed, or to accumulate more work for more efficient batched
processing later.

If no other tasks are running on the cpu, it can take advantgae of
the available cpu cycles to complete the existing accumulated work for
immediate processing to minimize delay, otherwise it will yield.

Signed-off-by: Tim Chen <tim.c.chen@linux.intel.com>
---
 include/linux/sched.h |  1 +
 kernel/sched/core.c   | 12 ++++++++++++
 2 files changed, 13 insertions(+)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index 7cb07fd..eefa943 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -168,6 +168,7 @@ extern int nr_threads;
 DECLARE_PER_CPU(unsigned long, process_counts);
 extern int nr_processes(void);
 extern unsigned long nr_running(void);
+extern bool single_task_running(void);
 extern unsigned long nr_iowait(void);
 extern unsigned long nr_iowait_cpu(int cpu);
 extern unsigned long this_cpu_load(void);
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 9cae286..03c8675 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -2283,6 +2283,18 @@ unsigned long nr_running(void)
 	return sum;
 }
 
+/*
+ * Check if only the current task is running on the cpu.
+ */
+bool single_task_running(void)
+{
+	if (cpu_rq(smp_processor_id())->nr_running == 1)
+		return true;
+	else
+		return false;
+}
+EXPORT_SYMBOL(single_task_running);
+
 unsigned long long nr_context_switches(void)
 {
 	int i;
-- 
1.7.11.7

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

* [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure
       [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
  2014-07-22 22:09 ` [PATCH v5 0/7] crypto: SHA1 multibuffer implementation Tim Chen
  2014-07-22 22:09 ` [PATCH v5 1/7] sched: Add function single_task_running to let a task check if it is the only task running on a cpu Tim Chen
@ 2014-07-22 22:09 ` Tim Chen
  2014-07-25  7:08   ` Peter Zijlstra
  2014-07-25  7:26   ` Peter Zijlstra
  2014-07-22 22:09 ` [PATCH v5 3/7] crypto: SHA1 multibuffer crypto opportunistic flush Tim Chen
                   ` (4 subsequent siblings)
  7 siblings, 2 replies; 15+ messages in thread
From: Tim Chen @ 2014-07-22 22:09 UTC (permalink / raw)
  To: Herbert Xu, H. Peter Anvin, David S.Miller, Peter Zijlstra, Ingo Molnar
  Cc: Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Tim Chen, Jussi Kivilinna, Thomas Gleixner,
	Tadeusz Struk, tkhai, linux-crypto, linux-kernel

This patch introduces the multi-buffer crypto daemon which is responsible
for submitting crypto jobs in a work queue to the responsible multi-buffer
crypto algorithm.  The idea of the multi-buffer algorihtm is to put
data streams from multiple jobs in a wide (AVX2) register and then
take advantage of SIMD instructions to do crypto computation on several
buffers simultaneously.

The multi-buffer crypto daemon is also responsbile for flushing the
remaining buffers to complete the computation if no new buffers arrive
for a while.

Signed-off-by: Tim Chen <tim.c.chen@linux.intel.com>
---
 crypto/Kconfig                 |  30 ++
 crypto/Makefile                |   1 +
 crypto/mcryptd.c               | 672 +++++++++++++++++++++++++++++++++++++++++
 include/crypto/internal/hash.h |   9 +
 include/crypto/mcryptd.h       | 112 +++++++
 5 files changed, 824 insertions(+)
 create mode 100644 crypto/mcryptd.c
 create mode 100644 include/crypto/mcryptd.h

diff --git a/crypto/Kconfig b/crypto/Kconfig
index ce4012a..5d3c711 100644
--- a/crypto/Kconfig
+++ b/crypto/Kconfig
@@ -157,6 +157,20 @@ config CRYPTO_CRYPTD
 	  converts an arbitrary synchronous software crypto algorithm
 	  into an asynchronous algorithm that executes in a kernel thread.
 
+config CRYPTO_MCRYPTD
+	tristate "Software async multi-buffer crypto daemon"
+	select CRYPTO_BLKCIPHER
+	select CRYPTO_HASH
+	select CRYPTO_MANAGER
+	select CRYPTO_WORKQUEUE
+	help
+	  This is a generic software asynchronous crypto daemon that
+	  provides the kernel thread to assist multi-buffer crypto
+	  algorithms for submitting jobs and flushing jobs in multi-buffer
+	  crypto algorithms.  Multi-buffer crypto algorithms are executed
+	  in the context of this kernel thread and drivers can post
+	  their crypto request asyncrhously and process by this daemon.
+
 config CRYPTO_AUTHENC
 	tristate "Authenc support"
 	select CRYPTO_AEAD
@@ -547,6 +561,22 @@ config CRYPTO_SHA1_PPC
 	  This is the powerpc hardware accelerated implementation of the
 	  SHA-1 secure hash standard (FIPS 180-1/DFIPS 180-2).
 
+config CRYPTO_SHA1_MB
+	tristate "SHA1 digest algorithm (x86_64 Multi-Buffer, Experimental)"
+	depends on X86 && 64BIT
+	select CRYPTO_SHA1
+	select CRYPTO_HASH
+	select CRYPTO_MCRYPTD
+	help
+	  SHA-1 secure hash standard (FIPS 180-1/DFIPS 180-2) implemented
+	  using multi-buffer technique.  This algorithm computes on
+	  multiple data lanes concurrently with SIMD instructions for
+	  better throughput.  It should not be enabled by default but
+	  used when there is significant amount of work to keep the keep
+	  the data lanes filled to get performance benefit.  If the data
+	  lanes remain unfilled, a flush operation will be initiated to
+	  process the crypto jobs, adding a slight latency.
+
 config CRYPTO_SHA256
 	tristate "SHA224 and SHA256 digest algorithm"
 	select CRYPTO_HASH
diff --git a/crypto/Makefile b/crypto/Makefile
index 38e64231..27dbbad 100644
--- a/crypto/Makefile
+++ b/crypto/Makefile
@@ -60,6 +60,7 @@ obj-$(CONFIG_CRYPTO_GCM) += gcm.o
 obj-$(CONFIG_CRYPTO_CCM) += ccm.o
 obj-$(CONFIG_CRYPTO_PCRYPT) += pcrypt.o
 obj-$(CONFIG_CRYPTO_CRYPTD) += cryptd.o
+obj-$(CONFIG_CRYPTO_MCRYPTD) += mcryptd.o
 obj-$(CONFIG_CRYPTO_DES) += des_generic.o
 obj-$(CONFIG_CRYPTO_FCRYPT) += fcrypt.o
 obj-$(CONFIG_CRYPTO_BLOWFISH) += blowfish_generic.o
diff --git a/crypto/mcryptd.c b/crypto/mcryptd.c
new file mode 100644
index 0000000..622d6b4
--- /dev/null
+++ b/crypto/mcryptd.c
@@ -0,0 +1,672 @@
+/*
+ * Software multibuffer async crypto daemon.
+ *
+ * Copyright (c) 2014 Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ * Adapted from crypto daemon.
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License as published by the Free
+ * Software Foundation; either version 2 of the License, or (at your option)
+ * any later version.
+ *
+ */
+
+#include <crypto/algapi.h>
+#include <crypto/internal/hash.h>
+#include <crypto/internal/aead.h>
+#include <crypto/mcryptd.h>
+#include <crypto/crypto_wq.h>
+#include <linux/err.h>
+#include <linux/init.h>
+#include <linux/kernel.h>
+#include <linux/list.h>
+#include <linux/module.h>
+#include <linux/scatterlist.h>
+#include <linux/sched.h>
+#include <linux/slab.h>
+#include <linux/hardirq.h>
+
+#define MCRYPTD_MAX_CPU_QLEN 100
+#define MCRYPTD_BATCH 9
+
+static void *mcryptd_alloc_instance(struct crypto_alg *alg, unsigned int head,
+				   unsigned int tail);
+
+struct mcryptd_flush_list {
+	struct list_head list;
+	struct mutex lock;
+};
+
+struct mcryptd_flush_list __percpu *mcryptd_flist;
+
+struct hashd_instance_ctx {
+	struct crypto_shash_spawn spawn;
+	struct mcryptd_queue *queue;
+};
+
+static void mcryptd_queue_worker(struct work_struct *work);
+
+void mcryptd_arm_flusher(struct mcryptd_alg_cstate *cstate, unsigned long delay)
+{
+	struct mcryptd_flush_list *flist;
+
+	if (!cstate->flusher_engaged) {
+		/* put the flusher on the flush list */
+		flist = per_cpu_ptr(mcryptd_flist, smp_processor_id());
+		mutex_lock(&flist->lock);
+		list_add_tail(&cstate->flush_list, &flist->list);
+		cstate->flusher_engaged = true;
+		cstate->next_flush = jiffies + delay;
+		queue_delayed_work_on(smp_processor_id(), kcrypto_wq,
+			&cstate->flush, delay);
+		mutex_unlock(&flist->lock);
+	}
+}
+EXPORT_SYMBOL(mcryptd_arm_flusher);
+
+static int mcryptd_init_queue(struct mcryptd_queue *queue,
+			     unsigned int max_cpu_qlen)
+{
+	int cpu;
+	struct mcryptd_cpu_queue *cpu_queue;
+
+	queue->cpu_queue = alloc_percpu(struct mcryptd_cpu_queue);
+	pr_debug("mqueue:%p mcryptd_cpu_queue %p\n", queue, queue->cpu_queue);
+	if (!queue->cpu_queue)
+		return -ENOMEM;
+	for_each_possible_cpu(cpu) {
+		cpu_queue = per_cpu_ptr(queue->cpu_queue, cpu);
+		pr_debug("cpu_queue #%d %p\n", cpu, queue->cpu_queue);
+		crypto_init_queue(&cpu_queue->queue, max_cpu_qlen);
+		INIT_WORK(&cpu_queue->work, mcryptd_queue_worker);
+	}
+	return 0;
+}
+
+static void mcryptd_fini_queue(struct mcryptd_queue *queue)
+{
+	int cpu;
+	struct mcryptd_cpu_queue *cpu_queue;
+
+	for_each_possible_cpu(cpu) {
+		cpu_queue = per_cpu_ptr(queue->cpu_queue, cpu);
+		BUG_ON(cpu_queue->queue.qlen);
+	}
+	free_percpu(queue->cpu_queue);
+}
+
+static int mcryptd_enqueue_request(struct mcryptd_queue *queue,
+				  struct crypto_async_request *request,
+				  struct mcryptd_hash_request_ctx *rctx)
+{
+	int cpu, err;
+	struct mcryptd_cpu_queue *cpu_queue;
+
+	cpu = get_cpu();
+	cpu_queue = this_cpu_ptr(queue->cpu_queue);
+	rctx->tag.cpu = cpu;
+
+	err = crypto_enqueue_request(&cpu_queue->queue, request);
+	pr_debug("enqueue request: cpu %d cpu_queue %p request %p\n",
+		 cpu, cpu_queue, request);
+	queue_work_on(cpu, kcrypto_wq, &cpu_queue->work);
+	put_cpu();
+
+	return err;
+}
+
+/* Called in workqueue context, do one real cryption work (via
+ * req->complete) and reschedule itself if there are more work to
+ * do. */
+static void mcryptd_queue_worker(struct work_struct *work)
+{
+	struct mcryptd_cpu_queue *cpu_queue;
+	struct crypto_async_request *req, *backlog;
+	int i;
+
+	/*
+	 * Need to loop through more than once for multi-buffer to
+	 * be effective.
+	 */
+
+	cpu_queue = container_of(work, struct mcryptd_cpu_queue, work);
+	i = 0;
+	while (i < MCRYPTD_BATCH || single_task_running()) {
+		/*
+		 * preempt_disable/enable is used to prevent
+		 * being preempted by mcryptd_enqueue_request()
+		 */
+		local_bh_disable();
+		preempt_disable();
+		backlog = crypto_get_backlog(&cpu_queue->queue);
+		req = crypto_dequeue_request(&cpu_queue->queue);
+		preempt_enable();
+		local_bh_enable();
+
+		if (!req)
+			return;
+
+		if (backlog)
+			backlog->complete(backlog, -EINPROGRESS);
+		req->complete(req, 0);
+		if (!cpu_queue->queue.qlen)
+			return;
+		++i;
+	}
+	if (cpu_queue->queue.qlen)
+		queue_work(kcrypto_wq, &cpu_queue->work);
+}
+
+void mcryptd_flusher(struct work_struct *__work)
+{
+	struct	mcryptd_alg_cstate	*alg_cpu_state;
+	struct	mcryptd_alg_state	*alg_state;
+	struct	mcryptd_flush_list	*flist;
+	int	cpu;
+
+	cpu = smp_processor_id();
+	alg_cpu_state = container_of(to_delayed_work(__work),
+				     struct mcryptd_alg_cstate, flush);
+	alg_state = alg_cpu_state->alg_state;
+	if (alg_cpu_state->cpu != cpu)
+		pr_debug("mcryptd error: work on cpu %d, should be cpu %d\n",
+				cpu, alg_cpu_state->cpu);
+
+	if (alg_cpu_state->flusher_engaged) {
+		flist = per_cpu_ptr(mcryptd_flist, cpu);
+		mutex_lock(&flist->lock);
+		list_del(&alg_cpu_state->flush_list);
+		alg_cpu_state->flusher_engaged = false;
+		mutex_unlock(&flist->lock);
+		alg_state->flusher(alg_cpu_state);
+	}
+}
+EXPORT_SYMBOL_GPL(mcryptd_flusher);
+
+static inline struct mcryptd_queue *mcryptd_get_queue(struct crypto_tfm *tfm)
+{
+	struct crypto_instance *inst = crypto_tfm_alg_instance(tfm);
+	struct mcryptd_instance_ctx *ictx = crypto_instance_ctx(inst);
+
+	return ictx->queue;
+}
+
+static void *mcryptd_alloc_instance(struct crypto_alg *alg, unsigned int head,
+				   unsigned int tail)
+{
+	char *p;
+	struct crypto_instance *inst;
+	int err;
+
+	p = kzalloc(head + sizeof(*inst) + tail, GFP_KERNEL);
+	if (!p)
+		return ERR_PTR(-ENOMEM);
+
+	inst = (void *)(p + head);
+
+	err = -ENAMETOOLONG;
+	if (snprintf(inst->alg.cra_driver_name, CRYPTO_MAX_ALG_NAME,
+		    "mcryptd(%s)", alg->cra_driver_name) >= CRYPTO_MAX_ALG_NAME)
+		goto out_free_inst;
+
+	memcpy(inst->alg.cra_name, alg->cra_name, CRYPTO_MAX_ALG_NAME);
+
+	inst->alg.cra_priority = alg->cra_priority + 50;
+	inst->alg.cra_blocksize = alg->cra_blocksize;
+	inst->alg.cra_alignmask = alg->cra_alignmask;
+
+out:
+	return p;
+
+out_free_inst:
+	kfree(p);
+	p = ERR_PTR(err);
+	goto out;
+}
+
+static int mcryptd_hash_init_tfm(struct crypto_tfm *tfm)
+{
+	struct crypto_instance *inst = crypto_tfm_alg_instance(tfm);
+	struct hashd_instance_ctx *ictx = crypto_instance_ctx(inst);
+	struct crypto_shash_spawn *spawn = &ictx->spawn;
+	struct mcryptd_hash_ctx *ctx = crypto_tfm_ctx(tfm);
+	struct crypto_shash *hash;
+
+	hash = crypto_spawn_shash(spawn);
+	if (IS_ERR(hash))
+		return PTR_ERR(hash);
+
+	ctx->child = hash;
+	crypto_ahash_set_reqsize(__crypto_ahash_cast(tfm),
+				 sizeof(struct mcryptd_hash_request_ctx) +
+				 crypto_shash_descsize(hash));
+	return 0;
+}
+
+static void mcryptd_hash_exit_tfm(struct crypto_tfm *tfm)
+{
+	struct mcryptd_hash_ctx *ctx = crypto_tfm_ctx(tfm);
+
+	crypto_free_shash(ctx->child);
+}
+
+static int mcryptd_hash_setkey(struct crypto_ahash *parent,
+				   const u8 *key, unsigned int keylen)
+{
+	struct mcryptd_hash_ctx *ctx   = crypto_ahash_ctx(parent);
+	struct crypto_shash *child = ctx->child;
+	int err;
+
+	crypto_shash_clear_flags(child, CRYPTO_TFM_REQ_MASK);
+	crypto_shash_set_flags(child, crypto_ahash_get_flags(parent) &
+				      CRYPTO_TFM_REQ_MASK);
+	err = crypto_shash_setkey(child, key, keylen);
+	crypto_ahash_set_flags(parent, crypto_shash_get_flags(child) &
+				       CRYPTO_TFM_RES_MASK);
+	return err;
+}
+
+static int mcryptd_hash_enqueue(struct ahash_request *req,
+				crypto_completion_t complete)
+{
+	int ret;
+
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+	struct crypto_ahash *tfm = crypto_ahash_reqtfm(req);
+	struct mcryptd_queue *queue =
+		mcryptd_get_queue(crypto_ahash_tfm(tfm));
+
+	rctx->complete = req->base.complete;
+	req->base.complete = complete;
+
+	ret = mcryptd_enqueue_request(queue, &req->base, rctx);
+
+	return ret;
+}
+
+static void mcryptd_hash_init(struct crypto_async_request *req_async, int err)
+{
+	struct mcryptd_hash_ctx *ctx = crypto_tfm_ctx(req_async->tfm);
+	struct crypto_shash *child = ctx->child;
+	struct ahash_request *req = ahash_request_cast(req_async);
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+	struct shash_desc *desc = &rctx->desc;
+
+	if (unlikely(err == -EINPROGRESS))
+		goto out;
+
+	desc->tfm = child;
+	desc->flags = CRYPTO_TFM_REQ_MAY_SLEEP;
+
+	err = crypto_shash_init(desc);
+
+	req->base.complete = rctx->complete;
+
+out:
+	local_bh_disable();
+	rctx->complete(&req->base, err);
+	local_bh_enable();
+}
+
+static int mcryptd_hash_init_enqueue(struct ahash_request *req)
+{
+	return mcryptd_hash_enqueue(req, mcryptd_hash_init);
+}
+
+static void mcryptd_hash_update(struct crypto_async_request *req_async, int err)
+{
+	struct ahash_request *req = ahash_request_cast(req_async);
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+
+	if (unlikely(err == -EINPROGRESS))
+		goto out;
+
+	err = shash_ahash_mcryptd_update(req, &rctx->desc);
+	if (err) {
+		req->base.complete = rctx->complete;
+		goto out;
+	}
+
+	return;
+out:
+	local_bh_disable();
+	rctx->complete(&req->base, err);
+	local_bh_enable();
+}
+
+static int mcryptd_hash_update_enqueue(struct ahash_request *req)
+{
+	return mcryptd_hash_enqueue(req, mcryptd_hash_update);
+}
+
+static void mcryptd_hash_final(struct crypto_async_request *req_async, int err)
+{
+	struct ahash_request *req = ahash_request_cast(req_async);
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+
+	if (unlikely(err == -EINPROGRESS))
+		goto out;
+
+	err = shash_ahash_mcryptd_final(req, &rctx->desc);
+	if (err) {
+		req->base.complete = rctx->complete;
+		goto out;
+	}
+
+	return;
+out:
+	local_bh_disable();
+	rctx->complete(&req->base, err);
+	local_bh_enable();
+}
+
+static int mcryptd_hash_final_enqueue(struct ahash_request *req)
+{
+	return mcryptd_hash_enqueue(req, mcryptd_hash_final);
+}
+
+static void mcryptd_hash_finup(struct crypto_async_request *req_async, int err)
+{
+	struct ahash_request *req = ahash_request_cast(req_async);
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+
+	if (unlikely(err == -EINPROGRESS))
+		goto out;
+
+	err = shash_ahash_mcryptd_finup(req, &rctx->desc);
+
+	if (err) {
+		req->base.complete = rctx->complete;
+		goto out;
+	}
+
+	return;
+out:
+	local_bh_disable();
+	rctx->complete(&req->base, err);
+	local_bh_enable();
+}
+
+static int mcryptd_hash_finup_enqueue(struct ahash_request *req)
+{
+	return mcryptd_hash_enqueue(req, mcryptd_hash_finup);
+}
+
+static void mcryptd_hash_digest(struct crypto_async_request *req_async, int err)
+{
+	struct mcryptd_hash_ctx *ctx = crypto_tfm_ctx(req_async->tfm);
+	struct crypto_shash *child = ctx->child;
+	struct ahash_request *req = ahash_request_cast(req_async);
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+	struct shash_desc *desc = &rctx->desc;
+
+	if (unlikely(err == -EINPROGRESS))
+		goto out;
+
+	desc->tfm = child;
+	desc->flags = CRYPTO_TFM_REQ_MAY_SLEEP;  /* check this again */
+
+	err = shash_ahash_mcryptd_digest(req, desc);
+
+	if (err) {
+		req->base.complete = rctx->complete;
+		goto out;
+	}
+
+	return;
+out:
+	local_bh_disable();
+	rctx->complete(&req->base, err);
+	local_bh_enable();
+}
+
+static int mcryptd_hash_digest_enqueue(struct ahash_request *req)
+{
+	return mcryptd_hash_enqueue(req, mcryptd_hash_digest);
+}
+
+static int mcryptd_hash_export(struct ahash_request *req, void *out)
+{
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+
+	return crypto_shash_export(&rctx->desc, out);
+}
+
+static int mcryptd_hash_import(struct ahash_request *req, const void *in)
+{
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+
+	return crypto_shash_import(&rctx->desc, in);
+}
+
+static int mcryptd_create_hash(struct crypto_template *tmpl, struct rtattr **tb,
+			      struct mcryptd_queue *queue)
+{
+	struct hashd_instance_ctx *ctx;
+	struct ahash_instance *inst;
+	struct shash_alg *salg;
+	struct crypto_alg *alg;
+	int err;
+
+	salg = shash_attr_alg(tb[1], 0, 0);
+	if (IS_ERR(salg))
+		return PTR_ERR(salg);
+
+	alg = &salg->base;
+	pr_debug("crypto: mcryptd hash alg: %s\n", alg->cra_name);
+	inst = mcryptd_alloc_instance(alg, ahash_instance_headroom(),
+					sizeof(*ctx));
+	err = PTR_ERR(inst);
+	if (IS_ERR(inst))
+		goto out_put_alg;
+
+	ctx = ahash_instance_ctx(inst);
+	ctx->queue = queue;
+
+	err = crypto_init_shash_spawn(&ctx->spawn, salg,
+				      ahash_crypto_instance(inst));
+	if (err)
+		goto out_free_inst;
+
+	inst->alg.halg.base.cra_flags = CRYPTO_ALG_ASYNC;
+
+	inst->alg.halg.digestsize = salg->digestsize;
+	inst->alg.halg.base.cra_ctxsize = sizeof(struct mcryptd_hash_ctx);
+
+	inst->alg.halg.base.cra_init = mcryptd_hash_init_tfm;
+	inst->alg.halg.base.cra_exit = mcryptd_hash_exit_tfm;
+
+	inst->alg.init   = mcryptd_hash_init_enqueue;
+	inst->alg.update = mcryptd_hash_update_enqueue;
+	inst->alg.final  = mcryptd_hash_final_enqueue;
+	inst->alg.finup  = mcryptd_hash_finup_enqueue;
+	inst->alg.export = mcryptd_hash_export;
+	inst->alg.import = mcryptd_hash_import;
+	inst->alg.setkey = mcryptd_hash_setkey;
+	inst->alg.digest = mcryptd_hash_digest_enqueue;
+
+	err = ahash_register_instance(tmpl, inst);
+	if (err) {
+		crypto_drop_shash(&ctx->spawn);
+out_free_inst:
+		kfree(inst);
+	}
+
+out_put_alg:
+	crypto_mod_put(alg);
+	return err;
+}
+
+static struct mcryptd_queue mqueue;
+
+static int mcryptd_create(struct crypto_template *tmpl, struct rtattr **tb)
+{
+	struct crypto_attr_type *algt;
+
+	algt = crypto_get_attr_type(tb);
+	if (IS_ERR(algt))
+		return PTR_ERR(algt);
+
+	switch (algt->type & algt->mask & CRYPTO_ALG_TYPE_MASK) {
+	case CRYPTO_ALG_TYPE_DIGEST:
+		return mcryptd_create_hash(tmpl, tb, &mqueue);
+	break;
+	}
+
+	return -EINVAL;
+}
+
+static void mcryptd_free(struct crypto_instance *inst)
+{
+	struct mcryptd_instance_ctx *ctx = crypto_instance_ctx(inst);
+	struct hashd_instance_ctx *hctx = crypto_instance_ctx(inst);
+
+	switch (inst->alg.cra_flags & CRYPTO_ALG_TYPE_MASK) {
+	case CRYPTO_ALG_TYPE_AHASH:
+		crypto_drop_shash(&hctx->spawn);
+		kfree(ahash_instance(inst));
+		return;
+	default:
+		crypto_drop_spawn(&ctx->spawn);
+		kfree(inst);
+	}
+}
+
+static struct crypto_template mcryptd_tmpl = {
+	.name = "mcryptd",
+	.create = mcryptd_create,
+	.free = mcryptd_free,
+	.module = THIS_MODULE,
+};
+
+struct mcryptd_ahash *mcryptd_alloc_ahash(const char *alg_name,
+					u32 type, u32 mask)
+{
+	char mcryptd_alg_name[CRYPTO_MAX_ALG_NAME];
+	struct crypto_ahash *tfm;
+
+	if (snprintf(mcryptd_alg_name, CRYPTO_MAX_ALG_NAME,
+		     "mcryptd(%s)", alg_name) >= CRYPTO_MAX_ALG_NAME)
+		return ERR_PTR(-EINVAL);
+	tfm = crypto_alloc_ahash(mcryptd_alg_name, type, mask);
+	if (IS_ERR(tfm))
+		return ERR_CAST(tfm);
+	if (tfm->base.__crt_alg->cra_module != THIS_MODULE) {
+		crypto_free_ahash(tfm);
+		return ERR_PTR(-EINVAL);
+	}
+
+	return __mcryptd_ahash_cast(tfm);
+}
+EXPORT_SYMBOL_GPL(mcryptd_alloc_ahash);
+
+int shash_ahash_mcryptd_digest(struct ahash_request *req,
+			       struct shash_desc *desc)
+{
+	int err;
+
+	err = crypto_shash_init(desc) ?:
+	      shash_ahash_mcryptd_finup(req, desc);
+
+	return err;
+}
+EXPORT_SYMBOL_GPL(shash_ahash_mcryptd_digest);
+
+int shash_ahash_mcryptd_update(struct ahash_request *req,
+			       struct shash_desc *desc)
+{
+	struct crypto_shash *tfm = desc->tfm;
+	struct shash_alg *shash = crypto_shash_alg(tfm);
+
+	/* alignment is to be done by multi-buffer crypto algorithm if needed */
+
+	return shash->update(desc, NULL, 0);
+}
+EXPORT_SYMBOL_GPL(shash_ahash_mcryptd_update);
+
+int shash_ahash_mcryptd_finup(struct ahash_request *req,
+			      struct shash_desc *desc)
+{
+	struct crypto_shash *tfm = desc->tfm;
+	struct shash_alg *shash = crypto_shash_alg(tfm);
+
+	/* alignment is to be done by multi-buffer crypto algorithm if needed */
+
+	return shash->finup(desc, NULL, 0, req->result);
+}
+EXPORT_SYMBOL_GPL(shash_ahash_mcryptd_finup);
+
+int shash_ahash_mcryptd_final(struct ahash_request *req,
+			      struct shash_desc *desc)
+{
+	struct crypto_shash *tfm = desc->tfm;
+	struct shash_alg *shash = crypto_shash_alg(tfm);
+
+	/* alignment is to be done by multi-buffer crypto algorithm if needed */
+
+	return shash->final(desc, req->result);
+}
+EXPORT_SYMBOL_GPL(shash_ahash_mcryptd_final);
+
+struct crypto_shash *mcryptd_ahash_child(struct mcryptd_ahash *tfm)
+{
+	struct mcryptd_hash_ctx *ctx = crypto_ahash_ctx(&tfm->base);
+
+	return ctx->child;
+}
+EXPORT_SYMBOL_GPL(mcryptd_ahash_child);
+
+struct shash_desc *mcryptd_shash_desc(struct ahash_request *req)
+{
+	struct mcryptd_hash_request_ctx *rctx = ahash_request_ctx(req);
+	return &rctx->desc;
+}
+EXPORT_SYMBOL_GPL(mcryptd_shash_desc);
+
+void mcryptd_free_ahash(struct mcryptd_ahash *tfm)
+{
+	crypto_free_ahash(&tfm->base);
+}
+EXPORT_SYMBOL_GPL(mcryptd_free_ahash);
+
+
+static int __init mcryptd_init(void)
+{
+	int err, cpu;
+	struct mcryptd_flush_list *flist;
+
+	mcryptd_flist = alloc_percpu(struct mcryptd_flush_list);
+	for_each_possible_cpu(cpu) {
+		flist = per_cpu_ptr(mcryptd_flist, cpu);
+		INIT_LIST_HEAD(&flist->list);
+		mutex_init(&flist->lock);
+	}
+
+	err = mcryptd_init_queue(&mqueue, MCRYPTD_MAX_CPU_QLEN);
+	if (err) {
+		free_percpu(mcryptd_flist);
+		return err;
+	}
+
+	err = crypto_register_template(&mcryptd_tmpl);
+	if (err) {
+		mcryptd_fini_queue(&mqueue);
+		free_percpu(mcryptd_flist);
+	}
+
+	return err;
+}
+
+static void __exit mcryptd_exit(void)
+{
+	mcryptd_fini_queue(&mqueue);
+	crypto_unregister_template(&mcryptd_tmpl);
+	free_percpu(mcryptd_flist);
+}
+
+subsys_initcall(mcryptd_init);
+module_exit(mcryptd_exit);
+
+MODULE_LICENSE("GPL");
+MODULE_DESCRIPTION("Software async multibuffer crypto daemon");
diff --git a/include/crypto/internal/hash.h b/include/crypto/internal/hash.h
index 9b6f32a..3b4af1d 100644
--- a/include/crypto/internal/hash.h
+++ b/include/crypto/internal/hash.h
@@ -117,6 +117,15 @@ int shash_ahash_update(struct ahash_request *req, struct shash_desc *desc);
 int shash_ahash_finup(struct ahash_request *req, struct shash_desc *desc);
 int shash_ahash_digest(struct ahash_request *req, struct shash_desc *desc);
 
+int shash_ahash_mcryptd_update(struct ahash_request *req,
+			       struct shash_desc *desc);
+int shash_ahash_mcryptd_final(struct ahash_request *req,
+			      struct shash_desc *desc);
+int shash_ahash_mcryptd_finup(struct ahash_request *req,
+			      struct shash_desc *desc);
+int shash_ahash_mcryptd_digest(struct ahash_request *req,
+			       struct shash_desc *desc);
+
 int crypto_init_shash_ops_async(struct crypto_tfm *tfm);
 
 static inline void *crypto_ahash_ctx(struct crypto_ahash *tfm)
diff --git a/include/crypto/mcryptd.h b/include/crypto/mcryptd.h
new file mode 100644
index 0000000..c23ee1f
--- /dev/null
+++ b/include/crypto/mcryptd.h
@@ -0,0 +1,112 @@
+/*
+ * Software async multibuffer crypto daemon headers
+ *
+ *    Author:
+ *             Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *    Copyright (c) 2014, Intel Corporation.
+ */
+
+#ifndef _CRYPTO_MCRYPT_H
+#define _CRYPTO_MCRYPT_H
+
+#include <linux/crypto.h>
+#include <linux/kernel.h>
+#include <crypto/hash.h>
+
+struct mcryptd_ahash {
+	struct crypto_ahash base;
+};
+
+static inline struct mcryptd_ahash *__mcryptd_ahash_cast(
+	struct crypto_ahash *tfm)
+{
+	return (struct mcryptd_ahash *)tfm;
+}
+
+struct mcryptd_cpu_queue {
+	struct crypto_queue queue;
+	struct work_struct work;
+};
+
+struct mcryptd_queue {
+	struct mcryptd_cpu_queue __percpu *cpu_queue;
+};
+
+struct mcryptd_instance_ctx {
+	struct crypto_spawn spawn;
+	struct mcryptd_queue *queue;
+};
+
+struct mcryptd_hash_ctx {
+	struct crypto_shash *child;
+	struct mcryptd_alg_state *alg_state;
+};
+
+struct mcryptd_tag {
+	/* seq number of request */
+	unsigned seq_num;
+	/* arrival time of request */
+	unsigned long arrival;
+	unsigned long expire;
+	int	cpu;
+};
+
+struct mcryptd_hash_request_ctx {
+	struct list_head waiter;
+	crypto_completion_t complete;
+	struct mcryptd_tag tag;
+	struct crypto_hash_walk walk;
+	u8 *out;
+	int flag;
+	struct shash_desc desc;
+};
+
+struct mcryptd_ahash *mcryptd_alloc_ahash(const char *alg_name,
+					u32 type, u32 mask);
+struct crypto_shash *mcryptd_ahash_child(struct mcryptd_ahash *tfm);
+struct shash_desc *mcryptd_shash_desc(struct ahash_request *req);
+void mcryptd_free_ahash(struct mcryptd_ahash *tfm);
+void mcryptd_flusher(struct work_struct *work);
+
+enum mcryptd_req_type {
+	MCRYPTD_NONE,
+	MCRYPTD_UPDATE,
+	MCRYPTD_FINUP,
+	MCRYPTD_DIGEST,
+	MCRYPTD_FINAL
+};
+
+struct mcryptd_alg_cstate {
+	unsigned long next_flush;
+	unsigned next_seq_num;
+	bool	flusher_engaged;
+	struct  delayed_work flush;
+	int	cpu;
+	struct  mcryptd_alg_state *alg_state;
+	void	*mgr;
+	spinlock_t work_lock;
+	struct list_head work_list;
+	struct list_head flush_list;
+};
+
+struct mcryptd_alg_state {
+	struct mcryptd_alg_cstate __percpu *alg_cstate;
+	unsigned long (*flusher)(struct mcryptd_alg_cstate *cstate);
+};
+
+/* return delay in jiffies from current time */
+static inline unsigned long get_delay(unsigned long t)
+{
+	long delay;
+
+	delay = (long) t - (long) jiffies;
+	if (delay <= 0)
+		return 0;
+	else
+		return (unsigned long) delay;
+}
+
+void mcryptd_arm_flusher(struct mcryptd_alg_cstate *cstate, unsigned long delay);
+
+#endif
-- 
1.7.11.7

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

* [PATCH v5 3/7] crypto: SHA1 multibuffer crypto opportunistic flush
       [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
                   ` (2 preceding siblings ...)
  2014-07-22 22:09 ` [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure Tim Chen
@ 2014-07-22 22:09 ` Tim Chen
  2014-07-25  7:22   ` Peter Zijlstra
  2014-07-22 22:09 ` [PATCH v5 4/7] crypto: SHA1 multibuffer algorithm data structures Tim Chen
                   ` (3 subsequent siblings)
  7 siblings, 1 reply; 15+ messages in thread
From: Tim Chen @ 2014-07-22 22:09 UTC (permalink / raw)
  To: Herbert Xu, H. Peter Anvin, David S.Miller, Peter Zijlstra, Ingo Molnar
  Cc: Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Tim Chen, Jussi Kivilinna, Thomas Gleixner,
	Tadeusz Struk, tkhai, linux-crypto, linux-kernel

The crypto daemon can take advantage of available cpu
cycles to flush any unfinished jobs if it is the
only task running on the cpu, and there are no more crypto
jobs to process.

Signed-off-by: Tim Chen <tim.c.chen@linux.intel.com>
---
 crypto/mcryptd.c | 39 ++++++++++++++++++++++++++++++++++++---
 1 file changed, 36 insertions(+), 3 deletions(-)

diff --git a/crypto/mcryptd.c b/crypto/mcryptd.c
index 622d6b4..dbc20d1 100644
--- a/crypto/mcryptd.c
+++ b/crypto/mcryptd.c
@@ -116,9 +116,40 @@ static int mcryptd_enqueue_request(struct mcryptd_queue *queue,
 	return err;
 }
 
-/* Called in workqueue context, do one real cryption work (via
+/*
+ * Try to opportunisticlly flush the partially completed jobs if
+ * crypto daemon is the only task running.
+ */
+static void mcryptd_opportunistic_flush(void)
+{
+	struct mcryptd_flush_list *flist;
+	struct mcryptd_alg_cstate *cstate;
+
+	flist = per_cpu_ptr(mcryptd_flist, smp_processor_id());
+	while (single_task_running()) {
+		mutex_lock(&flist->lock);
+		if (list_empty(&flist->list)) {
+			mutex_unlock(&flist->lock);
+			return;
+		}
+		cstate = list_entry(flist->list.next,
+				struct mcryptd_alg_cstate, flush_list);
+		if (!cstate->flusher_engaged) {
+			mutex_unlock(&flist->lock);
+			return;
+		}
+		list_del(&cstate->flush_list);
+		cstate->flusher_engaged = false;
+		mutex_unlock(&flist->lock);
+		cstate->alg_state->flusher(cstate);
+	}
+}
+
+/*
+ * Called in workqueue context, do one real cryption work (via
  * req->complete) and reschedule itself if there are more work to
- * do. */
+ * do.
+ */
 static void mcryptd_queue_worker(struct work_struct *work)
 {
 	struct mcryptd_cpu_queue *cpu_queue;
@@ -144,8 +175,10 @@ static void mcryptd_queue_worker(struct work_struct *work)
 		preempt_enable();
 		local_bh_enable();
 
-		if (!req)
+		if (!req) {
+			mcryptd_opportunistic_flush();
 			return;
+		}
 
 		if (backlog)
 			backlog->complete(backlog, -EINPROGRESS);
-- 
1.7.11.7

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

* [PATCH v5 4/7] crypto: SHA1 multibuffer algorithm data structures
       [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
                   ` (3 preceding siblings ...)
  2014-07-22 22:09 ` [PATCH v5 3/7] crypto: SHA1 multibuffer crypto opportunistic flush Tim Chen
@ 2014-07-22 22:09 ` Tim Chen
  2014-07-22 22:09 ` [PATCH v5 5/7] crypto: SHA1 multibuffer submit and flush routines for AVX2 Tim Chen
                   ` (2 subsequent siblings)
  7 siblings, 0 replies; 15+ messages in thread
From: Tim Chen @ 2014-07-22 22:09 UTC (permalink / raw)
  To: Herbert Xu, H. Peter Anvin, David S.Miller, Peter Zijlstra, Ingo Molnar
  Cc: Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Tim Chen, Jussi Kivilinna, Thomas Gleixner,
	Tadeusz Struk, tkhai, linux-crypto, linux-kernel

This patch introduces the data structures and prototypes of functions
needed for computing SHA1 hash using multi-buffer.  Included are the
structures of the multi-buffer SHA1 job, job scheduler in C and x86
assembly.

Signed-off-by: Tim Chen <tim.c.chen@linux.intel.com>
---
 arch/x86/crypto/sha-mb/sha1_mb_mgr_datastruct.S | 287 ++++++++++++++++++++++++
 arch/x86/crypto/sha-mb/sha_mb_ctx.h             | 136 +++++++++++
 arch/x86/crypto/sha-mb/sha_mb_mgr.h             | 110 +++++++++
 3 files changed, 533 insertions(+)
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb_mgr_datastruct.S
 create mode 100644 arch/x86/crypto/sha-mb/sha_mb_ctx.h
 create mode 100644 arch/x86/crypto/sha-mb/sha_mb_mgr.h

diff --git a/arch/x86/crypto/sha-mb/sha1_mb_mgr_datastruct.S b/arch/x86/crypto/sha-mb/sha1_mb_mgr_datastruct.S
new file mode 100644
index 0000000..86688c6
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/sha1_mb_mgr_datastruct.S
@@ -0,0 +1,287 @@
+/*
+ * Header file for multi buffer SHA1 algorithm data structure
+ *
+ * This file is provided under a dual BSD/GPLv2 license.  When using or
+ * redistributing this file, you may do so under either license.
+ *
+ * GPL LICENSE SUMMARY
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  This program is free software; you can redistribute it and/or modify
+ *  it under the terms of version 2 of the GNU General Public License as
+ *  published by the Free Software Foundation.
+ *
+ *  This program is distributed in the hope that it will be useful, but
+ *  WITHOUT ANY WARRANTY; without even the implied warranty of
+ *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ *  General Public License for more details.
+ *
+ *  Contact Information:
+ *      James Guilford <james.guilford@intel.com>
+ *	Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *  BSD LICENSE
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  Redistribution and use in source and binary forms, with or without
+ *  modification, are permitted provided that the following conditions
+ *  are met:
+ *
+ *    * Redistributions of source code must retain the above copyright
+ *      notice, this list of conditions and the following disclaimer.
+ *    * Redistributions in binary form must reproduce the above copyright
+ *      notice, this list of conditions and the following disclaimer in
+ *      the documentation and/or other materials provided with the
+ *      distribution.
+ *    * Neither the name of Intel Corporation nor the names of its
+ *      contributors may be used to endorse or promote products derived
+ *      from this software without specific prior written permission.
+ *
+ *  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+# Macros for defining data structures
+
+# Usage example
+
+#START_FIELDS	# JOB_AES
+###	name		size	align
+#FIELD	_plaintext,	8,	8	# pointer to plaintext
+#FIELD	_ciphertext,	8,	8	# pointer to ciphertext
+#FIELD	_IV,		16,	8	# IV
+#FIELD	_keys,		8,	8	# pointer to keys
+#FIELD	_len,		4,	4	# length in bytes
+#FIELD	_status,	4,	4	# status enumeration
+#FIELD	_user_data,	8,	8	# pointer to user data
+#UNION  _union,         size1,  align1, \
+#	                size2,  align2, \
+#	                size3,  align3, \
+#	                ...
+#END_FIELDS
+#%assign _JOB_AES_size	_FIELD_OFFSET
+#%assign _JOB_AES_align	_STRUCT_ALIGN
+
+#########################################################################
+
+# Alternate "struc-like" syntax:
+#	STRUCT job_aes2
+#	RES_Q	.plaintext,	1
+#	RES_Q	.ciphertext,	1
+#	RES_DQ	.IV,		1
+#	RES_B	.nested,	_JOB_AES_SIZE, _JOB_AES_ALIGN
+#	RES_U	.union,		size1, align1, \
+#				size2, align2, \
+#				...
+#	ENDSTRUCT
+#	# Following only needed if nesting
+#	%assign job_aes2_size	_FIELD_OFFSET
+#	%assign job_aes2_align	_STRUCT_ALIGN
+#
+# RES_* macros take a name, a count and an optional alignment.
+# The count in in terms of the base size of the macro, and the
+# default alignment is the base size.
+# The macros are:
+# Macro    Base size
+# RES_B	    1
+# RES_W	    2
+# RES_D     4
+# RES_Q     8
+# RES_DQ   16
+# RES_Y    32
+# RES_Z    64
+#
+# RES_U defines a union. It's arguments are a name and two or more
+# pairs of "size, alignment"
+#
+# The two assigns are only needed if this structure is being nested
+# within another. Even if the assigns are not done, one can still use
+# STRUCT_NAME_size as the size of the structure.
+#
+# Note that for nesting, you still need to assign to STRUCT_NAME_size.
+#
+# The differences between this and using "struc" directly are that each
+# type is implicitly aligned to its natural length (although this can be
+# over-ridden with an explicit third parameter), and that the structure
+# is padded at the end to its overall alignment.
+#
+
+#########################################################################
+
+#ifndef _SHA1_MB_MGR_DATASTRUCT_ASM_
+#define _SHA1_MB_MGR_DATASTRUCT_ASM_
+
+## START_FIELDS
+.macro START_FIELDS
+ _FIELD_OFFSET = 0
+ _STRUCT_ALIGN = 0
+.endm
+
+## FIELD name size align
+.macro FIELD name size align
+ _FIELD_OFFSET = (_FIELD_OFFSET + (\align) - 1) & (~ ((\align)-1))
+ \name	= _FIELD_OFFSET
+ _FIELD_OFFSET = _FIELD_OFFSET + (\size)
+.if (\align > _STRUCT_ALIGN)
+ _STRUCT_ALIGN = \align
+.endif
+.endm
+
+## END_FIELDS
+.macro END_FIELDS
+ _FIELD_OFFSET = (_FIELD_OFFSET + _STRUCT_ALIGN-1) & (~ (_STRUCT_ALIGN-1))
+.endm
+
+########################################################################
+
+.macro STRUCT p1
+START_FIELDS
+.struc \p1
+.endm
+
+.macro ENDSTRUCT
+ tmp = _FIELD_OFFSET
+ END_FIELDS
+ tmp = (_FIELD_OFFSET - %%tmp)
+.if (tmp > 0)
+	.lcomm	tmp
+.endif
+.endstruc
+.endm
+
+## RES_int name size align
+.macro RES_int p1 p2 p3
+ name = \p1
+ size = \p2
+ align = .\p3
+
+ _FIELD_OFFSET = (_FIELD_OFFSET + (align) - 1) & (~ ((align)-1))
+.align align
+.lcomm name size
+ _FIELD_OFFSET = _FIELD_OFFSET + (size)
+.if (align > _STRUCT_ALIGN)
+ _STRUCT_ALIGN = align
+.endif
+.endm
+
+
+
+# macro RES_B name, size [, align]
+.macro RES_B _name, _size, _align=1
+RES_int _name _size _align
+.endm
+
+# macro RES_W name, size [, align]
+.macro RES_W _name, _size, _align=2
+RES_int _name 2*(_size) _align
+.endm
+
+# macro RES_D name, size [, align]
+.macro RES_D _name, _size, _align=4
+RES_int _name 4*(_size) _align
+.endm
+
+# macro RES_Q name, size [, align]
+.macro RES_Q _name, _size, _align=8
+RES_int _name 8*(_size) _align
+.endm
+
+# macro RES_DQ name, size [, align]
+.macro RES_DQ _name, _size, _align=16
+RES_int _name 16*(_size) _align
+.endm
+
+# macro RES_Y name, size [, align]
+.macro RES_Y _name, _size, _align=32
+RES_int _name 32*(_size) _align
+.endm
+
+# macro RES_Z name, size [, align]
+.macro RES_Z _name, _size, _align=64
+RES_int _name 64*(_size) _align
+.endm
+
+
+#endif
+
+########################################################################
+#### Define constants
+########################################################################
+
+########################################################################
+#### Define SHA1 Out Of Order Data Structures
+########################################################################
+
+START_FIELDS    # LANE_DATA
+###     name            size    align
+FIELD   _job_in_lane,   8,      8       # pointer to job object
+END_FIELDS
+
+_LANE_DATA_size = _FIELD_OFFSET
+_LANE_DATA_align = _STRUCT_ALIGN
+
+########################################################################
+
+START_FIELDS    # SHA1_ARGS_X8
+###     name            size    align
+FIELD   _digest,        4*5*8,  16      # transposed digest
+FIELD   _data_ptr,      8*8,    8       # array of pointers to data
+END_FIELDS
+
+_SHA1_ARGS_X4_size =     _FIELD_OFFSET
+_SHA1_ARGS_X4_align =    _STRUCT_ALIGN
+_SHA1_ARGS_X8_size =     _FIELD_OFFSET
+_SHA1_ARGS_X8_align =    _STRUCT_ALIGN
+
+########################################################################
+
+START_FIELDS    # MB_MGR
+###     name            size    align
+FIELD   _args,          _SHA1_ARGS_X4_size, _SHA1_ARGS_X4_align
+FIELD   _lens,          4*8,    8
+FIELD   _unused_lanes,  8,      8
+FIELD   _ldata,         _LANE_DATA_size*8, _LANE_DATA_align
+END_FIELDS
+
+_MB_MGR_size =   _FIELD_OFFSET
+_MB_MGR_align =  _STRUCT_ALIGN
+
+_args_digest    =     _args + _digest
+_args_data_ptr  =     _args + _data_ptr
+
+
+########################################################################
+#### Define constants
+########################################################################
+
+#define STS_UNKNOWN             0
+#define STS_BEING_PROCESSED     1
+#define STS_COMPLETED           2
+
+########################################################################
+#### Define JOB_SHA1 structure
+########################################################################
+
+START_FIELDS    # JOB_SHA1
+
+###     name                            size    align
+FIELD   _buffer,                        8,      8       # pointer to buffer
+FIELD   _len,                           4,      4       # length in bytes
+FIELD   _result_digest,                 5*4,    32      # Digest (output)
+FIELD   _status,                        4,      4
+FIELD   _user_data,                     8,      8
+END_FIELDS
+
+_JOB_SHA1_size =  _FIELD_OFFSET
+_JOB_SHA1_align = _STRUCT_ALIGN
diff --git a/arch/x86/crypto/sha-mb/sha_mb_ctx.h b/arch/x86/crypto/sha-mb/sha_mb_ctx.h
new file mode 100644
index 0000000..e36069d
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/sha_mb_ctx.h
@@ -0,0 +1,136 @@
+/*
+ * Header file for multi buffer SHA context
+ *
+ * This file is provided under a dual BSD/GPLv2 license.  When using or
+ * redistributing this file, you may do so under either license.
+ *
+ * GPL LICENSE SUMMARY
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  This program is free software; you can redistribute it and/or modify
+ *  it under the terms of version 2 of the GNU General Public License as
+ *  published by the Free Software Foundation.
+ *
+ *  This program is distributed in the hope that it will be useful, but
+ *  WITHOUT ANY WARRANTY; without even the implied warranty of
+ *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ *  General Public License for more details.
+ *
+ *  Contact Information:
+ *	Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *  BSD LICENSE
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  Redistribution and use in source and binary forms, with or without
+ *  modification, are permitted provided that the following conditions
+ *  are met:
+ *
+ *    * Redistributions of source code must retain the above copyright
+ *      notice, this list of conditions and the following disclaimer.
+ *    * Redistributions in binary form must reproduce the above copyright
+ *      notice, this list of conditions and the following disclaimer in
+ *      the documentation and/or other materials provided with the
+ *      distribution.
+ *    * Neither the name of Intel Corporation nor the names of its
+ *      contributors may be used to endorse or promote products derived
+ *      from this software without specific prior written permission.
+ *
+ *  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _SHA_MB_CTX_INTERNAL_H
+#define _SHA_MB_CTX_INTERNAL_H
+
+#include "sha_mb_mgr.h"
+
+#define HASH_UPDATE          0x00
+#define HASH_FIRST           0x01
+#define HASH_LAST            0x02
+#define HASH_ENTIRE          0x03
+#define HASH_DONE	     0x04
+#define HASH_FINAL	     0x08
+
+#define HASH_CTX_STS_IDLE       0x00
+#define HASH_CTX_STS_PROCESSING 0x01
+#define HASH_CTX_STS_LAST       0x02
+#define HASH_CTX_STS_COMPLETE   0x04
+
+enum hash_ctx_error {
+	HASH_CTX_ERROR_NONE               =  0,
+	HASH_CTX_ERROR_INVALID_FLAGS      = -1,
+	HASH_CTX_ERROR_ALREADY_PROCESSING = -2,
+	HASH_CTX_ERROR_ALREADY_COMPLETED  = -3,
+
+#ifdef HASH_CTX_DEBUG
+	HASH_CTX_ERROR_DEBUG_DIGEST_MISMATCH = -4,
+#endif
+};
+
+
+#define hash_ctx_user_data(ctx)  ((ctx)->user_data)
+#define hash_ctx_digest(ctx)     ((ctx)->job.result_digest)
+#define hash_ctx_processing(ctx) ((ctx)->status & HASH_CTX_STS_PROCESSING)
+#define hash_ctx_complete(ctx)   ((ctx)->status == HASH_CTX_STS_COMPLETE)
+#define hash_ctx_status(ctx)     ((ctx)->status)
+#define hash_ctx_error(ctx)      ((ctx)->error)
+#define hash_ctx_init(ctx) \
+	do { \
+		(ctx)->error = HASH_CTX_ERROR_NONE; \
+		(ctx)->status = HASH_CTX_STS_COMPLETE; \
+	} while (0)
+
+
+/* Hash Constants and Typedefs */
+#define SHA1_DIGEST_LENGTH          5
+#define SHA1_LOG2_BLOCK_SIZE        6
+
+#define SHA1_PADLENGTHFIELD_SIZE    8
+
+#ifdef SHA_MB_DEBUG
+#define assert(expr) \
+do { \
+	if (unlikely(!(expr))) { \
+		printk(KERN_ERR "Assertion failed! %s,%s,%s,line=%d\n", \
+		#expr, __FILE__, __func__, __LINE__); \
+	} \
+} while (0)
+#else
+#define assert(expr) do {} while (0)
+#endif
+
+struct sha1_ctx_mgr {
+	struct sha1_mb_mgr mgr;
+};
+
+/* typedef struct sha1_ctx_mgr sha1_ctx_mgr; */
+
+struct sha1_hash_ctx {
+	/* Must be at struct offset 0 */
+	struct job_sha1       job;
+	/* status flag */
+	int status;
+	/* error flag */
+	int error;
+
+	uint32_t	total_length;
+	const void	*incoming_buffer;
+	uint32_t	incoming_buffer_length;
+	uint8_t		partial_block_buffer[SHA1_BLOCK_SIZE * 2];
+	uint32_t	partial_block_buffer_length;
+	void		*user_data;
+};
+
+#endif
diff --git a/arch/x86/crypto/sha-mb/sha_mb_mgr.h b/arch/x86/crypto/sha-mb/sha_mb_mgr.h
new file mode 100644
index 0000000..08ad1a9
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/sha_mb_mgr.h
@@ -0,0 +1,110 @@
+/*
+ * Header file for multi buffer SHA1 algorithm manager
+ *
+ * This file is provided under a dual BSD/GPLv2 license.  When using or
+ * redistributing this file, you may do so under either license.
+ *
+ * GPL LICENSE SUMMARY
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  This program is free software; you can redistribute it and/or modify
+ *  it under the terms of version 2 of the GNU General Public License as
+ *  published by the Free Software Foundation.
+ *
+ *  This program is distributed in the hope that it will be useful, but
+ *  WITHOUT ANY WARRANTY; without even the implied warranty of
+ *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ *  General Public License for more details.
+ *
+ *  Contact Information:
+ *      James Guilford <james.guilford@intel.com>
+ *	Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *  BSD LICENSE
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  Redistribution and use in source and binary forms, with or without
+ *  modification, are permitted provided that the following conditions
+ *  are met:
+ *
+ *    * Redistributions of source code must retain the above copyright
+ *      notice, this list of conditions and the following disclaimer.
+ *    * Redistributions in binary form must reproduce the above copyright
+ *      notice, this list of conditions and the following disclaimer in
+ *      the documentation and/or other materials provided with the
+ *      distribution.
+ *    * Neither the name of Intel Corporation nor the names of its
+ *      contributors may be used to endorse or promote products derived
+ *      from this software without specific prior written permission.
+ *
+ *  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+#ifndef __SHA_MB_MGR_H
+#define __SHA_MB_MGR_H
+
+
+#include <linux/types.h>
+
+#define NUM_SHA1_DIGEST_WORDS 5
+
+enum job_sts {	STS_UNKNOWN = 0,
+		STS_BEING_PROCESSED = 1,
+		STS_COMPLETED = 2,
+		STS_INTERNAL_ERROR = 3,
+		STS_ERROR = 4
+};
+
+struct job_sha1 {
+	u8	*buffer;
+	u32	len;
+	u32	result_digest[NUM_SHA1_DIGEST_WORDS] __aligned(32);
+	enum	job_sts status;
+	void	*user_data;
+};
+
+/* SHA1 out-of-order scheduler */
+
+/* typedef uint32_t sha1_digest_array[5][8]; */
+
+struct sha1_args_x8 {
+	uint32_t	digest[5][8];
+	uint8_t		*data_ptr[8];
+};
+
+struct sha1_lane_data {
+	struct job_sha1 *job_in_lane;
+};
+
+struct sha1_mb_mgr {
+	struct sha1_args_x8 args;
+
+	uint32_t lens[8];
+
+	/* each byte is index (0...7) of unused lanes */
+	uint64_t unused_lanes;
+	/* byte 4 is set to FF as a flag */
+	struct sha1_lane_data ldata[8];
+};
+
+
+#define SHA1_MB_MGR_NUM_LANES_AVX2 8
+
+void sha1_mb_mgr_init_avx2(struct sha1_mb_mgr *state);
+struct job_sha1 *sha1_mb_mgr_submit_avx2(struct sha1_mb_mgr *state,
+					 struct job_sha1 *job);
+struct job_sha1 *sha1_mb_mgr_flush_avx2(struct sha1_mb_mgr *state);
+struct job_sha1 *sha1_mb_mgr_get_comp_job_avx2(struct sha1_mb_mgr *state);
+
+#endif
-- 
1.7.11.7

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

* [PATCH v5 5/7] crypto: SHA1 multibuffer submit and flush routines for AVX2
       [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
                   ` (4 preceding siblings ...)
  2014-07-22 22:09 ` [PATCH v5 4/7] crypto: SHA1 multibuffer algorithm data structures Tim Chen
@ 2014-07-22 22:09 ` Tim Chen
  2014-07-22 22:09 ` [PATCH v5 6/7] crypto: SHA1 multibuffer crypto computation (x8 AVX2) Tim Chen
  2014-07-22 22:09 ` [PATCH v5 7/7] crypto: SHA1 multibuffer scheduler Tim Chen
  7 siblings, 0 replies; 15+ messages in thread
From: Tim Chen @ 2014-07-22 22:09 UTC (permalink / raw)
  To: Herbert Xu, H. Peter Anvin, David S.Miller, Peter Zijlstra, Ingo Molnar
  Cc: Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Tim Chen, Jussi Kivilinna, Thomas Gleixner,
	Tadeusz Struk, tkhai, linux-crypto, linux-kernel

This patch introduces the routines used to submit and flush buffers
belonging to SHA1 crypto jobs to the SHA1 multibuffer algorithm.  It is
implemented mostly in assembly optimized with AVX2 instructions.

Signed-off-by: Tim Chen <tim.c.chen@linux.intel.com>
---
 arch/x86/crypto/sha-mb/sha1_mb_mgr_flush_avx2.S  | 327 +++++++++++++++++++++++
 arch/x86/crypto/sha-mb/sha1_mb_mgr_init_avx2.c   |  64 +++++
 arch/x86/crypto/sha-mb/sha1_mb_mgr_submit_avx2.S | 228 ++++++++++++++++
 3 files changed, 619 insertions(+)
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb_mgr_flush_avx2.S
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb_mgr_init_avx2.c
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb_mgr_submit_avx2.S

diff --git a/arch/x86/crypto/sha-mb/sha1_mb_mgr_flush_avx2.S b/arch/x86/crypto/sha-mb/sha1_mb_mgr_flush_avx2.S
new file mode 100644
index 0000000..85c4e1c
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/sha1_mb_mgr_flush_avx2.S
@@ -0,0 +1,327 @@
+/*
+ * Flush routine for SHA1 multibuffer
+ *
+ * This file is provided under a dual BSD/GPLv2 license.  When using or
+ * redistributing this file, you may do so under either license.
+ *
+ * GPL LICENSE SUMMARY
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  This program is free software; you can redistribute it and/or modify
+ *  it under the terms of version 2 of the GNU General Public License as
+ *  published by the Free Software Foundation.
+ *
+ *  This program is distributed in the hope that it will be useful, but
+ *  WITHOUT ANY WARRANTY; without even the implied warranty of
+ *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ *  General Public License for more details.
+ *
+ *  Contact Information:
+ *      James Guilford <james.guilford@intel.com>
+ *	Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *  BSD LICENSE
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  Redistribution and use in source and binary forms, with or without
+ *  modification, are permitted provided that the following conditions
+ *  are met:
+ *
+ *    * Redistributions of source code must retain the above copyright
+ *      notice, this list of conditions and the following disclaimer.
+ *    * Redistributions in binary form must reproduce the above copyright
+ *      notice, this list of conditions and the following disclaimer in
+ *      the documentation and/or other materials provided with the
+ *      distribution.
+ *    * Neither the name of Intel Corporation nor the names of its
+ *      contributors may be used to endorse or promote products derived
+ *      from this software without specific prior written permission.
+ *
+ *  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+#include <linux/linkage.h>
+#include "sha1_mb_mgr_datastruct.S"
+
+
+.extern sha1_x8_avx2
+
+# LINUX register definitions
+#define arg1    %rdi
+#define arg2    %rsi
+
+# Common definitions
+#define state   arg1
+#define job     arg2
+#define len2    arg2
+
+# idx must be a register not clobbered by sha1_x8_avx2
+#define idx		%r8
+#define DWORD_idx	%r8d
+
+#define unused_lanes    %rbx
+#define lane_data       %rbx
+#define tmp2            %rbx
+#define tmp2_w		%ebx
+
+#define job_rax         %rax
+#define tmp1            %rax
+#define size_offset     %rax
+#define tmp             %rax
+#define start_offset    %rax
+
+#define tmp3            %arg1
+
+#define extra_blocks    %arg2
+#define p               %arg2
+
+
+# STACK_SPACE needs to be an odd multiple of 8
+_XMM_SAVE_SIZE  = 10*16
+_GPR_SAVE_SIZE  = 8*8
+_ALIGN_SIZE     = 8
+
+_XMM_SAVE       = 0
+_GPR_SAVE       = _XMM_SAVE + _XMM_SAVE_SIZE
+STACK_SPACE     = _GPR_SAVE + _GPR_SAVE_SIZE + _ALIGN_SIZE
+
+.macro LABEL prefix n
+\prefix\n\():
+.endm
+
+.macro JNE_SKIP i
+jne     skip_\i
+.endm
+
+.altmacro
+.macro SET_OFFSET _offset
+offset = \_offset
+.endm
+.noaltmacro
+
+# JOB* sha1_mb_mgr_flush_avx2(MB_MGR *state)
+# arg 1 : rcx : state
+ENTRY(sha1_mb_mgr_flush_avx2)
+	mov	%rsp, %r10
+	sub     $STACK_SPACE, %rsp
+	and     $~31, %rsp
+	mov     %rbx, _GPR_SAVE(%rsp)
+	mov     %r10, _GPR_SAVE+8*1(%rsp) #save rsp
+	mov	%rbp, _GPR_SAVE+8*3(%rsp)
+	mov	%r12, _GPR_SAVE+8*4(%rsp)
+	mov	%r13, _GPR_SAVE+8*5(%rsp)
+	mov	%r14, _GPR_SAVE+8*6(%rsp)
+	mov	%r15, _GPR_SAVE+8*7(%rsp)
+
+	# If bit (32+3) is set, then all lanes are empty
+	mov     _unused_lanes(state), unused_lanes
+	bt      $32+3, unused_lanes
+	jc      return_null
+
+	# find a lane with a non-null job
+	xor     idx, idx
+	offset = (_ldata + 1 * _LANE_DATA_size + _job_in_lane)
+	cmpq    $0, offset(state)
+	cmovne  one(%rip), idx
+	offset = (_ldata + 2 * _LANE_DATA_size + _job_in_lane)
+	cmpq    $0, offset(state)
+	cmovne  two(%rip), idx
+	offset = (_ldata + 3 * _LANE_DATA_size + _job_in_lane)
+	cmpq    $0, offset(state)
+	cmovne  three(%rip), idx
+	offset = (_ldata + 4 * _LANE_DATA_size + _job_in_lane)
+	cmpq    $0, offset(state)
+	cmovne  four(%rip), idx
+	offset = (_ldata + 5 * _LANE_DATA_size + _job_in_lane)
+	cmpq    $0, offset(state)
+	cmovne  five(%rip), idx
+	offset = (_ldata + 6 * _LANE_DATA_size + _job_in_lane)
+	cmpq    $0, offset(state)
+	cmovne  six(%rip), idx
+	offset = (_ldata + 7 * _LANE_DATA_size + _job_in_lane)
+	cmpq    $0, offset(state)
+	cmovne  seven(%rip), idx
+
+	# copy idx to empty lanes
+copy_lane_data:
+	offset =  (_args + _data_ptr)
+	mov     offset(state,idx,8), tmp
+
+	I = 0
+.rep 8
+	offset =  (_ldata + I * _LANE_DATA_size + _job_in_lane)
+	cmpq    $0, offset(state)
+.altmacro
+	JNE_SKIP %I
+	offset =  (_args + _data_ptr + 8*I)
+	mov     tmp, offset(state)
+	offset =  (_lens + 4*I)
+	movl    $0xFFFFFFFF, offset(state)
+LABEL skip_ %I
+	I = (I+1)
+.noaltmacro
+.endr
+
+	# Find min length
+	vmovdqa _lens+0*16(state), %xmm0
+	vmovdqa _lens+1*16(state), %xmm1
+
+	vpminud %xmm1, %xmm0, %xmm2     # xmm2 has {D,C,B,A}
+	vpalignr $8, %xmm2, %xmm3, %xmm3   # xmm3 has {x,x,D,C}
+	vpminud %xmm3, %xmm2, %xmm2        # xmm2 has {x,x,E,F}
+	vpalignr $4, %xmm2, %xmm3, %xmm3    # xmm3 has {x,x,x,E}
+	vpminud %xmm3, %xmm2, %xmm2        # xmm2 has min value in low dword
+
+	vmovd   %xmm2, DWORD_idx
+	mov	idx, len2
+	and	$0xF, idx
+	shr	$4, len2
+	jz	len_is_0
+
+	vpand   clear_low_nibble(%rip), %xmm2, %xmm2
+	vpshufd $0, %xmm2, %xmm2
+
+	vpsubd  %xmm2, %xmm0, %xmm0
+	vpsubd  %xmm2, %xmm1, %xmm1
+
+	vmovdqa %xmm0, _lens+0*16(state)
+	vmovdqa %xmm1, _lens+1*16(state)
+
+	# "state" and "args" are the same address, arg1
+	# len is arg2
+	call	sha1_x8_avx2
+	# state and idx are intact
+
+
+len_is_0:
+	# process completed job "idx"
+	imul    $_LANE_DATA_size, idx, lane_data
+	lea     _ldata(state, lane_data), lane_data
+
+	mov     _job_in_lane(lane_data), job_rax
+	movq    $0, _job_in_lane(lane_data)
+	movl    $STS_COMPLETED, _status(job_rax)
+	mov     _unused_lanes(state), unused_lanes
+	shl     $4, unused_lanes
+	or      idx, unused_lanes
+	mov     unused_lanes, _unused_lanes(state)
+
+	movl	$0xFFFFFFFF, _lens(state, idx, 4)
+
+	vmovd    _args_digest(state , idx, 4) , %xmm0
+	vpinsrd  $1, _args_digest+1*32(state, idx, 4), %xmm0, %xmm0
+	vpinsrd  $2, _args_digest+2*32(state, idx, 4), %xmm0, %xmm0
+	vpinsrd  $3, _args_digest+3*32(state, idx, 4), %xmm0, %xmm0
+	movl    _args_digest+4*32(state, idx, 4), tmp2_w
+
+	vmovdqu  %xmm0, _result_digest(job_rax)
+	offset =  (_result_digest + 1*16)
+	mov     tmp2_w, offset(job_rax)
+
+return:
+
+	mov     _GPR_SAVE(%rsp), %rbx
+	mov     _GPR_SAVE+8*1(%rsp), %r10 #saved rsp
+	mov	_GPR_SAVE+8*3(%rsp), %rbp
+	mov	_GPR_SAVE+8*4(%rsp), %r12
+	mov	_GPR_SAVE+8*5(%rsp), %r13
+	mov	_GPR_SAVE+8*6(%rsp), %r14
+	mov	_GPR_SAVE+8*7(%rsp), %r15
+	mov     %r10, %rsp
+
+	ret
+
+return_null:
+	xor     job_rax, job_rax
+	jmp     return
+ENDPROC(sha1_mb_mgr_flush_avx2)
+
+
+#################################################################
+
+.align 16
+ENTRY(sha1_mb_mgr_get_comp_job_avx2)
+	push    %rbx
+
+	## if bit 32+3 is set, then all lanes are empty
+	mov     _unused_lanes(state), unused_lanes
+	bt      $(32+3), unused_lanes
+	jc      .return_null
+
+	# Find min length
+	vmovdqa _lens(state), %xmm0
+	vmovdqa _lens+1*16(state), %xmm1
+
+	vpminud %xmm1, %xmm0, %xmm2        # xmm2 has {D,C,B,A}
+	vpalignr $8, %xmm2, %xmm3, %xmm3   # xmm3 has {x,x,D,C}
+	vpminud %xmm3, %xmm2, %xmm2        # xmm2 has {x,x,E,F}
+	vpalignr $4, %xmm2, %xmm3, %xmm3    # xmm3 has {x,x,x,E}
+	vpminud %xmm3, %xmm2, %xmm2        # xmm2 has min value in low dword
+
+	vmovd   %xmm2, DWORD_idx
+	test    $~0xF, idx
+	jnz     .return_null
+
+	# process completed job "idx"
+	imul    $_LANE_DATA_size, idx, lane_data
+	lea     _ldata(state, lane_data), lane_data
+
+	mov     _job_in_lane(lane_data), job_rax
+	movq    $0,  _job_in_lane(lane_data)
+	movl    $STS_COMPLETED, _status(job_rax)
+	mov     _unused_lanes(state), unused_lanes
+	shl     $4, unused_lanes
+	or      idx, unused_lanes
+	mov     unused_lanes, _unused_lanes(state)
+
+	movl    $0xFFFFFFFF, _lens(state,  idx, 4)
+
+	vmovd   _args_digest(state, idx, 4), %xmm0
+	vpinsrd $1, _args_digest+1*32(state, idx, 4), %xmm0, %xmm0
+	vpinsrd $2, _args_digest+2*32(state, idx, 4), %xmm0, %xmm0
+	vpinsrd $3, _args_digest+3*32(state, idx, 4), %xmm0, %xmm0
+	movl    _args_digest+4*32(state, idx, 4), tmp2_w
+
+	vmovdqu %xmm0, _result_digest(job_rax)
+	movl    tmp2_w, _result_digest+1*16(job_rax)
+
+	pop     %rbx
+
+	ret
+
+.return_null:
+	xor     job_rax, job_rax
+	pop     %rbx
+	ret
+ENDPROC(sha1_mb_mgr_get_comp_job_avx2)
+
+.data
+
+.align 16
+clear_low_nibble:
+.octa	0x000000000000000000000000FFFFFFF0
+one:
+.quad  1
+two:
+.quad  2
+three:
+.quad  3
+four:
+.quad  4
+five:
+.quad  5
+six:
+.quad  6
+seven:
+.quad  7
diff --git a/arch/x86/crypto/sha-mb/sha1_mb_mgr_init_avx2.c b/arch/x86/crypto/sha-mb/sha1_mb_mgr_init_avx2.c
new file mode 100644
index 0000000..4ca7e16
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/sha1_mb_mgr_init_avx2.c
@@ -0,0 +1,64 @@
+/*
+ * Initialization code for multi buffer SHA1 algorithm for AVX2
+ *
+ * This file is provided under a dual BSD/GPLv2 license.  When using or
+ * redistributing this file, you may do so under either license.
+ *
+ * GPL LICENSE SUMMARY
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  This program is free software; you can redistribute it and/or modify
+ *  it under the terms of version 2 of the GNU General Public License as
+ *  published by the Free Software Foundation.
+ *
+ *  This program is distributed in the hope that it will be useful, but
+ *  WITHOUT ANY WARRANTY; without even the implied warranty of
+ *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ *  General Public License for more details.
+ *
+ *  Contact Information:
+ *	Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *  BSD LICENSE
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  Redistribution and use in source and binary forms, with or without
+ *  modification, are permitted provided that the following conditions
+ *  are met:
+ *
+ *    * Redistributions of source code must retain the above copyright
+ *      notice, this list of conditions and the following disclaimer.
+ *    * Redistributions in binary form must reproduce the above copyright
+ *      notice, this list of conditions and the following disclaimer in
+ *      the documentation and/or other materials provided with the
+ *      distribution.
+ *    * Neither the name of Intel Corporation nor the names of its
+ *      contributors may be used to endorse or promote products derived
+ *      from this software without specific prior written permission.
+ *
+ *  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include "sha_mb_mgr.h"
+
+void sha1_mb_mgr_init_avx2(struct sha1_mb_mgr *state)
+{
+	unsigned int j;
+	state->unused_lanes = 0xF76543210;
+	for (j = 0; j < 8; j++) {
+		state->lens[j] = 0xFFFFFFFF;
+		state->ldata[j].job_in_lane = NULL;
+	}
+}
diff --git a/arch/x86/crypto/sha-mb/sha1_mb_mgr_submit_avx2.S b/arch/x86/crypto/sha-mb/sha1_mb_mgr_submit_avx2.S
new file mode 100644
index 0000000..2ab9560
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/sha1_mb_mgr_submit_avx2.S
@@ -0,0 +1,228 @@
+/*
+ * Buffer submit code for multi buffer SHA1 algorithm
+ *
+ * This file is provided under a dual BSD/GPLv2 license.  When using or
+ * redistributing this file, you may do so under either license.
+ *
+ * GPL LICENSE SUMMARY
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  This program is free software; you can redistribute it and/or modify
+ *  it under the terms of version 2 of the GNU General Public License as
+ *  published by the Free Software Foundation.
+ *
+ *  This program is distributed in the hope that it will be useful, but
+ *  WITHOUT ANY WARRANTY; without even the implied warranty of
+ *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ *  General Public License for more details.
+ *
+ *  Contact Information:
+ *      James Guilford <james.guilford@intel.com>
+ *	Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *  BSD LICENSE
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  Redistribution and use in source and binary forms, with or without
+ *  modification, are permitted provided that the following conditions
+ *  are met:
+ *
+ *    * Redistributions of source code must retain the above copyright
+ *      notice, this list of conditions and the following disclaimer.
+ *    * Redistributions in binary form must reproduce the above copyright
+ *      notice, this list of conditions and the following disclaimer in
+ *      the documentation and/or other materials provided with the
+ *      distribution.
+ *    * Neither the name of Intel Corporation nor the names of its
+ *      contributors may be used to endorse or promote products derived
+ *      from this software without specific prior written permission.
+ *
+ *  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <linux/linkage.h>
+#include "sha1_mb_mgr_datastruct.S"
+
+
+.extern sha1_x8_avx
+
+# LINUX register definitions
+arg1    = %rdi
+arg2    = %rsi
+size_offset	= %rcx
+tmp2		= %rcx
+extra_blocks	= %rdx
+
+# Common definitions
+#define state   arg1
+#define job     %rsi
+#define len2    arg2
+#define p2      arg2
+
+# idx must be a register not clobberred by sha1_x8_avx2
+idx		= %r8
+DWORD_idx	= %r8d
+last_len	= %r8
+
+p               = %r11
+start_offset    = %r11
+
+unused_lanes    = %rbx
+BYTE_unused_lanes = %bl
+
+job_rax         = %rax
+len             = %rax
+DWORD_len	= %eax
+
+lane            = %rbp
+tmp3            = %rbp
+
+tmp             = %r9
+DWORD_tmp	= %r9d
+
+lane_data       = %r10
+
+# STACK_SPACE needs to be an odd multiple of 8
+STACK_SPACE     = 8*8 + 16*10 + 8
+
+# JOB* submit_mb_mgr_submit_avx2(MB_MGR *state, job_sha1 *job)
+# arg 1 : rcx : state
+# arg 2 : rdx : job
+ENTRY(sha1_mb_mgr_submit_avx2)
+
+	mov	%rsp, %r10
+	sub     $STACK_SPACE, %rsp
+	and	$~31, %rsp
+
+	mov     %rbx, (%rsp)
+	mov	%r10, 8*2(%rsp)	#save old rsp
+	mov     %rbp, 8*3(%rsp)
+	mov	%r12, 8*4(%rsp)
+	mov	%r13, 8*5(%rsp)
+	mov	%r14, 8*6(%rsp)
+	mov	%r15, 8*7(%rsp)
+
+	mov     _unused_lanes(state), unused_lanes
+	mov	unused_lanes, lane
+	and	$0xF, lane
+	shr     $4, unused_lanes
+	imul    $_LANE_DATA_size, lane, lane_data
+	movl    $STS_BEING_PROCESSED, _status(job)
+	lea     _ldata(state, lane_data), lane_data
+	mov     unused_lanes, _unused_lanes(state)
+	movl    _len(job),  DWORD_len
+
+	mov	job, _job_in_lane(lane_data)
+	shl	$4, len
+	or	lane, len
+
+	movl    DWORD_len,  _lens(state , lane, 4)
+
+	# Load digest words from result_digest
+	vmovdqu	_result_digest(job), %xmm0
+	mov	_result_digest+1*16(job), DWORD_tmp
+	vmovd    %xmm0, _args_digest(state, lane, 4)
+	vpextrd  $1, %xmm0, _args_digest+1*32(state , lane, 4)
+	vpextrd  $2, %xmm0, _args_digest+2*32(state , lane, 4)
+	vpextrd  $3, %xmm0, _args_digest+3*32(state , lane, 4)
+	movl    DWORD_tmp, _args_digest+4*32(state , lane, 4)
+
+	mov     _buffer(job), p
+	mov     p, _args_data_ptr(state, lane, 8)
+
+	cmp     $0xF, unused_lanes
+	jne     return_null
+
+start_loop:
+	# Find min length
+	vmovdqa _lens(state), %xmm0
+	vmovdqa _lens+1*16(state), %xmm1
+
+	vpminud %xmm1, %xmm0, %xmm2        # xmm2 has {D,C,B,A}
+	vpalignr $8, %xmm2, %xmm3, %xmm3   # xmm3 has {x,x,D,C}
+	vpminud %xmm3, %xmm2, %xmm2        # xmm2 has {x,x,E,F}
+	vpalignr $4, %xmm2, %xmm3, %xmm3   # xmm3 has {x,x,x,E}
+	vpminud %xmm3, %xmm2, %xmm2        # xmm2 has min value in low dword
+
+	vmovd   %xmm2, DWORD_idx
+	mov    idx, len2
+	and    $0xF, idx
+	shr    $4, len2
+	jz     len_is_0
+
+	vpand   clear_low_nibble(%rip), %xmm2, %xmm2
+	vpshufd $0, %xmm2, %xmm2
+
+	vpsubd  %xmm2, %xmm0, %xmm0
+	vpsubd  %xmm2, %xmm1, %xmm1
+
+	vmovdqa %xmm0, _lens + 0*16(state)
+	vmovdqa %xmm1, _lens + 1*16(state)
+
+
+	# "state" and "args" are the same address, arg1
+	# len is arg2
+	call    sha1_x8_avx2
+
+	# state and idx are intact
+
+len_is_0:
+	# process completed job "idx"
+	imul    $_LANE_DATA_size, idx, lane_data
+	lea     _ldata(state, lane_data), lane_data
+
+	mov     _job_in_lane(lane_data), job_rax
+	mov     _unused_lanes(state), unused_lanes
+	movq    $0, _job_in_lane(lane_data)
+	movl    $STS_COMPLETED, _status(job_rax)
+	shl     $4, unused_lanes
+	or      idx, unused_lanes
+	mov     unused_lanes, _unused_lanes(state)
+
+	movl	$0xFFFFFFFF, _lens(state, idx, 4)
+
+	vmovd    _args_digest(state, idx, 4), %xmm0
+	vpinsrd  $1, _args_digest+1*32(state , idx, 4), %xmm0, %xmm0
+	vpinsrd  $2, _args_digest+2*32(state , idx, 4), %xmm0, %xmm0
+	vpinsrd  $3, _args_digest+3*32(state , idx, 4), %xmm0, %xmm0
+	movl    4*32(state, idx, 4), DWORD_tmp
+
+	vmovdqu  %xmm0, _result_digest(job_rax)
+	movl    DWORD_tmp, _result_digest+1*16(job_rax)
+
+return:
+
+	mov     (%rsp), %rbx
+	mov	8*2(%rsp), %r10	#save old rsp
+	mov     8*3(%rsp), %rbp
+	mov	8*4(%rsp), %r12
+	mov	8*5(%rsp), %r13
+	mov	8*6(%rsp), %r14
+	mov	8*7(%rsp), %r15
+	mov     %r10, %rsp
+
+	ret
+
+return_null:
+	xor     job_rax, job_rax
+	jmp     return
+
+ENDPROC(sha1_mb_mgr_submit_avx2)
+
+.data
+
+.align 16
+clear_low_nibble:
+	.octa	0x000000000000000000000000FFFFFFF0
-- 
1.7.11.7

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

* [PATCH v5 6/7] crypto: SHA1 multibuffer crypto computation (x8 AVX2)
       [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
                   ` (5 preceding siblings ...)
  2014-07-22 22:09 ` [PATCH v5 5/7] crypto: SHA1 multibuffer submit and flush routines for AVX2 Tim Chen
@ 2014-07-22 22:09 ` Tim Chen
  2014-07-22 22:09 ` [PATCH v5 7/7] crypto: SHA1 multibuffer scheduler Tim Chen
  7 siblings, 0 replies; 15+ messages in thread
From: Tim Chen @ 2014-07-22 22:09 UTC (permalink / raw)
  To: Herbert Xu, H. Peter Anvin, David S.Miller, Peter Zijlstra, Ingo Molnar
  Cc: Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Tim Chen, Jussi Kivilinna, Thomas Gleixner,
	Tadeusz Struk, tkhai, linux-crypto, linux-kernel

This patch introduces the assembly routines to do SHA1 computation on
buffers belonging to serveral jobs at once.  The assembly routines are
optimized with AVX2 instructions that have 8 data lanes and using AVX2
registers.

Signed-off-by: Tim Chen <tim.c.chen@linux.intel.com>
---
 arch/x86/crypto/sha-mb/sha1_x8_avx2.S | 472 ++++++++++++++++++++++++++++++++++
 1 file changed, 472 insertions(+)
 create mode 100644 arch/x86/crypto/sha-mb/sha1_x8_avx2.S

diff --git a/arch/x86/crypto/sha-mb/sha1_x8_avx2.S b/arch/x86/crypto/sha-mb/sha1_x8_avx2.S
new file mode 100644
index 0000000..8e1b477
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/sha1_x8_avx2.S
@@ -0,0 +1,472 @@
+/*
+ * Multi-buffer SHA1 algorithm hash compute routine
+ *
+ * This file is provided under a dual BSD/GPLv2 license.  When using or
+ * redistributing this file, you may do so under either license.
+ *
+ * GPL LICENSE SUMMARY
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  This program is free software; you can redistribute it and/or modify
+ *  it under the terms of version 2 of the GNU General Public License as
+ *  published by the Free Software Foundation.
+ *
+ *  This program is distributed in the hope that it will be useful, but
+ *  WITHOUT ANY WARRANTY; without even the implied warranty of
+ *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ *  General Public License for more details.
+ *
+ *  Contact Information:
+ *      James Guilford <james.guilford@intel.com>
+ *	Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *  BSD LICENSE
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  Redistribution and use in source and binary forms, with or without
+ *  modification, are permitted provided that the following conditions
+ *  are met:
+ *
+ *    * Redistributions of source code must retain the above copyright
+ *      notice, this list of conditions and the following disclaimer.
+ *    * Redistributions in binary form must reproduce the above copyright
+ *      notice, this list of conditions and the following disclaimer in
+ *      the documentation and/or other materials provided with the
+ *      distribution.
+ *    * Neither the name of Intel Corporation nor the names of its
+ *      contributors may be used to endorse or promote products derived
+ *      from this software without specific prior written permission.
+ *
+ *  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <linux/linkage.h>
+#include "sha1_mb_mgr_datastruct.S"
+
+## code to compute oct SHA1 using SSE-256
+## outer calling routine takes care of save and restore of XMM registers
+
+## Function clobbers: rax, rcx, rdx,   rbx, rsi, rdi, r9-r15# ymm0-15
+##
+## Linux clobbers:    rax rbx rcx rdx rsi            r9 r10 r11 r12 r13 r14 r15
+## Linux preserves:                       rdi rbp r8
+##
+## clobbers ymm0-15
+
+
+# TRANSPOSE8 r0, r1, r2, r3, r4, r5, r6, r7, t0, t1
+# "transpose" data in {r0...r7} using temps {t0...t1}
+# Input looks like: {r0 r1 r2 r3 r4 r5 r6 r7}
+# r0 = {a7 a6 a5 a4   a3 a2 a1 a0}
+# r1 = {b7 b6 b5 b4   b3 b2 b1 b0}
+# r2 = {c7 c6 c5 c4   c3 c2 c1 c0}
+# r3 = {d7 d6 d5 d4   d3 d2 d1 d0}
+# r4 = {e7 e6 e5 e4   e3 e2 e1 e0}
+# r5 = {f7 f6 f5 f4   f3 f2 f1 f0}
+# r6 = {g7 g6 g5 g4   g3 g2 g1 g0}
+# r7 = {h7 h6 h5 h4   h3 h2 h1 h0}
+#
+# Output looks like: {r0 r1 r2 r3 r4 r5 r6 r7}
+# r0 = {h0 g0 f0 e0   d0 c0 b0 a0}
+# r1 = {h1 g1 f1 e1   d1 c1 b1 a1}
+# r2 = {h2 g2 f2 e2   d2 c2 b2 a2}
+# r3 = {h3 g3 f3 e3   d3 c3 b3 a3}
+# r4 = {h4 g4 f4 e4   d4 c4 b4 a4}
+# r5 = {h5 g5 f5 e5   d5 c5 b5 a5}
+# r6 = {h6 g6 f6 e6   d6 c6 b6 a6}
+# r7 = {h7 g7 f7 e7   d7 c7 b7 a7}
+#
+
+.macro TRANSPOSE8 r0 r1 r2 r3 r4 r5 r6 r7 t0 t1
+	# process top half (r0..r3) {a...d}
+	vshufps  $0x44, \r1, \r0, \t0 # t0 = {b5 b4 a5 a4   b1 b0 a1 a0}
+	vshufps  $0xEE, \r1, \r0, \r0 # r0 = {b7 b6 a7 a6   b3 b2 a3 a2}
+	vshufps  $0x44, \r3, \r2, \t1 # t1 = {d5 d4 c5 c4   d1 d0 c1 c0}
+	vshufps  $0xEE, \r3, \r2, \r2 # r2 = {d7 d6 c7 c6   d3 d2 c3 c2}
+	vshufps  $0xDD, \t1, \t0, \r3 # r3 = {d5 c5 b5 a5   d1 c1 b1 a1}
+	vshufps  $0x88, \r2, \r0, \r1 # r1 = {d6 c6 b6 a6   d2 c2 b2 a2}
+	vshufps  $0xDD, \r2, \r0, \r0 # r0 = {d7 c7 b7 a7   d3 c3 b3 a3}
+	vshufps  $0x88, \t1, \t0, \t0 # t0 = {d4 c4 b4 a4   d0 c0 b0 a0}
+
+	# use r2 in place of t0
+	# process bottom half (r4..r7) {e...h}
+	vshufps  $0x44, \r5, \r4, \r2 # r2 = {f5 f4 e5 e4   f1 f0 e1 e0}
+	vshufps  $0xEE, \r5, \r4, \r4 # r4 = {f7 f6 e7 e6   f3 f2 e3 e2}
+	vshufps  $0x44, \r7, \r6, \t1 # t1 = {h5 h4 g5 g4   h1 h0 g1 g0}
+	vshufps  $0xEE, \r7, \r6, \r6 # r6 = {h7 h6 g7 g6   h3 h2 g3 g2}
+	vshufps  $0xDD, \t1, \r2, \r7 # r7 = {h5 g5 f5 e5   h1 g1 f1 e1}
+	vshufps  $0x88, \r6, \r4, \r5 # r5 = {h6 g6 f6 e6   h2 g2 f2 e2}
+	vshufps  $0xDD, \r6, \r4, \r4 # r4 = {h7 g7 f7 e7   h3 g3 f3 e3}
+	vshufps  $0x88, \t1, \r2, \t1 # t1 = {h4 g4 f4 e4   h0 g0 f0 e0}
+
+	vperm2f128      $0x13, \r1, \r5, \r6  # h6...a6
+	vperm2f128      $0x02, \r1, \r5, \r2  # h2...a2
+	vperm2f128      $0x13, \r3, \r7, \r5  # h5...a5
+	vperm2f128      $0x02, \r3, \r7, \r1  # h1...a1
+	vperm2f128      $0x13, \r0, \r4, \r7  # h7...a7
+	vperm2f128      $0x02, \r0, \r4, \r3  # h3...a3
+	vperm2f128      $0x13, \t0, \t1, \r4  # h4...a4
+	vperm2f128      $0x02, \t0, \t1, \r0  # h0...a0
+
+.endm
+##
+## Magic functions defined in FIPS 180-1
+##
+# macro MAGIC_F0 F,B,C,D,T   ## F = (D ^ (B & (C ^ D)))
+.macro MAGIC_F0 regF regB regC regD regT
+    vpxor \regD, \regC, \regF
+    vpand \regB, \regF, \regF
+    vpxor \regD, \regF, \regF
+.endm
+
+# macro MAGIC_F1 F,B,C,D,T   ## F = (B ^ C ^ D)
+.macro MAGIC_F1 regF regB regC regD regT
+    vpxor  \regC, \regD, \regF
+    vpxor  \regB, \regF, \regF
+.endm
+
+# macro MAGIC_F2 F,B,C,D,T   ## F = ((B & C) | (B & D) | (C & D))
+.macro MAGIC_F2 regF regB regC regD regT
+    vpor  \regC, \regB, \regF
+    vpand \regC, \regB, \regT
+    vpand \regD, \regF, \regF
+    vpor  \regT, \regF, \regF
+.endm
+
+# macro MAGIC_F3 F,B,C,D,T   ## F = (B ^ C ^ D)
+.macro MAGIC_F3 regF regB regC regD regT
+    MAGIC_F1 \regF,\regB,\regC,\regD,\regT
+.endm
+
+# PROLD reg, imm, tmp
+.macro PROLD reg imm tmp
+	vpsrld  $(32-\imm), \reg, \tmp
+	vpslld  $\imm, \reg, \reg
+	vpor    \tmp, \reg, \reg
+.endm
+
+.macro PROLD_nd reg imm tmp src
+	vpsrld  $(32-\imm), \src, \tmp
+	vpslld  $\imm, \src, \reg
+	vpor	\tmp, \reg, \reg
+.endm
+
+.macro SHA1_STEP_00_15 regA regB regC regD regE regT regF memW immCNT MAGIC
+	vpaddd	\immCNT, \regE, \regE
+	vpaddd	\memW*32(%rsp), \regE, \regE
+	PROLD_nd \regT, 5, \regF, \regA
+	vpaddd	\regT, \regE, \regE
+	\MAGIC  \regF, \regB, \regC, \regD, \regT
+        PROLD   \regB, 30, \regT
+        vpaddd  \regF, \regE, \regE
+.endm
+
+.macro SHA1_STEP_16_79 regA regB regC regD regE regT regF memW immCNT MAGIC
+	vpaddd	\immCNT, \regE, \regE
+	offset = ((\memW - 14) & 15) * 32
+	vmovdqu offset(%rsp), W14
+	vpxor	W14, W16, W16
+	offset = ((\memW -  8) & 15) * 32
+	vpxor	offset(%rsp), W16, W16
+	offset = ((\memW -  3) & 15) * 32
+	vpxor	offset(%rsp), W16, W16
+	vpsrld	$(32-1), W16, \regF
+	vpslld	$1, W16, W16
+	vpor	W16, \regF, \regF
+
+	ROTATE_W
+
+	offset = ((\memW - 0) & 15) * 32
+	vmovdqu	\regF, offset(%rsp)
+	vpaddd	\regF, \regE, \regE
+	PROLD_nd \regT, 5, \regF, \regA
+	vpaddd	\regT, \regE, \regE
+	\MAGIC \regF,\regB,\regC,\regD,\regT      ## FUN  = MAGIC_Fi(B,C,D)
+	PROLD   \regB,30, \regT
+	vpaddd  \regF, \regE, \regE
+.endm
+
+########################################################################
+########################################################################
+########################################################################
+
+## FRAMESZ plus pushes must be an odd multiple of 8
+YMM_SAVE = (15-15)*32
+FRAMESZ = 32*16 + YMM_SAVE
+_YMM  =   FRAMESZ - YMM_SAVE
+
+#define VMOVPS   vmovups
+
+IDX  = %rax
+inp0 = %r9
+inp1 = %r10
+inp2 = %r11
+inp3 = %r12
+inp4 = %r13
+inp5 = %r14
+inp6 = %r15
+inp7 = %rcx
+arg1 = %rdi
+arg2 = %rsi
+RSP_SAVE = %rdx
+
+# ymm0 A
+# ymm1 B
+# ymm2 C
+# ymm3 D
+# ymm4 E
+# ymm5         F       AA
+# ymm6         T0      BB
+# ymm7         T1      CC
+# ymm8         T2      DD
+# ymm9         T3      EE
+# ymm10                T4      TMP
+# ymm11                T5      FUN
+# ymm12                T6      K
+# ymm13                T7      W14
+# ymm14                T8      W15
+# ymm15                T9      W16
+
+
+A  =     %ymm0
+B  =     %ymm1
+C  =     %ymm2
+D  =     %ymm3
+E  =     %ymm4
+F  =     %ymm5
+T0 =	 %ymm6
+T1 =     %ymm7
+T2 =     %ymm8
+T3 =     %ymm9
+T4 =     %ymm10
+T5 =     %ymm11
+T6 =     %ymm12
+T7 =     %ymm13
+T8  =     %ymm14
+T9  =     %ymm15
+
+AA  =     %ymm5
+BB  =     %ymm6
+CC  =     %ymm7
+DD  =     %ymm8
+EE  =     %ymm9
+TMP =     %ymm10
+FUN =     %ymm11
+K   =     %ymm12
+W14 =     %ymm13
+W15 =     %ymm14
+W16 =     %ymm15
+
+.macro ROTATE_ARGS
+ TMP_ = E
+ E = D
+ D = C
+ C = B
+ B = A
+ A = TMP_
+.endm
+
+.macro ROTATE_W
+TMP_  = W16
+W16  = W15
+W15  = W14
+W14  = TMP_
+.endm
+
+# 8 streams x 5 32bit words per digest x 4 bytes per word
+#define DIGEST_SIZE (8*5*4)
+
+.align 32
+
+# void sha1_x8_avx2(void **input_data, UINT128 *digest, UINT32 size)
+# arg 1 : pointer to array[4] of pointer to input data
+# arg 2 : size (in blocks) ;; assumed to be >= 1
+#
+ENTRY(sha1_x8_avx2)
+
+	push	RSP_SAVE
+
+	#save rsp
+	mov	%rsp, RSP_SAVE
+	sub     $FRAMESZ, %rsp
+
+	#align rsp to 32 Bytes
+	and	$~0x1F, %rsp
+
+	## Initialize digests
+	vmovdqu  0*32(arg1), A
+	vmovdqu  1*32(arg1), B
+	vmovdqu  2*32(arg1), C
+	vmovdqu  3*32(arg1), D
+	vmovdqu  4*32(arg1), E
+
+	## transpose input onto stack
+	mov     _data_ptr+0*8(arg1),inp0
+	mov     _data_ptr+1*8(arg1),inp1
+	mov     _data_ptr+2*8(arg1),inp2
+	mov     _data_ptr+3*8(arg1),inp3
+	mov     _data_ptr+4*8(arg1),inp4
+	mov     _data_ptr+5*8(arg1),inp5
+	mov     _data_ptr+6*8(arg1),inp6
+	mov     _data_ptr+7*8(arg1),inp7
+
+	xor     IDX, IDX
+lloop:
+	vmovdqu  PSHUFFLE_BYTE_FLIP_MASK(%rip), F
+	I=0
+.rep 2
+	VMOVPS   (inp0, IDX), T0
+	VMOVPS   (inp1, IDX), T1
+	VMOVPS   (inp2, IDX), T2
+	VMOVPS   (inp3, IDX), T3
+	VMOVPS   (inp4, IDX), T4
+	VMOVPS   (inp5, IDX), T5
+	VMOVPS   (inp6, IDX), T6
+	VMOVPS   (inp7, IDX), T7
+
+	TRANSPOSE8       T0, T1, T2, T3, T4, T5, T6, T7, T8, T9
+	vpshufb  F, T0, T0
+	vmovdqu  T0, (I*8)*32(%rsp)
+	vpshufb  F, T1, T1
+	vmovdqu  T1, (I*8+1)*32(%rsp)
+	vpshufb  F, T2, T2
+	vmovdqu  T2, (I*8+2)*32(%rsp)
+	vpshufb  F, T3, T3
+	vmovdqu  T3, (I*8+3)*32(%rsp)
+	vpshufb  F, T4, T4
+	vmovdqu  T4, (I*8+4)*32(%rsp)
+	vpshufb  F, T5, T5
+	vmovdqu  T5, (I*8+5)*32(%rsp)
+	vpshufb  F, T6, T6
+	vmovdqu  T6, (I*8+6)*32(%rsp)
+	vpshufb  F, T7, T7
+	vmovdqu  T7, (I*8+7)*32(%rsp)
+	add     $32, IDX
+	I = (I+1)
+.endr
+	# save old digests
+	vmovdqu  A,AA
+	vmovdqu  B,BB
+	vmovdqu  C,CC
+	vmovdqu  D,DD
+	vmovdqu  E,EE
+
+##
+## perform 0-79 steps
+##
+	vmovdqu  K00_19(%rip), K
+## do rounds 0...15
+	I = 0
+.rep 16
+	SHA1_STEP_00_15 A,B,C,D,E, TMP,FUN, I, K, MAGIC_F0
+	ROTATE_ARGS
+	I = (I+1)
+.endr
+
+## do rounds 16...19
+	vmovdqu  ((16 - 16) & 15) * 32 (%rsp), W16
+	vmovdqu  ((16 - 15) & 15) * 32 (%rsp), W15
+.rep 4
+	SHA1_STEP_16_79 A,B,C,D,E, TMP,FUN, I, K, MAGIC_F0
+	ROTATE_ARGS
+	I = (I+1)
+.endr
+
+## do rounds 20...39
+	vmovdqu  K20_39(%rip), K
+.rep 20
+	SHA1_STEP_16_79 A,B,C,D,E, TMP,FUN, I, K, MAGIC_F1
+	ROTATE_ARGS
+	I = (I+1)
+.endr
+
+## do rounds 40...59
+	vmovdqu  K40_59(%rip), K
+.rep 20
+	SHA1_STEP_16_79 A,B,C,D,E, TMP,FUN, I, K, MAGIC_F2
+	ROTATE_ARGS
+	I = (I+1)
+.endr
+
+## do rounds 60...79
+	vmovdqu  K60_79(%rip), K
+.rep 20
+	SHA1_STEP_16_79 A,B,C,D,E, TMP,FUN, I, K, MAGIC_F3
+	ROTATE_ARGS
+	I = (I+1)
+.endr
+
+	vpaddd   AA,A,A
+	vpaddd   BB,B,B
+	vpaddd   CC,C,C
+	vpaddd   DD,D,D
+	vpaddd   EE,E,E
+
+	sub     $1, arg2
+	jne     lloop
+
+	# write out digests
+	vmovdqu  A, 0*32(arg1)
+	vmovdqu  B, 1*32(arg1)
+	vmovdqu  C, 2*32(arg1)
+	vmovdqu  D, 3*32(arg1)
+	vmovdqu  E, 4*32(arg1)
+
+	# update input pointers
+	add     IDX, inp0
+	add     IDX, inp1
+	add     IDX, inp2
+	add     IDX, inp3
+	add     IDX, inp4
+	add     IDX, inp5
+	add     IDX, inp6
+	add     IDX, inp7
+	mov     inp0, _data_ptr (arg1)
+	mov     inp1, _data_ptr + 1*8(arg1)
+	mov     inp2, _data_ptr + 2*8(arg1)
+	mov     inp3, _data_ptr + 3*8(arg1)
+	mov     inp4, _data_ptr + 4*8(arg1)
+	mov     inp5, _data_ptr + 5*8(arg1)
+	mov     inp6, _data_ptr + 6*8(arg1)
+	mov     inp7, _data_ptr + 7*8(arg1)
+
+	################
+	## Postamble
+
+	mov     RSP_SAVE, %rsp
+	pop	RSP_SAVE
+
+	ret
+ENDPROC(sha1_x8_avx2)
+
+
+.data
+
+.align 32
+K00_19:
+.octa 0x5A8279995A8279995A8279995A827999
+.octa 0x5A8279995A8279995A8279995A827999
+K20_39:
+.octa 0x6ED9EBA16ED9EBA16ED9EBA16ED9EBA1
+.octa 0x6ED9EBA16ED9EBA16ED9EBA16ED9EBA1
+K40_59:
+.octa 0x8F1BBCDC8F1BBCDC8F1BBCDC8F1BBCDC
+.octa 0x8F1BBCDC8F1BBCDC8F1BBCDC8F1BBCDC
+K60_79:
+.octa 0xCA62C1D6CA62C1D6CA62C1D6CA62C1D6
+.octa 0xCA62C1D6CA62C1D6CA62C1D6CA62C1D6
+PSHUFFLE_BYTE_FLIP_MASK:
+.octa 0x0c0d0e0f08090a0b0405060700010203
+.octa 0x0c0d0e0f08090a0b0405060700010203
-- 
1.7.11.7

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

* [PATCH v5 7/7] crypto: SHA1 multibuffer scheduler
       [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
                   ` (6 preceding siblings ...)
  2014-07-22 22:09 ` [PATCH v5 6/7] crypto: SHA1 multibuffer crypto computation (x8 AVX2) Tim Chen
@ 2014-07-22 22:09 ` Tim Chen
  7 siblings, 0 replies; 15+ messages in thread
From: Tim Chen @ 2014-07-22 22:09 UTC (permalink / raw)
  To: Herbert Xu, H. Peter Anvin, David S.Miller, Peter Zijlstra, Ingo Molnar
  Cc: Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Tim Chen, Jussi Kivilinna, Thomas Gleixner,
	Tadeusz Struk, tkhai, linux-crypto, linux-kernel

This patch introduces the multi-buffer scheduler which is responsible
for submitting scatter-gather buffers from several SHA1 jobs to the
multi-buffer algorithm.  It also contains the flush routine to that's
called by the crypto daemon to complete the job when no new jobs arrive
before the deadline of maximum latency of a SHA1 crypto job.

The SHA1 multi-buffer crypto algorithm is defined and initialized in
this patch.

Signed-off-by: Tim Chen <tim.c.chen@linux.intel.com>
---
 arch/x86/crypto/Makefile         |   2 +
 arch/x86/crypto/sha-mb/Makefile  |  11 +
 arch/x86/crypto/sha-mb/sha1_mb.c | 935 +++++++++++++++++++++++++++++++++++++++
 3 files changed, 948 insertions(+)
 create mode 100644 arch/x86/crypto/sha-mb/Makefile
 create mode 100644 arch/x86/crypto/sha-mb/sha1_mb.c

diff --git a/arch/x86/crypto/Makefile b/arch/x86/crypto/Makefile
index 61d6e28..1189267 100644
--- a/arch/x86/crypto/Makefile
+++ b/arch/x86/crypto/Makefile
@@ -25,6 +25,7 @@ obj-$(CONFIG_CRYPTO_GHASH_CLMUL_NI_INTEL) += ghash-clmulni-intel.o
 
 obj-$(CONFIG_CRYPTO_CRC32C_INTEL) += crc32c-intel.o
 obj-$(CONFIG_CRYPTO_SHA1_SSSE3) += sha1-ssse3.o
+obj-$(CONFIG_CRYPTO_SHA1_MB) += sha-mb/
 obj-$(CONFIG_CRYPTO_CRC32_PCLMUL) += crc32-pclmul.o
 obj-$(CONFIG_CRYPTO_SHA256_SSSE3) += sha256-ssse3.o
 obj-$(CONFIG_CRYPTO_SHA512_SSSE3) += sha512-ssse3.o
@@ -88,3 +89,4 @@ crc32-pclmul-y := crc32-pclmul_asm.o crc32-pclmul_glue.o
 sha256-ssse3-y := sha256-ssse3-asm.o sha256-avx-asm.o sha256-avx2-asm.o sha256_ssse3_glue.o
 sha512-ssse3-y := sha512-ssse3-asm.o sha512-avx-asm.o sha512-avx2-asm.o sha512_ssse3_glue.o
 crct10dif-pclmul-y := crct10dif-pcl-asm_64.o crct10dif-pclmul_glue.o
+
diff --git a/arch/x86/crypto/sha-mb/Makefile b/arch/x86/crypto/sha-mb/Makefile
new file mode 100644
index 0000000..2f87563
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/Makefile
@@ -0,0 +1,11 @@
+#
+# Arch-specific CryptoAPI modules.
+#
+
+avx2_supported := $(call as-instr,vpgatherdd %ymm0$(comma)(%eax$(comma)%ymm1\
+                                $(comma)4)$(comma)%ymm2,yes,no)
+ifeq ($(avx2_supported),yes)
+	obj-$(CONFIG_CRYPTO_SHA1_MB) += sha1-mb.o
+	sha1-mb-y := sha1_mb.o sha1_mb_mgr_flush_avx2.o \
+	     sha1_mb_mgr_init_avx2.o sha1_mb_mgr_submit_avx2.o sha1_x8_avx2.o
+endif
diff --git a/arch/x86/crypto/sha-mb/sha1_mb.c b/arch/x86/crypto/sha-mb/sha1_mb.c
new file mode 100644
index 0000000..e27ff31
--- /dev/null
+++ b/arch/x86/crypto/sha-mb/sha1_mb.c
@@ -0,0 +1,935 @@
+/*
+ * Multi buffer SHA1 algorithm Glue Code
+ *
+ * This file is provided under a dual BSD/GPLv2 license.  When using or
+ * redistributing this file, you may do so under either license.
+ *
+ * GPL LICENSE SUMMARY
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  This program is free software; you can redistribute it and/or modify
+ *  it under the terms of version 2 of the GNU General Public License as
+ *  published by the Free Software Foundation.
+ *
+ *  This program is distributed in the hope that it will be useful, but
+ *  WITHOUT ANY WARRANTY; without even the implied warranty of
+ *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ *  General Public License for more details.
+ *
+ *  Contact Information:
+ *	Tim Chen <tim.c.chen@linux.intel.com>
+ *
+ *  BSD LICENSE
+ *
+ *  Copyright(c) 2014 Intel Corporation.
+ *
+ *  Redistribution and use in source and binary forms, with or without
+ *  modification, are permitted provided that the following conditions
+ *  are met:
+ *
+ *    * Redistributions of source code must retain the above copyright
+ *      notice, this list of conditions and the following disclaimer.
+ *    * Redistributions in binary form must reproduce the above copyright
+ *      notice, this list of conditions and the following disclaimer in
+ *      the documentation and/or other materials provided with the
+ *      distribution.
+ *    * Neither the name of Intel Corporation nor the names of its
+ *      contributors may be used to endorse or promote products derived
+ *      from this software without specific prior written permission.
+ *
+ *  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#define pr_fmt(fmt)	KBUILD_MODNAME ": " fmt
+
+#include <crypto/internal/hash.h>
+#include <linux/init.h>
+#include <linux/module.h>
+#include <linux/mm.h>
+#include <linux/cryptohash.h>
+#include <linux/types.h>
+#include <linux/list.h>
+#include <crypto/scatterwalk.h>
+#include <crypto/sha.h>
+#include <crypto/mcryptd.h>
+#include <crypto/crypto_wq.h>
+#include <asm/byteorder.h>
+#include <asm/i387.h>
+#include <asm/xcr.h>
+#include <asm/xsave.h>
+#include <linux/hardirq.h>
+#include <asm/fpu-internal.h>
+#include "sha_mb_ctx.h"
+
+#define FLUSH_INTERVAL 1000 /* in usec */
+
+struct mcryptd_alg_state sha1_mb_alg_state;
+
+struct sha1_mb_ctx {
+	struct mcryptd_ahash *mcryptd_tfm;
+};
+
+static inline struct mcryptd_hash_request_ctx *cast_hash_to_mcryptd_ctx(struct sha1_hash_ctx *hash_ctx)
+{
+	struct shash_desc *desc;
+
+	desc = container_of((void *) hash_ctx, struct shash_desc, __ctx);
+	return container_of(desc, struct mcryptd_hash_request_ctx, desc);
+}
+
+static inline struct ahash_request *cast_mcryptd_ctx_to_req(struct mcryptd_hash_request_ctx *ctx)
+{
+	return container_of((void *) ctx, struct ahash_request, __ctx);
+}
+
+static void req_ctx_init(struct mcryptd_hash_request_ctx *rctx,
+				struct shash_desc *desc)
+{
+	rctx->flag = HASH_UPDATE;
+}
+
+asmlinkage void (*sha1_job_mgr_init)(struct sha1_mb_mgr *state);
+asmlinkage struct job_sha1* (*sha1_job_mgr_submit)(struct sha1_mb_mgr *state,
+							  struct job_sha1 *job);
+asmlinkage struct job_sha1* (*sha1_job_mgr_flush)(struct sha1_mb_mgr *state);
+asmlinkage struct job_sha1* (*sha1_job_mgr_get_comp_job)(struct sha1_mb_mgr *state);
+
+inline void sha1_init_digest(uint32_t *digest)
+{
+	static const uint32_t initial_digest[SHA1_DIGEST_LENGTH] = {SHA1_H0,
+					SHA1_H1, SHA1_H2, SHA1_H3, SHA1_H4 };
+	memcpy(digest, initial_digest, sizeof(initial_digest));
+}
+
+inline uint32_t sha1_pad(uint8_t padblock[SHA1_BLOCK_SIZE * 2],
+			 uint32_t total_len)
+{
+	uint32_t i = total_len & (SHA1_BLOCK_SIZE - 1);
+
+	memset(&padblock[i], 0, SHA1_BLOCK_SIZE);
+	padblock[i] = 0x80;
+
+	i += ((SHA1_BLOCK_SIZE - 1) &
+	      (0 - (total_len + SHA1_PADLENGTHFIELD_SIZE + 1)))
+	     + 1 + SHA1_PADLENGTHFIELD_SIZE;
+
+#if SHA1_PADLENGTHFIELD_SIZE == 16
+	*((uint64_t *) &padblock[i - 16]) = 0;
+#endif
+
+	*((uint64_t *) &padblock[i - 8]) = cpu_to_be64(total_len << 3);
+
+	/* Number of extra blocks to hash */
+	return i >> SHA1_LOG2_BLOCK_SIZE;
+}
+
+static struct sha1_hash_ctx *sha1_ctx_mgr_resubmit(struct sha1_ctx_mgr *mgr, struct sha1_hash_ctx *ctx)
+{
+	while (ctx) {
+		if (ctx->status & HASH_CTX_STS_COMPLETE) {
+			/* Clear PROCESSING bit */
+			ctx->status = HASH_CTX_STS_COMPLETE;
+			return ctx;
+		}
+
+		/*
+		 * If the extra blocks are empty, begin hashing what remains
+		 * in the user's buffer.
+		 */
+		if (ctx->partial_block_buffer_length == 0 &&
+		    ctx->incoming_buffer_length) {
+
+			const void *buffer = ctx->incoming_buffer;
+			uint32_t len = ctx->incoming_buffer_length;
+			uint32_t copy_len;
+
+			/*
+			 * Only entire blocks can be hashed.
+			 * Copy remainder to extra blocks buffer.
+			 */
+			copy_len = len & (SHA1_BLOCK_SIZE-1);
+
+			if (copy_len) {
+				len -= copy_len;
+				memcpy(ctx->partial_block_buffer,
+				       ((const char *) buffer + len),
+				       copy_len);
+				ctx->partial_block_buffer_length = copy_len;
+			}
+
+			ctx->incoming_buffer_length = 0;
+
+			/* len should be a multiple of the block size now */
+			assert((len % SHA1_BLOCK_SIZE) == 0);
+
+			/* Set len to the number of blocks to be hashed */
+			len >>= SHA1_LOG2_BLOCK_SIZE;
+
+			if (len) {
+
+				ctx->job.buffer = (uint8_t *) buffer;
+				ctx->job.len = len;
+				ctx = (struct sha1_hash_ctx *) sha1_job_mgr_submit(&mgr->mgr,
+										  &ctx->job);
+				continue;
+			}
+		}
+
+		/*
+		 * If the extra blocks are not empty, then we are
+		 * either on the last block(s) or we need more
+		 * user input before continuing.
+		 */
+		if (ctx->status & HASH_CTX_STS_LAST) {
+
+			uint8_t *buf = ctx->partial_block_buffer;
+			uint32_t n_extra_blocks = sha1_pad(buf, ctx->total_length);
+
+			ctx->status = (HASH_CTX_STS_PROCESSING |
+				       HASH_CTX_STS_COMPLETE);
+			ctx->job.buffer = buf;
+			ctx->job.len = (uint32_t) n_extra_blocks;
+			ctx = (struct sha1_hash_ctx *) sha1_job_mgr_submit(&mgr->mgr, &ctx->job);
+			continue;
+		}
+
+		if (ctx)
+			ctx->status = HASH_CTX_STS_IDLE;
+		return ctx;
+	}
+
+	return NULL;
+}
+
+struct sha1_hash_ctx *sha1_ctx_mgr_get_comp_ctx(struct sha1_ctx_mgr *mgr)
+{
+	/*
+	 * If get_comp_job returns NULL, there are no jobs complete.
+	 * If get_comp_job returns a job, verify that it is safe to return to the user.
+	 * If it is not ready, resubmit the job to finish processing.
+	 * If sha1_ctx_mgr_resubmit returned a job, it is ready to be returned.
+	 * Otherwise, all jobs currently being managed by the hash_ctx_mgr still need processing.
+	 */
+	struct sha1_hash_ctx *ctx;
+
+	ctx = (struct sha1_hash_ctx *) sha1_job_mgr_get_comp_job(&mgr->mgr);
+	return sha1_ctx_mgr_resubmit(mgr, ctx);
+}
+
+void sha1_ctx_mgr_init(struct sha1_ctx_mgr *mgr)
+{
+	sha1_job_mgr_init(&mgr->mgr);
+}
+
+struct sha1_hash_ctx *sha1_ctx_mgr_submit(struct sha1_ctx_mgr *mgr,
+					  struct sha1_hash_ctx *ctx,
+					  const void *buffer,
+					  uint32_t len,
+					  int flags)
+{
+	if (flags & (~HASH_ENTIRE)) {
+		/* User should not pass anything other than FIRST, UPDATE, or LAST */
+		ctx->error = HASH_CTX_ERROR_INVALID_FLAGS;
+		return ctx;
+	}
+
+	if (ctx->status & HASH_CTX_STS_PROCESSING) {
+		/* Cannot submit to a currently processing job. */
+		ctx->error = HASH_CTX_ERROR_ALREADY_PROCESSING;
+		return ctx;
+	}
+
+	if ((ctx->status & HASH_CTX_STS_COMPLETE) && !(flags & HASH_FIRST)) {
+		/* Cannot update a finished job. */
+		ctx->error = HASH_CTX_ERROR_ALREADY_COMPLETED;
+		return ctx;
+	}
+
+
+	if (flags & HASH_FIRST) {
+		/* Init digest */
+		sha1_init_digest(ctx->job.result_digest);
+
+		/* Reset byte counter */
+		ctx->total_length = 0;
+
+		/* Clear extra blocks */
+		ctx->partial_block_buffer_length = 0;
+	}
+
+	/* If we made it here, there were no errors during this call to submit */
+	ctx->error = HASH_CTX_ERROR_NONE;
+
+	/* Store buffer ptr info from user */
+	ctx->incoming_buffer = buffer;
+	ctx->incoming_buffer_length = len;
+
+	/* Store the user's request flags and mark this ctx as currently being processed. */
+	ctx->status = (flags & HASH_LAST) ?
+			(HASH_CTX_STS_PROCESSING | HASH_CTX_STS_LAST) :
+			HASH_CTX_STS_PROCESSING;
+
+	/* Advance byte counter */
+	ctx->total_length += len;
+
+	/*
+	 * If there is anything currently buffered in the extra blocks,
+	 * append to it until it contains a whole block.
+	 * Or if the user's buffer contains less than a whole block,
+	 * append as much as possible to the extra block.
+	 */
+	if ((ctx->partial_block_buffer_length) | (len < SHA1_BLOCK_SIZE)) {
+		/* Compute how many bytes to copy from user buffer into extra block */
+		uint32_t copy_len = SHA1_BLOCK_SIZE - ctx->partial_block_buffer_length;
+		if (len < copy_len)
+			copy_len = len;
+
+		if (copy_len) {
+			/* Copy and update relevant pointers and counters */
+			memcpy(&ctx->partial_block_buffer[ctx->partial_block_buffer_length],
+				buffer, copy_len);
+
+			ctx->partial_block_buffer_length += copy_len;
+			ctx->incoming_buffer = (const void *)((const char *)buffer + copy_len);
+			ctx->incoming_buffer_length = len - copy_len;
+		}
+
+		/* The extra block should never contain more than 1 block here */
+		assert(ctx->partial_block_buffer_length <= SHA1_BLOCK_SIZE);
+
+		/* If the extra block buffer contains exactly 1 block, it can be hashed. */
+		if (ctx->partial_block_buffer_length >= SHA1_BLOCK_SIZE) {
+			ctx->partial_block_buffer_length = 0;
+
+			ctx->job.buffer = ctx->partial_block_buffer;
+			ctx->job.len = 1;
+			ctx = (struct sha1_hash_ctx *) sha1_job_mgr_submit(&mgr->mgr, &ctx->job);
+		}
+	}
+
+	return sha1_ctx_mgr_resubmit(mgr, ctx);
+}
+
+struct sha1_hash_ctx *sha1_ctx_mgr_flush(struct sha1_ctx_mgr *mgr)
+{
+	struct sha1_hash_ctx *ctx;
+
+	while (1) {
+		ctx = (struct sha1_hash_ctx *) sha1_job_mgr_flush(&mgr->mgr);
+
+		/* If flush returned 0, there are no more jobs in flight. */
+		if (!ctx)
+			return NULL;
+
+		/*
+		 * If flush returned a job, resubmit the job to finish processing.
+		 */
+		ctx = sha1_ctx_mgr_resubmit(mgr, ctx);
+
+		/*
+		 * If sha1_ctx_mgr_resubmit returned a job, it is ready to be returned.
+		 * Otherwise, all jobs currently being managed by the sha1_ctx_mgr
+		 * still need processing. Loop.
+		 */
+		if (ctx)
+			return ctx;
+	}
+}
+
+static int sha1_mb_init(struct shash_desc *desc)
+{
+	struct sha1_hash_ctx *sctx = shash_desc_ctx(desc);
+
+	hash_ctx_init(sctx);
+	sctx->job.result_digest[0] = SHA1_H0;
+	sctx->job.result_digest[1] = SHA1_H1;
+	sctx->job.result_digest[2] = SHA1_H2;
+	sctx->job.result_digest[3] = SHA1_H3;
+	sctx->job.result_digest[4] = SHA1_H4;
+	sctx->total_length = 0;
+	sctx->partial_block_buffer_length = 0;
+	sctx->status = HASH_CTX_STS_IDLE;
+
+	return 0;
+}
+
+static int sha1_mb_set_results(struct mcryptd_hash_request_ctx *rctx)
+{
+	int	i;
+	struct	sha1_hash_ctx *sctx = shash_desc_ctx(&rctx->desc);
+	__be32	*dst = (__be32 *) rctx->out;
+
+	for (i = 0; i < 5; ++i)
+		dst[i] = cpu_to_be32(sctx->job.result_digest[i]);
+
+	return 0;
+}
+
+static int sha_finish_walk(struct mcryptd_hash_request_ctx **ret_rctx,
+			struct mcryptd_alg_cstate *cstate, bool flush)
+{
+	int	flag = HASH_UPDATE;
+	int	nbytes, err = 0;
+	struct mcryptd_hash_request_ctx *rctx = *ret_rctx;
+	struct sha1_hash_ctx *sha_ctx;
+
+	/* more work ? */
+	while (!(rctx->flag & HASH_DONE)) {
+		nbytes = crypto_ahash_walk_done(&rctx->walk, 0);
+		if (nbytes < 0) {
+			err = nbytes;
+			goto out;
+		}
+		/* check if the walk is done */
+		if (crypto_ahash_walk_last(&rctx->walk)) {
+			rctx->flag |= HASH_DONE;
+			if (rctx->flag & HASH_FINAL)
+				flag |= HASH_LAST;
+
+		}
+		sha_ctx = (struct sha1_hash_ctx *) shash_desc_ctx(&rctx->desc);
+		kernel_fpu_begin();
+		sha_ctx = sha1_ctx_mgr_submit(cstate->mgr, sha_ctx, rctx->walk.data, nbytes, flag);
+		if (!sha_ctx) {
+			if (flush)
+				sha_ctx = sha1_ctx_mgr_flush(cstate->mgr);
+		}
+		kernel_fpu_end();
+		if (sha_ctx)
+			rctx = cast_hash_to_mcryptd_ctx(sha_ctx);
+		else {
+			rctx = NULL;
+			goto out;
+		}
+	}
+
+	/* copy the results */
+	if (rctx->flag & HASH_FINAL)
+		sha1_mb_set_results(rctx);
+
+out:
+	*ret_rctx = rctx;
+	return err;
+}
+
+static int sha_complete_job(struct mcryptd_hash_request_ctx *rctx,
+			    struct mcryptd_alg_cstate *cstate,
+			    int err)
+{
+	struct ahash_request *req = cast_mcryptd_ctx_to_req(rctx);
+	struct sha1_hash_ctx *sha_ctx;
+	struct mcryptd_hash_request_ctx *req_ctx;
+	int ret;
+
+	/* remove from work list */
+	spin_lock(&cstate->work_lock);
+	list_del(&rctx->waiter);
+	spin_unlock(&cstate->work_lock);
+
+	if (irqs_disabled())
+		rctx->complete(&req->base, err);
+	else {
+		local_bh_disable();
+		rctx->complete(&req->base, err);
+		local_bh_enable();
+	}
+
+	/* check to see if there are other jobs that are done */
+	sha_ctx = sha1_ctx_mgr_get_comp_ctx(cstate->mgr);
+	while (sha_ctx) {
+		req_ctx = cast_hash_to_mcryptd_ctx(sha_ctx);
+		ret = sha_finish_walk(&req_ctx, cstate, false);
+		if (req_ctx) {
+			spin_lock(&cstate->work_lock);
+			list_del(&req_ctx->waiter);
+			spin_unlock(&cstate->work_lock);
+
+			req = cast_mcryptd_ctx_to_req(req_ctx);
+			if (irqs_disabled())
+				rctx->complete(&req->base, ret);
+			else {
+				local_bh_disable();
+				rctx->complete(&req->base, ret);
+				local_bh_enable();
+			}
+		}
+		sha_ctx = sha1_ctx_mgr_get_comp_ctx(cstate->mgr);
+	}
+
+	return 0;
+}
+
+static void sha1_mb_add_list(struct mcryptd_hash_request_ctx *rctx,
+			     struct mcryptd_alg_cstate *cstate)
+{
+	unsigned long next_flush;
+	unsigned long delay = usecs_to_jiffies(FLUSH_INTERVAL);
+
+	/* initialize tag */
+	rctx->tag.arrival = jiffies;    /* tag the arrival time */
+	rctx->tag.seq_num = cstate->next_seq_num++;
+	next_flush = rctx->tag.arrival + delay;
+	rctx->tag.expire = next_flush;
+
+	spin_lock(&cstate->work_lock);
+	list_add_tail(&rctx->waiter, &cstate->work_list);
+	spin_unlock(&cstate->work_lock);
+
+	mcryptd_arm_flusher(cstate, delay);
+}
+
+static int sha1_mb_update(struct shash_desc *desc, const u8 *data,
+			  unsigned int len)
+{
+	struct mcryptd_hash_request_ctx *rctx =
+			container_of(desc, struct mcryptd_hash_request_ctx, desc);
+	struct mcryptd_alg_cstate *cstate =
+				this_cpu_ptr(sha1_mb_alg_state.alg_cstate);
+
+	struct ahash_request *req = cast_mcryptd_ctx_to_req(rctx);
+	struct sha1_hash_ctx *sha_ctx;
+	int ret = 0, nbytes;
+
+
+	/* sanity check */
+	if (rctx->tag.cpu != smp_processor_id()) {
+		pr_err("mcryptd error: cpu clash\n");
+		goto done;
+	}
+
+	/* need to init context */
+	req_ctx_init(rctx, desc);
+
+	nbytes = crypto_ahash_walk_first(req, &rctx->walk);
+
+	if (nbytes < 0) {
+		ret = nbytes;
+		goto done;
+	}
+
+	if (crypto_ahash_walk_last(&rctx->walk))
+		rctx->flag |= HASH_DONE;
+
+	/* submit */
+	sha_ctx = (struct sha1_hash_ctx *) shash_desc_ctx(desc);
+	sha1_mb_add_list(rctx, cstate);
+	kernel_fpu_begin();
+	sha_ctx = sha1_ctx_mgr_submit(cstate->mgr, sha_ctx, rctx->walk.data, nbytes, HASH_UPDATE);
+	kernel_fpu_end();
+
+	/* check if anything is returned */
+	if (!sha_ctx)
+		return -EINPROGRESS;
+
+	if (sha_ctx->error) {
+		ret = sha_ctx->error;
+		rctx = cast_hash_to_mcryptd_ctx(sha_ctx);
+		goto done;
+	}
+
+	rctx = cast_hash_to_mcryptd_ctx(sha_ctx);
+	ret = sha_finish_walk(&rctx, cstate, false);
+
+	if (!rctx)
+		return -EINPROGRESS;
+done:
+	sha_complete_job(rctx, cstate, ret);
+	return ret;
+}
+
+static int sha1_mb_finup(struct shash_desc *desc, const u8 *data,
+			     unsigned int len, u8 *out)
+{
+	struct mcryptd_hash_request_ctx *rctx =
+			container_of(desc, struct mcryptd_hash_request_ctx, desc);
+	struct mcryptd_alg_cstate *cstate =
+				this_cpu_ptr(sha1_mb_alg_state.alg_cstate);
+
+	struct ahash_request *req = cast_mcryptd_ctx_to_req(rctx);
+	struct sha1_hash_ctx *sha_ctx;
+	int ret = 0, flag = HASH_UPDATE, nbytes;
+
+	/* sanity check */
+	if (rctx->tag.cpu != smp_processor_id()) {
+		pr_err("mcryptd error: cpu clash\n");
+		goto done;
+	}
+
+	/* need to init context */
+	req_ctx_init(rctx, desc);
+
+	nbytes = crypto_ahash_walk_first(req, &rctx->walk);
+
+	if (nbytes < 0) {
+		ret = nbytes;
+		goto done;
+	}
+
+	if (crypto_ahash_walk_last(&rctx->walk)) {
+		rctx->flag |= HASH_DONE;
+		flag = HASH_LAST;
+	}
+	rctx->out = out;
+
+	/* submit */
+	rctx->flag |= HASH_FINAL;
+	sha_ctx = (struct sha1_hash_ctx *) shash_desc_ctx(desc);
+	sha1_mb_add_list(rctx, cstate);
+
+	kernel_fpu_begin();
+	sha_ctx = sha1_ctx_mgr_submit(cstate->mgr, sha_ctx, rctx->walk.data, nbytes, flag);
+	kernel_fpu_end();
+
+	/* check if anything is returned */
+	if (!sha_ctx)
+		return -EINPROGRESS;
+
+	if (sha_ctx->error) {
+		ret = sha_ctx->error;
+		goto done;
+	}
+
+	rctx = cast_hash_to_mcryptd_ctx(sha_ctx);
+	ret = sha_finish_walk(&rctx, cstate, false);
+	if (!rctx)
+		return -EINPROGRESS;
+done:
+	sha_complete_job(rctx, cstate, ret);
+	return ret;
+}
+
+static int sha1_mb_final(struct shash_desc *desc, u8 *out)
+{
+	struct mcryptd_hash_request_ctx *rctx =
+			container_of(desc, struct mcryptd_hash_request_ctx, desc);
+	struct mcryptd_alg_cstate *cstate =
+				this_cpu_ptr(sha1_mb_alg_state.alg_cstate);
+
+	struct sha1_hash_ctx *sha_ctx;
+	int ret = 0;
+	u8 data;
+
+	/* sanity check */
+	if (rctx->tag.cpu != smp_processor_id()) {
+		pr_err("mcryptd error: cpu clash\n");
+		goto done;
+	}
+
+	/* need to init context */
+	req_ctx_init(rctx, desc);
+
+	rctx->out = out;
+	rctx->flag |= HASH_DONE | HASH_FINAL;
+
+	sha_ctx = (struct sha1_hash_ctx *) shash_desc_ctx(desc);
+	/* flag HASH_FINAL and 0 data size */
+	sha1_mb_add_list(rctx, cstate);
+	kernel_fpu_begin();
+	sha_ctx = sha1_ctx_mgr_submit(cstate->mgr, sha_ctx, &data, 0, HASH_LAST);
+	kernel_fpu_end();
+
+	/* check if anything is returned */
+	if (!sha_ctx)
+		return -EINPROGRESS;
+
+	if (sha_ctx->error) {
+		ret = sha_ctx->error;
+		rctx = cast_hash_to_mcryptd_ctx(sha_ctx);
+		goto done;
+	}
+
+	rctx = cast_hash_to_mcryptd_ctx(sha_ctx);
+	ret = sha_finish_walk(&rctx, cstate, false);
+	if (!rctx)
+		return -EINPROGRESS;
+done:
+	sha_complete_job(rctx, cstate, ret);
+	return ret;
+}
+
+static int sha1_mb_export(struct shash_desc *desc, void *out)
+{
+	struct sha1_hash_ctx *sctx = shash_desc_ctx(desc);
+
+	memcpy(out, sctx, sizeof(*sctx));
+
+	return 0;
+}
+
+static int sha1_mb_import(struct shash_desc *desc, const void *in)
+{
+	struct sha1_hash_ctx *sctx = shash_desc_ctx(desc);
+
+	memcpy(sctx, in, sizeof(*sctx));
+
+	return 0;
+}
+
+
+static struct shash_alg sha1_mb_shash_alg = {
+	.digestsize	=	SHA1_DIGEST_SIZE,
+	.init		=	sha1_mb_init,
+	.update		=	sha1_mb_update,
+	.final		=	sha1_mb_final,
+	.finup		=	sha1_mb_finup,
+	.export		=	sha1_mb_export,
+	.import		=	sha1_mb_import,
+	.descsize	=	sizeof(struct sha1_hash_ctx),
+	.statesize	=	sizeof(struct sha1_hash_ctx),
+	.base		=	{
+		.cra_name	 = "__sha1-mb",
+		.cra_driver_name = "__intel_sha1-mb",
+		.cra_priority	 = 100,
+		/*
+		 * use ASYNC flag as some buffers in multi-buffer
+		 * algo may not have completed before hashing thread sleep
+		 */
+		.cra_flags	 = CRYPTO_ALG_TYPE_SHASH | CRYPTO_ALG_ASYNC,
+		.cra_blocksize	 = SHA1_BLOCK_SIZE,
+		.cra_module	 = THIS_MODULE,
+		.cra_list	 = LIST_HEAD_INIT(sha1_mb_shash_alg.base.cra_list),
+	}
+};
+
+static int sha1_mb_async_init(struct ahash_request *req)
+{
+	struct crypto_ahash *tfm = crypto_ahash_reqtfm(req);
+	struct sha1_mb_ctx *ctx = crypto_ahash_ctx(tfm);
+	struct ahash_request *mcryptd_req = ahash_request_ctx(req);
+	struct mcryptd_ahash *mcryptd_tfm = ctx->mcryptd_tfm;
+
+	memcpy(mcryptd_req, req, sizeof(*req));
+	ahash_request_set_tfm(mcryptd_req, &mcryptd_tfm->base);
+	return crypto_ahash_init(mcryptd_req);
+}
+
+static int sha1_mb_async_update(struct ahash_request *req)
+{
+	struct ahash_request *mcryptd_req = ahash_request_ctx(req);
+
+	struct crypto_ahash *tfm = crypto_ahash_reqtfm(req);
+	struct sha1_mb_ctx *ctx = crypto_ahash_ctx(tfm);
+	struct mcryptd_ahash *mcryptd_tfm = ctx->mcryptd_tfm;
+
+	memcpy(mcryptd_req, req, sizeof(*req));
+	ahash_request_set_tfm(mcryptd_req, &mcryptd_tfm->base);
+	return crypto_ahash_update(mcryptd_req);
+}
+
+static int sha1_mb_async_finup(struct ahash_request *req)
+{
+	struct ahash_request *mcryptd_req = ahash_request_ctx(req);
+
+	struct crypto_ahash *tfm = crypto_ahash_reqtfm(req);
+	struct sha1_mb_ctx *ctx = crypto_ahash_ctx(tfm);
+	struct mcryptd_ahash *mcryptd_tfm = ctx->mcryptd_tfm;
+
+	memcpy(mcryptd_req, req, sizeof(*req));
+	ahash_request_set_tfm(mcryptd_req, &mcryptd_tfm->base);
+	return crypto_ahash_finup(mcryptd_req);
+}
+
+static int sha1_mb_async_final(struct ahash_request *req)
+{
+	struct ahash_request *mcryptd_req = ahash_request_ctx(req);
+
+	struct crypto_ahash *tfm = crypto_ahash_reqtfm(req);
+	struct sha1_mb_ctx *ctx = crypto_ahash_ctx(tfm);
+	struct mcryptd_ahash *mcryptd_tfm = ctx->mcryptd_tfm;
+
+	memcpy(mcryptd_req, req, sizeof(*req));
+	ahash_request_set_tfm(mcryptd_req, &mcryptd_tfm->base);
+	return crypto_ahash_final(mcryptd_req);
+}
+
+int sha1_mb_async_digest(struct ahash_request *req)
+{
+	struct crypto_ahash *tfm = crypto_ahash_reqtfm(req);
+	struct sha1_mb_ctx *ctx = crypto_ahash_ctx(tfm);
+	struct ahash_request *mcryptd_req = ahash_request_ctx(req);
+	struct mcryptd_ahash *mcryptd_tfm = ctx->mcryptd_tfm;
+
+	memcpy(mcryptd_req, req, sizeof(*req));
+	ahash_request_set_tfm(mcryptd_req, &mcryptd_tfm->base);
+	return crypto_ahash_digest(mcryptd_req);
+}
+
+static int sha1_mb_async_init_tfm(struct crypto_tfm *tfm)
+{
+	struct mcryptd_ahash *mcryptd_tfm;
+	struct sha1_mb_ctx *ctx = crypto_tfm_ctx(tfm);
+	struct mcryptd_hash_ctx *mctx;
+
+	mcryptd_tfm = mcryptd_alloc_ahash("__intel_sha1-mb", 0, 0);
+	if (IS_ERR(mcryptd_tfm))
+		return PTR_ERR(mcryptd_tfm);
+	mctx = crypto_ahash_ctx(&mcryptd_tfm->base);
+	mctx->alg_state = &sha1_mb_alg_state;
+	ctx->mcryptd_tfm = mcryptd_tfm;
+	crypto_ahash_set_reqsize(__crypto_ahash_cast(tfm),
+				 sizeof(struct ahash_request) +
+				 crypto_ahash_reqsize(&mcryptd_tfm->base));
+
+	return 0;
+}
+
+static void sha1_mb_async_exit_tfm(struct crypto_tfm *tfm)
+{
+	struct sha1_mb_ctx *ctx = crypto_tfm_ctx(tfm);
+
+	mcryptd_free_ahash(ctx->mcryptd_tfm);
+}
+
+static struct ahash_alg sha1_mb_async_alg = {
+	.init           = sha1_mb_async_init,
+	.update         = sha1_mb_async_update,
+	.final          = sha1_mb_async_final,
+	.finup          = sha1_mb_async_finup,
+	.digest         = sha1_mb_async_digest,
+	.halg = {
+		.digestsize     = SHA1_DIGEST_SIZE,
+		.base = {
+			.cra_name               = "sha1",
+			.cra_driver_name        = "sha1_mb",
+			.cra_priority           = 200,
+			.cra_flags              = CRYPTO_ALG_TYPE_AHASH | CRYPTO_ALG_ASYNC,
+			.cra_blocksize          = SHA1_BLOCK_SIZE,
+			.cra_type               = &crypto_ahash_type,
+			.cra_module             = THIS_MODULE,
+			.cra_list               = LIST_HEAD_INIT(sha1_mb_async_alg.halg.base.cra_list),
+			.cra_init               = sha1_mb_async_init_tfm,
+			.cra_exit               = sha1_mb_async_exit_tfm,
+			.cra_ctxsize		= sizeof(struct sha1_mb_ctx),
+			.cra_alignmask		= 0,
+		},
+	},
+};
+
+unsigned long sha1_mb_flusher(struct mcryptd_alg_cstate *cstate)
+{
+	struct mcryptd_hash_request_ctx *rctx;
+	unsigned long cur_time;
+	unsigned long next_flush = 0;
+	struct sha1_hash_ctx *sha_ctx;
+
+
+	cur_time = jiffies;
+
+	while (!list_empty(&cstate->work_list)) {
+		rctx = list_entry(cstate->work_list.next,
+				struct mcryptd_hash_request_ctx, waiter);
+		if time_before(cur_time, rctx->tag.expire)
+			break;
+		kernel_fpu_begin();
+		sha_ctx = (struct sha1_hash_ctx *) sha1_ctx_mgr_flush(cstate->mgr);
+		kernel_fpu_end();
+		if (!sha_ctx) {
+			pr_err("sha1_mb error: nothing got flushed for non-empty list\n");
+			break;
+		}
+		rctx = cast_hash_to_mcryptd_ctx(sha_ctx);
+		sha_finish_walk(&rctx, cstate, true);
+		sha_complete_job(rctx, cstate, 0);
+	}
+
+	if (!list_empty(&cstate->work_list)) {
+		rctx = list_entry(cstate->work_list.next,
+				struct mcryptd_hash_request_ctx, waiter);
+		/* get the hash context and then flush time */
+		next_flush = rctx->tag.expire;
+		mcryptd_arm_flusher(cstate, get_delay(next_flush));
+	}
+	return next_flush;
+}
+
+static int __init sha1_mb_mod_init(void)
+{
+
+	int cpu;
+	int err;
+	struct mcryptd_alg_cstate *cpu_state;
+
+	/* check for dependent cpu features */
+	if (!boot_cpu_has(X86_FEATURE_AVX2) ||
+	    !boot_cpu_has(X86_FEATURE_BMI2))
+		return -ENODEV;
+
+	/* initialize multibuffer structures */
+	sha1_mb_alg_state.alg_cstate = alloc_percpu(struct mcryptd_alg_cstate);
+
+	sha1_job_mgr_init = sha1_mb_mgr_init_avx2;
+	sha1_job_mgr_submit = sha1_mb_mgr_submit_avx2;
+	sha1_job_mgr_flush = sha1_mb_mgr_flush_avx2;
+	sha1_job_mgr_get_comp_job = sha1_mb_mgr_get_comp_job_avx2;
+
+	if (!sha1_mb_alg_state.alg_cstate)
+		return -ENOMEM;
+	for_each_possible_cpu(cpu) {
+		cpu_state = per_cpu_ptr(sha1_mb_alg_state.alg_cstate, cpu);
+		cpu_state->next_flush = 0;
+		cpu_state->next_seq_num = 0;
+		cpu_state->flusher_engaged = false;
+		INIT_DELAYED_WORK(&cpu_state->flush, mcryptd_flusher);
+		cpu_state->cpu = cpu;
+		cpu_state->alg_state = &sha1_mb_alg_state;
+		cpu_state->mgr = (struct sha1_ctx_mgr *) kzalloc(sizeof(struct sha1_ctx_mgr), GFP_KERNEL);
+		if (!cpu_state->mgr)
+			goto err2;
+		sha1_ctx_mgr_init(cpu_state->mgr);
+		INIT_LIST_HEAD(&cpu_state->work_list);
+		spin_lock_init(&cpu_state->work_lock);
+	}
+	sha1_mb_alg_state.flusher = &sha1_mb_flusher;
+
+	err = crypto_register_shash(&sha1_mb_shash_alg);
+	if (err)
+		goto err2;
+	err = crypto_register_ahash(&sha1_mb_async_alg);
+	if (err)
+		goto err1;
+
+
+	return 0;
+err1:
+	crypto_unregister_shash(&sha1_mb_shash_alg);
+err2:
+	for_each_possible_cpu(cpu) {
+		cpu_state = per_cpu_ptr(sha1_mb_alg_state.alg_cstate, cpu);
+		kfree(cpu_state->mgr);
+	}
+	free_percpu(sha1_mb_alg_state.alg_cstate);
+	return -ENODEV;
+}
+
+static void __exit sha1_mb_mod_fini(void)
+{
+	int cpu;
+	struct mcryptd_alg_cstate *cpu_state;
+
+	crypto_unregister_ahash(&sha1_mb_async_alg);
+	crypto_unregister_shash(&sha1_mb_shash_alg);
+	for_each_possible_cpu(cpu) {
+		cpu_state = per_cpu_ptr(sha1_mb_alg_state.alg_cstate, cpu);
+		kfree(cpu_state->mgr);
+	}
+	free_percpu(sha1_mb_alg_state.alg_cstate);
+}
+
+module_init(sha1_mb_mod_init);
+module_exit(sha1_mb_mod_fini);
+
+MODULE_LICENSE("GPL");
+MODULE_DESCRIPTION("SHA1 Secure Hash Algorithm, multi buffer accelerated");
+
+MODULE_ALIAS("sha1");
-- 
1.7.11.7

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

* Re: [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure
  2014-07-22 22:09 ` [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure Tim Chen
@ 2014-07-25  7:08   ` Peter Zijlstra
  2014-07-25 16:25     ` Tim Chen
  2014-07-25  7:26   ` Peter Zijlstra
  1 sibling, 1 reply; 15+ messages in thread
From: Peter Zijlstra @ 2014-07-25  7:08 UTC (permalink / raw)
  To: Tim Chen
  Cc: Herbert Xu, H. Peter Anvin, David S.Miller, Ingo Molnar,
	Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Jussi Kivilinna, Thomas Gleixner, Tadeusz Struk,
	tkhai, linux-crypto, linux-kernel

On Tue, Jul 22, 2014 at 03:09:32PM -0700, Tim Chen wrote:
> +/* Called in workqueue context, do one real cryption work (via
> + * req->complete) and reschedule itself if there are more work to
> + * do. */

You seem to manage the 'normal' comment style in other places, this one
'special' for a reason?

> +static void mcryptd_queue_worker(struct work_struct *work)
> +{
> +	struct mcryptd_cpu_queue *cpu_queue;
> +	struct crypto_async_request *req, *backlog;
> +	int i;
> +
> +	/*
> +	 * Need to loop through more than once for multi-buffer to
> +	 * be effective.
> +	 */
> +
> +	cpu_queue = container_of(work, struct mcryptd_cpu_queue, work);
> +	i = 0;
> +	while (i < MCRYPTD_BATCH || single_task_running()) {
> +		/*
> +		 * preempt_disable/enable is used to prevent
> +		 * being preempted by mcryptd_enqueue_request()
> +		 */
> +		local_bh_disable();
> +		preempt_disable();
> +		backlog = crypto_get_backlog(&cpu_queue->queue);
> +		req = crypto_dequeue_request(&cpu_queue->queue);
> +		preempt_enable();
> +		local_bh_enable();
> +
> +		if (!req)
> +			return;
> +
> +		if (backlog)
> +			backlog->complete(backlog, -EINPROGRESS);
> +		req->complete(req, 0);
> +		if (!cpu_queue->queue.qlen)
> +			return;
> +		++i;
> +	}
> +	if (cpu_queue->queue.qlen)
> +		queue_work(kcrypto_wq, &cpu_queue->work);
> +}

Right, so I don't think you need that single_task_running() thing for
this. Also its a rather inconclusive test, a task can come in while
processing the request, preempt the processing, complete and by the time
we're back to check if we want to continue the loop its only us again.

So its actually misleading..

You could do that, but then you have to keep preemption disabled across
the entire thing, and break out on need_resched(), that would entirely
obviate the need for single_task_running(), but would increase the
preemption latency by however long the req processing takes, so that's
bad too.

But you can get similar 'fuzzy' semantics as above by using something
like:

static inline bool did_context_switch(unsigned long *nivcs)
{
	if (*nivcs != current->nivcs) {
		*nivcs = current->nivcs;
		return true;
	}
	return false;
}

static void mcryptd_queue_worker()
{
	...
	unsigned long nivcs = current->nivcs;

	...

	while (!did_context_switch(&nivcs) || i < MCRYPTD_BATCH)

	...
}

It'll terminate earlier I suppose, its the inverse test. But I think you
want to terminate early if there's any activity.

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

* Re: [PATCH v5 3/7] crypto: SHA1 multibuffer crypto opportunistic flush
  2014-07-22 22:09 ` [PATCH v5 3/7] crypto: SHA1 multibuffer crypto opportunistic flush Tim Chen
@ 2014-07-25  7:22   ` Peter Zijlstra
  0 siblings, 0 replies; 15+ messages in thread
From: Peter Zijlstra @ 2014-07-25  7:22 UTC (permalink / raw)
  To: Tim Chen
  Cc: Herbert Xu, H. Peter Anvin, David S.Miller, Ingo Molnar,
	Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Jussi Kivilinna, Thomas Gleixner, Tadeusz Struk,
	tkhai, linux-crypto, linux-kernel

On Tue, Jul 22, 2014 at 03:09:35PM -0700, Tim Chen wrote:
> The crypto daemon can take advantage of available cpu
> cycles to flush any unfinished jobs if it is the
> only task running on the cpu, and there are no more crypto
> jobs to process.

You conveniently forgot to mention energy efficiency, and the problem
with short idle durations. Although the latter issue seems overshadowed
by the not so very robust idle detection.

In any case, same as the other patch, you can get similarly inconclusive
semantics without single_task_running().

> Signed-off-by: Tim Chen <tim.c.chen@linux.intel.com>
> ---
>  crypto/mcryptd.c | 39 ++++++++++++++++++++++++++++++++++++---
>  1 file changed, 36 insertions(+), 3 deletions(-)

> +/*
> + * Called in workqueue context, do one real cryption work (via
>   * req->complete) and reschedule itself if there are more work to
> - * do. */
> + * do.
> + */

See, this should've been in the previous patch :-)

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

* Re: [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure
  2014-07-22 22:09 ` [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure Tim Chen
  2014-07-25  7:08   ` Peter Zijlstra
@ 2014-07-25  7:26   ` Peter Zijlstra
  2014-07-25 16:47     ` Tim Chen
  1 sibling, 1 reply; 15+ messages in thread
From: Peter Zijlstra @ 2014-07-25  7:26 UTC (permalink / raw)
  To: Tim Chen
  Cc: Herbert Xu, H. Peter Anvin, David S.Miller, Ingo Molnar,
	Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Jussi Kivilinna, Thomas Gleixner, Tadeusz Struk,
	tkhai, linux-crypto, linux-kernel

On Tue, Jul 22, 2014 at 03:09:32PM -0700, Tim Chen wrote:
> This patch introduces the multi-buffer crypto daemon which is responsible
> for submitting crypto jobs in a work queue to the responsible multi-buffer
> crypto algorithm.  The idea of the multi-buffer algorihtm is to put
> data streams from multiple jobs in a wide (AVX2) register and then
> take advantage of SIMD instructions to do crypto computation on several
> buffers simultaneously.
> 
> The multi-buffer crypto daemon is also responsbile for flushing the
> remaining buffers to complete the computation if no new buffers arrive
> for a while.

How is any of this SHA1 specific as the Subject says it is?

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

* Re: [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure
  2014-07-25  7:08   ` Peter Zijlstra
@ 2014-07-25 16:25     ` Tim Chen
  2014-07-29 22:28       ` Tim Chen
  0 siblings, 1 reply; 15+ messages in thread
From: Tim Chen @ 2014-07-25 16:25 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Herbert Xu, H. Peter Anvin, David S.Miller, Ingo Molnar,
	Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Jussi Kivilinna, Thomas Gleixner, Tadeusz Struk,
	tkhai, linux-crypto, linux-kernel

On Fri, 2014-07-25 at 09:08 +0200, Peter Zijlstra wrote:
> On Tue, Jul 22, 2014 at 03:09:32PM -0700, Tim Chen wrote:
> > +/* Called in workqueue context, do one real cryption work (via
> > + * req->complete) and reschedule itself if there are more work to
> > + * do. */
> 
> You seem to manage the 'normal' comment style in other places, this one
> 'special' for a reason?

Yes, I'll need to correct it.

> 
> > +static void mcryptd_queue_worker(struct work_struct *work)
> > +{
> > +	struct mcryptd_cpu_queue *cpu_queue;
> > +	struct crypto_async_request *req, *backlog;
> > +	int i;
> > +
> > +	/*
> > +	 * Need to loop through more than once for multi-buffer to
> > +	 * be effective.
> > +	 */
> > +
> > +	cpu_queue = container_of(work, struct mcryptd_cpu_queue, work);
> > +	i = 0;
> > +	while (i < MCRYPTD_BATCH || single_task_running()) {
> > +		/*
> > +		 * preempt_disable/enable is used to prevent
> > +		 * being preempted by mcryptd_enqueue_request()
> > +		 */
> > +		local_bh_disable();
> > +		preempt_disable();
> > +		backlog = crypto_get_backlog(&cpu_queue->queue);
> > +		req = crypto_dequeue_request(&cpu_queue->queue);
> > +		preempt_enable();
> > +		local_bh_enable();
> > +
> > +		if (!req)
> > +			return;
> > +
> > +		if (backlog)
> > +			backlog->complete(backlog, -EINPROGRESS);
> > +		req->complete(req, 0);
> > +		if (!cpu_queue->queue.qlen)
> > +			return;
> > +		++i;
> > +	}
> > +	if (cpu_queue->queue.qlen)
> > +		queue_work(kcrypto_wq, &cpu_queue->work);
> > +}
> 
> Right, so I don't think you need that single_task_running() thing for
> this. Also its a rather inconclusive test, a task can come in while
> processing the request, preempt the processing, complete and by the time
> we're back to check if we want to continue the loop its only us again.
> 
> So its actually misleading..
> 
> You could do that, but then you have to keep preemption disabled across
> the entire thing, and break out on need_resched(), that would entirely
> obviate the need for single_task_running(), but would increase the
> preemption latency by however long the req processing takes, so that's
> bad too.
> 
> But you can get similar 'fuzzy' semantics as above by using something
> like:
> 
> static inline bool did_context_switch(unsigned long *nivcs)
> {
> 	if (*nivcs != current->nivcs) {
> 		*nivcs = current->nivcs;
> 		return true;
> 	}
> 	return false;
> }
> 
> static void mcryptd_queue_worker()
> {
> 	...
> 	unsigned long nivcs = current->nivcs;
> 
> 	...
> 
> 	while (!did_context_switch(&nivcs) || i < MCRYPTD_BATCH)

Won't I need something like 

while ((!did_context_switch(&nivcs) && single_task_running()) 
	|| i < MCRYPTD_BATCH)

in case I got some new task in my run queue but I did not get
pre-empted?  I should not continue to run in that case.

> 
> 	...
> }
> 
> It'll terminate earlier I suppose, its the inverse test. But I think you
> want to terminate early if there's any activity.

I have some doubts here.
If I'm still the only task running, why not continue, even if I've been
pre-empted?  Since there's nothing else to run, why not take
advantage of the cpu?

Thanks.

Tim

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

* Re: [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure
  2014-07-25  7:26   ` Peter Zijlstra
@ 2014-07-25 16:47     ` Tim Chen
  0 siblings, 0 replies; 15+ messages in thread
From: Tim Chen @ 2014-07-25 16:47 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Herbert Xu, H. Peter Anvin, David S.Miller, Ingo Molnar,
	Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Jussi Kivilinna, Thomas Gleixner, Tadeusz Struk,
	tkhai, linux-crypto, linux-kernel

On Fri, 2014-07-25 at 09:26 +0200, Peter Zijlstra wrote:
> On Tue, Jul 22, 2014 at 03:09:32PM -0700, Tim Chen wrote:
> > This patch introduces the multi-buffer crypto daemon which is responsible
> > for submitting crypto jobs in a work queue to the responsible multi-buffer
> > crypto algorithm.  The idea of the multi-buffer algorihtm is to put
> > data streams from multiple jobs in a wide (AVX2) register and then
> > take advantage of SIMD instructions to do crypto computation on several
> > buffers simultaneously.
> > 
> > The multi-buffer crypto daemon is also responsbile for flushing the
> > remaining buffers to complete the computation if no new buffers arrive
> > for a while.
> 
> How is any of this SHA1 specific as the Subject says it is?

Yes, I should simply say multibuffer crypto hash infrastructure in the
subject line.

Tim

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

* Re: [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure
  2014-07-25 16:25     ` Tim Chen
@ 2014-07-29 22:28       ` Tim Chen
  2014-07-30 10:01         ` Peter Zijlstra
  0 siblings, 1 reply; 15+ messages in thread
From: Tim Chen @ 2014-07-29 22:28 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Herbert Xu, H. Peter Anvin, David S.Miller, Ingo Molnar,
	Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Jussi Kivilinna, Thomas Gleixner, Tadeusz Struk,
	tkhai, linux-crypto, linux-kernel

On Fri, 2014-07-25 at 09:25 -0700, Tim Chen wrote:
> On Fri, 2014-07-25 at 09:08 +0200, Peter Zijlstra wrote:
> > On Tue, Jul 22, 2014 at 03:09:32PM -0700, Tim Chen wrote:
> > > +/* Called in workqueue context, do one real cryption work (via
> > > + * req->complete) and reschedule itself if there are more work to
> > > + * do. */
> > 
> > You seem to manage the 'normal' comment style in other places, this one
> > 'special' for a reason?
> 
> Yes, I'll need to correct it.
> 
> > 
> > > +static void mcryptd_queue_worker(struct work_struct *work)
> > > +{
> > > +	struct mcryptd_cpu_queue *cpu_queue;
> > > +	struct crypto_async_request *req, *backlog;
> > > +	int i;
> > > +
> > > +	/*
> > > +	 * Need to loop through more than once for multi-buffer to
> > > +	 * be effective.
> > > +	 */
> > > +
> > > +	cpu_queue = container_of(work, struct mcryptd_cpu_queue, work);
> > > +	i = 0;
> > > +	while (i < MCRYPTD_BATCH || single_task_running()) {
> > > +		/*
> > > +		 * preempt_disable/enable is used to prevent
> > > +		 * being preempted by mcryptd_enqueue_request()
> > > +		 */
> > > +		local_bh_disable();
> > > +		preempt_disable();
> > > +		backlog = crypto_get_backlog(&cpu_queue->queue);
> > > +		req = crypto_dequeue_request(&cpu_queue->queue);
> > > +		preempt_enable();
> > > +		local_bh_enable();
> > > +
> > > +		if (!req)
> > > +			return;
> > > +
> > > +		if (backlog)
> > > +			backlog->complete(backlog, -EINPROGRESS);
> > > +		req->complete(req, 0);
> > > +		if (!cpu_queue->queue.qlen)
> > > +			return;
> > > +		++i;
> > > +	}
> > > +	if (cpu_queue->queue.qlen)
> > > +		queue_work(kcrypto_wq, &cpu_queue->work);
> > > +}
> > 
> > Right, so I don't think you need that single_task_running() thing for
> > this. Also its a rather inconclusive test, a task can come in while
> > processing the request, preempt the processing, complete and by the time
> > we're back to check if we want to continue the loop its only us again.
> > 
> > So its actually misleading..
> > 
> > You could do that, but then you have to keep preemption disabled across
> > the entire thing, and break out on need_resched(), that would entirely
> > obviate the need for single_task_running(), but would increase the
> > preemption latency by however long the req processing takes, so that's
> > bad too.
> > 
> > But you can get similar 'fuzzy' semantics as above by using something
> > like:
> > 
> > static inline bool did_context_switch(unsigned long *nivcs)
> > {
> > 	if (*nivcs != current->nivcs) {
> > 		*nivcs = current->nivcs;
> > 		return true;
> > 	}
> > 	return false;
> > }
> > 
> > static void mcryptd_queue_worker()
> > {
> > 	...
> > 	unsigned long nivcs = current->nivcs;
> > 
> > 	...
> > 
> > 	while (!did_context_switch(&nivcs) || i < MCRYPTD_BATCH)
> 
> Won't I need something like 
> 
> while ((!did_context_switch(&nivcs) && single_task_running()) 
> 	|| i < MCRYPTD_BATCH)
> 
> in case I got some new task in my run queue but I did not get
> pre-empted?  I should not continue to run in that case.
> 
> > 
> > 	...
> > }
> > 
> > It'll terminate earlier I suppose, its the inverse test. But I think you
> > want to terminate early if there's any activity.
> 
> I have some doubts here.
> If I'm still the only task running, why not continue, even if I've been
> pre-empted?  Since there's nothing else to run, why not take
> advantage of the cpu?
> 

Peter,

Is my explanation adequate or you still have objection to the implementation?  I'm
trying to decide here whether to extend our batch processing by
the crypto daemon (prolong our busy period) 
based on whether there are other tasks to run.  Pre-emption and
resuming the crypto daemon is okay. Even if there's a pre-emption and
context switch in between, we can still do extended processing if
there's nothing else to run. Otherwise the cpu will go idle. 

Tim

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

* Re: [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure
  2014-07-29 22:28       ` Tim Chen
@ 2014-07-30 10:01         ` Peter Zijlstra
  0 siblings, 0 replies; 15+ messages in thread
From: Peter Zijlstra @ 2014-07-30 10:01 UTC (permalink / raw)
  To: Tim Chen
  Cc: Herbert Xu, H. Peter Anvin, David S.Miller, Ingo Molnar,
	Chandramouli Narayanan, Vinodh Gopal, James Guilford,
	Wajdi Feghali, Jussi Kivilinna, Thomas Gleixner, Tadeusz Struk,
	tkhai, linux-crypto, linux-kernel

[-- Attachment #1: Type: text/plain, Size: 748 bytes --]

On Tue, Jul 29, 2014 at 03:28:25PM -0700, Tim Chen wrote:
> Peter,
> 
> Is my explanation adequate or you still have objection to the implementation?  I'm
> trying to decide here whether to extend our batch processing by
> the crypto daemon (prolong our busy period) 
> based on whether there are other tasks to run.  Pre-emption and
> resuming the crypto daemon is okay. Even if there's a pre-emption and
> context switch in between, we can still do extended processing if
> there's nothing else to run. Otherwise the cpu will go idle. 

I suppose.. I still don't really like it but what the heck.

That single_task_running thing is hard to use and a weird interface so
lets hope people don't find more 'creative' users for it.



[-- Attachment #2: Type: application/pgp-signature, Size: 836 bytes --]

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

end of thread, other threads:[~2014-07-30 10:02 UTC | newest]

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
     [not found] <cover.1406033477.git.tim.c.chen@linux.intel.com>
2014-07-22 22:09 ` [PATCH v5 0/7] crypto: SHA1 multibuffer implementation Tim Chen
2014-07-22 22:09 ` [PATCH v5 1/7] sched: Add function single_task_running to let a task check if it is the only task running on a cpu Tim Chen
2014-07-22 22:09 ` [PATCH v5 2/7] crypto: SHA1 multibuffer crypto hash infrastructure Tim Chen
2014-07-25  7:08   ` Peter Zijlstra
2014-07-25 16:25     ` Tim Chen
2014-07-29 22:28       ` Tim Chen
2014-07-30 10:01         ` Peter Zijlstra
2014-07-25  7:26   ` Peter Zijlstra
2014-07-25 16:47     ` Tim Chen
2014-07-22 22:09 ` [PATCH v5 3/7] crypto: SHA1 multibuffer crypto opportunistic flush Tim Chen
2014-07-25  7:22   ` Peter Zijlstra
2014-07-22 22:09 ` [PATCH v5 4/7] crypto: SHA1 multibuffer algorithm data structures Tim Chen
2014-07-22 22:09 ` [PATCH v5 5/7] crypto: SHA1 multibuffer submit and flush routines for AVX2 Tim Chen
2014-07-22 22:09 ` [PATCH v5 6/7] crypto: SHA1 multibuffer crypto computation (x8 AVX2) Tim Chen
2014-07-22 22:09 ` [PATCH v5 7/7] crypto: SHA1 multibuffer scheduler Tim Chen

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.