2022-07-26 20:17:40

by Dmitry Safonov

[permalink] [raw]
Subject: [PATCH 0/6] net/crypto: Introduce crypto_pool

Add crypto_pool - an API for allocating per-CPU array of crypto requests
on slow-path (in sleep'able context) and to use them on a fast-path,
which is RX/TX for net/ users (or in any other bh-disabled users).
The design is based on the current implementations of md5sig_pool.

Previously, I've suggested to add such API on TCP-AO patch submission [1],
where Herbert kindly suggested to help with introducing new crypto API.

New API will allow:
- to reuse per-CPU ahash_request(s) for different users
- to allocate only one per-CPU scratch buffer rather than a new one for
each user
- to have a common API for net/ users that need ahash on RX/TX fast path

In this version I've wired up TCP-MD5 and IPv6-SR-HMAC as users.
Potentially, xfrm_ipcomp and xfrm_ah can be converted as well.
The initial reason for patches would be to have TCP-AO as a user, which
would let it share per-CPU crypto_request for any supported hashing
algorithm.

While at it, I've also made TCP-MD5 static key dynamically switchable.
This means that after TCP-MD5 was used and the last key got destroyed,
the static branch is disabled and any potential penalty for checking
tcp_md5sig_info is gone, and the system's tcp performance should be as
if it never had TCP-MD5 key defined.

[1]: http://lkml.kernel.org/r/[email protected]

Cc: Andy Lutomirski <[email protected]>
Cc: Ard Biesheuvel <[email protected]>
Cc: David Ahern <[email protected]>
Cc: "David S. Miller" <[email protected]>
Cc: Dmitry Safonov <[email protected]>
Cc: Eric Biggers <[email protected]>
Cc: Eric Dumazet <[email protected]>
Cc: Francesco Ruggeri <[email protected]>
Cc: Herbert Xu <[email protected]>
Cc: Hideaki YOSHIFUJI <[email protected]>
Cc: Jakub Kicinski <[email protected]>
Cc: Leonard Crestez <[email protected]>
Cc: Paolo Abeni <[email protected]>
Cc: Salam Noureddine <[email protected]>
Cc: [email protected]
Cc: [email protected]
Cc: [email protected]

Dmitry Safonov (6):
crypto: Introduce crypto_pool
crypto_pool: Add crypto_pool_reserve_scratch()
net/tcp: Separate tcp_md5sig_info allocation into
tcp_md5sig_info_add()
net/tcp: Disable TCP-MD5 static key on tcp_md5sig_info destruction
net/tcp: Use crypto_pool for TCP-MD5
net/ipv6: sr: Switch to using crypto_pool

crypto/Kconfig | 12 ++
crypto/Makefile | 1 +
crypto/crypto_pool.c | 316 +++++++++++++++++++++++++++++++++++++++
include/crypto/pool.h | 33 ++++
include/net/seg6_hmac.h | 7 -
include/net/tcp.h | 32 ++--
net/ipv4/Kconfig | 2 +-
net/ipv4/tcp.c | 102 ++++---------
net/ipv4/tcp_ipv4.c | 153 ++++++++++++-------
net/ipv4/tcp_minisocks.c | 23 ++-
net/ipv4/tcp_output.c | 4 +-
net/ipv6/Kconfig | 2 +-
net/ipv6/seg6.c | 3 -
net/ipv6/seg6_hmac.c | 204 ++++++++++---------------
net/ipv6/tcp_ipv6.c | 63 ++++----
15 files changed, 634 insertions(+), 323 deletions(-)
create mode 100644 crypto/crypto_pool.c
create mode 100644 include/crypto/pool.h


base-commit: 058affafc65a74cf54499fb578b66ad0b18f939b
--
2.36.1


2022-07-26 20:17:40

by Dmitry Safonov

[permalink] [raw]
Subject: [PATCH 3/6] net/tcp: Separate tcp_md5sig_info allocation into tcp_md5sig_info_add()

Add a helper to allocate tcp_md5sig_info, that will help later to
do/allocate things when info allocated, once per socket.

Signed-off-by: Dmitry Safonov <[email protected]>
---
net/ipv4/tcp_ipv4.c | 30 +++++++++++++++++++++---------
1 file changed, 21 insertions(+), 9 deletions(-)

diff --git a/net/ipv4/tcp_ipv4.c b/net/ipv4/tcp_ipv4.c
index 0c83780dc9bf..55e4092209a5 100644
--- a/net/ipv4/tcp_ipv4.c
+++ b/net/ipv4/tcp_ipv4.c
@@ -1152,6 +1152,24 @@ struct tcp_md5sig_key *tcp_v4_md5_lookup(const struct sock *sk,
}
EXPORT_SYMBOL(tcp_v4_md5_lookup);

+static int tcp_md5sig_info_add(struct sock *sk, gfp_t gfp)
+{
+ struct tcp_sock *tp = tcp_sk(sk);
+ struct tcp_md5sig_info *md5sig;
+
+ if (rcu_dereference_protected(tp->md5sig_info, lockdep_sock_is_held(sk)))
+ return 0;
+
+ md5sig = kmalloc(sizeof(*md5sig), gfp);
+ if (!md5sig)
+ return -ENOMEM;
+
+ sk_gso_disable(sk);
+ INIT_HLIST_HEAD(&md5sig->head);
+ rcu_assign_pointer(tp->md5sig_info, md5sig);
+ return 0;
+}
+
/* This can be called on a newly created socket, from other files */
int tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
int family, u8 prefixlen, int l3index, u8 flags,
@@ -1182,17 +1200,11 @@ int tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
return 0;
}

+ if (tcp_md5sig_info_add(sk, gfp))
+ return -ENOMEM;
+
md5sig = rcu_dereference_protected(tp->md5sig_info,
lockdep_sock_is_held(sk));
- if (!md5sig) {
- md5sig = kmalloc(sizeof(*md5sig), gfp);
- if (!md5sig)
- return -ENOMEM;
-
- sk_gso_disable(sk);
- INIT_HLIST_HEAD(&md5sig->head);
- rcu_assign_pointer(tp->md5sig_info, md5sig);
- }

key = sock_kmalloc(sk, sizeof(*key), gfp | __GFP_ZERO);
if (!key)
--
2.36.1

2022-07-26 20:17:40

by Dmitry Safonov

[permalink] [raw]
Subject: [PATCH 1/6] crypto: Introduce crypto_pool

Introduce a per-CPU pool of async crypto requests that can be used
in bh-disabled contexts (designed with net RX/TX softirqs as users in
mind). Allocation can sleep and is a slow-path.
Initial implementation has only ahash as a backend and a fix-sized array
of possible algorithms used in parallel.

Signed-off-by: Dmitry Safonov <[email protected]>
---
crypto/Kconfig | 6 +
crypto/Makefile | 1 +
crypto/crypto_pool.c | 287 ++++++++++++++++++++++++++++++++++++++++++
include/crypto/pool.h | 34 +++++
4 files changed, 328 insertions(+)
create mode 100644 crypto/crypto_pool.c
create mode 100644 include/crypto/pool.h

diff --git a/crypto/Kconfig b/crypto/Kconfig
index bb427a835e44..aeddaa3dcc77 100644
--- a/crypto/Kconfig
+++ b/crypto/Kconfig
@@ -2128,6 +2128,12 @@ config CRYPTO_STATS
config CRYPTO_HASH_INFO
bool

+config CRYPTO_POOL
+ tristate "Per-CPU crypto pool"
+ default n
+ help
+ Per-CPU pool of crypto requests ready for usage in atomic contexts.
+
source "drivers/crypto/Kconfig"
source "crypto/asymmetric_keys/Kconfig"
source "certs/Kconfig"
diff --git a/crypto/Makefile b/crypto/Makefile
index 167c004dbf4f..6d1d9801b76b 100644
--- a/crypto/Makefile
+++ b/crypto/Makefile
@@ -63,6 +63,7 @@ obj-$(CONFIG_CRYPTO_ACOMP2) += crypto_acompress.o
cryptomgr-y := algboss.o testmgr.o

