All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH v2 net-next 0/4] Add support for async socket operations
@ 2015-03-16 16:15 Tadeusz Struk
  2015-03-16 16:15 ` [PATCH v2 net-next 1/4] net: socket: add support for async operations Tadeusz Struk
                   ` (4 more replies)
  0 siblings, 5 replies; 11+ messages in thread
From: Tadeusz Struk @ 2015-03-16 16:15 UTC (permalink / raw)
  To: davem
  Cc: linux-aio, herbert, netdev, ying.xue, bcrl, viro, linux-fsdevel,
	hch, linux-crypto

After the iocb parameter has been removed from sendmsg() and recvmsg() ops
the socket layer, and the network stack no longer support async operations.
This patch set adds support for asynchronous operations on sockets back.

--
Tadeusz Struk (4):
      net: socket: add support for async operations
      aio: prefer aio_op op over inter_op
      crypto: af_alg - Allow to link sgl
      crypto: algif - change algif_skcipher to be asynchronous

Changes in v2:
* removed redundant total_size param from aio_sendmsg and aio_recvmsg functions

 crypto/af_alg.c         |   18 +++-
 crypto/algif_skcipher.c |  224 ++++++++++++++++++++++++++++++++++++++++++++++-
 fs/aio.c                |    8 +-
 include/crypto/if_alg.h |    4 +
 include/linux/net.h     |    5 +
 net/socket.c            |   63 +++++++++++++
 6 files changed, 308 insertions(+), 14 deletions(-)
-- 

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