obj-$(CONFIG_CRYPTO_MANAGER2) += cryptomgr.o
+obj-$(CONFIG_CRYPTO_POOL) += crypto_pool.o
obj-$(CONFIG_CRYPTO_USER) += crypto_user.o
crypto_user-y := crypto_user_base.o
crypto_user-$(CONFIG_CRYPTO_STATS) += crypto_user_stat.o
diff --git a/crypto/crypto_pool.c b/crypto/crypto_pool.c
new file mode 100644
index 000000000000..c668c02499b7
--- /dev/null
+++ b/crypto/crypto_pool.c
@@ -0,0 +1,287 @@
+// SPDX-License-Identifier: GPL-2.0-or-later
+
+#include <crypto/pool.h>
+#include <linux/kref.h>
+#include <linux/module.h>
+#include <linux/mutex.h>
+#include <linux/percpu.h>
+#include <linux/workqueue.h>
+
+static unsigned long scratch_size = DEFAULT_CRYPTO_POOL_SCRATCH_SZ;
+static DEFINE_PER_CPU(void *, crypto_pool_scratch);
+
+struct crypto_pool_entry {
+ struct ahash_request * __percpu *req;
+ const char *alg;
+ struct kref kref;
+ bool needs_key;
+};
+
+#define CPOOL_SIZE (PAGE_SIZE/sizeof(struct crypto_pool_entry))
+static struct crypto_pool_entry cpool[CPOOL_SIZE];
+static int last_allocated;
+static DEFINE_MUTEX(cpool_mutex);
+
+static int crypto_pool_scratch_alloc(void)
+{
+ int cpu;
+
+ lockdep_assert_held(&cpool_mutex);
+
+ for_each_possible_cpu(cpu) {
+ void *scratch = per_cpu(crypto_pool_scratch, cpu);
+
+ if (scratch)
+ continue;
+
+ scratch = kmalloc_node(scratch_size, GFP_KERNEL,
+ cpu_to_node(cpu));
+ if (!scratch)
+ return -ENOMEM;
+ per_cpu(crypto_pool_scratch, cpu) = scratch;
+ }
+ return 0;
+}
+
+static void crypto_pool_scratch_free(void)
+{
+ int cpu;
+
+ lockdep_assert_held(&cpool_mutex);
+
+ for_each_possible_cpu(cpu) {
+ void *scratch = per_cpu(crypto_pool_scratch, cpu);
+
+ if (!scratch)
+ continue;
+ per_cpu(crypto_pool_scratch, cpu) = NULL;
+ kfree(scratch);
+ }
+}
+
+static int __cpool_alloc_ahash(struct crypto_pool_entry *e, const char *alg)
+{
+ struct crypto_ahash *hash;
+ int cpu, ret = -ENOMEM;
+
+ e->alg = kstrdup(alg, GFP_KERNEL);
+ if (!e->alg)
+ return -ENOMEM;
+
+ e->req = alloc_percpu(struct ahash_request *);
+ if (!e->req)
+ goto out_free_alg;
+
+ hash = crypto_alloc_ahash(alg, 0, CRYPTO_ALG_ASYNC);
+ if (IS_ERR(hash)) {
+ ret = PTR_ERR(hash);
+ goto out_free_req;
+ }
+
+ /* If hash has .setkey(), allocate ahash per-cpu, not only request */
+ e->needs_key = crypto_ahash_get_flags(hash) & CRYPTO_TFM_NEED_KEY;
+
+ for_each_possible_cpu(cpu) {
+ struct ahash_request *req;
+
+ if (!hash)
+ hash = crypto_alloc_ahash(alg, 0, CRYPTO_ALG_ASYNC);
+ if (IS_ERR(hash))
+ goto out_free;
+
+ req = ahash_request_alloc(hash, GFP_KERNEL);
+ if (!req)
+ goto out_free;
+
+ ahash_request_set_callback(req, 0, NULL, NULL);
+
+ *per_cpu_ptr(e->req, cpu) = req;
+
+ if (e->needs_key)
+ hash = NULL;
+ }
+ kref_init(&e->kref);
+ return 0;
+
+out_free:
+ if (!IS_ERR_OR_NULL(hash) && e->needs_key)
+ crypto_free_ahash(hash);
+
+ for_each_possible_cpu(cpu) {
+ if (*per_cpu_ptr(e->req, cpu) == NULL)
+ break;
+ hash = crypto_ahash_reqtfm(*per_cpu_ptr(e->req, cpu));
+ ahash_request_free(*per_cpu_ptr(e->req, cpu));
+ if (e->needs_key) {
+ crypto_free_ahash(hash);
+ hash = NULL;
+ }
+ }
+
+ if (hash)
+ crypto_free_ahash(hash);
+out_free_req:
+ free_percpu(e->req);
+out_free_alg:
+ kfree(e->alg);
+ e->alg = NULL;
+ return ret;
+}
+
+/**
+ * crypto_pool_alloc_ahash - allocates pool for ahash requests
+ * @alg: name of async hash algorithm
+ */
+int crypto_pool_alloc_ahash(const char *alg)
+{
+ unsigned int i;
+ int err;
+
+ /* slow-path */
+ mutex_lock(&cpool_mutex);
+ err = crypto_pool_scratch_alloc();
+ if (err)
+ goto out;
+
+ for (i = 0; i < last_allocated; i++) {
+ if (cpool[i].alg && !strcmp(cpool[i].alg, alg)) {
+ kref_get(&cpool[i].kref);
+ goto out;
+ }
+ }
+
+ for (i = 0; i < last_allocated; i++) {
+ if (!cpool[i].alg)
+ break;
+ }
+ if (i >= CPOOL_SIZE) {
+ err = -ENOSPC;
+ goto out;
+ }
+
+ err = __cpool_alloc_ahash(&cpool[i], alg);
+ if (!err && last_allocated <= i)
+ last_allocated++;
+out:
+ mutex_unlock(&cpool_mutex);
+ return err ?: (int)i;
+}
+EXPORT_SYMBOL_GPL(crypto_pool_alloc_ahash);
+
+static void __cpool_free_entry(struct crypto_pool_entry *e)
+{
+ struct crypto_ahash *hash = NULL;
+ int cpu;
+
+ for_each_possible_cpu(cpu) {
+ if (*per_cpu_ptr(e->req, cpu) == NULL)
+ continue;
+
+ hash = crypto_ahash_reqtfm(*per_cpu_ptr(e->req, cpu));
+ ahash_request_free(*per_cpu_ptr(e->req, cpu));
+ if (e->needs_key) {
+ crypto_free_ahash(hash);
+ hash = NULL;
+ }
+ }
+ if (hash)
+ crypto_free_ahash(hash);
+ free_percpu(e->req);
+ kfree(e->alg);
+ memset(e, 0, sizeof(*e));
+}
+
+static void cpool_cleanup_work_cb(struct work_struct *work)
+{
+ unsigned int i;
+ bool free_scratch = true;
+
+ mutex_lock(&cpool_mutex);
+ for (i = 0; i < last_allocated; i++) {
+ if (kref_read(&cpool[i].kref) > 0) {
+ free_scratch = false;
+ continue;
+ }
+ if (!cpool[i].alg)
+ continue;
+ __cpool_free_entry(&cpool[i]);
+ }
+ if (free_scratch)
+ crypto_pool_scratch_free();
+ mutex_unlock(&cpool_mutex);
+}
+
+static DECLARE_WORK(cpool_cleanup_work, cpool_cleanup_work_cb);
+static void cpool_schedule_cleanup(struct kref *kref)
+{
+ schedule_work(&cpool_cleanup_work);
+}
+
+/**
+ * crypto_pool_release - decreases number of users for a pool. If it was
+ * the last user of the pool, releases any memory that was consumed.
+ * @id: crypto_pool that was previously allocated by crypto_pool_alloc_ahash()
+ */
+void crypto_pool_release(unsigned int id)
+{
+ if (WARN_ON_ONCE(id > last_allocated || !cpool[id].alg))
+ return;
+
+ /* slow-path */
+ kref_put(&cpool[id].kref, cpool_schedule_cleanup);
+}
+EXPORT_SYMBOL_GPL(crypto_pool_release);
+
+/**
+ * crypto_pool_add - increases number of users (refcounter) for a pool
+ * @id: crypto_pool that was previously allocated by crypto_pool_alloc_ahash()
+ */
+void crypto_pool_add(unsigned int id)
+{
+ if (WARN_ON_ONCE(id > last_allocated || !cpool[id].alg))
+ return;
+ kref_get(&cpool[id].kref);
+}
+EXPORT_SYMBOL_GPL(crypto_pool_add);
+
+/**
+ * crypto_pool_get - disable bh and start using crypto_pool
+ * @id: crypto_pool that was previously allocated by crypto_pool_alloc_ahash()
+ * @c: returned crypto_pool for usage (uninitialized on failure)
+ */
+int crypto_pool_get(unsigned int id, struct crypto_pool *c)
+{
+ struct crypto_pool_ahash *ret = (struct crypto_pool_ahash *)c;
+
+ local_bh_disable();
+ if (WARN_ON_ONCE(id > last_allocated || !cpool[id].alg)) {
+ local_bh_enable();
+ return -EINVAL;
+ }
+ ret->req = *this_cpu_ptr(cpool[id].req);
+ ret->base.scratch = this_cpu_read(crypto_pool_scratch);
+ return 0;
+}
+EXPORT_SYMBOL_GPL(crypto_pool_get);
+
+/**
+ * crypto_pool_algo - return algorithm of crypto_pool
+ * @id: crypto_pool that was previously allocated by crypto_pool_alloc_ahash()
+ * @buf: buffer to return name of algorithm
+ * @buf_len: size of @buf
+ */
+size_t crypto_pool_algo(unsigned int id, char *buf, size_t buf_len)
+{
+ size_t ret = 0;
+
+ /* slow-path */
+ mutex_lock(&cpool_mutex);
+ if (cpool[id].alg)
+ ret = strscpy(buf, cpool[id].alg, buf_len);
+ mutex_unlock(&cpool_mutex);
+ return ret;
+}
+EXPORT_SYMBOL_GPL(crypto_pool_algo);
+
+MODULE_LICENSE("GPL");
+MODULE_DESCRIPTION("Per-CPU pool of crypto requests");
diff --git a/include/crypto/pool.h b/include/crypto/pool.h
new file mode 100644
index 000000000000..2c61aa45faff
--- /dev/null
+++ b/include/crypto/pool.h
@@ -0,0 +1,34 @@
+/* SPDX-License-Identifier: GPL-2.0-or-later */
+#ifndef _CRYPTO_POOL_H
+#define _CRYPTO_POOL_H
+
+#include <crypto/hash.h>
+
+#define DEFAULT_CRYPTO_POOL_SCRATCH_SZ 128
+
+struct crypto_pool {
+ void *scratch;
+};
+
+/*
+ * struct crypto_pool_ahash - per-CPU pool of ahash_requests
+ * @base: common members that can be used by any async crypto ops
+ * @req: pre-allocated ahash request
+ */
+struct crypto_pool_ahash {
+ struct crypto_pool base;
+ struct ahash_request *req;
+};
+
+int crypto_pool_alloc_ahash(const char *alg);
+void crypto_pool_add(unsigned int id);
+void crypto_pool_release(unsigned int id);
+
+int crypto_pool_get(unsigned int id, struct crypto_pool *c);
+static inline void crypto_pool_put(void)
+{
+ local_bh_enable();
+}
+size_t crypto_pool_algo(unsigned int id, char *buf, size_t buf_len);
+
+#endif /* _CRYPTO_POOL_H */
--
2.36.1

2022-07-26 20:18:54

by Dmitry Safonov

[permalink] [raw]
Subject: [PATCH 6/6] net/ipv6: sr: Switch to using crypto_pool

The conversion to use crypto_pool has the following upsides:
- now SR uses asynchronous API which may potentially free CPU cycles and
improve performance for of CPU crypto algorithm providers;
- hash descriptors now don't have to be allocated on boot, but only at
the moment SR starts using HMAC and until the last HMAC secret is
deleted;
- potentially reuse ahash_request(s) for different users
- allocate only one per-CPU scratch buffer rather than a new one for
each user
- have a common API for net/ users that need ahash on RX/TX fast path

Signed-off-by: Dmitry Safonov <[email protected]>
---
include/net/seg6_hmac.h | 7 --
net/ipv6/Kconfig | 2 +-
net/ipv6/seg6.c | 3 -
net/ipv6/seg6_hmac.c | 204 ++++++++++++++++------------------------
4 files changed, 80 insertions(+), 136 deletions(-)

diff --git a/include/net/seg6_hmac.h b/include/net/seg6_hmac.h
index 2b5d2ee5613e..d6b7820ecda2 100644
--- a/include/net/seg6_hmac.h
+++ b/include/net/seg6_hmac.h
@@ -32,13 +32,6 @@ struct seg6_hmac_info {
u8 alg_id;
};

-struct seg6_hmac_algo {
- u8 alg_id;
- char name[64];
- struct crypto_shash * __percpu *tfms;
- struct shash_desc * __percpu *shashs;
-};
-
extern int seg6_hmac_compute(struct seg6_hmac_info *hinfo,
struct ipv6_sr_hdr *hdr, struct in6_addr *saddr,
u8 *output);
diff --git a/net/ipv6/Kconfig b/net/ipv6/Kconfig
index 658bfed1df8b..5be1dab0f178 100644
--- a/net/ipv6/Kconfig
+++ b/net/ipv6/Kconfig
@@ -304,7 +304,7 @@ config IPV6_SEG6_LWTUNNEL
config IPV6_SEG6_HMAC
bool "IPv6: Segment Routing HMAC support"
depends on IPV6
- select CRYPTO
+ select CRYPTO_POOL
select CRYPTO_HMAC
select CRYPTO_SHA1
select CRYPTO_SHA256
diff --git a/net/ipv6/seg6.c b/net/ipv6/seg6.c
index 73aaabf0e966..96b80e1d04c1 100644
--- a/net/ipv6/seg6.c
+++ b/net/ipv6/seg6.c
@@ -552,9 +552,6 @@ int __init seg6_init(void)

void seg6_exit(void)
{
-#ifdef CONFIG_IPV6_SEG6_HMAC
- seg6_hmac_exit();
-#endif
#ifdef CONFIG_IPV6_SEG6_LWTUNNEL
seg6_iptunnel_exit();
#endif
diff --git a/net/ipv6/seg6_hmac.c b/net/ipv6/seg6_hmac.c
index d43c50a7310d..3732dd993925 100644
--- a/net/ipv6/seg6_hmac.c
+++ b/net/ipv6/seg6_hmac.c
@@ -35,6 +35,7 @@
#include <net/xfrm.h>

#include <crypto/hash.h>
+#include <crypto/pool.h>
#include <net/seg6.h>
#include <net/genetlink.h>
#include <net/seg6_hmac.h>
@@ -70,6 +71,12 @@ static const struct rhashtable_params rht_params = {
.obj_cmpfn = seg6_hmac_cmpfn,
};

+struct seg6_hmac_algo {
+ u8 alg_id;
+ char name[64];
+ int crypto_pool_id;
+};
+
static struct seg6_hmac_algo hmac_algos[] = {
{
.alg_id = SEG6_HMAC_ALGO_SHA1,
@@ -115,55 +122,17 @@ static struct seg6_hmac_algo *__hmac_get_algo(u8 alg_id)
return NULL;
}

-static int __do_hmac(struct seg6_hmac_info *hinfo, const char *text, u8 psize,
- u8 *output, int outlen)
-{
- struct seg6_hmac_algo *algo;
- struct crypto_shash *tfm;
- struct shash_desc *shash;
- int ret, dgsize;
-
- algo = __hmac_get_algo(hinfo->alg_id);
- if (!algo)
- return -ENOENT;
-
- tfm = *this_cpu_ptr(algo->tfms);
-
- dgsize = crypto_shash_digestsize(tfm);
- if (dgsize > outlen) {
- pr_debug("sr-ipv6: __do_hmac: digest size too big (%d / %d)\n",
- dgsize, outlen);
- return -ENOMEM;
- }
-
- ret = crypto_shash_setkey(tfm, hinfo->secret, hinfo->slen);
- if (ret < 0) {
- pr_debug("sr-ipv6: crypto_shash_setkey failed: err %d\n", ret);
- goto failed;
- }
-
- shash = *this_cpu_ptr(algo->shashs);
- shash->tfm = tfm;
-
- ret = crypto_shash_digest(shash, text, psize, output);
- if (ret < 0) {
- pr_debug("sr-ipv6: crypto_shash_digest failed: err %d\n", ret);
- goto failed;
- }
-
- return dgsize;
-
-failed:
- return ret;
-}
-
int seg6_hmac_compute(struct seg6_hmac_info *hinfo, struct ipv6_sr_hdr *hdr,
struct in6_addr *saddr, u8 *output)
{
__be32 hmackeyid = cpu_to_be32(hinfo->hmackeyid);
- u8 tmp_out[SEG6_HMAC_MAX_DIGESTSIZE];
+ struct crypto_pool_ahash hp;
+ struct seg6_hmac_algo *algo;
int plen, i, dgsize, wrsize;
+ struct crypto_ahash *tfm;
+ struct scatterlist sg;
char *ring, *off;
+ int err;

/* a 160-byte buffer for digest output allows to store highest known
* hash function (RadioGatun) with up to 1216 bits
@@ -176,6 +145,10 @@ int seg6_hmac_compute(struct seg6_hmac_info *hinfo, struct ipv6_sr_hdr *hdr,
if (plen >= SEG6_HMAC_RING_SIZE)
return -EMSGSIZE;

+ algo = __hmac_get_algo(hinfo->alg_id);
+ if (!algo)
+ return -ENOENT;
+
/* Let's build the HMAC text on the ring buffer. The text is composed
* as follows, in order:
*
@@ -186,8 +159,36 @@ int seg6_hmac_compute(struct seg6_hmac_info *hinfo, struct ipv6_sr_hdr *hdr,
* 5. All segments in the segments list (n * 128 bits)
*/

- local_bh_disable();
+ err = crypto_pool_get(algo->crypto_pool_id, (struct crypto_pool *)&hp);
+ if (err)
+ return err;
+
ring = this_cpu_ptr(hmac_ring);
+
+ sg_init_one(&sg, ring, plen);
+
+ tfm = crypto_ahash_reqtfm(hp.req);
+ dgsize = crypto_ahash_digestsize(tfm);
+ if (dgsize > SEG6_HMAC_MAX_DIGESTSIZE) {
+ pr_debug("digest size too big (%d / %d)\n",
+ dgsize, SEG6_HMAC_MAX_DIGESTSIZE);
+ err = -ENOMEM;
+ goto err_put_pool;
+ }
+
+ err = crypto_ahash_setkey(tfm, hinfo->secret, hinfo->slen);
+ if (err) {
+ pr_debug("crypto_ahash_setkey failed: err %d\n", err);
+ goto err_put_pool;
+ }
+
+ err = crypto_ahash_init(hp.req);
+ if (err)
+ goto err_put_pool;
+
+ ahash_request_set_crypt(hp.req, &sg,
+ hp.base.scratch, SEG6_HMAC_MAX_DIGESTSIZE);
+
off = ring;

/* source address */
@@ -210,21 +211,25 @@ int seg6_hmac_compute(struct seg6_hmac_info *hinfo, struct ipv6_sr_hdr *hdr,
off += 16;
}

- dgsize = __do_hmac(hinfo, ring, plen, tmp_out,
- SEG6_HMAC_MAX_DIGESTSIZE);
- local_bh_enable();
+ err = crypto_ahash_update(hp.req);
+ if (err)
+ goto err_put_pool;

- if (dgsize < 0)
- return dgsize;
+ err = crypto_ahash_final(hp.req);
+ if (err)
+ goto err_put_pool;

wrsize = SEG6_HMAC_FIELD_LEN;
if (wrsize > dgsize)
wrsize = dgsize;

memset(output, 0, SEG6_HMAC_FIELD_LEN);
- memcpy(output, tmp_out, wrsize);
+ memcpy(output, hp.base.scratch, wrsize);

- return 0;
+err_put_pool:
+ crypto_pool_put();
+
+ return err;
}
EXPORT_SYMBOL(seg6_hmac_compute);

@@ -291,12 +296,24 @@ EXPORT_SYMBOL(seg6_hmac_info_lookup);
int seg6_hmac_info_add(struct net *net, u32 key, struct seg6_hmac_info *hinfo)
{
struct seg6_pernet_data *sdata = seg6_pernet(net);
- int err;
+ struct seg6_hmac_algo *algo;
+ int ret;

- err = rhashtable_lookup_insert_fast(&sdata->hmac_infos, &hinfo->node,
+ algo = __hmac_get_algo(hinfo->alg_id);
+ if (!algo)
+ return -ENOENT;
+
+ ret = crypto_pool_alloc_ahash(algo->name);
+ if (ret < 0)
+ return ret;
+ algo->crypto_pool_id = ret;
+
+ ret = rhashtable_lookup_insert_fast(&sdata->hmac_infos, &hinfo->node,
rht_params);
+ if (ret)
+ crypto_pool_release(algo->crypto_pool_id);

- return err;
+ return ret;
}
EXPORT_SYMBOL(seg6_hmac_info_add);

@@ -304,6 +321,7 @@ int seg6_hmac_info_del(struct net *net, u32 key)
{
struct seg6_pernet_data *sdata = seg6_pernet(net);
struct seg6_hmac_info *hinfo;
+ struct seg6_hmac_algo *algo;
int err = -ENOENT;

hinfo = rhashtable_lookup_fast(&sdata->hmac_infos, &key, rht_params);
@@ -315,6 +333,12 @@ int seg6_hmac_info_del(struct net *net, u32 key)
if (err)
goto out;

+ algo = __hmac_get_algo(hinfo->alg_id);
+ if (algo)
+ crypto_pool_release(algo->crypto_pool_id);
+ else
+ WARN_ON_ONCE(1);
+
seg6_hinfo_release(hinfo);

out:
@@ -348,56 +372,9 @@ int seg6_push_hmac(struct net *net, struct in6_addr *saddr,
}
EXPORT_SYMBOL(seg6_push_hmac);

-static int seg6_hmac_init_algo(void)
-{
- struct seg6_hmac_algo *algo;
- struct crypto_shash *tfm;
- struct shash_desc *shash;
- int i, alg_count, cpu;
-
- alg_count = ARRAY_SIZE(hmac_algos);
-
- for (i = 0; i < alg_count; i++) {
- struct crypto_shash **p_tfm;
- int shsize;
-
- algo = &hmac_algos[i];
- algo->tfms = alloc_percpu(struct crypto_shash *);
- if (!algo->tfms)
- return -ENOMEM;
-
- for_each_possible_cpu(cpu) {
- tfm = crypto_alloc_shash(algo->name, 0, 0);
- if (IS_ERR(tfm))
- return PTR_ERR(tfm);
- p_tfm = per_cpu_ptr(algo->tfms, cpu);
- *p_tfm = tfm;
- }
-
- p_tfm = raw_cpu_ptr(algo->tfms);
- tfm = *p_tfm;
-
- shsize = sizeof(*shash) + crypto_shash_descsize(tfm);
-
- algo->shashs = alloc_percpu(struct shash_desc *);
- if (!algo->shashs)
- return -ENOMEM;
-
- for_each_possible_cpu(cpu) {
- shash = kzalloc_node(shsize, GFP_KERNEL,
- cpu_to_node(cpu));
- if (!shash)
- return -ENOMEM;
- *per_cpu_ptr(algo->shashs, cpu) = shash;
- }
- }
-
- return 0;
-}
-
int __init seg6_hmac_init(void)
{
- return seg6_hmac_init_algo();
+ return crypto_pool_reserve_scratch(SEG6_HMAC_MAX_DIGESTSIZE);
}

int __net_init seg6_hmac_net_init(struct net *net)
@@ -407,29 +384,6 @@ int __net_init seg6_hmac_net_init(struct net *net)
return rhashtable_init(&sdata->hmac_infos, &rht_params);
}

-void seg6_hmac_exit(void)
-{
- struct seg6_hmac_algo *algo = NULL;
- int i, alg_count, cpu;
-
- alg_count = ARRAY_SIZE(hmac_algos);
- for (i = 0; i < alg_count; i++) {
- algo = &hmac_algos[i];
- for_each_possible_cpu(cpu) {
- struct crypto_shash *tfm;
- struct shash_desc *shash;
-
- shash = *per_cpu_ptr(algo->shashs, cpu);
- kfree(shash);
- tfm = *per_cpu_ptr(algo->tfms, cpu);
- crypto_free_shash(tfm);
- }
- free_percpu(algo->tfms);
- free_percpu(algo->shashs);
- }
-}
-EXPORT_SYMBOL(seg6_hmac_exit);
-
void __net_exit seg6_hmac_net_exit(struct net *net)
{
struct seg6_pernet_data *sdata = seg6_pernet(net);
--
2.36.1

2022-07-26 20:19:46

by Dmitry Safonov

[permalink] [raw]
Subject: [PATCH 4/6] net/tcp: Disable TCP-MD5 static key on tcp_md5sig_info destruction

To do that, separate two scenarios:
- where it's the first MD5 key on the system, which means that enabling
of the static key may need to sleep;
- copying of an existing key from a listening socket to the request
socket upon receiving a signed TCP segment, where static key was
already enabled (when the key was added to the listening socket).

Now the life-time of the static branch for TCP-MD5 is until:
- last tcp_md5sig_info is destroyed
- last socket in time-wait state with MD5 key is closed.

Which means that after all sockets with TCP-MD5 keys are gone, the
system gets back the performance of disabled md5-key static branch.

Signed-off-by: Dmitry Safonov <[email protected]>
---
include/net/tcp.h | 10 ++++++---
net/ipv4/tcp.c | 5 +----
net/ipv4/tcp_ipv4.c | 45 +++++++++++++++++++++++++++++++---------
net/ipv4/tcp_minisocks.c | 9 +++++---
net/ipv4/tcp_output.c | 4 ++--
net/ipv6/tcp_ipv6.c | 10 ++++-----
6 files changed, 55 insertions(+), 28 deletions(-)

diff --git a/include/net/tcp.h b/include/net/tcp.h
index f366626cbbba..aa735f963723 100644
--- a/include/net/tcp.h
+++ b/include/net/tcp.h
@@ -1663,7 +1663,11 @@ int tcp_v4_md5_hash_skb(char *md5_hash, const struct tcp_md5sig_key *key,
const struct sock *sk, const struct sk_buff *skb);
int tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
int family, u8 prefixlen, int l3index, u8 flags,
- const u8 *newkey, u8 newkeylen, gfp_t gfp);
+ const u8 *newkey, u8 newkeylen);
+int tcp_md5_key_copy(struct sock *sk, const union tcp_md5_addr *addr,
+ int family, u8 prefixlen, int l3index,
+ struct tcp_md5sig_key *key);
+
int tcp_md5_do_del(struct sock *sk, const union tcp_md5_addr *addr,
int family, u8 prefixlen, int l3index, u8 flags);
struct tcp_md5sig_key *tcp_v4_md5_lookup(const struct sock *sk,
@@ -1671,7 +1675,7 @@ struct tcp_md5sig_key *tcp_v4_md5_lookup(const struct sock *sk,

#ifdef CONFIG_TCP_MD5SIG
#include <linux/jump_label.h>
-extern struct static_key_false tcp_md5_needed;
+extern struct static_key_false_deferred tcp_md5_needed;
struct tcp_md5sig_key *__tcp_md5_do_lookup(const struct sock *sk, int l3index,
const union tcp_md5_addr *addr,
int family);
@@ -1679,7 +1683,7 @@ static inline struct tcp_md5sig_key *
tcp_md5_do_lookup(const struct sock *sk, int l3index,
const union tcp_md5_addr *addr, int family)
{
- if (!static_branch_unlikely(&tcp_md5_needed))
+ if (!static_branch_unlikely(&tcp_md5_needed.key))
return NULL;
return __tcp_md5_do_lookup(sk, l3index, addr, family);
}
diff --git a/net/ipv4/tcp.c b/net/ipv4/tcp.c
index 635c6782cdbb..300056008e90 100644
--- a/net/ipv4/tcp.c
+++ b/net/ipv4/tcp.c
@@ -4404,11 +4404,8 @@ bool tcp_alloc_md5sig_pool(void)
if (unlikely(!tcp_md5sig_pool_populated)) {
mutex_lock(&tcp_md5sig_mutex);

- if (!tcp_md5sig_pool_populated) {
+ if (!tcp_md5sig_pool_populated)
__tcp_alloc_md5sig_pool();
- if (tcp_md5sig_pool_populated)
- static_branch_inc(&tcp_md5_needed);
- }

mutex_unlock(&tcp_md5sig_mutex);
}
diff --git a/net/ipv4/tcp_ipv4.c b/net/ipv4/tcp_ipv4.c
index 55e4092209a5..5b0caaea5029 100644
--- a/net/ipv4/tcp_ipv4.c
+++ b/net/ipv4/tcp_ipv4.c
@@ -1044,7 +1044,7 @@ static void tcp_v4_reqsk_destructor(struct request_sock *req)
* We need to maintain these in the sk structure.
*/

-DEFINE_STATIC_KEY_FALSE(tcp_md5_needed);
+DEFINE_STATIC_KEY_DEFERRED_FALSE(tcp_md5_needed, HZ);
EXPORT_SYMBOL(tcp_md5_needed);

static bool better_md5_match(struct tcp_md5sig_key *old, struct tcp_md5sig_key *new)
@@ -1171,9 +1171,9 @@ static int tcp_md5sig_info_add(struct sock *sk, gfp_t gfp)
}

/* This can be called on a newly created socket, from other files */
-int tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
- int family, u8 prefixlen, int l3index, u8 flags,
- const u8 *newkey, u8 newkeylen, gfp_t gfp)
+int __tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
+ int family, u8 prefixlen, int l3index, u8 flags,
+ const u8 *newkey, u8 newkeylen, gfp_t gfp)
{
/* Add Key to the list */
struct tcp_md5sig_key *key;
@@ -1200,9 +1200,6 @@ int tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
return 0;
}

- if (tcp_md5sig_info_add(sk, gfp))
- return -ENOMEM;
-
md5sig = rcu_dereference_protected(tp->md5sig_info,
lockdep_sock_is_held(sk));

@@ -1226,8 +1223,36 @@ int tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
hlist_add_head_rcu(&key->node, &md5sig->head);
return 0;
}
+
+int tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
+ int family, u8 prefixlen, int l3index, u8 flags,
+ const u8 *newkey, u8 newkeylen)
+{
+ if (tcp_md5sig_info_add(sk, GFP_KERNEL))
+ return -ENOMEM;
+
+ static_branch_inc(&tcp_md5_needed.key);
+
+ return __tcp_md5_do_add(sk, addr, family, prefixlen, l3index, flags,
+ newkey, newkeylen, GFP_KERNEL);
+}
EXPORT_SYMBOL(tcp_md5_do_add);