* [PATCH v2 net-next 1/4] net: socket: add support for async operations
  2015-03-16 16:15 [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
@ 2015-03-16 16:15 ` Tadeusz Struk
  2015-03-19 16:20   ` Al Viro
  2015-03-16 16:15 ` [PATCH v2 net-next 2/4] aio: prefer aio_op op over iter_op Tadeusz Struk
                   ` (3 subsequent siblings)
  4 siblings, 1 reply; 11+ messages in thread
From: Tadeusz Struk @ 2015-03-16 16:15 UTC (permalink / raw)
  To: davem
  Cc: linux-aio, herbert, netdev, ying.xue, bcrl, viro, linux-fsdevel,
	hch, linux-crypto

Add support for async operations.

Signed-off-by: Tadeusz Struk <tadeusz.struk@intel.com>
---
 include/linux/net.h |    5 ++++
 net/socket.c        |   63 +++++++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 68 insertions(+)

diff --git a/include/linux/net.h b/include/linux/net.h
index e74114b..e9180bf 100644
--- a/include/linux/net.h
+++ b/include/linux/net.h
@@ -25,6 +25,7 @@
 #include <linux/kmemcheck.h>
 #include <linux/rcupdate.h>
 #include <linux/jump_label.h>
+#include <linux/aio.h>
 #include <uapi/linux/net.h>
 
 struct poll_table_struct;
@@ -173,6 +174,10 @@ struct proto_ops {
 	 */
 	int		(*recvmsg)   (struct socket *sock, struct msghdr *m,
 				      size_t total_len, int flags);
+	int		(*aio_sendmsg)(struct kiocb *iocb, struct socket *sock,
+				       struct msghdr *m);
+	int		(*aio_recvmsg)(struct kiocb *iocb, struct socket *sock,
+				       struct msghdr *m, int flags);
 	int		(*mmap)	     (struct file *file, struct socket *sock,
 				      struct vm_area_struct * vma);
 	ssize_t		(*sendpage)  (struct socket *sock, struct page *page,
diff --git a/net/socket.c b/net/socket.c
index 95d3085..1364dfb 100644
--- a/net/socket.c
+++ b/net/socket.c
@@ -132,6 +132,11 @@ static ssize_t sock_splice_read(struct file *file, loff_t *ppos,
 				struct pipe_inode_info *pipe, size_t len,
 				unsigned int flags);
 
+static ssize_t sock_aio_read(struct kiocb *iocb, const struct iovec *iov,
+			     unsigned long nr_segs, loff_t loff);
+static ssize_t sock_aio_write(struct kiocb *iocb, const struct iovec *iov,
+			      unsigned long nr_segs, loff_t loff);
+
 /*
  *	Socket files have a set of 'special' operations as well as the generic file ones. These don't appear
  *	in the operation structures but are done directly via the socketcall() multiplexor.
@@ -144,6 +149,8 @@ static const struct file_operations socket_file_ops = {
 	.write =	new_sync_write,
 	.read_iter =	sock_read_iter,
 	.write_iter =	sock_write_iter,
+	.aio_read =	sock_aio_read,
+	.aio_write =	sock_aio_write,
 	.poll =		sock_poll,
 	.unlocked_ioctl = sock_ioctl,
 #ifdef CONFIG_COMPAT
@@ -836,6 +843,62 @@ static ssize_t sock_write_iter(struct kiocb *iocb, struct iov_iter *from)
 	return res;
 }
 
+static ssize_t sock_aio_read(struct kiocb *iocb, const struct iovec *iov,
+			     unsigned long nr_segs, loff_t loff)
+{
+	struct file *file = iocb->ki_filp;
+	struct socket *sock = file->private_data;
+	struct iov_iter iter;
+	struct msghdr msg = {0};
+	ssize_t res;
+
+	if (file->f_flags & O_NONBLOCK)
+		msg.msg_flags = MSG_DONTWAIT;
+
+	if (iocb->ki_pos != 0)
+		return -ESPIPE;
+
+	if (iocb->ki_nbytes == 0)
+		return 0;
+
+	if (sock->ops->aio_recvmsg) {
+		iov_iter_init(&iter, READ, iov, nr_segs, iocb->ki_nbytes);
+		msg.msg_iter = iter;
+		res = sock->ops->aio_recvmsg(iocb, sock, &msg, msg.msg_flags);
+	} else {
+		res = -EOPNOTSUPP;
+	}
+	return res;
+}
+
+static ssize_t sock_aio_write(struct kiocb *iocb, const struct iovec *iov,
+			      unsigned long nr_segs, loff_t loff)
+{
+	struct file *file = iocb->ki_filp;
+	struct socket *sock = file->private_data;
+	struct iov_iter iter;
+	struct msghdr msg = {0};
+	ssize_t res;
+
+	if (iocb->ki_pos != 0)
+		return -ESPIPE;
+
+	if (file->f_flags & O_NONBLOCK)
+		msg.msg_flags = MSG_DONTWAIT;
+
+	if (sock->type == SOCK_SEQPACKET)
+		msg.msg_flags |= MSG_EOR;
+
+	if (sock->ops->aio_sendmsg) {
+		iov_iter_init(&iter, WRITE, iov, nr_segs, iocb->ki_nbytes);
+		msg.msg_iter = iter;
+		res = sock->ops->aio_sendmsg(iocb, sock, &msg);
+	} else {
+		res = -EOPNOTSUPP;
+	}
+	return res;
+}
+
 /*
  * Atomic setting of ioctl hooks to avoid race
  * with module unload.

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

* [PATCH v2 net-next 2/4] aio: prefer aio_op op over iter_op
  2015-03-16 16:15 [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
  2015-03-16 16:15 ` [PATCH v2 net-next 1/4] net: socket: add support for async operations Tadeusz Struk
@ 2015-03-16 16:15 ` Tadeusz Struk
  2015-03-19 16:22   ` Al Viro
  2015-03-16 16:15 ` [PATCH v2 net-next 3/4] crypto: af_alg - Allow to link sgl Tadeusz Struk
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 11+ messages in thread
From: Tadeusz Struk @ 2015-03-16 16:15 UTC (permalink / raw)
  To: davem
  Cc: linux-aio, herbert, netdev, ying.xue, bcrl, viro, linux-fsdevel,
	hch, linux-crypto

AIO interface should prefer AIO operations over iter_op

Signed-off-by: Tadeusz Struk <tadeusz.struk@intel.com>
---
 fs/aio.c |    8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git a/fs/aio.c b/fs/aio.c
index f8e52a1..389f4dd 100644
--- a/fs/aio.c
+++ b/fs/aio.c
@@ -1449,11 +1449,15 @@ rw_common:
 		if (rw == WRITE)
 			file_start_write(file);
 
-		if (iter_op) {
+		if (rw_op) {
+			ret = rw_op(req, iovec, nr_segs, req->ki_pos);
+		} else if (iter_op) {
 			iov_iter_init(&iter, rw, iovec, nr_segs, req->ki_nbytes);
 			ret = iter_op(req, &iter);
 		} else {
-			ret = rw_op(req, iovec, nr_segs, req->ki_pos);
+			if (iovec != inline_vecs)
+				kfree(iovec);
+			return -EINVAL;
 		}
 
 		if (rw == WRITE)

--
To unsubscribe, send a message with 'unsubscribe linux-aio' in
the body to majordomo@kvack.org.  For more info on Linux AIO,
see: http://www.kvack.org/aio/
Don't email: <a href=mailto:"aart@kvack.org">aart@kvack.org</a>

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

* [PATCH v2 net-next 3/4] crypto: af_alg - Allow to link sgl
  2015-03-16 16:15 [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
  2015-03-16 16:15 ` [PATCH v2 net-next 1/4] net: socket: add support for async operations Tadeusz Struk
  2015-03-16 16:15 ` [PATCH v2 net-next 2/4] aio: prefer aio_op op over iter_op Tadeusz Struk
@ 2015-03-16 16:15 ` Tadeusz Struk
  2015-03-16 16:15 ` [PATCH v2 net-next 4/4] crypto: algif - change algif_skcipher to be asynchronous Tadeusz Struk
  2015-03-18 20:59 ` [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
  4 siblings, 0 replies; 11+ messages in thread
From: Tadeusz Struk @ 2015-03-16 16:15 UTC (permalink / raw)
  To: davem
  Cc: linux-aio, herbert, netdev, ying.xue, bcrl, viro, linux-fsdevel,
	hch, linux-crypto

From: Tadeusz Struk <tadeusz.struk@intel.com>

Allow to link af_alg sgls.

Signed-off-by: Tadeusz Struk <tadeusz.struk@intel.com>
---
 crypto/af_alg.c         |   18 +++++++++++++-----
 include/crypto/if_alg.h |    4 +++-
 2 files changed, 16 insertions(+), 6 deletions(-)

diff --git a/crypto/af_alg.c b/crypto/af_alg.c
index 7f8b7edc..26089d1 100644
--- a/crypto/af_alg.c
+++ b/crypto/af_alg.c
@@ -358,8 +358,8 @@ int af_alg_make_sg(struct af_alg_sgl *sgl, struct iov_iter *iter, int len)
 	npages = (off + n + PAGE_SIZE - 1) >> PAGE_SHIFT;
 	if (WARN_ON(npages == 0))
 		return -EINVAL;
-
-	sg_init_table(sgl->sg, npages);
+	/* Add one extra for linking */
+	sg_init_table(sgl->sg, npages + 1);
 
 	for (i = 0, len = n; i < npages; i++) {
 		int plen = min_t(int, len, PAGE_SIZE - off);
@@ -369,18 +369,26 @@ int af_alg_make_sg(struct af_alg_sgl *sgl, struct iov_iter *iter, int len)
 		off = 0;
 		len -= plen;
 	}
+	sg_mark_end(sgl->sg + npages - 1);
+	sgl->npages = npages;
+
 	return n;
 }
 EXPORT_SYMBOL_GPL(af_alg_make_sg);
 
+void af_alg_link_sg(struct af_alg_sgl *sgl_prev, struct af_alg_sgl *sgl_new)
+{
+	sg_unmark_end(sgl_prev->sg + sgl_prev->npages - 1);
+	sg_chain(sgl_prev->sg, sgl_prev->npages + 1, sgl_new->sg);
+}
+EXPORT_SYMBOL(af_alg_link_sg);
+
 void af_alg_free_sg(struct af_alg_sgl *sgl)
 {
 	int i;
 
-	i = 0;
-	do {
+	for (i = 0; i < sgl->npages; i++)
 		put_page(sgl->pages[i]);
-	} while (!sg_is_last(sgl->sg + (i++)));
 }
 EXPORT_SYMBOL_GPL(af_alg_free_sg);
 
diff --git a/include/crypto/if_alg.h b/include/crypto/if_alg.h
index 178525e..018afb2 100644
--- a/include/crypto/if_alg.h
+++ b/include/crypto/if_alg.h
@@ -58,8 +58,9 @@ struct af_alg_type {
 };
 
 struct af_alg_sgl {
-	struct scatterlist sg[ALG_MAX_PAGES];
+	struct scatterlist sg[ALG_MAX_PAGES + 1];
 	struct page *pages[ALG_MAX_PAGES];
+	unsigned int npages;
 };
 
 int af_alg_register_type(const struct af_alg_type *type);
@@ -70,6 +71,7 @@ int af_alg_accept(struct sock *sk, struct socket *newsock);
 
 int af_alg_make_sg(struct af_alg_sgl *sgl, struct iov_iter *iter, int len);
 void af_alg_free_sg(struct af_alg_sgl *sgl);
+void af_alg_link_sg(struct af_alg_sgl *sgl_prev, struct af_alg_sgl *sgl_new);
 
 int af_alg_cmsg_send(struct msghdr *msg, struct af_alg_control *con);
 

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

* [PATCH v2 net-next 4/4] crypto: algif - change algif_skcipher to be asynchronous
  2015-03-16 16:15 [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
                   ` (2 preceding siblings ...)
  2015-03-16 16:15 ` [PATCH v2 net-next 3/4] crypto: af_alg - Allow to link sgl Tadeusz Struk
@ 2015-03-16 16:15 ` Tadeusz Struk
  2015-03-18 20:59 ` [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
  4 siblings, 0 replies; 11+ messages in thread
From: Tadeusz Struk @ 2015-03-16 16:15 UTC (permalink / raw)
  To: davem
  Cc: linux-aio, herbert, netdev, ying.xue, bcrl, viro, linux-fsdevel,
	hch, linux-crypto

From: Tadeusz Struk <tadeusz.struk@intel.com>

The way the algif_skcipher works currently is that on sendmsg/sendpage it
builds an sgl for the input data and then on read/recvmsg it sends the job
for encryption putting the user to sleep till the data is processed.
This way it can only handle one job at a given time.
This patch changes it to be asynchronous by adding AIO support.

Signed-off-by: Tadeusz Struk <tadeusz.struk@intel.com>
---
 crypto/algif_skcipher.c |  224 ++++++++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 218 insertions(+), 6 deletions(-)

diff --git a/crypto/algif_skcipher.c b/crypto/algif_skcipher.c
index b9743dc..5664c2b 100644
--- a/crypto/algif_skcipher.c
+++ b/crypto/algif_skcipher.c
@@ -39,6 +39,7 @@ struct skcipher_ctx {
 
 	struct af_alg_completion completion;
 
+	atomic_t inflight;
 	unsigned used;
 
 	unsigned int len;
@@ -49,9 +50,65 @@ struct skcipher_ctx {
 	struct ablkcipher_request req;
 };
 
+struct skcipher_async_rsgl {
+	struct af_alg_sgl sgl;
+	struct list_head list;
+};
+
+struct skcipher_async_req {
+	struct kiocb *iocb;
+	struct skcipher_async_rsgl first_sgl;
+	struct list_head list;
+	struct scatterlist *tsg;
+	char iv[];
+};
+
+#define GET_SREQ(areq, ctx) (struct skcipher_async_req *)((char *)areq + \
+	crypto_ablkcipher_reqsize(crypto_ablkcipher_reqtfm(&ctx->req)))
+
+#define GET_REQ_SIZE(ctx) \
+	crypto_ablkcipher_reqsize(crypto_ablkcipher_reqtfm(&ctx->req))
+
+#define GET_IV_SIZE(ctx) \
+	crypto_ablkcipher_ivsize(crypto_ablkcipher_reqtfm(&ctx->req))
+
 #define MAX_SGL_ENTS ((4096 - sizeof(struct skcipher_sg_list)) / \
 		      sizeof(struct scatterlist) - 1)
 
+static void skcipher_free_async_sgls(struct skcipher_async_req *sreq)
+{
+	struct skcipher_async_rsgl *rsgl, *tmp;
+	struct scatterlist *sgl;
+	struct scatterlist *sg;
+	int i, n;
+
+	list_for_each_entry_safe(rsgl, tmp, &sreq->list, list) {
+		af_alg_free_sg(&rsgl->sgl);
+		if (rsgl != &sreq->first_sgl)
+			kfree(rsgl);
+	}
+	sgl = sreq->tsg;
+	n = sg_nents(sgl);
+	for_each_sg(sgl, sg, n, i)
+		put_page(sg_page(sg));
+
+	kfree(sreq->tsg);
+}
+
+static void skcipher_async_cb(struct crypto_async_request *req, int err)
+{
+	struct sock *sk = req->data;
+	struct alg_sock *ask = alg_sk(sk);
+	struct skcipher_ctx *ctx = ask->private;
+	struct skcipher_async_req *sreq = GET_SREQ(req, ctx);
+	struct kiocb *iocb = sreq->iocb;
+
+	atomic_dec(&ctx->inflight);
+	skcipher_free_async_sgls(sreq);
+	kfree(req);
+	aio_complete(iocb, err, err);
+}
+
 static inline int skcipher_sndbuf(struct sock *sk)
 {
 	struct alg_sock *ask = alg_sk(sk);
@@ -96,7 +153,7 @@ static int skcipher_alloc_sgl(struct sock *sk)
 	return 0;
 }
 
-static void skcipher_pull_sgl(struct sock *sk, int used)
+static void skcipher_pull_sgl(struct sock *sk, int used, int put)
 {
 	struct alg_sock *ask = alg_sk(sk);
 	struct skcipher_ctx *ctx = ask->private;
@@ -123,8 +180,8 @@ static void skcipher_pull_sgl(struct sock *sk, int used)
 
 			if (sg[i].length)
 				return;
-
-			put_page(sg_page(sg + i));
+			if (put)
+				put_page(sg_page(sg + i));
 			sg_assign_page(sg + i, NULL);
 		}
 
@@ -143,7 +200,7 @@ static void skcipher_free_sgl(struct sock *sk)
 	struct alg_sock *ask = alg_sk(sk);
 	struct skcipher_ctx *ctx = ask->private;
 
-	skcipher_pull_sgl(sk, ctx->used);
+	skcipher_pull_sgl(sk, ctx->used, 1);
 }
 
 static int skcipher_wait_for_wmem(struct sock *sk, unsigned flags)
@@ -424,6 +481,147 @@ unlock:
 	return err ?: size;
 }
 
+static int skcipher_all_sg_nents(struct skcipher_ctx *ctx)
+{
+	struct skcipher_sg_list *sgl;
+	struct scatterlist *sg;
+	int nents = 0;
+
+	list_for_each_entry(sgl, &ctx->tsgl, list) {
+		sg = sgl->sg;
+
+		while (!sg->length)
+			sg++;
+
+		nents += sg_nents(sg);
+	}
+	return nents;
+}
+
+static int skcipher_recvmsg_async(struct kiocb *iocb, struct socket *sock,
+				  struct msghdr *msg, int flags)
+{
+	struct sock *sk = sock->sk;
+	struct alg_sock *ask = alg_sk(sk);
+	struct skcipher_ctx *ctx = ask->private;
+	struct skcipher_sg_list *sgl;
+	struct scatterlist *sg;
+	struct skcipher_async_req *sreq;
+	struct ablkcipher_request *req;
+	struct skcipher_async_rsgl *last_rsgl = NULL;
+	unsigned int len = 0, tx_nents = skcipher_all_sg_nents(ctx);
+	unsigned int reqlen = sizeof(struct skcipher_async_req) +
+				GET_REQ_SIZE(ctx) + GET_IV_SIZE(ctx);
+	int i = 0;
+	int err = -ENOMEM;
+
+	lock_sock(sk);
+	req = kmalloc(reqlen, GFP_KERNEL);
+	if (unlikely(!req))
+		goto unlock;
+
+	sreq = GET_SREQ(req, ctx);
+	sreq->iocb = iocb;
+	memset(&sreq->first_sgl, '\0', sizeof(struct skcipher_async_rsgl));
+	INIT_LIST_HEAD(&sreq->list);
+	sreq->tsg = kcalloc(tx_nents, sizeof(*sg), GFP_KERNEL);
+	if (unlikely(!sreq->tsg)) {
+		kfree(req);
+		goto unlock;
+	}
+	sg_init_table(sreq->tsg, tx_nents);
+	memcpy(sreq->iv, ctx->iv, GET_IV_SIZE(ctx));
+	ablkcipher_request_set_tfm(req, crypto_ablkcipher_reqtfm(&ctx->req));
+	ablkcipher_request_set_callback(req, CRYPTO_TFM_REQ_MAY_BACKLOG,
+					skcipher_async_cb, sk);
+
+	while (iov_iter_count(&msg->msg_iter)) {
+		struct skcipher_async_rsgl *rsgl;
+		unsigned long used;
+
+		if (!ctx->used) {
+			err = skcipher_wait_for_data(sk, flags);
+			if (err)
+				goto free;
+		}
+		sgl = list_first_entry(&ctx->tsgl,
+				       struct skcipher_sg_list, list);
+		sg = sgl->sg;
+
+		while (!sg->length)
+			sg++;
+
+		used = min_t(unsigned long, ctx->used,
+			     iov_iter_count(&msg->msg_iter));
+		used = min_t(unsigned long, used, sg->length);
+
+		if (i == tx_nents) {
+			struct scatterlist *tmp;
+			int x;
+			/* Ran out of tx slots in async request
+			 * need to expand */
+			tmp = kcalloc(tx_nents * 2, sizeof(*tmp),
+				      GFP_KERNEL);
+			if (!tmp)
+				goto free;
+
+			sg_init_table(tmp, tx_nents * 2);
+			for (x = 0; x < tx_nents; x++)
+				sg_set_page(&tmp[x], sg_page(&sreq->tsg[x]),
+					    sreq->tsg[x].length,
+					    sreq->tsg[x].offset);
+			kfree(sreq->tsg);
+			sreq->tsg = tmp;
+			tx_nents *= 2;
+		}
+		/* Need to take over the tx sgl from ctx
+		 * to the asynch req - these sgls will be freed later */
+		sg_set_page(sreq->tsg + i++, sg_page(sg), sg->length,
+			    sg->offset);
+
+		if (list_empty(&sreq->list)) {
+			rsgl = &sreq->first_sgl;
+			list_add_tail(&rsgl->list, &sreq->list);
+		} else {
+			rsgl = kzalloc(sizeof(*rsgl), GFP_KERNEL);
+			if (!rsgl) {
+				err = -ENOMEM;
+				goto free;
+			}
+			list_add_tail(&rsgl->list, &sreq->list);
+		}
+
+		used = af_alg_make_sg(&rsgl->sgl, &msg->msg_iter, used);
+		err = used;
+		if (used < 0)
+			goto free;
+		if (last_rsgl)
+			af_alg_link_sg(&last_rsgl->sgl, &rsgl->sgl);
+
+		last_rsgl = rsgl;
+		len += used;
+		skcipher_pull_sgl(sk, used, 0);
+		iov_iter_advance(&msg->msg_iter, used);
+	}
+
+	ablkcipher_request_set_crypt(req, sreq->tsg, sreq->first_sgl.sgl.sg,
+				     len, sreq->iv);
+	err = ctx->enc ? crypto_ablkcipher_encrypt(req) :
+			 crypto_ablkcipher_decrypt(req);
+	if (err == -EINPROGRESS) {
+		atomic_inc(&ctx->inflight);
+		err = -EIOCBQUEUED;
+		goto unlock;
+	}
+free:
+	skcipher_free_async_sgls(sreq);
+	kfree(req);
+unlock:
+	skcipher_wmem_wakeup(sk);
+	release_sock(sk);
+	return err;
+}
+
 static int skcipher_recvmsg(struct socket *sock, struct msghdr *msg,
 			    size_t ignored, int flags)
 {
@@ -484,7 +682,7 @@ free:
 			goto unlock;
 
 		copied += used;
-		skcipher_pull_sgl(sk, used);
+		skcipher_pull_sgl(sk, used, 1);
 		iov_iter_advance(&msg->msg_iter, used);
 	}
 
@@ -497,7 +695,6 @@ unlock:
 	return copied ?: err;
 }
 