+int tcp_md5_key_copy(struct sock *sk, const union tcp_md5_addr *addr,
+ int family, u8 prefixlen, int l3index,
+ struct tcp_md5sig_key *key)
+{
+ if (tcp_md5sig_info_add(sk, sk_gfp_mask(sk, GFP_ATOMIC)))
+ return -ENOMEM;
+
+ atomic_inc(&tcp_md5_needed.key.key.enabled);
+
+ return __tcp_md5_do_add(sk, addr, family, prefixlen, l3index,
+ key->flags, key->key, key->keylen,
+ sk_gfp_mask(sk, GFP_ATOMIC));
+}
+EXPORT_SYMBOL(tcp_md5_key_copy);
+
int tcp_md5_do_del(struct sock *sk, const union tcp_md5_addr *addr, int family,
u8 prefixlen, int l3index, u8 flags)
{
@@ -1314,7 +1339,7 @@ static int tcp_v4_parse_md5_keys(struct sock *sk, int optname,
return -EINVAL;

return tcp_md5_do_add(sk, addr, AF_INET, prefixlen, l3index, flags,
- cmd.tcpm_key, cmd.tcpm_keylen, GFP_KERNEL);
+ cmd.tcpm_key, cmd.tcpm_keylen);
}

static int tcp_v4_md5_hash_headers(struct tcp_md5sig_pool *hp,
@@ -1571,8 +1596,7 @@ struct sock *tcp_v4_syn_recv_sock(const struct sock *sk, struct sk_buff *skb,
* memory, then we end up not copying the key
* across. Shucks.
*/
- tcp_md5_do_add(newsk, addr, AF_INET, 32, l3index, key->flags,
- key->key, key->keylen, GFP_ATOMIC);
+ tcp_md5_key_copy(newsk, addr, AF_INET, 32, l3index, key);
sk_gso_disable(newsk);
}
#endif
@@ -2260,6 +2284,7 @@ void tcp_v4_destroy_sock(struct sock *sk)
tcp_clear_md5_list(sk);
kfree_rcu(rcu_dereference_protected(tp->md5sig_info, 1), rcu);
tp->md5sig_info = NULL;
+ static_branch_slow_dec_deferred(&tcp_md5_needed);
}
#endif

diff --git a/net/ipv4/tcp_minisocks.c b/net/ipv4/tcp_minisocks.c
index cb95d88497ae..5d475a45a478 100644
--- a/net/ipv4/tcp_minisocks.c
+++ b/net/ipv4/tcp_minisocks.c
@@ -291,13 +291,14 @@ void tcp_time_wait(struct sock *sk, int state, int timeo)
*/
do {
tcptw->tw_md5_key = NULL;
- if (static_branch_unlikely(&tcp_md5_needed)) {
+ if (static_branch_unlikely(&tcp_md5_needed.key)) {
struct tcp_md5sig_key *key;

key = tp->af_specific->md5_lookup(sk, sk);
if (key) {
tcptw->tw_md5_key = kmemdup(key, sizeof(*key), GFP_ATOMIC);
BUG_ON(tcptw->tw_md5_key && !tcp_alloc_md5sig_pool());
+ atomic_inc(&tcp_md5_needed.key.key.enabled);
}
}
} while (0);
@@ -337,11 +338,13 @@ EXPORT_SYMBOL(tcp_time_wait);
void tcp_twsk_destructor(struct sock *sk)
{
#ifdef CONFIG_TCP_MD5SIG
- if (static_branch_unlikely(&tcp_md5_needed)) {
+ if (static_branch_unlikely(&tcp_md5_needed.key)) {
struct tcp_timewait_sock *twsk = tcp_twsk(sk);

- if (twsk->tw_md5_key)
+ if (twsk->tw_md5_key) {
kfree_rcu(twsk->tw_md5_key, rcu);
+ static_branch_slow_dec_deferred(&tcp_md5_needed);
+ }
}
#endif
}
diff --git a/net/ipv4/tcp_output.c b/net/ipv4/tcp_output.c
index 78b654ff421b..9e12845a8758 100644
--- a/net/ipv4/tcp_output.c
+++ b/net/ipv4/tcp_output.c
@@ -766,7 +766,7 @@ static unsigned int tcp_syn_options(struct sock *sk, struct sk_buff *skb,

*md5 = NULL;
#ifdef CONFIG_TCP_MD5SIG
- if (static_branch_unlikely(&tcp_md5_needed) &&
+ if (static_branch_unlikely(&tcp_md5_needed.key) &&
rcu_access_pointer(tp->md5sig_info)) {
*md5 = tp->af_specific->md5_lookup(sk, sk);
if (*md5) {
@@ -922,7 +922,7 @@ static unsigned int tcp_established_options(struct sock *sk, struct sk_buff *skb

*md5 = NULL;
#ifdef CONFIG_TCP_MD5SIG
- if (static_branch_unlikely(&tcp_md5_needed) &&
+ if (static_branch_unlikely(&tcp_md5_needed.key) &&
rcu_access_pointer(tp->md5sig_info)) {
*md5 = tp->af_specific->md5_lookup(sk, sk);
if (*md5) {
diff --git a/net/ipv6/tcp_ipv6.c b/net/ipv6/tcp_ipv6.c
index e54eee80ce5f..cb891a71db0d 100644
--- a/net/ipv6/tcp_ipv6.c
+++ b/net/ipv6/tcp_ipv6.c
@@ -658,12 +658,11 @@ static int tcp_v6_parse_md5_keys(struct sock *sk, int optname,
if (ipv6_addr_v4mapped(&sin6->sin6_addr))
return tcp_md5_do_add(sk, (union tcp_md5_addr *)&sin6->sin6_addr.s6_addr32[3],
AF_INET, prefixlen, l3index, flags,
- cmd.tcpm_key, cmd.tcpm_keylen,
- GFP_KERNEL);
+ cmd.tcpm_key, cmd.tcpm_keylen);

return tcp_md5_do_add(sk, (union tcp_md5_addr *)&sin6->sin6_addr,
AF_INET6, prefixlen, l3index, flags,
- cmd.tcpm_key, cmd.tcpm_keylen, GFP_KERNEL);
+ cmd.tcpm_key, cmd.tcpm_keylen);
}

static int tcp_v6_md5_hash_headers(struct tcp_md5sig_pool *hp,
@@ -1359,9 +1358,8 @@ static struct sock *tcp_v6_syn_recv_sock(const struct sock *sk, struct sk_buff *
* memory, then we end up not copying the key
* across. Shucks.
*/
- tcp_md5_do_add(newsk, (union tcp_md5_addr *)&newsk->sk_v6_daddr,
- AF_INET6, 128, l3index, key->flags, key->key, key->keylen,
- sk_gfp_mask(sk, GFP_ATOMIC));
+ tcp_md5_key_copy(newsk, (union tcp_md5_addr *)&newsk->sk_v6_daddr,
+ AF_INET6, 128, l3index, key);
}
#endif

--
2.36.1

2022-07-26 20:22:22

by Dmitry Safonov

[permalink] [raw]
Subject: [PATCH 5/6] net/tcp: Use crypto_pool for TCP-MD5

Use crypto_pool API that was designed with tcp_md5sig_pool in mind.
The conversion to use crypto_pool will allow:
- to reuse ahash_request(s) for different users
- to allocate only one per-CPU scratch buffer rather than a new one for
each user
- to have a common API for net/ users that need ahash on RX/TX fast path

Signed-off-by: Dmitry Safonov <[email protected]>
---
include/net/tcp.h | 22 +++------
net/ipv4/Kconfig | 2 +-
net/ipv4/tcp.c | 99 +++++++++++-----------------------------
net/ipv4/tcp_ipv4.c | 90 +++++++++++++++++++++---------------
net/ipv4/tcp_minisocks.c | 22 +++++++--
net/ipv6/tcp_ipv6.c | 53 ++++++++++-----------
6 files changed, 127 insertions(+), 161 deletions(-)

diff --git a/include/net/tcp.h b/include/net/tcp.h
index aa735f963723..7060dd1cf6cd 100644
--- a/include/net/tcp.h
+++ b/include/net/tcp.h
@@ -1652,12 +1652,6 @@ union tcp_md5sum_block {
#endif
};

-/* - pool: digest algorithm, hash description and scratch buffer */
-struct tcp_md5sig_pool {
- struct ahash_request *md5_req;
- void *scratch;
-};
-
/* - functions */
int tcp_v4_md5_hash_skb(char *md5_hash, const struct tcp_md5sig_key *key,
const struct sock *sk, const struct sk_buff *skb);
@@ -1713,17 +1707,15 @@ tcp_inbound_md5_hash(const struct sock *sk, const struct sk_buff *skb,
#define tcp_twsk_md5_key(twsk) NULL
#endif

-bool tcp_alloc_md5sig_pool(void);
-
-struct tcp_md5sig_pool *tcp_get_md5sig_pool(void);
-static inline void tcp_put_md5sig_pool(void)
-{
- local_bh_enable();
-}
+struct crypto_pool_ahash;
+int tcp_md5_alloc_crypto_pool(void);
+void tcp_md5_release_crypto_pool(void);
+void tcp_md5_add_crypto_pool(void);
+extern int tcp_md5_crypto_pool_id;

-int tcp_md5_hash_skb_data(struct tcp_md5sig_pool *, const struct sk_buff *,
+int tcp_md5_hash_skb_data(struct crypto_pool_ahash *, const struct sk_buff *,
unsigned int header_len);
-int tcp_md5_hash_key(struct tcp_md5sig_pool *hp,
+int tcp_md5_hash_key(struct crypto_pool_ahash *hp,
const struct tcp_md5sig_key *key);

/* From tcp_fastopen.c */
diff --git a/net/ipv4/Kconfig b/net/ipv4/Kconfig
index e983bb0c5012..c341864e4398 100644
--- a/net/ipv4/Kconfig
+++ b/net/ipv4/Kconfig
@@ -733,7 +733,7 @@ config DEFAULT_TCP_CONG

config TCP_MD5SIG
bool "TCP: MD5 Signature Option support (RFC2385)"
- select CRYPTO
+ select CRYPTO_POOL
select CRYPTO_MD5
help
RFC2385 specifies a method of giving MD5 protection to TCP sessions.
diff --git a/net/ipv4/tcp.c b/net/ipv4/tcp.c
index 300056008e90..73abb00e12bf 100644
--- a/net/ipv4/tcp.c
+++ b/net/ipv4/tcp.c
@@ -244,6 +244,7 @@
#define pr_fmt(fmt) "TCP: " fmt

#include <crypto/hash.h>
+#include <crypto/pool.h>
#include <linux/kernel.h>
#include <linux/module.h>
#include <linux/types.h>
@@ -4355,92 +4356,43 @@ int tcp_getsockopt(struct sock *sk, int level, int optname, char __user *optval,
EXPORT_SYMBOL(tcp_getsockopt);

#ifdef CONFIG_TCP_MD5SIG
-static DEFINE_PER_CPU(struct tcp_md5sig_pool, tcp_md5sig_pool);
-static DEFINE_MUTEX(tcp_md5sig_mutex);
-static bool tcp_md5sig_pool_populated = false;
+int tcp_md5_crypto_pool_id = -1;
+EXPORT_SYMBOL(tcp_md5_crypto_pool_id);

-static void __tcp_alloc_md5sig_pool(void)
+int tcp_md5_alloc_crypto_pool(void)
{
- struct crypto_ahash *hash;
- int cpu;
-
- hash = crypto_alloc_ahash("md5", 0, CRYPTO_ALG_ASYNC);
- if (IS_ERR(hash))
- return;
-
- for_each_possible_cpu(cpu) {
- void *scratch = per_cpu(tcp_md5sig_pool, cpu).scratch;
- struct ahash_request *req;
-
- if (!scratch) {
- scratch = kmalloc_node(sizeof(union tcp_md5sum_block) +
- sizeof(struct tcphdr),
- GFP_KERNEL,
- cpu_to_node(cpu));
- if (!scratch)
- return;
- per_cpu(tcp_md5sig_pool, cpu).scratch = scratch;
- }
- if (per_cpu(tcp_md5sig_pool, cpu).md5_req)
- continue;
-
- req = ahash_request_alloc(hash, GFP_KERNEL);
- if (!req)
- return;
+ int ret;

- ahash_request_set_callback(req, 0, NULL, NULL);
+ ret = crypto_pool_reserve_scratch(sizeof(union tcp_md5sum_block) +
+ sizeof(struct tcphdr));
+ if (ret)
+ return ret;

- per_cpu(tcp_md5sig_pool, cpu).md5_req = req;
- }
- /* before setting tcp_md5sig_pool_populated, we must commit all writes
- * to memory. See smp_rmb() in tcp_get_md5sig_pool()
- */
- smp_wmb();
- tcp_md5sig_pool_populated = true;
+ ret = crypto_pool_alloc_ahash("md5");
+ if (ret >= 0)
+ tcp_md5_crypto_pool_id = ret;
+ return ret;
}
+EXPORT_SYMBOL(tcp_md5_alloc_crypto_pool);

-bool tcp_alloc_md5sig_pool(void)
+void tcp_md5_release_crypto_pool(void)
{
- if (unlikely(!tcp_md5sig_pool_populated)) {
- mutex_lock(&tcp_md5sig_mutex);
-
- if (!tcp_md5sig_pool_populated)
- __tcp_alloc_md5sig_pool();
-
- mutex_unlock(&tcp_md5sig_mutex);
- }
- return tcp_md5sig_pool_populated;
+ crypto_pool_release(tcp_md5_crypto_pool_id);
}
-EXPORT_SYMBOL(tcp_alloc_md5sig_pool);
+EXPORT_SYMBOL(tcp_md5_release_crypto_pool);

-
-/**
- * tcp_get_md5sig_pool - get md5sig_pool for this user
- *
- * We use percpu structure, so if we succeed, we exit with preemption
- * and BH disabled, to make sure another thread or softirq handling
- * wont try to get same context.
- */
-struct tcp_md5sig_pool *tcp_get_md5sig_pool(void)
+void tcp_md5_add_crypto_pool(void)
{
- local_bh_disable();
-
- if (tcp_md5sig_pool_populated) {
- /* coupled with smp_wmb() in __tcp_alloc_md5sig_pool() */
- smp_rmb();
- return this_cpu_ptr(&tcp_md5sig_pool);
- }
- local_bh_enable();
- return NULL;
+ crypto_pool_add(tcp_md5_crypto_pool_id);
}
-EXPORT_SYMBOL(tcp_get_md5sig_pool);
+EXPORT_SYMBOL(tcp_md5_add_crypto_pool);

-int tcp_md5_hash_skb_data(struct tcp_md5sig_pool *hp,
+int tcp_md5_hash_skb_data(struct crypto_pool_ahash *hp,
const struct sk_buff *skb, unsigned int header_len)
{
struct scatterlist sg;
const struct tcphdr *tp = tcp_hdr(skb);
- struct ahash_request *req = hp->md5_req;
+ struct ahash_request *req = hp->req;
unsigned int i;
const unsigned int head_data_len = skb_headlen(skb) > header_len ?
skb_headlen(skb) - header_len : 0;
@@ -4474,16 +4426,17 @@ int tcp_md5_hash_skb_data(struct tcp_md5sig_pool *hp,
}
EXPORT_SYMBOL(tcp_md5_hash_skb_data);

-int tcp_md5_hash_key(struct tcp_md5sig_pool *hp, const struct tcp_md5sig_key *key)
+int tcp_md5_hash_key(struct crypto_pool_ahash *hp,
+ const struct tcp_md5sig_key *key)
{
u8 keylen = READ_ONCE(key->keylen); /* paired with WRITE_ONCE() in tcp_md5_do_add */
struct scatterlist sg;

sg_init_one(&sg, key->key, keylen);
- ahash_request_set_crypt(hp->md5_req, &sg, NULL, keylen);
+ ahash_request_set_crypt(hp->req, &sg, NULL, keylen);

/* We use data_race() because tcp_md5_do_add() might change key->key under us */
- return data_race(crypto_ahash_update(hp->md5_req));
+ return data_race(crypto_ahash_update(hp->req));
}
EXPORT_SYMBOL(tcp_md5_hash_key);

diff --git a/net/ipv4/tcp_ipv4.c b/net/ipv4/tcp_ipv4.c
index 5b0caaea5029..100e142ed03a 100644
--- a/net/ipv4/tcp_ipv4.c
+++ b/net/ipv4/tcp_ipv4.c
@@ -79,6 +79,7 @@
#include <linux/btf_ids.h>

#include <crypto/hash.h>
+#include <crypto/pool.h>
#include <linux/scatterlist.h>

#include <trace/events/tcp.h>
@@ -1206,10 +1207,6 @@ int __tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
key = sock_kmalloc(sk, sizeof(*key), gfp | __GFP_ZERO);
if (!key)
return -ENOMEM;
- if (!tcp_alloc_md5sig_pool()) {
- sock_kfree_s(sk, key, sizeof(*key));
- return -ENOMEM;
- }

memcpy(key->key, newkey, newkeylen);
key->keylen = newkeylen;
@@ -1228,8 +1225,13 @@ int tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
int family, u8 prefixlen, int l3index, u8 flags,
const u8 *newkey, u8 newkeylen)
{
- if (tcp_md5sig_info_add(sk, GFP_KERNEL))
+ if (tcp_md5_alloc_crypto_pool())
+ return -ENOMEM;
+
+ if (tcp_md5sig_info_add(sk, GFP_KERNEL)) {
+ tcp_md5_release_crypto_pool();
return -ENOMEM;
+ }

static_branch_inc(&tcp_md5_needed.key);

@@ -1242,8 +1244,12 @@ int tcp_md5_key_copy(struct sock *sk, const union tcp_md5_addr *addr,
int family, u8 prefixlen, int l3index,
struct tcp_md5sig_key *key)
{
- if (tcp_md5sig_info_add(sk, sk_gfp_mask(sk, GFP_ATOMIC)))
+ tcp_md5_add_crypto_pool();
+
+ if (tcp_md5sig_info_add(sk, sk_gfp_mask(sk, GFP_ATOMIC))) {
+ tcp_md5_release_crypto_pool();
return -ENOMEM;
+ }

atomic_inc(&tcp_md5_needed.key.key.enabled);

@@ -1342,7 +1348,7 @@ static int tcp_v4_parse_md5_keys(struct sock *sk, int optname,
cmd.tcpm_key, cmd.tcpm_keylen);
}

-static int tcp_v4_md5_hash_headers(struct tcp_md5sig_pool *hp,
+static int tcp_v4_md5_hash_headers(struct crypto_pool_ahash *hp,
__be32 daddr, __be32 saddr,
const struct tcphdr *th, int nbytes)
{
@@ -1350,7 +1356,7 @@ static int tcp_v4_md5_hash_headers(struct tcp_md5sig_pool *hp,
struct scatterlist sg;
struct tcphdr *_th;

- bp = hp->scratch;
+ bp = hp->base.scratch;
bp->saddr = saddr;
bp->daddr = daddr;
bp->pad = 0;
@@ -1362,37 +1368,34 @@ static int tcp_v4_md5_hash_headers(struct tcp_md5sig_pool *hp,
_th->check = 0;

sg_init_one(&sg, bp, sizeof(*bp) + sizeof(*th));
- ahash_request_set_crypt(hp->md5_req, &sg, NULL,
+ ahash_request_set_crypt(hp->req, &sg, NULL,
sizeof(*bp) + sizeof(*th));
- return crypto_ahash_update(hp->md5_req);
+ return crypto_ahash_update(hp->req);
}

static int tcp_v4_md5_hash_hdr(char *md5_hash, const struct tcp_md5sig_key *key,
__be32 daddr, __be32 saddr, const struct tcphdr *th)
{
- struct tcp_md5sig_pool *hp;
- struct ahash_request *req;
+ struct crypto_pool_ahash hp;

- hp = tcp_get_md5sig_pool();
- if (!hp)
+ if (crypto_pool_get(tcp_md5_crypto_pool_id, (struct crypto_pool *)&hp))
goto clear_hash_noput;
- req = hp->md5_req;

- if (crypto_ahash_init(req))
+ if (crypto_ahash_init(hp.req))
goto clear_hash;
- if (tcp_v4_md5_hash_headers(hp, daddr, saddr, th, th->doff << 2))
+ if (tcp_v4_md5_hash_headers(&hp, daddr, saddr, th, th->doff << 2))
goto clear_hash;
- if (tcp_md5_hash_key(hp, key))
+ if (tcp_md5_hash_key(&hp, key))
goto clear_hash;
- ahash_request_set_crypt(req, NULL, md5_hash, 0);
- if (crypto_ahash_final(req))
+ ahash_request_set_crypt(hp.req, NULL, md5_hash, 0);
+ if (crypto_ahash_final(hp.req))
goto clear_hash;

- tcp_put_md5sig_pool();
+ crypto_pool_put();
return 0;

clear_hash:
- tcp_put_md5sig_pool();
+ crypto_pool_put();
clear_hash_noput:
memset(md5_hash, 0, 16);
return 1;
@@ -1402,8 +1405,7 @@ int tcp_v4_md5_hash_skb(char *md5_hash, const struct tcp_md5sig_key *key,
const struct sock *sk,
const struct sk_buff *skb)
{
- struct tcp_md5sig_pool *hp;
- struct ahash_request *req;
+ struct crypto_pool_ahash hp;
const struct tcphdr *th = tcp_hdr(skb);
__be32 saddr, daddr;

@@ -1416,29 +1418,27 @@ int tcp_v4_md5_hash_skb(char *md5_hash, const struct tcp_md5sig_key *key,
daddr = iph->daddr;
}

- hp = tcp_get_md5sig_pool();
- if (!hp)
+ if (crypto_pool_get(tcp_md5_crypto_pool_id, (struct crypto_pool *)&hp))
goto clear_hash_noput;
- req = hp->md5_req;

- if (crypto_ahash_init(req))
+ if (crypto_ahash_init(hp.req))
goto clear_hash;

- if (tcp_v4_md5_hash_headers(hp, daddr, saddr, th, skb->len))
+ if (tcp_v4_md5_hash_headers(&hp, daddr, saddr, th, skb->len))
goto clear_hash;
- if (tcp_md5_hash_skb_data(hp, skb, th->doff << 2))
+ if (tcp_md5_hash_skb_data(&hp, skb, th->doff << 2))
goto clear_hash;
- if (tcp_md5_hash_key(hp, key))
+ if (tcp_md5_hash_key(&hp, key))
goto clear_hash;
- ahash_request_set_crypt(req, NULL, md5_hash, 0);
- if (crypto_ahash_final(req))
+ ahash_request_set_crypt(hp.req, NULL, md5_hash, 0);
+ if (crypto_ahash_final(hp.req))
goto clear_hash;

- tcp_put_md5sig_pool();
+ crypto_pool_put();
return 0;

clear_hash:
- tcp_put_md5sig_pool();
+ crypto_pool_put();
clear_hash_noput:
memset(md5_hash, 0, 16);
return 1;
@@ -2257,6 +2257,18 @@ static int tcp_v4_init_sock(struct sock *sk)
return 0;
}

+#ifdef CONFIG_TCP_MD5SIG
+static void tcp_md5sig_info_free_rcu(struct rcu_head *head)
+{
+ struct tcp_md5sig_info *md5sig;
+
+ md5sig = container_of(head, struct tcp_md5sig_info, rcu);
+ kfree(md5sig);
+ static_branch_slow_dec_deferred(&tcp_md5_needed);
+ tcp_md5_release_crypto_pool();
+}
+#endif
+
void tcp_v4_destroy_sock(struct sock *sk)
{
struct tcp_sock *tp = tcp_sk(sk);
@@ -2281,10 +2293,12 @@ void tcp_v4_destroy_sock(struct sock *sk)
#ifdef CONFIG_TCP_MD5SIG
/* Clean up the MD5 key list, if any */
if (tp->md5sig_info) {
+ struct tcp_md5sig_info *md5sig;
+
+ md5sig = rcu_dereference_protected(tp->md5sig_info, 1);
tcp_clear_md5_list(sk);
- kfree_rcu(rcu_dereference_protected(tp->md5sig_info, 1), rcu);
- tp->md5sig_info = NULL;
- static_branch_slow_dec_deferred(&tcp_md5_needed);
+ call_rcu(&md5sig->rcu, tcp_md5sig_info_free_rcu);
+ rcu_assign_pointer(tp->md5sig_info, NULL);
}
#endif

diff --git a/net/ipv4/tcp_minisocks.c b/net/ipv4/tcp_minisocks.c
index 5d475a45a478..d1d30337ffec 100644
--- a/net/ipv4/tcp_minisocks.c
+++ b/net/ipv4/tcp_minisocks.c
@@ -297,8 +297,10 @@ void tcp_time_wait(struct sock *sk, int state, int timeo)
key = tp->af_specific->md5_lookup(sk, sk);
if (key) {
tcptw->tw_md5_key = kmemdup(key, sizeof(*key), GFP_ATOMIC);
- BUG_ON(tcptw->tw_md5_key && !tcp_alloc_md5sig_pool());
+ if (WARN_ON(!tcptw->tw_md5_key))
+ break;
atomic_inc(&tcp_md5_needed.key.key.enabled);
+ tcp_md5_add_crypto_pool();
}
}
} while (0);
@@ -335,16 +337,26 @@ void tcp_time_wait(struct sock *sk, int state, int timeo)
}
EXPORT_SYMBOL(tcp_time_wait);

+#ifdef CONFIG_TCP_MD5SIG
+static void tcp_md5_twsk_free_rcu(struct rcu_head *head)
+{
+ struct tcp_md5sig_key *key;
+
+ key = container_of(head, struct tcp_md5sig_key, rcu);
+ kfree(key);
+ static_branch_slow_dec_deferred(&tcp_md5_needed);
+ tcp_md5_release_crypto_pool();
+}
+#endif
+
void tcp_twsk_destructor(struct sock *sk)
{
#ifdef CONFIG_TCP_MD5SIG
if (static_branch_unlikely(&tcp_md5_needed.key)) {
struct tcp_timewait_sock *twsk = tcp_twsk(sk);

- if (twsk->tw_md5_key) {
- kfree_rcu(twsk->tw_md5_key, rcu);
- static_branch_slow_dec_deferred(&tcp_md5_needed);
- }
+ if (twsk->tw_md5_key)
+ call_rcu(&twsk->tw_md5_key->rcu, tcp_md5_twsk_free_rcu);
}
#endif
}
diff --git a/net/ipv6/tcp_ipv6.c b/net/ipv6/tcp_ipv6.c
index cb891a71db0d..f75569f889e7 100644
--- a/net/ipv6/tcp_ipv6.c
+++ b/net/ipv6/tcp_ipv6.c
@@ -64,6 +64,7 @@
#include <linux/seq_file.h>

#include <crypto/hash.h>
+#include <crypto/pool.h>
#include <linux/scatterlist.h>

#include <trace/events/tcp.h>
@@ -665,7 +666,7 @@ static int tcp_v6_parse_md5_keys(struct sock *sk, int optname,
cmd.tcpm_key, cmd.tcpm_keylen);
}

-static int tcp_v6_md5_hash_headers(struct tcp_md5sig_pool *hp,
+static int tcp_v6_md5_hash_headers(struct crypto_pool_ahash *hp,
const struct in6_addr *daddr,
const struct in6_addr *saddr,
const struct tcphdr *th, int nbytes)
@@ -674,7 +675,7 @@ static int tcp_v6_md5_hash_headers(struct tcp_md5sig_pool *hp,
struct scatterlist sg;
struct tcphdr *_th;

- bp = hp->scratch;
+ bp = hp->base.scratch;
/* 1. TCP pseudo-header (RFC2460) */
bp->saddr = *saddr;
bp->daddr = *daddr;
@@ -686,38 +687,35 @@ static int tcp_v6_md5_hash_headers(struct tcp_md5sig_pool *hp,
_th->check = 0;

sg_init_one(&sg, bp, sizeof(*bp) + sizeof(*th));
- ahash_request_set_crypt(hp->md5_req, &sg, NULL,
+ ahash_request_set_crypt(hp->req, &sg, NULL,
sizeof(*bp) + sizeof(*th));
- return crypto_ahash_update(hp->md5_req);
+ return crypto_ahash_update(hp->req);
}

static int tcp_v6_md5_hash_hdr(char *md5_hash, const struct tcp_md5sig_key *key,
const struct in6_addr *daddr, struct in6_addr *saddr,
const struct tcphdr *th)
{
- struct tcp_md5sig_pool *hp;
- struct ahash_request *req;
+ struct crypto_pool_ahash hp;

- hp = tcp_get_md5sig_pool();
- if (!hp)
+ if (crypto_pool_get(tcp_md5_crypto_pool_id, (struct crypto_pool *)&hp))
goto clear_hash_noput;
- req = hp->md5_req;

- if (crypto_ahash_init(req))
+ if (crypto_ahash_init(hp.req))
goto clear_hash;
- if (tcp_v6_md5_hash_headers(hp, daddr, saddr, th, th->doff << 2))
+ if (tcp_v6_md5_hash_headers(&hp, daddr, saddr, th, th->doff << 2))
goto clear_hash;
- if (tcp_md5_hash_key(hp, key))
+ if (tcp_md5_hash_key(&hp, key))
goto clear_hash;
- ahash_request_set_crypt(req, NULL, md5_hash, 0);
- if (crypto_ahash_final(req))
+ ahash_request_set_crypt(hp.req, NULL, md5_hash, 0);
+ if (crypto_ahash_final(hp.req))
goto clear_hash;

- tcp_put_md5sig_pool();
+ crypto_pool_put();
return 0;

clear_hash:
- tcp_put_md5sig_pool();
+ crypto_pool_put();
clear_hash_noput:
memset(md5_hash, 0, 16);
return 1;
@@ -729,8 +727,7 @@ static int tcp_v6_md5_hash_skb(char *md5_hash,
const struct sk_buff *skb)
{
const struct in6_addr *saddr, *daddr;
- struct tcp_md5sig_pool *hp;
- struct ahash_request *req;
+ struct crypto_pool_ahash hp;
const struct tcphdr *th = tcp_hdr(skb);

if (sk) { /* valid for establish/request sockets */
@@ -742,29 +739,27 @@ static int tcp_v6_md5_hash_skb(char *md5_hash,
daddr = &ip6h->daddr;
}

- hp = tcp_get_md5sig_pool();
- if (!hp)
+ if (crypto_pool_get(tcp_md5_crypto_pool_id, (struct crypto_pool *)&hp))
goto clear_hash_noput;
- req = hp->md5_req;

- if (crypto_ahash_init(req))
+ if (crypto_ahash_init(hp.req))
goto clear_hash;

- if (tcp_v6_md5_hash_headers(hp, daddr, saddr, th, skb->len))
+ if (tcp_v6_md5_hash_headers(&hp, daddr, saddr, th, skb->len))
goto clear_hash;
- if (tcp_md5_hash_skb_data(hp, skb, th->doff << 2))
+ if (tcp_md5_hash_skb_data(&hp, skb, th->doff << 2))
goto clear_hash;
- if (tcp_md5_hash_key(hp, key))
+ if (tcp_md5_hash_key(&hp, key))
goto clear_hash;
- ahash_request_set_crypt(req, NULL, md5_hash, 0);
- if (crypto_ahash_final(req))
+ ahash_request_set_crypt(hp.req, NULL, md5_hash, 0);
+ if (crypto_ahash_final(hp.req))
goto clear_hash;

- tcp_put_md5sig_pool();
+ crypto_pool_put();
return 0;

clear_hash:
- tcp_put_md5sig_pool();
+ crypto_pool_put();
clear_hash_noput:
memset(md5_hash, 0, 16);
return 1;
--
2.36.1

2022-07-27 00:19:28

by Herbert Xu

[permalink] [raw]
Subject: Re: [PATCH 0/6] net/crypto: Introduce crypto_pool

On Tue, Jul 26, 2022 at 09:15:54PM +0100, Dmitry Safonov wrote:
> Add crypto_pool - an API for allocating per-CPU array of crypto requests
> on slow-path (in sleep'able context) and to use them on a fast-path,
> which is RX/TX for net/ users (or in any other bh-disabled users).
> The design is based on the current implementations of md5sig_pool.
>
> Previously, I've suggested to add such API on TCP-AO patch submission [1],
> where Herbert kindly suggested to help with introducing new crypto API.

What I was suggesting is modifying the actual ahash interface so
that the tfm can be shared between different key users by moving
the key into the request object.

Thanks,
--
Email: Herbert Xu <[email protected]>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

2022-07-27 02:53:23

by Jakub Kicinski

[permalink] [raw]
Subject: Re: [PATCH 6/6] net/ipv6: sr: Switch to using crypto_pool

On Tue, 26 Jul 2022 21:16:00 +0100 Dmitry Safonov wrote:
> The conversion to use crypto_pool has the following upsides:
> - now SR uses asynchronous API which may potentially free CPU cycles and
> improve performance for of CPU crypto algorithm providers;
> - hash descriptors now don't have to be allocated on boot, but only at
> the moment SR starts using HMAC and until the last HMAC secret is
> deleted;
> - potentially reuse ahash_request(s) for different users
> - allocate only one per-CPU scratch buffer rather than a new one for
> each user
> - have a common API for net/ users that need ahash on RX/TX fast path

Build problems on allmodconfig:

ERROR: modpost: "crypto_pool_reserve_scratch" [net/ipv6/ipv6.ko] undefined!

2022-07-27 16:03:32

by Leonard Crestez

[permalink] [raw]
Subject: Re: [PATCH 0/6] net/crypto: Introduce crypto_pool

On 7/27/22 03:17, Herbert Xu wrote:
> On Tue, Jul 26, 2022 at 09:15:54PM +0100, Dmitry Safonov wrote:
>> Add crypto_pool - an API for allocating per-CPU array of crypto requests
>> on slow-path (in sleep'able context) and to use them on a fast-path,
>> which is RX/TX for net/ users (or in any other bh-disabled users).
>> The design is based on the current implementations of md5sig_pool.
>>
>> Previously, I've suggested to add such API on TCP-AO patch submission [1],
>> where Herbert kindly suggested to help with introducing new crypto API.
>
> What I was suggesting is modifying the actual ahash interface so
> that the tfm can be shared between different key users by moving
> the key into the request object.

The fact that setkey is implemented at the crypto_ahash instead of the
ahash_request level is baked into all algorithm implementations
(including many hardware-specific ones). Changing this seems extremely
difficult.

Supporting setkey at the tfm level could be achieved by making it an
optional capability on a per-algorithm basis, then something like
crypto_pool could detect this scenario and avoid allocating a per-cpu
tfm. This would also require a crypto_pool_setkey wrapper.

As it stands right now multiple crypto-api users needs to duplicate
logic for allocating a percpu array of transforms so adding this "pool"
API is an useful step forward.

As far as I remember the requirement for a per-cpu scratch buffer is
based on weird architectures having limitations on what kind of memory
can be passed to crypto api so this will have to remain.

--
Regards,
Leonard

2022-07-28 09:30:46

by Herbert Xu

[permalink] [raw]
Subject: Re: [PATCH 0/6] net/crypto: Introduce crypto_pool

On Wed, Jul 27, 2022 at 06:52:27PM +0300, Leonard Crestez wrote:
>
> The fact that setkey is implemented at the crypto_ahash instead of the
> ahash_request level is baked into all algorithm implementations (including
> many hardware-specific ones). Changing this seems extremely difficult.

What I had in mind is simply making the tfm setkey optional. That
way you could then have an additional setkey at the request level.

If the key is provided in either place you're allowed to perform
the hash.

This should have minimal impact on existing code.

Cheers,
--
Email: Herbert Xu <[email protected]>
Home Page: http://gondor.apana.org.au/~herbert/
PGP Key: http://gondor.apana.org.au/~herbert/pubkey.txt

2022-07-29 16:20:14

by Dmitry Safonov

[permalink] [raw]
Subject: Re: [PATCH 0/6] net/crypto: Introduce crypto_pool

Hi Herbert,

On 7/27/22 01:17, Herbert Xu wrote:
> On Tue, Jul 26, 2022 at 09:15:54PM +0100, Dmitry Safonov wrote:
>> Add crypto_pool - an API for allocating per-CPU array of crypto requests
>> on slow-path (in sleep'able context) and to use them on a fast-path,
>> which is RX/TX for net/ users (or in any other bh-disabled users).
>> The design is based on the current implementations of md5sig_pool.
>>
>> Previously, I've suggested to add such API on TCP-AO patch submission [1],
>> where Herbert kindly suggested to help with introducing new crypto API.
>
> What I was suggesting is modifying the actual ahash interface so
> that the tfm can be shared between different key users by moving
> the key into the request object.

My impression here is that we're looking at different issues.
1. The necessity of allocating per-CPU ahash_requests.
2. Managing the lifetime and sharing of ahash_request between different
kernel users.

Removing (1) will allow saving (num_possible_cpus() - 1)*(sizeof(struct
ahash_request) + crypto_ahash_reqsize(tfm)) bytes. Which would be very
nice for the new fancy CPUs with hundreds of threads.

For (2) many kernel users try manage it themselves, resulting in
different implementations, as well as some users trying to avoid using
any complication like ref counting and allocating the request only once,
without freeing it until the module is unloaded. Here for example,
introducing TCP-AO would result in copy'n'paste of tcp_md5sig_pool code.
As well as RFC5925 for TCP-AO let user to have any supported hashing
algorithms, with the requirement from RFC5926 of hmac(sha1) & aes(cmac).
If a user wants more algorithms that implementation would need to be
patched.

I see quite a few net/ users that could use some common API for this
besides TCP-MD5 and TCP-AO. That have the same pattern of allocating
crypto algorithm on a slow-path (adding a key or module initialization)
and using it of a fast-path, which is RX/TX.
Besides of sharing and lifetime managing, those users need a temporary
buffer (usually the name is `scratch'), IIUC, it is needed for async
algorithms that could use some hardware accelerator instead of CPU and
need to write the result anywhere, but on vmapped stack.

So, here I'm trying to address (2) in order to avoid copy'n'pasting of
tcp_md5sig_pool code for introduction of TCP-AO support.
I've also patched tcp-md5 code to dynamically disable the static branch,
which is not crypto change.

There's also a chance I've misunderstood what is your proposal :-)

Thanks,
Dmitry

2022-08-14 16:15:26

by kernel test robot

[permalink] [raw]
Subject: Re: [PATCH 4/6] net/tcp: Disable TCP-MD5 static key on tcp_md5sig_info destruction

Hi Dmitry,

Thank you for the patch! Perhaps something to improve:

[auto build test WARNING on 058affafc65a74cf54499fb578b66ad0b18f939b]

url: https://github.com/intel-lab-lkp/linux/commits/Dmitry-Safonov/net-crypto-Introduce-crypto_pool/20220727-041830
base: 058affafc65a74cf54499fb578b66ad0b18f939b
config: i386-randconfig-a005 (https://download.01.org/0day-ci/archive/20220814/[email protected]/config)
compiler: gcc-11 (Debian 11.3.0-3) 11.3.0
reproduce (this is a W=1 build):
# https://github.com/intel-lab-lkp/linux/commit/a4ee3ecdaada036ed6747ed86eaf7270d3f27bab
git remote add linux-review https://github.com/intel-lab-lkp/linux
git fetch --no-tags linux-review Dmitry-Safonov/net-crypto-Introduce-crypto_pool/20220727-041830
git checkout a4ee3ecdaada036ed6747ed86eaf7270d3f27bab
# save the config file
mkdir build_dir && cp config build_dir/.config
make W=1 O=build_dir ARCH=i386 SHELL=/bin/bash net/ipv4/

If you fix the issue, kindly add following tag where applicable
Reported-by: kernel test robot <[email protected]>

All warnings (new ones prefixed by >>):

>> net/ipv4/tcp_ipv4.c:1174:5: warning: no previous prototype for '__tcp_md5_do_add' [-Wmissing-prototypes]
1174 | int __tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
| ^~~~~~~~~~~~~~~~


vim +/__tcp_md5_do_add +1174 net/ipv4/tcp_ipv4.c

1172
1173 /* This can be called on a newly created socket, from other files */
> 1174 int __tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
1175 int family, u8 prefixlen, int l3index, u8 flags,
1176 const u8 *newkey, u8 newkeylen, gfp_t gfp)
1177 {
1178 /* Add Key to the list */
1179 struct tcp_md5sig_key *key;
1180 struct tcp_sock *tp = tcp_sk(sk);
1181 struct tcp_md5sig_info *md5sig;
1182
1183 key = tcp_md5_do_lookup_exact(sk, addr, family, prefixlen, l3index, flags);
1184 if (key) {
1185 /* Pre-existing entry - just update that one.
1186 * Note that the key might be used concurrently.
1187 * data_race() is telling kcsan that we do not care of
1188 * key mismatches, since changing MD5 key on live flows
1189 * can lead to packet drops.
1190 */
1191 data_race(memcpy(key->key, newkey, newkeylen));
1192
1193 /* Pairs with READ_ONCE() in tcp_md5_hash_key().
1194 * Also note that a reader could catch new key->keylen value
1195 * but old key->key[], this is the reason we use __GFP_ZERO
1196 * at sock_kmalloc() time below these lines.
1197 */
1198 WRITE_ONCE(key->keylen, newkeylen);
1199
1200 return 0;
1201 }
1202
1203 md5sig = rcu_dereference_protected(tp->md5sig_info,
1204 lockdep_sock_is_held(sk));
1205
1206 key = sock_kmalloc(sk, sizeof(*key), gfp | __GFP_ZERO);
1207 if (!key)
1208 return -ENOMEM;
1209 if (!tcp_alloc_md5sig_pool()) {
1210 sock_kfree_s(sk, key, sizeof(*key));
1211 return -ENOMEM;
1212 }
1213
1214 memcpy(key->key, newkey, newkeylen);
1215 key->keylen = newkeylen;
1216 key->family = family;
1217 key->prefixlen = prefixlen;
1218 key->l3index = l3index;
1219 key->flags = flags;
1220 memcpy(&key->addr, addr,
1221 (IS_ENABLED(CONFIG_IPV6) && family == AF_INET6) ? sizeof(struct in6_addr) :
1222 sizeof(struct in_addr));
1223 hlist_add_head_rcu(&key->node, &md5sig->head);
1224 return 0;
1225 }
1226

--
0-DAY CI Kernel Test Service
https://01.org/lkp

2022-08-14 16:15:47

by kernel test robot

[permalink] [raw]
Subject: Re: [PATCH 4/6] net/tcp: Disable TCP-MD5 static key on tcp_md5sig_info destruction

Hi Dmitry,

Thank you for the patch! Perhaps something to improve:

[auto build test WARNING on 058affafc65a74cf54499fb578b66ad0b18f939b]

url: https://github.com/intel-lab-lkp/linux/commits/Dmitry-Safonov/net-crypto-Introduce-crypto_pool/20220727-041830
base: 058affafc65a74cf54499fb578b66ad0b18f939b
config: x86_64-defconfig (https://download.01.org/0day-ci/archive/20220814/[email protected]/config)
compiler: gcc-11 (Debian 11.3.0-3) 11.3.0
reproduce (this is a W=1 build):
# https://github.com/intel-lab-lkp/linux/commit/a4ee3ecdaada036ed6747ed86eaf7270d3f27bab
git remote add linux-review https://github.com/intel-lab-lkp/linux
git fetch --no-tags linux-review Dmitry-Safonov/net-crypto-Introduce-crypto_pool/20220727-041830
git checkout a4ee3ecdaada036ed6747ed86eaf7270d3f27bab
# save the config file
mkdir build_dir && cp config build_dir/.config
make W=1 O=build_dir ARCH=x86_64 SHELL=/bin/bash net/ipv4/

If you fix the issue, kindly add following tag where applicable
Reported-by: kernel test robot <[email protected]>

All warnings (new ones prefixed by >>):

>> net/ipv4/tcp_ipv4.c:1174:5: warning: no previous prototype for '__tcp_md5_do_add' [-Wmissing-prototypes]
1174 | int __tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
| ^~~~~~~~~~~~~~~~


vim +/__tcp_md5_do_add +1174 net/ipv4/tcp_ipv4.c

1172
1173 /* This can be called on a newly created socket, from other files */
> 1174 int __tcp_md5_do_add(struct sock *sk, const union tcp_md5_addr *addr,
1175 int family, u8 prefixlen, int l3index, u8 flags,
1176 const u8 *newkey, u8 newkeylen, gfp_t gfp)
1177 {
1178 /* Add Key to the list */
1179 struct tcp_md5sig_key *key;
1180 struct tcp_sock *tp = tcp_sk(sk);
1181 struct tcp_md5sig_info *md5sig;
1182
1183 key = tcp_md5_do_lookup_exact(sk, addr, family, prefixlen, l3index, flags);
1184 if (key) {
1185 /* Pre-existing entry - just update that one.
1186 * Note that the key might be used concurrently.
1187 * data_race() is telling kcsan that we do not care of
1188 * key mismatches, since changing MD5 key on live flows
1189 * can lead to packet drops.
1190 */
1191 data_race(memcpy(key->key, newkey, newkeylen));
1192
1193 /* Pairs with READ_ONCE() in tcp_md5_hash_key().
1194 * Also note that a reader could catch new key->keylen value
1195 * but old key->key[], this is the reason we use __GFP_ZERO
1196 * at sock_kmalloc() time below these lines.
1197 */
1198 WRITE_ONCE(key->keylen, newkeylen);
1199
1200 return 0;
1201 }
1202
1203 md5sig = rcu_dereference_protected(tp->md5sig_info,
1204 lockdep_sock_is_held(sk));
1205
1206 key = sock_kmalloc(sk, sizeof(*key), gfp | __GFP_ZERO);
1207 if (!key)
1208 return -ENOMEM;
1209 if (!tcp_alloc_md5sig_pool()) {
1210 sock_kfree_s(sk, key, sizeof(*key));
1211 return -ENOMEM;
1212 }
1213
1214 memcpy(key->key, newkey, newkeylen);
1215 key->keylen = newkeylen;
1216 key->family = family;
1217 key->prefixlen = prefixlen;
1218 key->l3index = l3index;
1219 key->flags = flags;
1220 memcpy(&key->addr, addr,
1221 (IS_ENABLED(CONFIG_IPV6) && family == AF_INET6) ? sizeof(struct in6_addr) :
1222 sizeof(struct in_addr));
1223 hlist_add_head_rcu(&key->node, &md5sig->head);
1224 return 0;
1225 }
1226

--
0-DAY CI Kernel Test Service
https://01.org/lkp

2022-08-23 20:08:30

by Dmitry Safonov

[permalink] [raw]
Subject: Re: [PATCH 6/6] net/ipv6: sr: Switch to using crypto_pool

On 8/8/22 04:40, liulongfang wrote:
> On 2022/7/27 4:16, Dmitry Safonov Wrote:
>> The conversion to use crypto_pool has the following upsides:
>> - now SR uses asynchronous API which may potentially free CPU cycles and
>> improve performance for of CPU crypto algorithm providers;
>
> Is there any specific test data for this performance improvement?
> How much has the memory increased?
> How much has the performance improved?

I have no test data for this.
The idea behind this improvement is the same as behind crypto ahash: if
your CPU can benefit by offloading work to hw hash accelerator and there
is a driver that provides crypto backend, you may certainly win some
performance. The number depends on platform/driver in use.

>> - hash descriptors now don't have to be allocated on boot, but only at
>> the moment SR starts using HMAC and until the last HMAC secret is
>> deleted;
>> - potentially reuse ahash_request(s) for different users
>> - allocate only one per-CPU scratch buffer rather than a new one for
>> each user
>> - have a common API for net/ users that need ahash on RX/TX fast path

[..]

Thanks,
Dmitry