-
 static unsigned int skcipher_poll(struct file *file, struct socket *sock,
 				  poll_table *wait)
 {
@@ -537,6 +734,7 @@ static struct proto_ops algif_skcipher_ops = {
 	.sendmsg	=	skcipher_sendmsg,
 	.sendpage	=	skcipher_sendpage,
 	.recvmsg	=	skcipher_recvmsg,
+	.aio_recvmsg	=	skcipher_recvmsg_async,
 	.poll		=	skcipher_poll,
 };
 
@@ -555,12 +753,25 @@ static int skcipher_setkey(void *private, const u8 *key, unsigned int keylen)
 	return crypto_ablkcipher_setkey(private, key, keylen);
 }
 
+static void skcipher_wait(struct sock *sk)
+{
+	struct alg_sock *ask = alg_sk(sk);
+	struct skcipher_ctx *ctx = ask->private;
+	int ctr = 0;
+
+	while (atomic_read(&ctx->inflight) && ctr++ < 100)
+		msleep(100);
+}
+
 static void skcipher_sock_destruct(struct sock *sk)
 {
 	struct alg_sock *ask = alg_sk(sk);
 	struct skcipher_ctx *ctx = ask->private;
 	struct crypto_ablkcipher *tfm = crypto_ablkcipher_reqtfm(&ctx->req);
 
+	if (atomic_read(&ctx->inflight))
+		skcipher_wait(sk);
+
 	skcipher_free_sgl(sk);
 	sock_kzfree_s(sk, ctx->iv, crypto_ablkcipher_ivsize(tfm));
 	sock_kfree_s(sk, ctx, ctx->len);
@@ -592,6 +803,7 @@ static int skcipher_accept_parent(void *private, struct sock *sk)
 	ctx->more = 0;
 	ctx->merge = 0;
 	ctx->enc = 0;
+	atomic_set(&ctx->inflight, 0);
 	af_alg_init_completion(&ctx->completion);
 
 	ask->private = ctx;

--
To unsubscribe, send a message with 'unsubscribe linux-aio' in
the body to majordomo@kvack.org.  For more info on Linux AIO,
see: http://www.kvack.org/aio/
Don't email: <a href=mailto:"aart@kvack.org">aart@kvack.org</a>

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

* Re: [PATCH v2 net-next 0/4] Add support for async socket operations
  2015-03-16 16:15 [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
                   ` (3 preceding siblings ...)
  2015-03-16 16:15 ` [PATCH v2 net-next 4/4] crypto: algif - change algif_skcipher to be asynchronous Tadeusz Struk
@ 2015-03-18 20:59 ` Tadeusz Struk
  2015-03-18 23:29   ` David Miller
  4 siblings, 1 reply; 11+ messages in thread
From: Tadeusz Struk @ 2015-03-18 20:59 UTC (permalink / raw)
  To: davem
  Cc: linux-aio, herbert, netdev, ying.xue, bcrl, viro, linux-fsdevel,
	hch, linux-crypto

On 03/16/2015 09:15 AM, Tadeusz Struk wrote:
> After the iocb parameter has been removed from sendmsg() and recvmsg() ops
> the socket layer, and the network stack no longer support async operations.
> This patch set adds support for asynchronous operations on sockets back.
> 
> --
> Tadeusz Struk (4):
>       net: socket: add support for async operations
>       aio: prefer aio_op op over inter_op
>       crypto: af_alg - Allow to link sgl
>       crypto: algif - change algif_skcipher to be asynchronous
> 
> Changes in v2:
> * removed redundant total_size param from aio_sendmsg and aio_recvmsg functions
> 
>  crypto/af_alg.c         |   18 +++-
>  crypto/algif_skcipher.c |  224 ++++++++++++++++++++++++++++++++++++++++++++++-
>  fs/aio.c                |    8 +-
>  include/crypto/if_alg.h |    4 +
>  include/linux/net.h     |    5 +
>  net/socket.c            |   63 +++++++++++++
>  6 files changed, 308 insertions(+), 14 deletions(-)
> 

Hi Dave,
What's your take on this patch set?
Regards,
Tadeusz

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

* Re: [PATCH v2 net-next 0/4] Add support for async socket operations
  2015-03-18 20:59 ` [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
@ 2015-03-18 23:29   ` David Miller
  0 siblings, 0 replies; 11+ messages in thread
From: David Miller @ 2015-03-18 23:29 UTC (permalink / raw)
  To: tadeusz.struk
  Cc: linux-aio, herbert, netdev, ying.xue, bcrl, viro, linux-fsdevel,
	hch, linux-crypto

From: Tadeusz Struk <tadeusz.struk@intel.com>
Date: Wed, 18 Mar 2015 13:59:30 -0700

> What's your take on this patch set?

I'm really busy so waiting for Al to give some feedback first.

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

* Re: [PATCH v2 net-next 1/4] net: socket: add support for async operations
  2015-03-16 16:15 ` [PATCH v2 net-next 1/4] net: socket: add support for async operations Tadeusz Struk
@ 2015-03-19 16:20   ` Al Viro
  2015-03-19 17:43     ` Tadeusz Struk
  0 siblings, 1 reply; 11+ messages in thread
From: Al Viro @ 2015-03-19 16:20 UTC (permalink / raw)
  To: Tadeusz Struk
  Cc: davem, linux-aio, herbert, netdev, ying.xue, bcrl, linux-fsdevel,
	hch, linux-crypto

On Mon, Mar 16, 2015 at 09:15:14AM -0700, Tadeusz Struk wrote:
> Add support for async operations.

NAK.  For the same reason as the last time - 

> +static ssize_t sock_aio_read(struct kiocb *iocb, const struct iovec *iov,
> +			     unsigned long nr_segs, loff_t loff);
> +static ssize_t sock_aio_write(struct kiocb *iocb, const struct iovec *iov,
> +			      unsigned long nr_segs, loff_t loff);
> +

is completely pointless.  Just have sock_read_iter() and sock_write_iter()
check if your new methods are present and use those if those are.

What's more, I'm not at all sure that you want to pass iocb that way -
kernel-side msghdr isn't tied to userland one anymore, so we might as well
stash a pointer to iocb into it.  Voila - no new methods needed at all.

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

* Re: [PATCH v2 net-next 2/4] aio: prefer aio_op op over iter_op
  2015-03-16 16:15 ` [PATCH v2 net-next 2/4] aio: prefer aio_op op over iter_op Tadeusz Struk
@ 2015-03-19 16:22   ` Al Viro
  0 siblings, 0 replies; 11+ messages in thread
From: Al Viro @ 2015-03-19 16:22 UTC (permalink / raw)
  To: Tadeusz Struk
  Cc: davem, linux-aio, herbert, netdev, ying.xue, bcrl, linux-fsdevel,
	hch, linux-crypto

On Mon, Mar 16, 2015 at 09:15:19AM -0700, Tadeusz Struk wrote:
> AIO interface should prefer AIO operations over iter_op

Again, NAK - the only reason you need that one is your insistence on
having separate sock_aio_{read,write}().  Doctor, it hurts when I do it...

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

* Re: [PATCH v2 net-next 1/4] net: socket: add support for async operations
  2015-03-19 16:20   ` Al Viro
@ 2015-03-19 17:43     ` Tadeusz Struk
  2015-03-19 18:13       ` Al Viro
  0 siblings, 1 reply; 11+ messages in thread
From: Tadeusz Struk @ 2015-03-19 17:43 UTC (permalink / raw)
  To: Al Viro
  Cc: davem, linux-aio, herbert, netdev, ying.xue, bcrl, linux-fsdevel,
	hch, linux-crypto

On 03/19/2015 09:20 AM, Al Viro wrote:
> is completely pointless.  Just have sock_read_iter() and sock_write_iter()
> check if your new methods are present and use those if those are.
> 

Ok, that will work for me too.

> What's more, I'm not at all sure that you want to pass iocb that way -
> kernel-side msghdr isn't tied to userland one anymore, so we might as well
> stash a pointer to iocb into it.  Voila - no new methods needed at all.

Good point, so what do you prefer - to add iocd to msghdr or to call the new
methods from sock_read_iter() and sock_write_iter()?
Either way is good for me.

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

* Re: [PATCH v2 net-next 1/4] net: socket: add support for async operations
  2015-03-19 17:43     ` Tadeusz Struk
@ 2015-03-19 18:13       ` Al Viro
  0 siblings, 0 replies; 11+ messages in thread
From: Al Viro @ 2015-03-19 18:13 UTC (permalink / raw)
  To: Tadeusz Struk
  Cc: davem, linux-aio, herbert, netdev, ying.xue, bcrl, linux-fsdevel,
	hch, linux-crypto

On Thu, Mar 19, 2015 at 10:43:16AM -0700, Tadeusz Struk wrote:
> On 03/19/2015 09:20 AM, Al Viro wrote:
> > is completely pointless.  Just have sock_read_iter() and sock_write_iter()
> > check if your new methods are present and use those if those are.
> > 
> 
> Ok, that will work for me too.
> 
> > What's more, I'm not at all sure that you want to pass iocb that way -
> > kernel-side msghdr isn't tied to userland one anymore, so we might as well
> > stash a pointer to iocb into it.  Voila - no new methods needed at all.
> 
> Good point, so what do you prefer - to add iocd to msghdr or to call the new
> methods from sock_read_iter() and sock_write_iter()?
> Either way is good for me.

I'd probably add msg_iocb to the end of struct msghdr and explicitly zero it in
copy_msghdr_from_user() and get_compat_msghdr(), but you are asking the wrong
guy - that sort of choices in net/* falls on davem, not me.

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

end of thread, other threads:[~2015-03-19 18:13 UTC | newest]

Thread overview: 11+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-03-16 16:15 [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
2015-03-16 16:15 ` [PATCH v2 net-next 1/4] net: socket: add support for async operations Tadeusz Struk
2015-03-19 16:20   ` Al Viro
2015-03-19 17:43     ` Tadeusz Struk
2015-03-19 18:13       ` Al Viro
2015-03-16 16:15 ` [PATCH v2 net-next 2/4] aio: prefer aio_op op over iter_op Tadeusz Struk
2015-03-19 16:22   ` Al Viro
2015-03-16 16:15 ` [PATCH v2 net-next 3/4] crypto: af_alg - Allow to link sgl Tadeusz Struk
2015-03-16 16:15 ` [PATCH v2 net-next 4/4] crypto: algif - change algif_skcipher to be asynchronous Tadeusz Struk
2015-03-18 20:59 ` [PATCH v2 net-next 0/4] Add support for async socket operations Tadeusz Struk
2015-03-18 23:29   ` David Miller

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.