linux-rdma.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v3 00/26] NFS/RDMA client patches for next
@ 2021-04-19 18:01 Chuck Lever
  2021-04-19 18:01 ` [PATCH v3 01/26] SUNRPC: Move fault injection call sites Chuck Lever
                   ` (25 more replies)
  0 siblings, 26 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:01 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Hi Trond-

Anna suggested I send these directly to you for review. They include
the three SUNRPC patches you've already seen and all NFS/RDMA
client-related patches I'm interested in seeing in the next kernel
release. All of these have been posted before and have been updated
with changes requested by reviewers.

---

Chuck Lever (26):
      SUNRPC: Move fault injection call sites
      SUNRPC: Remove trace_xprt_transmit_queued
      SUNRPC: Add tracepoint that fires when an RPC is retransmitted
      xprtrdma: Avoid Receive Queue wrapping
      xprtrdma: Do not refresh Receive Queue while it is draining
      xprtrdma: Put flushed Receives on free list instead of destroying them
      xprtrdma: Improve locking around rpcrdma_rep destruction
      xprtrdma: Improve commentary around rpcrdma_reps_unmap()
      xprtrdma: Improve locking around rpcrdma_rep creation
      xprtrdma: Fix cwnd update ordering
      xprtrdma: Delete rpcrdma_recv_buffer_put()
      xprtrdma: rpcrdma_mr_pop() already does list_del_init()
      xprtrdma: Rename frwr_release_mr()
      xprtrdma: Clarify use of barrier in frwr_wc_localinv_done()
      xprtrdma: Do not recycle MR after FastReg/LocalInv flushes
      xprtrdma: Do not wake RPC consumer on a failed LocalInv
      xprtrdma: Avoid Send Queue wrapping
      xprtrdma: Add tracepoints showing FastReg WRs and remote invalidation
      xprtrdma: Add an rpcrdma_mr_completion_class
      xprtrdma: Don't display r_xprt memory addresses in tracepoints
      xprtrdma: Remove the RPC/RDMA QP event handler
      xprtrdma: Move fr_cid to struct rpcrdma_mr
      xprtrdma: Move cqe to struct rpcrdma_mr
      xprtrdma: Move fr_linv_done field to struct rpcrdma_mr
      xprtrdma: Move the Work Request union to struct rpcrdma_mr
      xprtrdma: Move fr_mr field to struct rpcrdma_mr


 include/trace/events/rpcrdma.h    | 146 ++++++++++-----------
 include/trace/events/sunrpc.h     |  41 +++++-
 net/sunrpc/xprt.c                 |   6 +-
 net/sunrpc/xprtrdma/backchannel.c |   4 +-
 net/sunrpc/xprtrdma/frwr_ops.c    | 208 +++++++++++++-----------------
 net/sunrpc/xprtrdma/rpc_rdma.c    |  39 +++++-
 net/sunrpc/xprtrdma/verbs.c       | 131 +++++++++----------
 net/sunrpc/xprtrdma/xprt_rdma.h   |  29 ++---
 8 files changed, 317 insertions(+), 287 deletions(-)

--
Chuck Lever


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

* [PATCH v3 01/26] SUNRPC: Move fault injection call sites
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
@ 2021-04-19 18:01 ` Chuck Lever
  2021-04-19 18:05   ` Chuck Lever III
  2021-04-19 18:01 ` [PATCH v3 02/26] SUNRPC: Remove trace_xprt_transmit_queued Chuck Lever
                   ` (24 subsequent siblings)
  25 siblings, 1 reply; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:01 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

I've hit some crashes that occur in the xprt_rdma_inject_disconnect
path. It appears that, for some provides, rdma_disconnect() can
take so long that the transport can disconnect and release its
hardware resources while rdma_disconnect() is still running,
resulting in a UAF in the provider.

The transport's fault injection method may depend on the stability
of transport data structures. That means it needs to be invoked
only from contexts that hold the transport write lock.

Fixes: 4a0682583988 ("SUNRPC: Transport fault injection")
Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/clnt.c               |    1 -
 net/sunrpc/xprt.c               |    6 ++++--
 net/sunrpc/xprtrdma/transport.c |    6 ++++--
 3 files changed, 8 insertions(+), 5 deletions(-)

diff --git a/net/sunrpc/clnt.c b/net/sunrpc/clnt.c
index 612f0a641f4c..c2a01125be1a 100644
--- a/net/sunrpc/clnt.c
+++ b/net/sunrpc/clnt.c
@@ -1799,7 +1799,6 @@ call_allocate(struct rpc_task *task)
 
 	status = xprt->ops->buf_alloc(task);
 	trace_rpc_buf_alloc(task, status);
-	xprt_inject_disconnect(xprt);
 	if (status == 0)
 		return;
 	if (status != -ENOMEM) {
diff --git a/net/sunrpc/xprt.c b/net/sunrpc/xprt.c
index 691ccf8049a4..d616b93751d8 100644
--- a/net/sunrpc/xprt.c
+++ b/net/sunrpc/xprt.c
@@ -1483,7 +1483,10 @@ bool xprt_prepare_transmit(struct rpc_task *task)
 
 void xprt_end_transmit(struct rpc_task *task)
 {
-	xprt_release_write(task->tk_rqstp->rq_xprt, task);
+	struct rpc_xprt	*xprt = task->tk_rqstp->rq_xprt;
+
+	xprt_inject_disconnect(xprt);
+	xprt_release_write(xprt, task);
 }
 
 /**
@@ -1885,7 +1888,6 @@ void xprt_release(struct rpc_task *task)
 	spin_unlock(&xprt->transport_lock);
 	if (req->rq_buffer)
 		xprt->ops->buf_free(task);
-	xprt_inject_disconnect(xprt);
 	xdr_free_bvec(&req->rq_rcv_buf);
 	xdr_free_bvec(&req->rq_snd_buf);
 	if (req->rq_cred != NULL)
diff --git a/net/sunrpc/xprtrdma/transport.c b/net/sunrpc/xprtrdma/transport.c
index 78d29d1bcc20..09953597d055 100644
--- a/net/sunrpc/xprtrdma/transport.c
+++ b/net/sunrpc/xprtrdma/transport.c
@@ -262,8 +262,10 @@ xprt_rdma_connect_worker(struct work_struct *work)
  * xprt_rdma_inject_disconnect - inject a connection fault
  * @xprt: transport context
  *
- * If @xprt is connected, disconnect it to simulate spurious connection
- * loss.
+ * If @xprt is connected, disconnect it to simulate spurious
+ * connection loss. Caller must hold @xprt's send lock to
+ * ensure that data structures and hardware resources are
+ * stable during the rdma_disconnect() call.
  */
 static void
 xprt_rdma_inject_disconnect(struct rpc_xprt *xprt)



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

* [PATCH v3 02/26] SUNRPC: Remove trace_xprt_transmit_queued
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
  2021-04-19 18:01 ` [PATCH v3 01/26] SUNRPC: Move fault injection call sites Chuck Lever
@ 2021-04-19 18:01 ` Chuck Lever
  2021-04-19 18:01 ` [PATCH v3 03/26] SUNRPC: Add tracepoint that fires when an RPC is retransmitted Chuck Lever
                   ` (23 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:01 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

This tracepoint can crash when dereferencing snd_task because
when some transports connect, they put a cookie in that field
instead of a pointer to an rpc_task.

BUG: KASAN: use-after-free in trace_event_raw_event_xprt_writelock_event+0x141/0x18e [sunrpc]
Read of size 2 at addr ffff8881a83bd3a0 by task git/331872

CPU: 11 PID: 331872 Comm: git Tainted: G S                5.12.0-rc2-00007-g3ab6e585a7f9 #1453
Hardware name: Supermicro SYS-6028R-T/X10DRi, BIOS 1.1a 10/16/2015
Call Trace:
 dump_stack+0x9c/0xcf
 print_address_description.constprop.0+0x18/0x239
 kasan_report+0x174/0x1b0
 trace_event_raw_event_xprt_writelock_event+0x141/0x18e [sunrpc]
 xprt_prepare_transmit+0x8e/0xc1 [sunrpc]
 call_transmit+0x4d/0xc6 [sunrpc]

Fixes: 9ce07ae5eb1d ("SUNRPC: Replace dprintk() call site in xprt_prepare_transmit")
Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 include/trace/events/sunrpc.h |    1 -
 net/sunrpc/xprt.c             |    2 --
 2 files changed, 3 deletions(-)

diff --git a/include/trace/events/sunrpc.h b/include/trace/events/sunrpc.h
index 036eb1f5c133..2f01314de73a 100644
--- a/include/trace/events/sunrpc.h
+++ b/include/trace/events/sunrpc.h
@@ -1141,7 +1141,6 @@ DECLARE_EVENT_CLASS(xprt_writelock_event,
 
 DEFINE_WRITELOCK_EVENT(reserve_xprt);
 DEFINE_WRITELOCK_EVENT(release_xprt);
-DEFINE_WRITELOCK_EVENT(transmit_queued);
 
 DECLARE_EVENT_CLASS(xprt_cong_event,
 	TP_PROTO(
diff --git a/net/sunrpc/xprt.c b/net/sunrpc/xprt.c
index d616b93751d8..11ebe8a127b8 100644
--- a/net/sunrpc/xprt.c
+++ b/net/sunrpc/xprt.c
@@ -1469,8 +1469,6 @@ bool xprt_prepare_transmit(struct rpc_task *task)
 	struct rpc_xprt	*xprt = req->rq_xprt;
 
 	if (!xprt_lock_write(xprt, task)) {
-		trace_xprt_transmit_queued(xprt, task);
-
 		/* Race breaker: someone may have transmitted us */
 		if (!test_bit(RPC_TASK_NEED_XMIT, &task->tk_runstate))
 			rpc_wake_up_queued_task_set_status(&xprt->sending,



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

* [PATCH v3 03/26] SUNRPC: Add tracepoint that fires when an RPC is retransmitted
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
  2021-04-19 18:01 ` [PATCH v3 01/26] SUNRPC: Move fault injection call sites Chuck Lever
  2021-04-19 18:01 ` [PATCH v3 02/26] SUNRPC: Remove trace_xprt_transmit_queued Chuck Lever
@ 2021-04-19 18:01 ` Chuck Lever
  2021-04-19 18:02 ` [PATCH v3 04/26] xprtrdma: Avoid Receive Queue wrapping Chuck Lever
                   ` (22 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:01 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

A separate tracepoint can be left enabled all the time to capture
rare but important retransmission events. So for example:

kworker/u26:3-568   [009]   156.967933: xprt_retransmit:      task:44093@5 xid=0xa25dbc79 nfsv3 WRITE ntrans=2

Or, for example, enable all nfs and nfs4 tracepoints, and set up a
trigger to disable tracing when xprt_retransmit fires to capture
everything that leads up to it.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 include/trace/events/sunrpc.h |   40 ++++++++++++++++++++++++++++++++++++++++
 net/sunrpc/xprt.c             |    4 +++-
 2 files changed, 43 insertions(+), 1 deletion(-)

diff --git a/include/trace/events/sunrpc.h b/include/trace/events/sunrpc.h
index 2f01314de73a..3c3c4c843c62 100644
--- a/include/trace/events/sunrpc.h
+++ b/include/trace/events/sunrpc.h
@@ -1079,6 +1079,46 @@ TRACE_EVENT(xprt_transmit,
 		__entry->seqno, __entry->status)
 );
 
+TRACE_EVENT(xprt_retransmit,
+	TP_PROTO(
+		const struct rpc_rqst *rqst
+	),
+
+	TP_ARGS(rqst),
+
+	TP_STRUCT__entry(
+		__field(unsigned int, task_id)
+		__field(unsigned int, client_id)
+		__field(u32, xid)
+		__field(int, ntrans)
+		__field(int, version)
+		__string(progname,
+			 rqst->rq_task->tk_client->cl_program->name)
+		__string(procedure,
+			 rqst->rq_task->tk_msg.rpc_proc->p_name)
+	),
+
+	TP_fast_assign(
+		struct rpc_task *task = rqst->rq_task;
+
+		__entry->task_id = task->tk_pid;
+		__entry->client_id = task->tk_client ?
+			task->tk_client->cl_clid : -1;
+		__entry->xid = be32_to_cpu(rqst->rq_xid);
+		__entry->ntrans = rqst->rq_ntrans;
+		__assign_str(progname,
+			     task->tk_client->cl_program->name)
+		__entry->version = task->tk_client->cl_vers;
+		__assign_str(procedure, task->tk_msg.rpc_proc->p_name)
+	),
+
+	TP_printk(
+		"task:%u@%u xid=0x%08x %sv%d %s ntrans=%d",
+		__entry->task_id, __entry->client_id, __entry->xid,
+		__get_str(progname), __entry->version, __get_str(procedure),
+		__entry->ntrans)
+);
+
 TRACE_EVENT(xprt_ping,
 	TP_PROTO(const struct rpc_xprt *xprt, int status),
 
diff --git a/net/sunrpc/xprt.c b/net/sunrpc/xprt.c
index 11ebe8a127b8..43bc093c66b9 100644
--- a/net/sunrpc/xprt.c
+++ b/net/sunrpc/xprt.c
@@ -1538,8 +1538,10 @@ xprt_request_transmit(struct rpc_rqst *req, struct rpc_task *snd_task)
 		return status;
 	}
 
-	if (is_retrans)
+	if (is_retrans) {
 		task->tk_client->cl_stats->rpcretrans++;
+		trace_xprt_retransmit(req);
+	}
 
 	xprt_inject_disconnect(xprt);
 



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

* [PATCH v3 04/26] xprtrdma: Avoid Receive Queue wrapping
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (2 preceding siblings ...)
  2021-04-19 18:01 ` [PATCH v3 03/26] SUNRPC: Add tracepoint that fires when an RPC is retransmitted Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-19 18:02 ` [PATCH v3 05/26] xprtrdma: Do not refresh Receive Queue while it is draining Chuck Lever
                   ` (21 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Commit e340c2d6ef2a ("xprtrdma: Reduce the doorbell rate (Receive)")
increased the number of Receive WRs that are posted by the client,
but did not increase the size of the Receive Queue allocated during
transport set-up.

This is usually not an issue because RPCRDMA_BACKWARD_WRS is defined
as (32) when SUNRPC_BACKCHANNEL is defined. In cases where it isn't,
there is a real risk of Receive Queue wrapping.

Fixes: e340c2d6ef2a ("xprtrdma: Reduce the doorbell rate (Receive)")
Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
Reviewed-by: Tom Talpey <tom@talpey.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c |    1 +
 1 file changed, 1 insertion(+)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index 766a1048a48a..132df9b59ab4 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -257,6 +257,7 @@ int frwr_query_device(struct rpcrdma_ep *ep, const struct ib_device *device)
 	ep->re_attr.cap.max_send_wr += 1; /* for ib_drain_sq */
 	ep->re_attr.cap.max_recv_wr = ep->re_max_requests;
 	ep->re_attr.cap.max_recv_wr += RPCRDMA_BACKWARD_WRS;
+	ep->re_attr.cap.max_recv_wr += RPCRDMA_MAX_RECV_BATCH;
 	ep->re_attr.cap.max_recv_wr += 1; /* for ib_drain_rq */
 
 	ep->re_max_rdma_segs =



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

* [PATCH v3 05/26] xprtrdma: Do not refresh Receive Queue while it is draining
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (3 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 04/26] xprtrdma: Avoid Receive Queue wrapping Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-19 18:02 ` [PATCH v3 06/26] xprtrdma: Put flushed Receives on free list instead of destroying them Chuck Lever
                   ` (20 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Currently the Receive completion handler refreshes the Receive Queue
whenever a successful Receive completion occurs.

On disconnect, xprtrdma drains the Receive Queue. The first few
Receive completions after a disconnect are typically successful,
until the first flushed Receive.

This means the Receive completion handler continues to post more
Receive WRs after the drain sentinel has been posted. The late-
posted Receives flush after the drain sentinel has completed,
leading to a crash later in rpcrdma_xprt_disconnect().

To prevent this crash, xprtrdma has to ensure that the Receive
handler stops posting Receives before ib_drain_rq() posts its
drain sentinel.

Suggested-by: Tom Talpey <tom@talpey.com>
Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/verbs.c     |   13 +++++++++++++
 net/sunrpc/xprtrdma/xprt_rdma.h |    1 +
 2 files changed, 14 insertions(+)

diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index ec912cf9c618..d8ed69442219 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -101,6 +101,12 @@ static void rpcrdma_xprt_drain(struct rpcrdma_xprt *r_xprt)
 	struct rpcrdma_ep *ep = r_xprt->rx_ep;
 	struct rdma_cm_id *id = ep->re_id;
 
+	/* Wait for rpcrdma_post_recvs() to leave its critical
+	 * section.
+	 */
+	if (atomic_inc_return(&ep->re_receiving) > 1)
+		wait_for_completion(&ep->re_done);
+
 	/* Flush Receives, then wait for deferred Reply work
 	 * to complete.
 	 */
@@ -414,6 +420,7 @@ static int rpcrdma_ep_create(struct rpcrdma_xprt *r_xprt)
 	__module_get(THIS_MODULE);
 	device = id->device;
 	ep->re_id = id;
+	reinit_completion(&ep->re_done);
 
 	ep->re_max_requests = r_xprt->rx_xprt.max_reqs;
 	ep->re_inline_send = xprt_rdma_max_inline_write;
@@ -1385,6 +1392,9 @@ void rpcrdma_post_recvs(struct rpcrdma_xprt *r_xprt, bool temp)
 	if (!temp)
 		needed += RPCRDMA_MAX_RECV_BATCH;
 
+	if (atomic_inc_return(&ep->re_receiving) > 1)
+		goto out;
+
 	/* fast path: all needed reps can be found on the free list */
 	wr = NULL;
 	while (needed) {
@@ -1410,6 +1420,9 @@ void rpcrdma_post_recvs(struct rpcrdma_xprt *r_xprt, bool temp)
 
 	rc = ib_post_recv(ep->re_id->qp, wr,
 			  (const struct ib_recv_wr **)&bad_wr);
+	if (atomic_dec_return(&ep->re_receiving) > 0)
+		complete(&ep->re_done);
+
 out:
 	trace_xprtrdma_post_recvs(r_xprt, count, rc);
 	if (rc) {
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index fe3be985e239..31404326f29f 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -83,6 +83,7 @@ struct rpcrdma_ep {
 	unsigned int		re_max_inline_recv;
 	int			re_async_rc;
 	int			re_connect_status;
+	atomic_t		re_receiving;
 	atomic_t		re_force_disconnect;
 	struct ib_qp_init_attr	re_attr;
 	wait_queue_head_t       re_connect_wait;



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

* [PATCH v3 06/26] xprtrdma: Put flushed Receives on free list instead of destroying them
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (4 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 05/26] xprtrdma: Do not refresh Receive Queue while it is draining Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-19 18:02 ` [PATCH v3 07/26] xprtrdma: Improve locking around rpcrdma_rep destruction Chuck Lever
                   ` (19 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Defer destruction of an rpcrdma_rep until transport tear-down to
preserve the rb_all_reps list while Receives flush.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
Reviewed-by: Tom Talpey <tom@talpey.com>
---
 net/sunrpc/xprtrdma/verbs.c |    4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index d8ed69442219..1b599a623eea 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -80,6 +80,8 @@ static void rpcrdma_sendctx_put_locked(struct rpcrdma_xprt *r_xprt,
 				       struct rpcrdma_sendctx *sc);
 static int rpcrdma_reqs_setup(struct rpcrdma_xprt *r_xprt);
 static void rpcrdma_reqs_reset(struct rpcrdma_xprt *r_xprt);
+static void rpcrdma_rep_put(struct rpcrdma_buffer *buf,
+			    struct rpcrdma_rep *rep);
 static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep);
 static void rpcrdma_reps_unmap(struct rpcrdma_xprt *r_xprt);
 static void rpcrdma_mrs_create(struct rpcrdma_xprt *r_xprt);
@@ -211,7 +213,7 @@ static void rpcrdma_wc_receive(struct ib_cq *cq, struct ib_wc *wc)
 
 out_flushed:
 	rpcrdma_flush_disconnect(r_xprt, wc);
-	rpcrdma_rep_destroy(rep);
+	rpcrdma_rep_put(&r_xprt->rx_buf, rep);
 }
 
 static void rpcrdma_update_cm_private(struct rpcrdma_ep *ep,



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

* [PATCH v3 07/26] xprtrdma: Improve locking around rpcrdma_rep destruction
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (5 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 06/26] xprtrdma: Put flushed Receives on free list instead of destroying them Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-23 21:06   ` Trond Myklebust
  2021-04-19 18:02 ` [PATCH v3 08/26] xprtrdma: Improve commentary around rpcrdma_reps_unmap() Chuck Lever
                   ` (18 subsequent siblings)
  25 siblings, 1 reply; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Currently rpcrdma_reps_destroy() assumes that, at transport
tear-down, the content of the rb_free_reps list is the same as the
content of the rb_all_reps list. Although that is usually true,
using the rb_all_reps list should be more reliable because of
the way it's managed. And, rpcrdma_reps_unmap() uses rb_all_reps;
these two functions should both traverse the "all" list.

Ensure that all rpcrdma_reps are always destroyed whether they are
on the rep free list or not.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/verbs.c |   31 ++++++++++++++++++++++++-------
 1 file changed, 24 insertions(+), 7 deletions(-)

diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index 1b599a623eea..482fdc9e25c2 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -1007,16 +1007,23 @@ struct rpcrdma_rep *rpcrdma_rep_create(struct rpcrdma_xprt *r_xprt,
 	return NULL;
 }
 
-/* No locking needed here. This function is invoked only by the
- * Receive completion handler, or during transport shutdown.
- */
-static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep)
+static void rpcrdma_rep_destroy_locked(struct rpcrdma_rep *rep)
 {
-	list_del(&rep->rr_all);
 	rpcrdma_regbuf_free(rep->rr_rdmabuf);
 	kfree(rep);
 }
 
+static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep)
+{
+	struct rpcrdma_buffer *buf = &rep->rr_rxprt->rx_buf;
+
+	spin_lock(&buf->rb_lock);
+	list_del(&rep->rr_all);
+	spin_unlock(&buf->rb_lock);
+
+	rpcrdma_rep_destroy_locked(rep);
+}
+
 static struct rpcrdma_rep *rpcrdma_rep_get_locked(struct rpcrdma_buffer *buf)
 {
 	struct llist_node *node;
@@ -1049,8 +1056,18 @@ static void rpcrdma_reps_destroy(struct rpcrdma_buffer *buf)
 {
 	struct rpcrdma_rep *rep;
 
-	while ((rep = rpcrdma_rep_get_locked(buf)) != NULL)
-		rpcrdma_rep_destroy(rep);
+	spin_lock(&buf->rb_lock);
+	while ((rep = list_first_entry_or_null(&buf->rb_all_reps,
+					       struct rpcrdma_rep,
+					       rr_all)) != NULL) {
+		list_del(&rep->rr_all);
+		spin_unlock(&buf->rb_lock);
+
+		rpcrdma_rep_destroy_locked(rep);
+
+		spin_lock(&buf->rb_lock);
+	}
+	spin_unlock(&buf->rb_lock);
 }
 
 /**



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

* [PATCH v3 08/26] xprtrdma: Improve commentary around rpcrdma_reps_unmap()
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (6 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 07/26] xprtrdma: Improve locking around rpcrdma_rep destruction Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-19 18:02 ` [PATCH v3 09/26] xprtrdma: Improve locking around rpcrdma_rep creation Chuck Lever
                   ` (17 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/verbs.c |    6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index 482fdc9e25c2..f3482fd67ec2 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -1041,6 +1041,10 @@ static void rpcrdma_rep_put(struct rpcrdma_buffer *buf,
 	llist_add(&rep->rr_node, &buf->rb_free_reps);
 }
 
+/* Caller must ensure the QP is quiescent (RQ is drained) before
+ * invoking this function, to guarantee rb_all_reps is not
+ * changing.
+ */
 static void rpcrdma_reps_unmap(struct rpcrdma_xprt *r_xprt)
 {
 	struct rpcrdma_buffer *buf = &r_xprt->rx_buf;
@@ -1048,7 +1052,7 @@ static void rpcrdma_reps_unmap(struct rpcrdma_xprt *r_xprt)
 
 	list_for_each_entry(rep, &buf->rb_all_reps, rr_all) {
 		rpcrdma_regbuf_dma_unmap(rep->rr_rdmabuf);
-		rep->rr_temp = true;
+		rep->rr_temp = true;	/* Mark this rep for destruction */
 	}
 }
 



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

* [PATCH v3 09/26] xprtrdma: Improve locking around rpcrdma_rep creation
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (7 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 08/26] xprtrdma: Improve commentary around rpcrdma_reps_unmap() Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-19 18:02 ` [PATCH v3 10/26] xprtrdma: Fix cwnd update ordering Chuck Lever
                   ` (16 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Defensive clean up: Protect the rb_all_reps list during rep
creation.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/verbs.c |    9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)

diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index f3482fd67ec2..0ade69501061 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -963,13 +963,11 @@ static void rpcrdma_reqs_reset(struct rpcrdma_xprt *r_xprt)
 		rpcrdma_req_reset(req);
 }
 
-/* No locking needed here. This function is called only by the
- * Receive completion handler.
- */
 static noinline
 struct rpcrdma_rep *rpcrdma_rep_create(struct rpcrdma_xprt *r_xprt,
 				       bool temp)
 {
+	struct rpcrdma_buffer *buf = &r_xprt->rx_buf;
 	struct rpcrdma_rep *rep;
 
 	rep = kzalloc(sizeof(*rep), GFP_KERNEL);
@@ -996,7 +994,10 @@ struct rpcrdma_rep *rpcrdma_rep_create(struct rpcrdma_xprt *r_xprt,
 	rep->rr_recv_wr.sg_list = &rep->rr_rdmabuf->rg_iov;
 	rep->rr_recv_wr.num_sge = 1;
 	rep->rr_temp = temp;
-	list_add(&rep->rr_all, &r_xprt->rx_buf.rb_all_reps);
+
+	spin_lock(&buf->rb_lock);
+	list_add(&rep->rr_all, &buf->rb_all_reps);
+	spin_unlock(&buf->rb_lock);
 	return rep;
 
 out_free_regbuf:



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

* [PATCH v3 10/26] xprtrdma: Fix cwnd update ordering
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (8 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 09/26] xprtrdma: Improve locking around rpcrdma_rep creation Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-19 18:02 ` [PATCH v3 11/26] xprtrdma: Delete rpcrdma_recv_buffer_put() Chuck Lever
                   ` (15 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

After a reconnect, the reply handler is opening the cwnd (and thus
enabling more RPC Calls to be sent) /before/ rpcrdma_post_recvs()
can post enough Receive WRs to receive their replies. This causes an
RNR and the new connection is lost immediately.

The race is most clearly exposed when KASAN and disconnect injection
are enabled. This slows down rpcrdma_rep_create() enough to allow
the send side to post a bunch of RPC Calls before the Receive
completion handler can invoke ib_post_recv().

Fixes: 2ae50ad68cd7 ("xprtrdma: Close window between waking RPC senders and posting Receives")
Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/rpc_rdma.c  |    3 ++-
 net/sunrpc/xprtrdma/verbs.c     |   10 +++++-----
 net/sunrpc/xprtrdma/xprt_rdma.h |    2 +-
 3 files changed, 8 insertions(+), 7 deletions(-)

diff --git a/net/sunrpc/xprtrdma/rpc_rdma.c b/net/sunrpc/xprtrdma/rpc_rdma.c
index 292f066d006e..21ddd78a8c35 100644
--- a/net/sunrpc/xprtrdma/rpc_rdma.c
+++ b/net/sunrpc/xprtrdma/rpc_rdma.c
@@ -1430,9 +1430,10 @@ void rpcrdma_reply_handler(struct rpcrdma_rep *rep)
 		credits = 1;	/* don't deadlock */
 	else if (credits > r_xprt->rx_ep->re_max_requests)
 		credits = r_xprt->rx_ep->re_max_requests;
+	rpcrdma_post_recvs(r_xprt, credits + (buf->rb_bc_srv_max_requests << 1),
+			   false);
 	if (buf->rb_credits != credits)
 		rpcrdma_update_cwnd(r_xprt, credits);
-	rpcrdma_post_recvs(r_xprt, false);
 
 	req = rpcr_to_rdmar(rqst);
 	if (unlikely(req->rl_reply))
diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index 0ade69501061..5a2871c4561f 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -544,7 +544,7 @@ int rpcrdma_xprt_connect(struct rpcrdma_xprt *r_xprt)
 	 * outstanding Receives.
 	 */
 	rpcrdma_ep_get(ep);
-	rpcrdma_post_recvs(r_xprt, true);
+	rpcrdma_post_recvs(r_xprt, 1, true);
 
 	rc = rdma_connect(ep->re_id, &ep->re_remote_cma);
 	if (rc)
@@ -1395,21 +1395,21 @@ int rpcrdma_post_sends(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 /**
  * rpcrdma_post_recvs - Refill the Receive Queue
  * @r_xprt: controlling transport instance
- * @temp: mark Receive buffers to be deleted after use
+ * @needed: current credit grant
+ * @temp: mark Receive buffers to be deleted after one use
  *
  */
-void rpcrdma_post_recvs(struct rpcrdma_xprt *r_xprt, bool temp)
+void rpcrdma_post_recvs(struct rpcrdma_xprt *r_xprt, int needed, bool temp)
 {
 	struct rpcrdma_buffer *buf = &r_xprt->rx_buf;
 	struct rpcrdma_ep *ep = r_xprt->rx_ep;
 	struct ib_recv_wr *wr, *bad_wr;
 	struct rpcrdma_rep *rep;
-	int needed, count, rc;
+	int count, rc;
 
 	rc = 0;
 	count = 0;
 
-	needed = buf->rb_credits + (buf->rb_bc_srv_max_requests << 1);
 	if (likely(ep->re_receive_count > needed))
 		goto out;
 	needed -= ep->re_receive_count;
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index 31404326f29f..2504f67af63e 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -462,7 +462,7 @@ int rpcrdma_xprt_connect(struct rpcrdma_xprt *r_xprt);
 void rpcrdma_xprt_disconnect(struct rpcrdma_xprt *r_xprt);
 
 int rpcrdma_post_sends(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req);
-void rpcrdma_post_recvs(struct rpcrdma_xprt *r_xprt, bool temp);
+void rpcrdma_post_recvs(struct rpcrdma_xprt *r_xprt, int needed, bool temp);
 
 /*
  * Buffer calls - xprtrdma/verbs.c



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

* [PATCH v3 11/26] xprtrdma: Delete rpcrdma_recv_buffer_put()
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (9 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 10/26] xprtrdma: Fix cwnd update ordering Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-19 18:02 ` [PATCH v3 12/26] xprtrdma: rpcrdma_mr_pop() already does list_del_init() Chuck Lever
                   ` (14 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up: The name recv_buffer_put() is a vestige of older code,
and the function is just a wrapper for the newer rpcrdma_rep_put().
In most of the existing call sites, a pointer to the owning
rpcrdma_buffer is already available.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/backchannel.c |    4 +++-
 net/sunrpc/xprtrdma/rpc_rdma.c    |    4 ++--
 net/sunrpc/xprtrdma/verbs.c       |   24 ++++++++----------------
 net/sunrpc/xprtrdma/xprt_rdma.h   |    2 +-
 4 files changed, 14 insertions(+), 20 deletions(-)

diff --git a/net/sunrpc/xprtrdma/backchannel.c b/net/sunrpc/xprtrdma/backchannel.c
index a249837d6a55..1151efd09b27 100644
--- a/net/sunrpc/xprtrdma/backchannel.c
+++ b/net/sunrpc/xprtrdma/backchannel.c
@@ -155,9 +155,11 @@ void xprt_rdma_bc_destroy(struct rpc_xprt *xprt, unsigned int reqs)
 void xprt_rdma_bc_free_rqst(struct rpc_rqst *rqst)
 {
 	struct rpcrdma_req *req = rpcr_to_rdmar(rqst);
+	struct rpcrdma_rep *rep = req->rl_reply;
 	struct rpc_xprt *xprt = rqst->rq_xprt;
+	struct rpcrdma_xprt *r_xprt = rpcx_to_rdmax(xprt);
 
-	rpcrdma_recv_buffer_put(req->rl_reply);
+	rpcrdma_rep_put(&r_xprt->rx_buf, rep);
 	req->rl_reply = NULL;
 
 	spin_lock(&xprt->bc_pa_lock);
diff --git a/net/sunrpc/xprtrdma/rpc_rdma.c b/net/sunrpc/xprtrdma/rpc_rdma.c
index 21ddd78a8c35..be4e888e78a3 100644
--- a/net/sunrpc/xprtrdma/rpc_rdma.c
+++ b/net/sunrpc/xprtrdma/rpc_rdma.c
@@ -1437,7 +1437,7 @@ void rpcrdma_reply_handler(struct rpcrdma_rep *rep)
 
 	req = rpcr_to_rdmar(rqst);
 	if (unlikely(req->rl_reply))
-		rpcrdma_recv_buffer_put(req->rl_reply);
+		rpcrdma_rep_put(buf, req->rl_reply);
 	req->rl_reply = rep;
 	rep->rr_rqst = rqst;
 
@@ -1465,5 +1465,5 @@ void rpcrdma_reply_handler(struct rpcrdma_rep *rep)
 	trace_xprtrdma_reply_short_err(rep);
 
 out:
-	rpcrdma_recv_buffer_put(rep);
+	rpcrdma_rep_put(buf, rep);
 }
diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index 5a2871c4561f..098d5c550e9b 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -80,8 +80,6 @@ static void rpcrdma_sendctx_put_locked(struct rpcrdma_xprt *r_xprt,
 				       struct rpcrdma_sendctx *sc);
 static int rpcrdma_reqs_setup(struct rpcrdma_xprt *r_xprt);
 static void rpcrdma_reqs_reset(struct rpcrdma_xprt *r_xprt);
-static void rpcrdma_rep_put(struct rpcrdma_buffer *buf,
-			    struct rpcrdma_rep *rep);
 static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep);
 static void rpcrdma_reps_unmap(struct rpcrdma_xprt *r_xprt);
 static void rpcrdma_mrs_create(struct rpcrdma_xprt *r_xprt);
@@ -1036,8 +1034,13 @@ static struct rpcrdma_rep *rpcrdma_rep_get_locked(struct rpcrdma_buffer *buf)
 	return llist_entry(node, struct rpcrdma_rep, rr_node);
 }
 
-static void rpcrdma_rep_put(struct rpcrdma_buffer *buf,
-			    struct rpcrdma_rep *rep)
+/**
+ * rpcrdma_rep_put - Release rpcrdma_rep back to free list
+ * @buf: buffer pool
+ * @rep: rep to release
+ *
+ */
+void rpcrdma_rep_put(struct rpcrdma_buffer *buf, struct rpcrdma_rep *rep)
 {
 	llist_add(&rep->rr_node, &buf->rb_free_reps);
 }
@@ -1252,17 +1255,6 @@ void rpcrdma_buffer_put(struct rpcrdma_buffer *buffers, struct rpcrdma_req *req)
 	spin_unlock(&buffers->rb_lock);
 }
 
-/**
- * rpcrdma_recv_buffer_put - Release rpcrdma_rep back to free list
- * @rep: rep to release
- *
- * Used after error conditions.
- */
-void rpcrdma_recv_buffer_put(struct rpcrdma_rep *rep)
-{
-	rpcrdma_rep_put(&rep->rr_rxprt->rx_buf, rep);
-}
-
 /* Returns a pointer to a rpcrdma_regbuf object, or NULL.
  *
  * xprtrdma uses a regbuf for posting an outgoing RDMA SEND, or for
@@ -1455,7 +1447,7 @@ void rpcrdma_post_recvs(struct rpcrdma_xprt *r_xprt, int needed, bool temp)
 
 			rep = container_of(wr, struct rpcrdma_rep, rr_recv_wr);
 			wr = wr->next;
-			rpcrdma_recv_buffer_put(rep);
+			rpcrdma_rep_put(buf, rep);
 			--count;
 		}
 	}
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index 2504f67af63e..9c5039d4634a 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -481,7 +481,7 @@ void rpcrdma_mrs_refresh(struct rpcrdma_xprt *r_xprt);
 struct rpcrdma_req *rpcrdma_buffer_get(struct rpcrdma_buffer *);
 void rpcrdma_buffer_put(struct rpcrdma_buffer *buffers,
 			struct rpcrdma_req *req);
-void rpcrdma_recv_buffer_put(struct rpcrdma_rep *);
+void rpcrdma_rep_put(struct rpcrdma_buffer *buf, struct rpcrdma_rep *rep);
 
 bool rpcrdma_regbuf_realloc(struct rpcrdma_regbuf *rb, size_t size,
 			    gfp_t flags);



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

* [PATCH v3 12/26] xprtrdma: rpcrdma_mr_pop() already does list_del_init()
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (10 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 11/26] xprtrdma: Delete rpcrdma_recv_buffer_put() Chuck Lever
@ 2021-04-19 18:02 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 13/26] xprtrdma: Rename frwr_release_mr() Chuck Lever
                   ` (13 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:02 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

The rpcrdma_mr_pop() earlier in the function has already cleared
out mr_list, so it must not be done again in the error path.

Fixes: 847568942f93 ("xprtrdma: Remove fr_state")
Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c |    1 -
 1 file changed, 1 deletion(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index 132df9b59ab4..aca2228095db 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -576,7 +576,6 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 		mr = container_of(frwr, struct rpcrdma_mr, frwr);
 		bad_wr = bad_wr->next;
 
-		list_del_init(&mr->mr_list);
 		frwr_mr_recycle(mr);
 	}
 }



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

* [PATCH v3 13/26] xprtrdma: Rename frwr_release_mr()
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (11 preceding siblings ...)
  2021-04-19 18:02 ` [PATCH v3 12/26] xprtrdma: rpcrdma_mr_pop() already does list_del_init() Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 14/26] xprtrdma: Clarify use of barrier in frwr_wc_localinv_done() Chuck Lever
                   ` (12 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up: To be consistent with other functions in this source file,
follow the naming convention of putting the object being acted upon
before the action itself.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c  |    6 +++---
 net/sunrpc/xprtrdma/verbs.c     |    4 ++--
 net/sunrpc/xprtrdma/xprt_rdma.h |    2 +-
 3 files changed, 6 insertions(+), 6 deletions(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index aca2228095db..bfc057fdb75f 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -50,11 +50,11 @@
 #endif
 
 /**
- * frwr_release_mr - Destroy one MR
+ * frwr_mr_release - Destroy one MR
  * @mr: MR allocated by frwr_mr_init
  *
  */
-void frwr_release_mr(struct rpcrdma_mr *mr)
+void frwr_mr_release(struct rpcrdma_mr *mr)
 {
 	int rc;
 
@@ -88,7 +88,7 @@ static void frwr_mr_recycle(struct rpcrdma_mr *mr)
 	r_xprt->rx_stats.mrs_recycled++;
 	spin_unlock(&r_xprt->rx_buf.rb_lock);
 
-	frwr_release_mr(mr);
+	frwr_mr_release(mr);
 }
 
 static void frwr_mr_put(struct rpcrdma_mr *mr)
diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index 098d5c550e9b..d4e573eef416 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -1138,7 +1138,7 @@ void rpcrdma_req_destroy(struct rpcrdma_req *req)
 		list_del(&mr->mr_all);
 		spin_unlock(&buf->rb_lock);
 
-		frwr_release_mr(mr);
+		frwr_mr_release(mr);
 	}
 
 	rpcrdma_regbuf_free(req->rl_recvbuf);
@@ -1169,7 +1169,7 @@ static void rpcrdma_mrs_destroy(struct rpcrdma_xprt *r_xprt)
 		list_del(&mr->mr_all);
 		spin_unlock(&buf->rb_lock);
 
-		frwr_release_mr(mr);
+		frwr_mr_release(mr);
 
 		spin_lock(&buf->rb_lock);
 	}
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index 9c5039d4634a..1b187d1dee8a 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -528,7 +528,7 @@ rpcrdma_data_dir(bool writing)
 void frwr_reset(struct rpcrdma_req *req);
 int frwr_query_device(struct rpcrdma_ep *ep, const struct ib_device *device);
 int frwr_mr_init(struct rpcrdma_xprt *r_xprt, struct rpcrdma_mr *mr);
-void frwr_release_mr(struct rpcrdma_mr *mr);
+void frwr_mr_release(struct rpcrdma_mr *mr);
 struct rpcrdma_mr_seg *frwr_map(struct rpcrdma_xprt *r_xprt,
 				struct rpcrdma_mr_seg *seg,
 				int nsegs, bool writing, __be32 xid,



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

* [PATCH v3 14/26] xprtrdma: Clarify use of barrier in frwr_wc_localinv_done()
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (12 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 13/26] xprtrdma: Rename frwr_release_mr() Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 15/26] xprtrdma: Do not recycle MR after FastReg/LocalInv flushes Chuck Lever
                   ` (11 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up: The comment and the placement of the memory barrier is
confusing. Humans want to read the function statements from head
to tail.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c |    8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index bfc057fdb75f..af85cec0ce31 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -592,14 +592,16 @@ static void frwr_wc_localinv_done(struct ib_cq *cq, struct ib_wc *wc)
 	struct rpcrdma_frwr *frwr =
 		container_of(cqe, struct rpcrdma_frwr, fr_cqe);
 	struct rpcrdma_mr *mr = container_of(frwr, struct rpcrdma_mr, frwr);
-	struct rpcrdma_rep *rep = mr->mr_req->rl_reply;
+	struct rpcrdma_rep *rep;
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
 	trace_xprtrdma_wc_li_done(wc, &frwr->fr_cid);
-	frwr_mr_done(wc, mr);
 
-	/* Ensure @rep is generated before frwr_mr_done */
+	/* Ensure that @rep is generated before the MR is released */
+	rep = mr->mr_req->rl_reply;
 	smp_rmb();
+
+	frwr_mr_done(wc, mr);
 	rpcrdma_complete_rqst(rep);
 
 	rpcrdma_flush_disconnect(cq->cq_context, wc);



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

* [PATCH v3 15/26] xprtrdma: Do not recycle MR after FastReg/LocalInv flushes
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (13 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 14/26] xprtrdma: Clarify use of barrier in frwr_wc_localinv_done() Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-25 14:19   ` Dan Aloni
  2021-04-19 18:03 ` [PATCH v3 16/26] xprtrdma: Do not wake RPC consumer on a failed LocalInv Chuck Lever
                   ` (10 subsequent siblings)
  25 siblings, 1 reply; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Better not to touch MRs involved in a flush or post error until the
Send and Receive Queues are drained and the transport is fully
quiescent. Simply don't insert such MRs back onto the free list.
They remain on mr_all and will be released when the connection is
torn down.

I had thought that recycling would prevent hardware resources from
being tied up for a long time. However, since v5.7, a transport
disconnect destroys the QP and other hardware-owned resources. The
MRs get cleaned up nicely at that point.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 include/trace/events/rpcrdma.h |    1 -
 net/sunrpc/xprtrdma/frwr_ops.c |   69 ++++++++++------------------------------
 2 files changed, 17 insertions(+), 53 deletions(-)

diff --git a/include/trace/events/rpcrdma.h b/include/trace/events/rpcrdma.h
index c838e7ac1c2d..e38e745d13b0 100644
--- a/include/trace/events/rpcrdma.h
+++ b/include/trace/events/rpcrdma.h
@@ -1014,7 +1014,6 @@ DEFINE_MR_EVENT(localinv);
 DEFINE_MR_EVENT(map);
 
 DEFINE_ANON_MR_EVENT(unmap);
-DEFINE_ANON_MR_EVENT(recycle);
 
 TRACE_EVENT(xprtrdma_dma_maperr,
 	TP_PROTO(
diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index af85cec0ce31..27087dc8ba3c 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -49,6 +49,16 @@
 # define RPCDBG_FACILITY	RPCDBG_TRANS
 #endif
 
+static void frwr_mr_unmap(struct rpcrdma_xprt *r_xprt, struct rpcrdma_mr *mr)
+{
+	if (mr->mr_device) {
+		trace_xprtrdma_mr_unmap(mr);
+		ib_dma_unmap_sg(mr->mr_device, mr->mr_sg, mr->mr_nents,
+				mr->mr_dir);
+		mr->mr_device = NULL;
+	}
+}
+
 /**
  * frwr_mr_release - Destroy one MR
  * @mr: MR allocated by frwr_mr_init
@@ -58,6 +68,8 @@ void frwr_mr_release(struct rpcrdma_mr *mr)
 {
 	int rc;
 
+	frwr_mr_unmap(mr->mr_xprt, mr);
+
 	rc = ib_dereg_mr(mr->frwr.fr_mr);
 	if (rc)
 		trace_xprtrdma_frwr_dereg(mr, rc);
@@ -65,32 +77,6 @@ void frwr_mr_release(struct rpcrdma_mr *mr)
 	kfree(mr);
 }
 
-static void frwr_mr_unmap(struct rpcrdma_xprt *r_xprt, struct rpcrdma_mr *mr)
-{
-	if (mr->mr_device) {
-		trace_xprtrdma_mr_unmap(mr);
-		ib_dma_unmap_sg(mr->mr_device, mr->mr_sg, mr->mr_nents,
-				mr->mr_dir);
-		mr->mr_device = NULL;
-	}
-}
-
-static void frwr_mr_recycle(struct rpcrdma_mr *mr)
-{
-	struct rpcrdma_xprt *r_xprt = mr->mr_xprt;
-
-	trace_xprtrdma_mr_recycle(mr);
-
-	frwr_mr_unmap(r_xprt, mr);
-
-	spin_lock(&r_xprt->rx_buf.rb_lock);
-	list_del(&mr->mr_all);
-	r_xprt->rx_stats.mrs_recycled++;
-	spin_unlock(&r_xprt->rx_buf.rb_lock);
-
-	frwr_mr_release(mr);
-}
-
 static void frwr_mr_put(struct rpcrdma_mr *mr)
 {
 	frwr_mr_unmap(mr->mr_xprt, mr);
@@ -365,6 +351,7 @@ struct rpcrdma_mr_seg *frwr_map(struct rpcrdma_xprt *r_xprt,
  * @cq: completion queue
  * @wc: WCE for a completed FastReg WR
  *
+ * Each flushed MR gets destroyed after the QP has drained.
  */
 static void frwr_wc_fastreg(struct ib_cq *cq, struct ib_wc *wc)
 {
@@ -374,7 +361,6 @@ static void frwr_wc_fastreg(struct ib_cq *cq, struct ib_wc *wc)
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
 	trace_xprtrdma_wc_fastreg(wc, &frwr->fr_cid);
-	/* The MR will get recycled when the associated req is retransmitted */
 
 	rpcrdma_flush_disconnect(cq->cq_context, wc);
 }
@@ -448,9 +434,7 @@ void frwr_reminv(struct rpcrdma_rep *rep, struct list_head *mrs)
 
 static void frwr_mr_done(struct ib_wc *wc, struct rpcrdma_mr *mr)
 {
-	if (wc->status != IB_WC_SUCCESS)
-		frwr_mr_recycle(mr);
-	else
+	if (likely(wc->status == IB_WC_SUCCESS))
 		frwr_mr_put(mr);
 }
 
@@ -567,17 +551,8 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	if (!rc)
 		return;
 
-	/* Recycle MRs in the LOCAL_INV chain that did not get posted.
-	 */
+	/* On error, the MRs get destroyed once the QP has drained. */
 	trace_xprtrdma_post_linv_err(req, rc);
-	while (bad_wr) {
-		frwr = container_of(bad_wr, struct rpcrdma_frwr,
-				    fr_invwr);
-		mr = container_of(frwr, struct rpcrdma_mr, frwr);
-		bad_wr = bad_wr->next;
-
-		frwr_mr_recycle(mr);
-	}
 }
 
 /**
@@ -621,7 +596,6 @@ void frwr_unmap_async(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 {
 	struct ib_send_wr *first, *last, **prev;
 	struct rpcrdma_ep *ep = r_xprt->rx_ep;
-	const struct ib_send_wr *bad_wr;
 	struct rpcrdma_frwr *frwr;
 	struct rpcrdma_mr *mr;
 	int rc;
@@ -663,21 +637,12 @@ void frwr_unmap_async(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	 * replaces the QP. The RPC reply handler won't call us
 	 * unless re_id->qp is a valid pointer.
 	 */
-	bad_wr = NULL;
-	rc = ib_post_send(ep->re_id->qp, first, &bad_wr);
+	rc = ib_post_send(ep->re_id->qp, first, NULL);
 	if (!rc)
 		return;
 
-	/* Recycle MRs in the LOCAL_INV chain that did not get posted.
-	 */
+	/* On error, the MRs get destroyed once the QP has drained. */
 	trace_xprtrdma_post_linv_err(req, rc);
-	while (bad_wr) {
-		frwr = container_of(bad_wr, struct rpcrdma_frwr, fr_invwr);
-		mr = container_of(frwr, struct rpcrdma_mr, frwr);
-		bad_wr = bad_wr->next;
-
-		frwr_mr_recycle(mr);
-	}
 
 	/* The final LOCAL_INV WR in the chain is supposed to
 	 * do the wake. If it was never posted, the wake will



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

* [PATCH v3 16/26] xprtrdma: Do not wake RPC consumer on a failed LocalInv
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (14 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 15/26] xprtrdma: Do not recycle MR after FastReg/LocalInv flushes Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 17/26] xprtrdma: Avoid Send Queue wrapping Chuck Lever
                   ` (9 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Throw away any reply where the LocalInv flushes or could not be
posted. The registered memory region is in an unknown state until
the disconnect completes.

rpcrdma_xprt_disconnect() will find and release the MR. No need to
put it back on the MR free list in this case.

The client retransmits pending RPC requests once it reestablishes a
fresh connection, so a replacement reply should be forthcoming on
the next connection instance.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c  |   17 +++++++++++------
 net/sunrpc/xprtrdma/rpc_rdma.c  |   32 +++++++++++++++++++++++++++++---
 net/sunrpc/xprtrdma/xprt_rdma.h |    1 +
 3 files changed, 41 insertions(+), 9 deletions(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index 27087dc8ba3c..951ae20485f3 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -576,10 +576,14 @@ static void frwr_wc_localinv_done(struct ib_cq *cq, struct ib_wc *wc)
 	rep = mr->mr_req->rl_reply;
 	smp_rmb();
 
-	frwr_mr_done(wc, mr);
+	if (wc->status != IB_WC_SUCCESS) {
+		if (rep)
+			rpcrdma_unpin_rqst(rep);
+		rpcrdma_flush_disconnect(cq->cq_context, wc);
+		return;
+	}
+	frwr_mr_put(mr);
 	rpcrdma_complete_rqst(rep);
-
-	rpcrdma_flush_disconnect(cq->cq_context, wc);
 }
 
 /**
@@ -645,8 +649,9 @@ void frwr_unmap_async(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	trace_xprtrdma_post_linv_err(req, rc);
 
 	/* The final LOCAL_INV WR in the chain is supposed to
-	 * do the wake. If it was never posted, the wake will
-	 * not happen, so wake here in that case.
+	 * do the wake. If it was never posted, the wake does
+	 * not happen. Unpin the rqst in preparation for its
+	 * retransmission.
 	 */
-	rpcrdma_complete_rqst(req->rl_reply);
+	rpcrdma_unpin_rqst(req->rl_reply);
 }
diff --git a/net/sunrpc/xprtrdma/rpc_rdma.c b/net/sunrpc/xprtrdma/rpc_rdma.c
index be4e888e78a3..649f7d8b9733 100644
--- a/net/sunrpc/xprtrdma/rpc_rdma.c
+++ b/net/sunrpc/xprtrdma/rpc_rdma.c
@@ -1326,9 +1326,35 @@ rpcrdma_decode_error(struct rpcrdma_xprt *r_xprt, struct rpcrdma_rep *rep,
 	return -EIO;
 }
 
-/* Perform XID lookup, reconstruction of the RPC reply, and
- * RPC completion while holding the transport lock to ensure
- * the rep, rqst, and rq_task pointers remain stable.
+/**
+ * rpcrdma_unpin_rqst - Release rqst without completing it
+ * @rep: RPC/RDMA Receive context
+ *
+ * This is done when a connection is lost so that a Reply
+ * can be dropped and its matching Call can be subsequently
+ * retransmitted on a new connection.
+ */
+void rpcrdma_unpin_rqst(struct rpcrdma_rep *rep)
+{
+	struct rpc_xprt *xprt = &rep->rr_rxprt->rx_xprt;
+	struct rpc_rqst *rqst = rep->rr_rqst;
+	struct rpcrdma_req *req = rpcr_to_rdmar(rqst);
+
+	req->rl_reply = NULL;
+	rep->rr_rqst = NULL;
+
+	spin_lock(&xprt->queue_lock);
+	xprt_unpin_rqst(rqst);
+	spin_unlock(&xprt->queue_lock);
+}
+
+/**
+ * rpcrdma_complete_rqst - Pass completed rqst back to RPC
+ * @rep: RPC/RDMA Receive context
+ *
+ * Reconstruct the RPC reply and complete the transaction
+ * while @rqst is still pinned to ensure the rep, rqst, and
+ * rq_task pointers remain stable.
  */
 void rpcrdma_complete_rqst(struct rpcrdma_rep *rep)
 {
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index 1b187d1dee8a..bb8aba390b88 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -561,6 +561,7 @@ int rpcrdma_marshal_req(struct rpcrdma_xprt *r_xprt, struct rpc_rqst *rqst);
 void rpcrdma_set_max_header_sizes(struct rpcrdma_ep *ep);
 void rpcrdma_reset_cwnd(struct rpcrdma_xprt *r_xprt);
 void rpcrdma_complete_rqst(struct rpcrdma_rep *rep);
+void rpcrdma_unpin_rqst(struct rpcrdma_rep *rep);
 void rpcrdma_reply_handler(struct rpcrdma_rep *rep);
 
 static inline void rpcrdma_set_xdrlen(struct xdr_buf *xdr, size_t len)



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

* [PATCH v3 17/26] xprtrdma: Avoid Send Queue wrapping
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (15 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 16/26] xprtrdma: Do not wake RPC consumer on a failed LocalInv Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 18/26] xprtrdma: Add tracepoints showing FastReg WRs and remote invalidation Chuck Lever
                   ` (8 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Send WRs can be signalled or unsignalled. A signalled Send WR
always has a matching Send completion, while a unsignalled Send
has a completion only if the Send WR fails.

xprtrdma has a Send account mechanism that is designed to reduce
the number of signalled Send WRs. This in turn mitigates the
interrupt rate of the underlying device.

RDMA consumers can't leave all Sends unsignaled, however, because
providers rely on Send completions to maintain their Send Queue head
and tail pointers. xprtrdma counts the number of unsignaled Send WRs
that have been posted to ensure that Sends are signalled often
enough to prevent the Send Queue from wrapping.

This mechanism neglected to account for FastReg WRs, which are
posted on the Send Queue but never signalled. As a result, the
Send Queue wrapped on occasion, resulting in duplication completions
of FastReg and LocalInv WRs.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c |   17 +++++++++++++++--
 net/sunrpc/xprtrdma/verbs.c    |   16 +---------------
 2 files changed, 16 insertions(+), 17 deletions(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index 951ae20485f3..43a412ea337a 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -390,11 +390,13 @@ static void frwr_cid_init(struct rpcrdma_ep *ep,
  */
 int frwr_send(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 {
+	struct ib_send_wr *post_wr, *send_wr = &req->rl_wr;
 	struct rpcrdma_ep *ep = r_xprt->rx_ep;
-	struct ib_send_wr *post_wr;
 	struct rpcrdma_mr *mr;
+	unsigned int num_wrs;
 
-	post_wr = &req->rl_wr;
+	num_wrs = 1;
+	post_wr = send_wr;
 	list_for_each_entry(mr, &req->rl_registered, mr_list) {
 		struct rpcrdma_frwr *frwr;
 
@@ -409,8 +411,19 @@ int frwr_send(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 		frwr->fr_regwr.wr.send_flags = 0;
 
 		post_wr = &frwr->fr_regwr.wr;
+		++num_wrs;
 	}
 
+	if ((kref_read(&req->rl_kref) > 1) || num_wrs > ep->re_send_count) {
+		send_wr->send_flags |= IB_SEND_SIGNALED;
+		ep->re_send_count = min_t(unsigned int, ep->re_send_batch,
+					  num_wrs - ep->re_send_count);
+	} else {
+		send_wr->send_flags &= ~IB_SEND_SIGNALED;
+		ep->re_send_count -= num_wrs;
+	}
+
+	trace_xprtrdma_post_send(req);
 	return ib_post_send(ep->re_id->qp, post_wr, NULL);
 }
 
diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index d4e573eef416..55c45cad2c8a 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -1365,21 +1365,7 @@ static void rpcrdma_regbuf_free(struct rpcrdma_regbuf *rb)
  */
 int rpcrdma_post_sends(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 {
-	struct ib_send_wr *send_wr = &req->rl_wr;
-	struct rpcrdma_ep *ep = r_xprt->rx_ep;
-	int rc;
-
-	if (!ep->re_send_count || kref_read(&req->rl_kref) > 1) {
-		send_wr->send_flags |= IB_SEND_SIGNALED;
-		ep->re_send_count = ep->re_send_batch;
-	} else {
-		send_wr->send_flags &= ~IB_SEND_SIGNALED;
-		--ep->re_send_count;
-	}
-
-	trace_xprtrdma_post_send(req);
-	rc = frwr_send(r_xprt, req);
-	if (rc)
+	if (frwr_send(r_xprt, req))
 		return -ENOTCONN;
 	return 0;
 }



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

* [PATCH v3 18/26] xprtrdma: Add tracepoints showing FastReg WRs and remote invalidation
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (16 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 17/26] xprtrdma: Avoid Send Queue wrapping Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 19/26] xprtrdma: Add an rpcrdma_mr_completion_class Chuck Lever
                   ` (7 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

The Send signaling logic is a little subtle, so add some
observability around it. For every xprtrdma_mr_fastreg event, there
should be an xprtrdma_mr_localinv or xprtrdma_mr_reminv event.

When these tracepoints are enabled, we can see exactly when an MR is
DMA-mapped, registered, invalidated (either locally or remotely) and
then DMA-unmapped.

   kworker/u25:2-190   [000]   787.979512: xprtrdma_mr_map:      task:351@5 mr.id=4 nents=2 5608@0x8679e0c8f6f56000:0x00000503 (TO_DEVICE)
   kworker/u25:2-190   [000]   787.979515: xprtrdma_chunk_read:  task:351@5 pos=148 5608@0x8679e0c8f6f56000:0x00000503 (last)
   kworker/u25:2-190   [000]   787.979519: xprtrdma_marshal:     task:351@5 xid=0x8679e0c8: hdr=52 xdr=148/5608/0 read list/inline
   kworker/u25:2-190   [000]   787.979525: xprtrdma_mr_fastreg:  task:351@5 mr.id=4 nents=2 5608@0x8679e0c8f6f56000:0x00000503 (TO_DEVICE)
   kworker/u25:2-190   [000]   787.979526: xprtrdma_post_send:   task:351@5 cq.id=0 cid=73 (2 SGEs)

 ...

    kworker/5:1H-219   [005]   787.980567: xprtrdma_wc_receive:  cq.id=1 cid=161 status=SUCCESS (0/0x0) received=164
    kworker/5:1H-219   [005]   787.980571: xprtrdma_post_recvs:  peer=[192.168.100.55]:20049 r_xprt=0xffff8884974d4000: 0 new recvs, 70 active (rc 0)
    kworker/5:1H-219   [005]   787.980573: xprtrdma_reply:       task:351@5 xid=0x8679e0c8 credits=64
    kworker/5:1H-219   [005]   787.980576: xprtrdma_mr_reminv:   task:351@5 mr.id=4 nents=2 5608@0x8679e0c8f6f56000:0x00000503 (TO_DEVICE)
    kworker/5:1H-219   [005]   787.980577: xprtrdma_mr_unmap:    mr.id=4 nents=2 5608@0x8679e0c8f6f56000:0x00000503 (TO_DEVICE)

Note that I've moved the xprtrdma_post_send tracepoint so that event
always appears after the xprtrdma_mr_fastreg tracepoint. Otherwise
the event log looks counterintuitive (FastReg is always supposed to
happen before Send).

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 include/trace/events/rpcrdma.h |    2 ++
 net/sunrpc/xprtrdma/frwr_ops.c |    2 ++
 2 files changed, 4 insertions(+)

diff --git a/include/trace/events/rpcrdma.h b/include/trace/events/rpcrdma.h
index e38e745d13b0..9462326b3535 100644
--- a/include/trace/events/rpcrdma.h
+++ b/include/trace/events/rpcrdma.h
@@ -1010,7 +1010,9 @@ TRACE_EVENT(xprtrdma_frwr_maperr,
 	)
 );
 
+DEFINE_MR_EVENT(fastreg);
 DEFINE_MR_EVENT(localinv);
+DEFINE_MR_EVENT(reminv);
 DEFINE_MR_EVENT(map);
 
 DEFINE_ANON_MR_EVENT(unmap);
diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index 43a412ea337a..0f47c1e24037 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -400,6 +400,7 @@ int frwr_send(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	list_for_each_entry(mr, &req->rl_registered, mr_list) {
 		struct rpcrdma_frwr *frwr;
 
+		trace_xprtrdma_mr_fastreg(mr);
 		frwr = &mr->frwr;
 
 		frwr->fr_cqe.done = frwr_wc_fastreg;
@@ -440,6 +441,7 @@ void frwr_reminv(struct rpcrdma_rep *rep, struct list_head *mrs)
 	list_for_each_entry(mr, mrs, mr_list)
 		if (mr->mr_handle == rep->rr_inv_rkey) {
 			list_del_init(&mr->mr_list);
+			trace_xprtrdma_mr_reminv(mr);
 			frwr_mr_put(mr);
 			break;	/* only one invalidated MR per RPC */
 		}



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

* [PATCH v3 19/26] xprtrdma: Add an rpcrdma_mr_completion_class
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (17 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 18/26] xprtrdma: Add tracepoints showing FastReg WRs and remote invalidation Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 20/26] xprtrdma: Don't display r_xprt memory addresses in tracepoints Chuck Lever
                   ` (6 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

I found it confusing that the MR_EVENT class displays the mr.id but
the associated COMPLETION_EVENT class displays a cid (that happens
to contain the mr.id!). To make it a little easier on humans who
have to read and interpret these events, create an MR_COMPLETION
class that displays the mr.id in the same way as the MR_EVENT class.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 include/trace/events/rpcrdma.h |   48 +++++++++++++++++++++++++++++++++++++---
 1 file changed, 44 insertions(+), 4 deletions(-)

diff --git a/include/trace/events/rpcrdma.h b/include/trace/events/rpcrdma.h
index 9462326b3535..3e6e4c69b533 100644
--- a/include/trace/events/rpcrdma.h
+++ b/include/trace/events/rpcrdma.h
@@ -60,6 +60,46 @@ DECLARE_EVENT_CLASS(rpcrdma_completion_class,
 				),					\
 				TP_ARGS(wc, cid))
 
+DECLARE_EVENT_CLASS(rpcrdma_mr_completion_class,
+	TP_PROTO(
+		const struct ib_wc *wc,
+		const struct rpc_rdma_cid *cid
+	),
+
+	TP_ARGS(wc, cid),
+
+	TP_STRUCT__entry(
+		__field(u32, cq_id)
+		__field(int, completion_id)
+		__field(unsigned long, status)
+		__field(unsigned int, vendor_err)
+	),
+
+	TP_fast_assign(
+		__entry->cq_id = cid->ci_queue_id;
+		__entry->completion_id = cid->ci_completion_id;
+		__entry->status = wc->status;
+		if (wc->status)
+			__entry->vendor_err = wc->vendor_err;
+		else
+			__entry->vendor_err = 0;
+	),
+
+	TP_printk("cq.id=%u mr.id=%d status=%s (%lu/0x%x)",
+		__entry->cq_id, __entry->completion_id,
+		rdma_show_wc_status(__entry->status),
+		__entry->status, __entry->vendor_err
+	)
+);
+
+#define DEFINE_MR_COMPLETION_EVENT(name)				\
+		DEFINE_EVENT(rpcrdma_mr_completion_class, name,		\
+				TP_PROTO(				\
+					const struct ib_wc *wc,		\
+					const struct rpc_rdma_cid *cid	\
+				),					\
+				TP_ARGS(wc, cid))
+
 DECLARE_EVENT_CLASS(rpcrdma_receive_completion_class,
 	TP_PROTO(
 		const struct ib_wc *wc,
@@ -886,10 +926,10 @@ TRACE_EVENT(xprtrdma_post_linv_err,
 DEFINE_RECEIVE_COMPLETION_EVENT(xprtrdma_wc_receive);
 
 DEFINE_COMPLETION_EVENT(xprtrdma_wc_send);
-DEFINE_COMPLETION_EVENT(xprtrdma_wc_fastreg);
-DEFINE_COMPLETION_EVENT(xprtrdma_wc_li);
-DEFINE_COMPLETION_EVENT(xprtrdma_wc_li_wake);
-DEFINE_COMPLETION_EVENT(xprtrdma_wc_li_done);
+DEFINE_MR_COMPLETION_EVENT(xprtrdma_wc_fastreg);
+DEFINE_MR_COMPLETION_EVENT(xprtrdma_wc_li);
+DEFINE_MR_COMPLETION_EVENT(xprtrdma_wc_li_wake);
+DEFINE_MR_COMPLETION_EVENT(xprtrdma_wc_li_done);
 
 TRACE_EVENT(xprtrdma_frwr_alloc,
 	TP_PROTO(



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

* [PATCH v3 20/26] xprtrdma: Don't display r_xprt memory addresses in tracepoints
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (18 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 19/26] xprtrdma: Add an rpcrdma_mr_completion_class Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 21/26] xprtrdma: Remove the RPC/RDMA QP event handler Chuck Lever
                   ` (5 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

The remote peer's IP address is sufficient, and does not expose
details of the kernel's memory layout.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 include/trace/events/rpcrdma.h |   51 +++++++++++++++-------------------------
 1 file changed, 19 insertions(+), 32 deletions(-)

diff --git a/include/trace/events/rpcrdma.h b/include/trace/events/rpcrdma.h
index 3e6e4c69b533..e38b8e33be2d 100644
--- a/include/trace/events/rpcrdma.h
+++ b/include/trace/events/rpcrdma.h
@@ -190,19 +190,17 @@ DECLARE_EVENT_CLASS(xprtrdma_rxprt,
 	TP_ARGS(r_xprt),
 
 	TP_STRUCT__entry(
-		__field(const void *, r_xprt)
 		__string(addr, rpcrdma_addrstr(r_xprt))
 		__string(port, rpcrdma_portstr(r_xprt))
 	),
 
 	TP_fast_assign(
-		__entry->r_xprt = r_xprt;
 		__assign_str(addr, rpcrdma_addrstr(r_xprt));
 		__assign_str(port, rpcrdma_portstr(r_xprt));
 	),
 
-	TP_printk("peer=[%s]:%s r_xprt=%p",
-		__get_str(addr), __get_str(port), __entry->r_xprt
+	TP_printk("peer=[%s]:%s",
+		__get_str(addr), __get_str(port)
 	)
 );
 
@@ -222,7 +220,6 @@ DECLARE_EVENT_CLASS(xprtrdma_connect_class,
 	TP_ARGS(r_xprt, rc),
 
 	TP_STRUCT__entry(
-		__field(const void *, r_xprt)
 		__field(int, rc)
 		__field(int, connect_status)
 		__string(addr, rpcrdma_addrstr(r_xprt))
@@ -230,15 +227,14 @@ DECLARE_EVENT_CLASS(xprtrdma_connect_class,
 	),
 
 	TP_fast_assign(
-		__entry->r_xprt = r_xprt;
 		__entry->rc = rc;
 		__entry->connect_status = r_xprt->rx_ep->re_connect_status;
 		__assign_str(addr, rpcrdma_addrstr(r_xprt));
 		__assign_str(port, rpcrdma_portstr(r_xprt));
 	),
 
-	TP_printk("peer=[%s]:%s r_xprt=%p: rc=%d connection status=%d",
-		__get_str(addr), __get_str(port), __entry->r_xprt,
+	TP_printk("peer=[%s]:%s rc=%d connection status=%d",
+		__get_str(addr), __get_str(port),
 		__entry->rc, __entry->connect_status
 	)
 );
@@ -535,22 +531,19 @@ TRACE_EVENT(xprtrdma_op_connect,
 	TP_ARGS(r_xprt, delay),
 
 	TP_STRUCT__entry(
-		__field(const void *, r_xprt)
 		__field(unsigned long, delay)
 		__string(addr, rpcrdma_addrstr(r_xprt))
 		__string(port, rpcrdma_portstr(r_xprt))
 	),
 
 	TP_fast_assign(
-		__entry->r_xprt = r_xprt;
 		__entry->delay = delay;
 		__assign_str(addr, rpcrdma_addrstr(r_xprt));
 		__assign_str(port, rpcrdma_portstr(r_xprt));
 	),
 
-	TP_printk("peer=[%s]:%s r_xprt=%p delay=%lu",
-		__get_str(addr), __get_str(port), __entry->r_xprt,
-		__entry->delay
+	TP_printk("peer=[%s]:%s delay=%lu",
+		__get_str(addr), __get_str(port), __entry->delay
 	)
 );
 
@@ -565,7 +558,6 @@ TRACE_EVENT(xprtrdma_op_set_cto,
 	TP_ARGS(r_xprt, connect, reconnect),
 
 	TP_STRUCT__entry(
-		__field(const void *, r_xprt)
 		__field(unsigned long, connect)
 		__field(unsigned long, reconnect)
 		__string(addr, rpcrdma_addrstr(r_xprt))
@@ -573,15 +565,14 @@ TRACE_EVENT(xprtrdma_op_set_cto,
 	),
 
 	TP_fast_assign(
-		__entry->r_xprt = r_xprt;
 		__entry->connect = connect;
 		__entry->reconnect = reconnect;
 		__assign_str(addr, rpcrdma_addrstr(r_xprt));
 		__assign_str(port, rpcrdma_portstr(r_xprt));
 	),
 
-	TP_printk("peer=[%s]:%s r_xprt=%p: connect=%lu reconnect=%lu",
-		__get_str(addr), __get_str(port), __entry->r_xprt,
+	TP_printk("peer=[%s]:%s connect=%lu reconnect=%lu",
+		__get_str(addr), __get_str(port),
 		__entry->connect / HZ, __entry->reconnect / HZ
 	)
 );
@@ -631,22 +622,19 @@ TRACE_EVENT(xprtrdma_createmrs,
 	TP_ARGS(r_xprt, count),
 
 	TP_STRUCT__entry(
-		__field(const void *, r_xprt)
 		__string(addr, rpcrdma_addrstr(r_xprt))
 		__string(port, rpcrdma_portstr(r_xprt))
 		__field(unsigned int, count)
 	),
 
 	TP_fast_assign(
-		__entry->r_xprt = r_xprt;
 		__entry->count = count;
 		__assign_str(addr, rpcrdma_addrstr(r_xprt));
 		__assign_str(port, rpcrdma_portstr(r_xprt));
 	),
 
-	TP_printk("peer=[%s]:%s r_xprt=%p: created %u MRs",
-		__get_str(addr), __get_str(port), __entry->r_xprt,
-		__entry->count
+	TP_printk("peer=[%s]:%s created %u MRs",
+		__get_str(addr), __get_str(port), __entry->count
 	)
 );
 
@@ -869,7 +857,7 @@ TRACE_EVENT(xprtrdma_post_recvs,
 	TP_ARGS(r_xprt, count, status),
 
 	TP_STRUCT__entry(
-		__field(const void *, r_xprt)
+		__field(u32, cq_id)
 		__field(unsigned int, count)
 		__field(int, status)
 		__field(int, posted)
@@ -878,16 +866,18 @@ TRACE_EVENT(xprtrdma_post_recvs,
 	),
 
 	TP_fast_assign(
-		__entry->r_xprt = r_xprt;
+		const struct rpcrdma_ep *ep = r_xprt->rx_ep;
+
+		__entry->cq_id = ep->re_attr.recv_cq->res.id;
 		__entry->count = count;
 		__entry->status = status;
-		__entry->posted = r_xprt->rx_ep->re_receive_count;
+		__entry->posted = ep->re_receive_count;
 		__assign_str(addr, rpcrdma_addrstr(r_xprt));
 		__assign_str(port, rpcrdma_portstr(r_xprt));
 	),
 
-	TP_printk("peer=[%s]:%s r_xprt=%p: %u new recvs, %d active (rc %d)",
-		__get_str(addr), __get_str(port), __entry->r_xprt,
+	TP_printk("peer=[%s]:%s cq.id=%d %u new recvs, %d active (rc %d)",
+		__get_str(addr), __get_str(port), __entry->cq_id,
 		__entry->count, __entry->posted, __entry->status
 	)
 );
@@ -1289,22 +1279,19 @@ TRACE_EVENT(xprtrdma_cb_setup,
 	TP_ARGS(r_xprt, reqs),
 
 	TP_STRUCT__entry(
-		__field(const void *, r_xprt)
 		__field(unsigned int, reqs)
 		__string(addr, rpcrdma_addrstr(r_xprt))
 		__string(port, rpcrdma_portstr(r_xprt))
 	),
 
 	TP_fast_assign(
-		__entry->r_xprt = r_xprt;
 		__entry->reqs = reqs;
 		__assign_str(addr, rpcrdma_addrstr(r_xprt));
 		__assign_str(port, rpcrdma_portstr(r_xprt));
 	),
 
-	TP_printk("peer=[%s]:%s r_xprt=%p: %u reqs",
-		__get_str(addr), __get_str(port),
-		__entry->r_xprt, __entry->reqs
+	TP_printk("peer=[%s]:%s %u reqs",
+		__get_str(addr), __get_str(port), __entry->reqs
 	)
 );
 



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

* [PATCH v3 21/26] xprtrdma: Remove the RPC/RDMA QP event handler
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (19 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 20/26] xprtrdma: Don't display r_xprt memory addresses in tracepoints Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:03 ` [PATCH v3 22/26] xprtrdma: Move fr_cid to struct rpcrdma_mr Chuck Lever
                   ` (4 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up: The handler only recorded a trace event. If indeed no
action is needed by the RPC/RDMA consumer, then the event can be
ignored.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 include/trace/events/rpcrdma.h |   32 --------------------------------
 net/sunrpc/xprtrdma/verbs.c    |   18 ------------------
 2 files changed, 50 deletions(-)

diff --git a/include/trace/events/rpcrdma.h b/include/trace/events/rpcrdma.h
index e38b8e33be2d..ef6166b840e7 100644
--- a/include/trace/events/rpcrdma.h
+++ b/include/trace/events/rpcrdma.h
@@ -577,38 +577,6 @@ TRACE_EVENT(xprtrdma_op_set_cto,
 	)
 );
 
-TRACE_EVENT(xprtrdma_qp_event,
-	TP_PROTO(
-		const struct rpcrdma_ep *ep,
-		const struct ib_event *event
-	),
-
-	TP_ARGS(ep, event),
-
-	TP_STRUCT__entry(
-		__field(unsigned long, event)
-		__string(name, event->device->name)
-		__array(unsigned char, srcaddr, sizeof(struct sockaddr_in6))
-		__array(unsigned char, dstaddr, sizeof(struct sockaddr_in6))
-	),
-
-	TP_fast_assign(
-		const struct rdma_cm_id *id = ep->re_id;
-
-		__entry->event = event->event;
-		__assign_str(name, event->device->name);
-		memcpy(__entry->srcaddr, &id->route.addr.src_addr,
-		       sizeof(struct sockaddr_in6));
-		memcpy(__entry->dstaddr, &id->route.addr.dst_addr,
-		       sizeof(struct sockaddr_in6));
-	),
-
-	TP_printk("%pISpc -> %pISpc device=%s %s (%lu)",
-		__entry->srcaddr, __entry->dstaddr, __get_str(name),
-		rdma_show_ib_event(__entry->event), __entry->event
-	)
-);
-
 /**
  ** Call events
  **/
diff --git a/net/sunrpc/xprtrdma/verbs.c b/net/sunrpc/xprtrdma/verbs.c
index 55c45cad2c8a..0aba8273b23e 100644
--- a/net/sunrpc/xprtrdma/verbs.c
+++ b/net/sunrpc/xprtrdma/verbs.c
@@ -120,22 +120,6 @@ static void rpcrdma_xprt_drain(struct rpcrdma_xprt *r_xprt)
 	rpcrdma_ep_put(ep);
 }
 
-/**
- * rpcrdma_qp_event_handler - Handle one QP event (error notification)
- * @event: details of the event
- * @context: ep that owns QP where event occurred
- *
- * Called from the RDMA provider (device driver) possibly in an interrupt
- * context. The QP is always destroyed before the ID, so the ID will be
- * reliably available when this handler is invoked.
- */
-static void rpcrdma_qp_event_handler(struct ib_event *event, void *context)
-{
-	struct rpcrdma_ep *ep = context;
-
-	trace_xprtrdma_qp_event(ep, event);
-}
-
 /* Ensure xprt_force_disconnect() is invoked exactly once when a
  * connection is closed or lost. (The important thing is it needs
  * to be invoked "at least" once).
@@ -431,8 +415,6 @@ static int rpcrdma_ep_create(struct rpcrdma_xprt *r_xprt)
 
 	r_xprt->rx_buf.rb_max_requests = cpu_to_be32(ep->re_max_requests);
 
-	ep->re_attr.event_handler = rpcrdma_qp_event_handler;
-	ep->re_attr.qp_context = ep;
 	ep->re_attr.srq = NULL;
 	ep->re_attr.cap.max_inline_data = 0;
 	ep->re_attr.sq_sig_type = IB_SIGNAL_REQ_WR;



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

* [PATCH v3 22/26] xprtrdma: Move fr_cid to struct rpcrdma_mr
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (20 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 21/26] xprtrdma: Remove the RPC/RDMA QP event handler Chuck Lever
@ 2021-04-19 18:03 ` Chuck Lever
  2021-04-19 18:04 ` [PATCH v3 23/26] xprtrdma: Move cqe " Chuck Lever
                   ` (3 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:03 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up (for several purposes):

- The MR's cid is initialized sooner so that tracepoints can show
  something reasonable even if the MR is never posted.
- The MR's res.id doesn't change so the cid won't change either.
  Initializing the cid once is sufficient.
- struct rpcrdma_frwr is going away soon.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c  |   31 +++++++++++++++----------------
 net/sunrpc/xprtrdma/xprt_rdma.h |    2 +-
 2 files changed, 16 insertions(+), 17 deletions(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index 0f47c1e24037..d3c18c776bf9 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -49,6 +49,15 @@
 # define RPCDBG_FACILITY	RPCDBG_TRANS
 #endif
 
+static void frwr_cid_init(struct rpcrdma_ep *ep,
+			  struct rpcrdma_mr *mr)
+{
+	struct rpc_rdma_cid *cid = &mr->mr_cid;
+
+	cid->ci_queue_id = ep->re_attr.send_cq->res.id;
+	cid->ci_completion_id = mr->frwr.fr_mr->res.id;
+}
+
 static void frwr_mr_unmap(struct rpcrdma_xprt *r_xprt, struct rpcrdma_mr *mr)
 {
 	if (mr->mr_device) {
@@ -134,6 +143,7 @@ int frwr_mr_init(struct rpcrdma_xprt *r_xprt, struct rpcrdma_mr *mr)
 	mr->mr_device = NULL;
 	INIT_LIST_HEAD(&mr->mr_list);
 	init_completion(&mr->frwr.fr_linv_done);
+	frwr_cid_init(ep, mr);
 
 	sg_init_table(sg, depth);
 	mr->mr_sg = sg;
@@ -358,22 +368,14 @@ static void frwr_wc_fastreg(struct ib_cq *cq, struct ib_wc *wc)
 	struct ib_cqe *cqe = wc->wr_cqe;
 	struct rpcrdma_frwr *frwr =
 		container_of(cqe, struct rpcrdma_frwr, fr_cqe);
+	struct rpcrdma_mr *mr = container_of(frwr, struct rpcrdma_mr, frwr);
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
-	trace_xprtrdma_wc_fastreg(wc, &frwr->fr_cid);
+	trace_xprtrdma_wc_fastreg(wc, &mr->mr_cid);
 
 	rpcrdma_flush_disconnect(cq->cq_context, wc);
 }
 
-static void frwr_cid_init(struct rpcrdma_ep *ep,
-			  struct rpcrdma_frwr *frwr)
-{
-	struct rpc_rdma_cid *cid = &frwr->fr_cid;
-
-	cid->ci_queue_id = ep->re_attr.send_cq->res.id;
-	cid->ci_completion_id = frwr->fr_mr->res.id;
-}
-
 /**
  * frwr_send - post Send WRs containing the RPC Call message
  * @r_xprt: controlling transport instance
@@ -404,7 +406,6 @@ int frwr_send(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 		frwr = &mr->frwr;
 
 		frwr->fr_cqe.done = frwr_wc_fastreg;
-		frwr_cid_init(ep, frwr);
 		frwr->fr_regwr.wr.next = post_wr;
 		frwr->fr_regwr.wr.wr_cqe = &frwr->fr_cqe;
 		frwr->fr_regwr.wr.num_sge = 0;
@@ -467,7 +468,7 @@ static void frwr_wc_localinv(struct ib_cq *cq, struct ib_wc *wc)
 	struct rpcrdma_mr *mr = container_of(frwr, struct rpcrdma_mr, frwr);
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
-	trace_xprtrdma_wc_li(wc, &frwr->fr_cid);
+	trace_xprtrdma_wc_li(wc, &mr->mr_cid);
 	frwr_mr_done(wc, mr);
 
 	rpcrdma_flush_disconnect(cq->cq_context, wc);
@@ -488,7 +489,7 @@ static void frwr_wc_localinv_wake(struct ib_cq *cq, struct ib_wc *wc)
 	struct rpcrdma_mr *mr = container_of(frwr, struct rpcrdma_mr, frwr);
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
-	trace_xprtrdma_wc_li_wake(wc, &frwr->fr_cid);
+	trace_xprtrdma_wc_li_wake(wc, &mr->mr_cid);
 	frwr_mr_done(wc, mr);
 	complete(&frwr->fr_linv_done);
 
@@ -529,7 +530,6 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 
 		frwr = &mr->frwr;
 		frwr->fr_cqe.done = frwr_wc_localinv;
-		frwr_cid_init(ep, frwr);
 		last = &frwr->fr_invwr;
 		last->next = NULL;
 		last->wr_cqe = &frwr->fr_cqe;
@@ -585,7 +585,7 @@ static void frwr_wc_localinv_done(struct ib_cq *cq, struct ib_wc *wc)
 	struct rpcrdma_rep *rep;
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
-	trace_xprtrdma_wc_li_done(wc, &frwr->fr_cid);
+	trace_xprtrdma_wc_li_done(wc, &mr->mr_cid);
 
 	/* Ensure that @rep is generated before the MR is released */
 	rep = mr->mr_req->rl_reply;
@@ -631,7 +631,6 @@ void frwr_unmap_async(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 
 		frwr = &mr->frwr;
 		frwr->fr_cqe.done = frwr_wc_localinv;
-		frwr_cid_init(ep, frwr);
 		last = &frwr->fr_invwr;
 		last->next = NULL;
 		last->wr_cqe = &frwr->fr_cqe;
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index bb8aba390b88..0cf073f0ee64 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -232,7 +232,6 @@ struct rpcrdma_sendctx {
 struct rpcrdma_frwr {
 	struct ib_mr			*fr_mr;
 	struct ib_cqe			fr_cqe;
-	struct rpc_rdma_cid		fr_cid;
 	struct completion		fr_linv_done;
 	union {
 		struct ib_reg_wr	fr_regwr;
@@ -254,6 +253,7 @@ struct rpcrdma_mr {
 	u32			mr_length;
 	u64			mr_offset;
 	struct list_head	mr_all;
+	struct rpc_rdma_cid	mr_cid;
 };
 
 /*



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

* [PATCH v3 23/26] xprtrdma: Move cqe to struct rpcrdma_mr
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (21 preceding siblings ...)
  2021-04-19 18:03 ` [PATCH v3 22/26] xprtrdma: Move fr_cid to struct rpcrdma_mr Chuck Lever
@ 2021-04-19 18:04 ` Chuck Lever
  2021-04-19 18:04 ` [PATCH v3 24/26] xprtrdma: Move fr_linv_done field " Chuck Lever
                   ` (2 subsequent siblings)
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:04 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up.

- Simplify variable initialization in the completion handlers.

- Move another field out of struct rpcrdma_frwr.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c  |   35 +++++++++++++++--------------------
 net/sunrpc/xprtrdma/xprt_rdma.h |    2 +-
 2 files changed, 16 insertions(+), 21 deletions(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index d3c18c776bf9..2a886a28d82b 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -366,9 +366,7 @@ struct rpcrdma_mr_seg *frwr_map(struct rpcrdma_xprt *r_xprt,
 static void frwr_wc_fastreg(struct ib_cq *cq, struct ib_wc *wc)
 {
 	struct ib_cqe *cqe = wc->wr_cqe;
-	struct rpcrdma_frwr *frwr =
-		container_of(cqe, struct rpcrdma_frwr, fr_cqe);
-	struct rpcrdma_mr *mr = container_of(frwr, struct rpcrdma_mr, frwr);
+	struct rpcrdma_mr *mr = container_of(cqe, struct rpcrdma_mr, mr_cqe);
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
 	trace_xprtrdma_wc_fastreg(wc, &mr->mr_cid);
@@ -405,9 +403,9 @@ int frwr_send(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 		trace_xprtrdma_mr_fastreg(mr);
 		frwr = &mr->frwr;
 
-		frwr->fr_cqe.done = frwr_wc_fastreg;
+		mr->mr_cqe.done = frwr_wc_fastreg;
 		frwr->fr_regwr.wr.next = post_wr;
-		frwr->fr_regwr.wr.wr_cqe = &frwr->fr_cqe;
+		frwr->fr_regwr.wr.wr_cqe = &mr->mr_cqe;
 		frwr->fr_regwr.wr.num_sge = 0;
 		frwr->fr_regwr.wr.opcode = IB_WR_REG_MR;
 		frwr->fr_regwr.wr.send_flags = 0;
@@ -463,9 +461,7 @@ static void frwr_mr_done(struct ib_wc *wc, struct rpcrdma_mr *mr)
 static void frwr_wc_localinv(struct ib_cq *cq, struct ib_wc *wc)
 {
 	struct ib_cqe *cqe = wc->wr_cqe;
-	struct rpcrdma_frwr *frwr =
-		container_of(cqe, struct rpcrdma_frwr, fr_cqe);
-	struct rpcrdma_mr *mr = container_of(frwr, struct rpcrdma_mr, frwr);
+	struct rpcrdma_mr *mr = container_of(cqe, struct rpcrdma_mr, mr_cqe);
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
 	trace_xprtrdma_wc_li(wc, &mr->mr_cid);
@@ -484,9 +480,8 @@ static void frwr_wc_localinv(struct ib_cq *cq, struct ib_wc *wc)
 static void frwr_wc_localinv_wake(struct ib_cq *cq, struct ib_wc *wc)
 {
 	struct ib_cqe *cqe = wc->wr_cqe;
-	struct rpcrdma_frwr *frwr =
-		container_of(cqe, struct rpcrdma_frwr, fr_cqe);
-	struct rpcrdma_mr *mr = container_of(frwr, struct rpcrdma_mr, frwr);
+	struct rpcrdma_mr *mr = container_of(cqe, struct rpcrdma_mr, mr_cqe);
+	struct rpcrdma_frwr *frwr = &mr->frwr;
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
 	trace_xprtrdma_wc_li_wake(wc, &mr->mr_cid);
@@ -529,16 +524,17 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 		r_xprt->rx_stats.local_inv_needed++;
 
 		frwr = &mr->frwr;
-		frwr->fr_cqe.done = frwr_wc_localinv;
 		last = &frwr->fr_invwr;
 		last->next = NULL;
-		last->wr_cqe = &frwr->fr_cqe;
+		last->wr_cqe = &mr->mr_cqe;
 		last->sg_list = NULL;
 		last->num_sge = 0;
 		last->opcode = IB_WR_LOCAL_INV;
 		last->send_flags = IB_SEND_SIGNALED;
 		last->ex.invalidate_rkey = mr->mr_handle;
 
+		last->wr_cqe->done = frwr_wc_localinv;
+
 		*prev = last;
 		prev = &last->next;
 	}
@@ -547,7 +543,7 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	 * last WR in the chain completes, all WRs in the chain
 	 * are complete.
 	 */
-	frwr->fr_cqe.done = frwr_wc_localinv_wake;
+	last->wr_cqe->done = frwr_wc_localinv_wake;
 	reinit_completion(&frwr->fr_linv_done);
 
 	/* Transport disconnect drains the receive CQ before it
@@ -579,9 +575,7 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 static void frwr_wc_localinv_done(struct ib_cq *cq, struct ib_wc *wc)
 {
 	struct ib_cqe *cqe = wc->wr_cqe;
-	struct rpcrdma_frwr *frwr =
-		container_of(cqe, struct rpcrdma_frwr, fr_cqe);
-	struct rpcrdma_mr *mr = container_of(frwr, struct rpcrdma_mr, frwr);
+	struct rpcrdma_mr *mr = container_of(cqe, struct rpcrdma_mr, mr_cqe);
 	struct rpcrdma_rep *rep;
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
@@ -630,16 +624,17 @@ void frwr_unmap_async(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 		r_xprt->rx_stats.local_inv_needed++;
 
 		frwr = &mr->frwr;
-		frwr->fr_cqe.done = frwr_wc_localinv;
 		last = &frwr->fr_invwr;
 		last->next = NULL;
-		last->wr_cqe = &frwr->fr_cqe;
+		last->wr_cqe = &mr->mr_cqe;
 		last->sg_list = NULL;
 		last->num_sge = 0;
 		last->opcode = IB_WR_LOCAL_INV;
 		last->send_flags = IB_SEND_SIGNALED;
 		last->ex.invalidate_rkey = mr->mr_handle;
 
+		last->wr_cqe->done = frwr_wc_localinv;
+
 		*prev = last;
 		prev = &last->next;
 	}
@@ -649,7 +644,7 @@ void frwr_unmap_async(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	 * are complete. The last completion will wake up the
 	 * RPC waiter.
 	 */
-	frwr->fr_cqe.done = frwr_wc_localinv_done;
+	last->wr_cqe->done = frwr_wc_localinv_done;
 
 	/* Transport disconnect drains the receive CQ before it
 	 * replaces the QP. The RPC reply handler won't call us
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index 0cf073f0ee64..f72b69c3f0ea 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -231,7 +231,6 @@ struct rpcrdma_sendctx {
  */
 struct rpcrdma_frwr {
 	struct ib_mr			*fr_mr;
-	struct ib_cqe			fr_cqe;
 	struct completion		fr_linv_done;
 	union {
 		struct ib_reg_wr	fr_regwr;
@@ -247,6 +246,7 @@ struct rpcrdma_mr {
 	struct scatterlist	*mr_sg;
 	int			mr_nents;
 	enum dma_data_direction	mr_dir;
+	struct ib_cqe		mr_cqe;
 	struct rpcrdma_frwr	frwr;
 	struct rpcrdma_xprt	*mr_xprt;
 	u32			mr_handle;



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

* [PATCH v3 24/26] xprtrdma: Move fr_linv_done field to struct rpcrdma_mr
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (22 preceding siblings ...)
  2021-04-19 18:04 ` [PATCH v3 23/26] xprtrdma: Move cqe " Chuck Lever
@ 2021-04-19 18:04 ` Chuck Lever
  2021-04-19 18:04 ` [PATCH v3 25/26] xprtrdma: Move the Work Request union " Chuck Lever
  2021-04-19 18:04 ` [PATCH v3 26/26] xprtrdma: Move fr_mr field " Chuck Lever
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:04 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up: Move more of struct rpcrdma_frwr into its parent.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c  |    9 ++++-----
 net/sunrpc/xprtrdma/xprt_rdma.h |    2 +-
 2 files changed, 5 insertions(+), 6 deletions(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index 2a886a28d82b..7d7a64d2ca4a 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -142,7 +142,7 @@ int frwr_mr_init(struct rpcrdma_xprt *r_xprt, struct rpcrdma_mr *mr)
 	mr->frwr.fr_mr = frmr;
 	mr->mr_device = NULL;
 	INIT_LIST_HEAD(&mr->mr_list);
-	init_completion(&mr->frwr.fr_linv_done);
+	init_completion(&mr->mr_linv_done);
 	frwr_cid_init(ep, mr);
 
 	sg_init_table(sg, depth);
@@ -481,12 +481,11 @@ static void frwr_wc_localinv_wake(struct ib_cq *cq, struct ib_wc *wc)
 {
 	struct ib_cqe *cqe = wc->wr_cqe;
 	struct rpcrdma_mr *mr = container_of(cqe, struct rpcrdma_mr, mr_cqe);
-	struct rpcrdma_frwr *frwr = &mr->frwr;
 
 	/* WARNING: Only wr_cqe and status are reliable at this point */
 	trace_xprtrdma_wc_li_wake(wc, &mr->mr_cid);
 	frwr_mr_done(wc, mr);
-	complete(&frwr->fr_linv_done);
+	complete(&mr->mr_linv_done);
 
 	rpcrdma_flush_disconnect(cq->cq_context, wc);
 }
@@ -544,7 +543,7 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	 * are complete.
 	 */
 	last->wr_cqe->done = frwr_wc_localinv_wake;
-	reinit_completion(&frwr->fr_linv_done);
+	reinit_completion(&mr->mr_linv_done);
 
 	/* Transport disconnect drains the receive CQ before it
 	 * replaces the QP. The RPC reply handler won't call us
@@ -558,7 +557,7 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	 * not happen, so don't wait in that case.
 	 */
 	if (bad_wr != first)
-		wait_for_completion(&frwr->fr_linv_done);
+		wait_for_completion(&mr->mr_linv_done);
 	if (!rc)
 		return;
 
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index f72b69c3f0ea..1374054b73fd 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -231,7 +231,6 @@ struct rpcrdma_sendctx {
  */
 struct rpcrdma_frwr {
 	struct ib_mr			*fr_mr;
-	struct completion		fr_linv_done;
 	union {
 		struct ib_reg_wr	fr_regwr;
 		struct ib_send_wr	fr_invwr;
@@ -247,6 +246,7 @@ struct rpcrdma_mr {
 	int			mr_nents;
 	enum dma_data_direction	mr_dir;
 	struct ib_cqe		mr_cqe;
+	struct completion	mr_linv_done;
 	struct rpcrdma_frwr	frwr;
 	struct rpcrdma_xprt	*mr_xprt;
 	u32			mr_handle;



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

* [PATCH v3 25/26] xprtrdma: Move the Work Request union to struct rpcrdma_mr
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (23 preceding siblings ...)
  2021-04-19 18:04 ` [PATCH v3 24/26] xprtrdma: Move fr_linv_done field " Chuck Lever
@ 2021-04-19 18:04 ` Chuck Lever
  2021-04-19 18:04 ` [PATCH v3 26/26] xprtrdma: Move fr_mr field " Chuck Lever
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:04 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 net/sunrpc/xprtrdma/frwr_ops.c  |   28 +++++++++-------------------
 net/sunrpc/xprtrdma/xprt_rdma.h |    8 ++++----
 2 files changed, 13 insertions(+), 23 deletions(-)

diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index 7d7a64d2ca4a..bb2b9c607c71 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -333,7 +333,7 @@ struct rpcrdma_mr_seg *frwr_map(struct rpcrdma_xprt *r_xprt,
 	key = (u8)(ibmr->rkey & 0x000000FF);
 	ib_update_fast_reg_key(ibmr, ++key);
 
-	reg_wr = &mr->frwr.fr_regwr;
+	reg_wr = &mr->mr_regwr;
 	reg_wr->mr = ibmr;
 	reg_wr->key = ibmr->rkey;
 	reg_wr->access = writing ?
@@ -398,19 +398,15 @@ int frwr_send(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	num_wrs = 1;
 	post_wr = send_wr;
 	list_for_each_entry(mr, &req->rl_registered, mr_list) {
-		struct rpcrdma_frwr *frwr;
-
 		trace_xprtrdma_mr_fastreg(mr);
-		frwr = &mr->frwr;
 
 		mr->mr_cqe.done = frwr_wc_fastreg;
-		frwr->fr_regwr.wr.next = post_wr;
-		frwr->fr_regwr.wr.wr_cqe = &mr->mr_cqe;
-		frwr->fr_regwr.wr.num_sge = 0;
-		frwr->fr_regwr.wr.opcode = IB_WR_REG_MR;
-		frwr->fr_regwr.wr.send_flags = 0;
-
-		post_wr = &frwr->fr_regwr.wr;
+		mr->mr_regwr.wr.next = post_wr;
+		mr->mr_regwr.wr.wr_cqe = &mr->mr_cqe;
+		mr->mr_regwr.wr.num_sge = 0;
+		mr->mr_regwr.wr.opcode = IB_WR_REG_MR;
+		mr->mr_regwr.wr.send_flags = 0;
+		post_wr = &mr->mr_regwr.wr;
 		++num_wrs;
 	}
 
@@ -506,7 +502,6 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	struct ib_send_wr *first, **prev, *last;
 	struct rpcrdma_ep *ep = r_xprt->rx_ep;
 	const struct ib_send_wr *bad_wr;
-	struct rpcrdma_frwr *frwr;
 	struct rpcrdma_mr *mr;
 	int rc;
 
@@ -515,15 +510,13 @@ void frwr_unmap_sync(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 	 * Chain the LOCAL_INV Work Requests and post them with
 	 * a single ib_post_send() call.
 	 */
-	frwr = NULL;
 	prev = &first;
 	while ((mr = rpcrdma_mr_pop(&req->rl_registered))) {
 
 		trace_xprtrdma_mr_localinv(mr);
 		r_xprt->rx_stats.local_inv_needed++;
 
-		frwr = &mr->frwr;
-		last = &frwr->fr_invwr;
+		last = &mr->mr_invwr;
 		last->next = NULL;
 		last->wr_cqe = &mr->mr_cqe;
 		last->sg_list = NULL;
@@ -608,22 +601,19 @@ void frwr_unmap_async(struct rpcrdma_xprt *r_xprt, struct rpcrdma_req *req)
 {
 	struct ib_send_wr *first, *last, **prev;
 	struct rpcrdma_ep *ep = r_xprt->rx_ep;
-	struct rpcrdma_frwr *frwr;
 	struct rpcrdma_mr *mr;
 	int rc;
 
 	/* Chain the LOCAL_INV Work Requests and post them with
 	 * a single ib_post_send() call.
 	 */
-	frwr = NULL;
 	prev = &first;
 	while ((mr = rpcrdma_mr_pop(&req->rl_registered))) {
 
 		trace_xprtrdma_mr_localinv(mr);
 		r_xprt->rx_stats.local_inv_needed++;
 
-		frwr = &mr->frwr;
-		last = &frwr->fr_invwr;
+		last = &mr->mr_invwr;
 		last->next = NULL;
 		last->wr_cqe = &mr->mr_cqe;
 		last->sg_list = NULL;
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index 1374054b73fd..99ef83d673d6 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -231,10 +231,6 @@ struct rpcrdma_sendctx {
  */
 struct rpcrdma_frwr {
 	struct ib_mr			*fr_mr;
-	union {
-		struct ib_reg_wr	fr_regwr;
-		struct ib_send_wr	fr_invwr;
-	};
 };
 
 struct rpcrdma_req;
@@ -247,6 +243,10 @@ struct rpcrdma_mr {
 	enum dma_data_direction	mr_dir;
 	struct ib_cqe		mr_cqe;
 	struct completion	mr_linv_done;
+	union {
+		struct ib_reg_wr	mr_regwr;
+		struct ib_send_wr	mr_invwr;
+	};
 	struct rpcrdma_frwr	frwr;
 	struct rpcrdma_xprt	*mr_xprt;
 	u32			mr_handle;



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

* [PATCH v3 26/26] xprtrdma: Move fr_mr field to struct rpcrdma_mr
  2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
                   ` (24 preceding siblings ...)
  2021-04-19 18:04 ` [PATCH v3 25/26] xprtrdma: Move the Work Request union " Chuck Lever
@ 2021-04-19 18:04 ` Chuck Lever
  25 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever @ 2021-04-19 18:04 UTC (permalink / raw)
  To: trondmy; +Cc: linux-nfs, linux-rdma

Clean up: The last remaining field in struct rpcrdma_frwr has been
removed, so the struct can be eliminated.

Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
---
 include/trace/events/rpcrdma.h  |   12 ++++++------
 net/sunrpc/xprtrdma/frwr_ops.c  |    8 ++++----
 net/sunrpc/xprtrdma/xprt_rdma.h |    7 ++-----
 3 files changed, 12 insertions(+), 15 deletions(-)

diff --git a/include/trace/events/rpcrdma.h b/include/trace/events/rpcrdma.h
index ef6166b840e7..bd55908c1bef 100644
--- a/include/trace/events/rpcrdma.h
+++ b/include/trace/events/rpcrdma.h
@@ -379,7 +379,7 @@ DECLARE_EVENT_CLASS(xprtrdma_mr_class,
 
 		__entry->task_id = task->tk_pid;
 		__entry->client_id = task->tk_client->cl_clid;
-		__entry->mr_id  = mr->frwr.fr_mr->res.id;
+		__entry->mr_id  = mr->mr_ibmr->res.id;
 		__entry->nents  = mr->mr_nents;
 		__entry->handle = mr->mr_handle;
 		__entry->length = mr->mr_length;
@@ -420,7 +420,7 @@ DECLARE_EVENT_CLASS(xprtrdma_anonymous_mr_class,
 	),
 
 	TP_fast_assign(
-		__entry->mr_id  = mr->frwr.fr_mr->res.id;
+		__entry->mr_id  = mr->mr_ibmr->res.id;
 		__entry->nents  = mr->mr_nents;
 		__entry->handle = mr->mr_handle;
 		__entry->length = mr->mr_length;
@@ -903,7 +903,7 @@ TRACE_EVENT(xprtrdma_frwr_alloc,
 	),
 
 	TP_fast_assign(
-		__entry->mr_id = mr->frwr.fr_mr->res.id;
+		__entry->mr_id = mr->mr_ibmr->res.id;
 		__entry->rc = rc;
 	),
 
@@ -931,7 +931,7 @@ TRACE_EVENT(xprtrdma_frwr_dereg,
 	),
 
 	TP_fast_assign(
-		__entry->mr_id  = mr->frwr.fr_mr->res.id;
+		__entry->mr_id  = mr->mr_ibmr->res.id;
 		__entry->nents  = mr->mr_nents;
 		__entry->handle = mr->mr_handle;
 		__entry->length = mr->mr_length;
@@ -964,7 +964,7 @@ TRACE_EVENT(xprtrdma_frwr_sgerr,
 	),
 
 	TP_fast_assign(
-		__entry->mr_id = mr->frwr.fr_mr->res.id;
+		__entry->mr_id = mr->mr_ibmr->res.id;
 		__entry->addr = mr->mr_sg->dma_address;
 		__entry->dir = mr->mr_dir;
 		__entry->nents = sg_nents;
@@ -994,7 +994,7 @@ TRACE_EVENT(xprtrdma_frwr_maperr,
 	),
 
 	TP_fast_assign(
-		__entry->mr_id = mr->frwr.fr_mr->res.id;
+		__entry->mr_id = mr->mr_ibmr->res.id;
 		__entry->addr = mr->mr_sg->dma_address;
 		__entry->dir = mr->mr_dir;
 		__entry->num_mapped = num_mapped;
diff --git a/net/sunrpc/xprtrdma/frwr_ops.c b/net/sunrpc/xprtrdma/frwr_ops.c
index bb2b9c607c71..285d73246fc2 100644
--- a/net/sunrpc/xprtrdma/frwr_ops.c
+++ b/net/sunrpc/xprtrdma/frwr_ops.c
@@ -55,7 +55,7 @@ static void frwr_cid_init(struct rpcrdma_ep *ep,
 	struct rpc_rdma_cid *cid = &mr->mr_cid;
 
 	cid->ci_queue_id = ep->re_attr.send_cq->res.id;
-	cid->ci_completion_id = mr->frwr.fr_mr->res.id;
+	cid->ci_completion_id = mr->mr_ibmr->res.id;
 }
 
 static void frwr_mr_unmap(struct rpcrdma_xprt *r_xprt, struct rpcrdma_mr *mr)
@@ -79,7 +79,7 @@ void frwr_mr_release(struct rpcrdma_mr *mr)
 
 	frwr_mr_unmap(mr->mr_xprt, mr);
 
-	rc = ib_dereg_mr(mr->frwr.fr_mr);
+	rc = ib_dereg_mr(mr->mr_ibmr);
 	if (rc)
 		trace_xprtrdma_frwr_dereg(mr, rc);
 	kfree(mr->mr_sg);
@@ -139,7 +139,7 @@ int frwr_mr_init(struct rpcrdma_xprt *r_xprt, struct rpcrdma_mr *mr)
 		goto out_list_err;
 
 	mr->mr_xprt = r_xprt;
-	mr->frwr.fr_mr = frmr;
+	mr->mr_ibmr = frmr;
 	mr->mr_device = NULL;
 	INIT_LIST_HEAD(&mr->mr_list);
 	init_completion(&mr->mr_linv_done);
@@ -323,7 +323,7 @@ struct rpcrdma_mr_seg *frwr_map(struct rpcrdma_xprt *r_xprt,
 		goto out_dmamap_err;
 	mr->mr_device = ep->re_id->device;
 
-	ibmr = mr->frwr.fr_mr;
+	ibmr = mr->mr_ibmr;
 	n = ib_map_mr_sg(ibmr, mr->mr_sg, dma_nents, NULL, PAGE_SIZE);
 	if (n != dma_nents)
 		goto out_mapmr_err;
diff --git a/net/sunrpc/xprtrdma/xprt_rdma.h b/net/sunrpc/xprtrdma/xprt_rdma.h
index 99ef83d673d6..436ad7312614 100644
--- a/net/sunrpc/xprtrdma/xprt_rdma.h
+++ b/net/sunrpc/xprtrdma/xprt_rdma.h
@@ -229,14 +229,12 @@ struct rpcrdma_sendctx {
  * An external memory region is any buffer or page that is registered
  * on the fly (ie, not pre-registered).
  */
-struct rpcrdma_frwr {
-	struct ib_mr			*fr_mr;
-};
-
 struct rpcrdma_req;
 struct rpcrdma_mr {
 	struct list_head	mr_list;
 	struct rpcrdma_req	*mr_req;
+
+	struct ib_mr		*mr_ibmr;
 	struct ib_device	*mr_device;
 	struct scatterlist	*mr_sg;
 	int			mr_nents;
@@ -247,7 +245,6 @@ struct rpcrdma_mr {
 		struct ib_reg_wr	mr_regwr;
 		struct ib_send_wr	mr_invwr;
 	};
-	struct rpcrdma_frwr	frwr;
 	struct rpcrdma_xprt	*mr_xprt;
 	u32			mr_handle;
 	u32			mr_length;



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

* Re: [PATCH v3 01/26] SUNRPC: Move fault injection call sites
  2021-04-19 18:01 ` [PATCH v3 01/26] SUNRPC: Move fault injection call sites Chuck Lever
@ 2021-04-19 18:05   ` Chuck Lever III
  0 siblings, 0 replies; 34+ messages in thread
From: Chuck Lever III @ 2021-04-19 18:05 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List, linux-rdma



> On Apr 19, 2021, at 2:01 PM, Chuck Lever <chuck.lever@oracle.com> wrote:
> 
> I've hit some crashes that occur in the xprt_rdma_inject_disconnect
> path. It appears that, for some provides, rdma_disconnect() can

^provides^providers

> take so long that the transport can disconnect and release its
> hardware resources while rdma_disconnect() is still running,
> resulting in a UAF in the provider.
> 
> The transport's fault injection method may depend on the stability
> of transport data structures. That means it needs to be invoked
> only from contexts that hold the transport write lock.
> 
> Fixes: 4a0682583988 ("SUNRPC: Transport fault injection")
> Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
> ---
> net/sunrpc/clnt.c               |    1 -
> net/sunrpc/xprt.c               |    6 ++++--
> net/sunrpc/xprtrdma/transport.c |    6 ++++--
> 3 files changed, 8 insertions(+), 5 deletions(-)
> 
> diff --git a/net/sunrpc/clnt.c b/net/sunrpc/clnt.c
> index 612f0a641f4c..c2a01125be1a 100644
> --- a/net/sunrpc/clnt.c
> +++ b/net/sunrpc/clnt.c
> @@ -1799,7 +1799,6 @@ call_allocate(struct rpc_task *task)
> 
> 	status = xprt->ops->buf_alloc(task);
> 	trace_rpc_buf_alloc(task, status);
> -	xprt_inject_disconnect(xprt);
> 	if (status == 0)
> 		return;
> 	if (status != -ENOMEM) {
> diff --git a/net/sunrpc/xprt.c b/net/sunrpc/xprt.c
> index 691ccf8049a4..d616b93751d8 100644
> --- a/net/sunrpc/xprt.c
> +++ b/net/sunrpc/xprt.c
> @@ -1483,7 +1483,10 @@ bool xprt_prepare_transmit(struct rpc_task *task)
> 
> void xprt_end_transmit(struct rpc_task *task)
> {
> -	xprt_release_write(task->tk_rqstp->rq_xprt, task);
> +	struct rpc_xprt	*xprt = task->tk_rqstp->rq_xprt;
> +
> +	xprt_inject_disconnect(xprt);
> +	xprt_release_write(xprt, task);
> }
> 
> /**
> @@ -1885,7 +1888,6 @@ void xprt_release(struct rpc_task *task)
> 	spin_unlock(&xprt->transport_lock);
> 	if (req->rq_buffer)
> 		xprt->ops->buf_free(task);
> -	xprt_inject_disconnect(xprt);
> 	xdr_free_bvec(&req->rq_rcv_buf);
> 	xdr_free_bvec(&req->rq_snd_buf);
> 	if (req->rq_cred != NULL)
> diff --git a/net/sunrpc/xprtrdma/transport.c b/net/sunrpc/xprtrdma/transport.c
> index 78d29d1bcc20..09953597d055 100644
> --- a/net/sunrpc/xprtrdma/transport.c
> +++ b/net/sunrpc/xprtrdma/transport.c
> @@ -262,8 +262,10 @@ xprt_rdma_connect_worker(struct work_struct *work)
>  * xprt_rdma_inject_disconnect - inject a connection fault
>  * @xprt: transport context
>  *
> - * If @xprt is connected, disconnect it to simulate spurious connection
> - * loss.
> + * If @xprt is connected, disconnect it to simulate spurious
> + * connection loss. Caller must hold @xprt's send lock to
> + * ensure that data structures and hardware resources are
> + * stable during the rdma_disconnect() call.
>  */
> static void
> xprt_rdma_inject_disconnect(struct rpc_xprt *xprt)
> 
> 

--
Chuck Lever




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

* Re: [PATCH v3 07/26] xprtrdma: Improve locking around rpcrdma_rep destruction
  2021-04-19 18:02 ` [PATCH v3 07/26] xprtrdma: Improve locking around rpcrdma_rep destruction Chuck Lever
@ 2021-04-23 21:06   ` Trond Myklebust
  2021-04-24 17:39     ` Chuck Lever III
  0 siblings, 1 reply; 34+ messages in thread
From: Trond Myklebust @ 2021-04-23 21:06 UTC (permalink / raw)
  To: chuck.lever; +Cc: linux-nfs, linux-rdma

On Mon, 2021-04-19 at 14:02 -0400, Chuck Lever wrote:
> Currently rpcrdma_reps_destroy() assumes that, at transport
> tear-down, the content of the rb_free_reps list is the same as the
> content of the rb_all_reps list. Although that is usually true,
> using the rb_all_reps list should be more reliable because of
> the way it's managed. And, rpcrdma_reps_unmap() uses rb_all_reps;
> these two functions should both traverse the "all" list.
> 
> Ensure that all rpcrdma_reps are always destroyed whether they are
> on the rep free list or not.
> 
> Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
> ---
>  net/sunrpc/xprtrdma/verbs.c |   31 ++++++++++++++++++++++++-------
>  1 file changed, 24 insertions(+), 7 deletions(-)
> 
> diff --git a/net/sunrpc/xprtrdma/verbs.c
> b/net/sunrpc/xprtrdma/verbs.c
> index 1b599a623eea..482fdc9e25c2 100644
> --- a/net/sunrpc/xprtrdma/verbs.c
> +++ b/net/sunrpc/xprtrdma/verbs.c
> @@ -1007,16 +1007,23 @@ struct rpcrdma_rep *rpcrdma_rep_create(struct
> rpcrdma_xprt *r_xprt,
>         return NULL;
>  }
>  
> -/* No locking needed here. This function is invoked only by the
> - * Receive completion handler, or during transport shutdown.
> - */
> -static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep)
> +static void rpcrdma_rep_destroy_locked(struct rpcrdma_rep *rep)

The name here is extremely confusing. As far as I can tell, this isn't
called with any lock?

>  {
> -       list_del(&rep->rr_all);
>         rpcrdma_regbuf_free(rep->rr_rdmabuf);
>         kfree(rep);
>  }
>  
> +static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep)
> +{
> +       struct rpcrdma_buffer *buf = &rep->rr_rxprt->rx_buf;
> +
> +       spin_lock(&buf->rb_lock);
> +       list_del(&rep->rr_all);
> +       spin_unlock(&buf->rb_lock);
> +
> +       rpcrdma_rep_destroy_locked(rep);
> +}
> +
>  static struct rpcrdma_rep *rpcrdma_rep_get_locked(struct
> rpcrdma_buffer *buf)
>  {
>         struct llist_node *node;
> @@ -1049,8 +1056,18 @@ static void rpcrdma_reps_destroy(struct
> rpcrdma_buffer *buf)
>  {
>         struct rpcrdma_rep *rep;
>  
> -       while ((rep = rpcrdma_rep_get_locked(buf)) != NULL)
> -               rpcrdma_rep_destroy(rep);
> +       spin_lock(&buf->rb_lock);
> +       while ((rep = list_first_entry_or_null(&buf->rb_all_reps,
> +                                              struct rpcrdma_rep,
> +                                              rr_all)) != NULL) {
> +               list_del(&rep->rr_all);
> +               spin_unlock(&buf->rb_lock);
> +
> +               rpcrdma_rep_destroy_locked(rep);
> +
> +               spin_lock(&buf->rb_lock);
> +       }
> +       spin_unlock(&buf->rb_lock);
>  }
>  
>  /**
> 
> 

-- 
Trond Myklebust
Linux NFS client maintainer, Hammerspace
trond.myklebust@hammerspace.com



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

* Re: [PATCH v3 07/26] xprtrdma: Improve locking around rpcrdma_rep destruction
  2021-04-23 21:06   ` Trond Myklebust
@ 2021-04-24 17:39     ` Chuck Lever III
  2021-04-24 17:59       ` Trond Myklebust
  0 siblings, 1 reply; 34+ messages in thread
From: Chuck Lever III @ 2021-04-24 17:39 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List, linux-rdma



> On Apr 23, 2021, at 5:06 PM, Trond Myklebust <trondmy@hammerspace.com> wrote:
> 
> On Mon, 2021-04-19 at 14:02 -0400, Chuck Lever wrote:
>> Currently rpcrdma_reps_destroy() assumes that, at transport
>> tear-down, the content of the rb_free_reps list is the same as the
>> content of the rb_all_reps list. Although that is usually true,
>> using the rb_all_reps list should be more reliable because of
>> the way it's managed. And, rpcrdma_reps_unmap() uses rb_all_reps;
>> these two functions should both traverse the "all" list.
>> 
>> Ensure that all rpcrdma_reps are always destroyed whether they are
>> on the rep free list or not.
>> 
>> Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
>> ---
>>  net/sunrpc/xprtrdma/verbs.c |   31 ++++++++++++++++++++++++-------
>>  1 file changed, 24 insertions(+), 7 deletions(-)
>> 
>> diff --git a/net/sunrpc/xprtrdma/verbs.c
>> b/net/sunrpc/xprtrdma/verbs.c
>> index 1b599a623eea..482fdc9e25c2 100644
>> --- a/net/sunrpc/xprtrdma/verbs.c
>> +++ b/net/sunrpc/xprtrdma/verbs.c
>> @@ -1007,16 +1007,23 @@ struct rpcrdma_rep *rpcrdma_rep_create(struct
>> rpcrdma_xprt *r_xprt,
>>         return NULL;
>>  }
>>  
>> -/* No locking needed here. This function is invoked only by the
>> - * Receive completion handler, or during transport shutdown.
>> - */
>> -static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep)
>> +static void rpcrdma_rep_destroy_locked(struct rpcrdma_rep *rep)
> 
> The name here is extremely confusing. As far as I can tell, this isn't
> called with any lock?

Fair enough.

I renamed it rpcrdma_rep_free() and it doesn't seem to have
any consequences for downstream commits in this series.

You could do a global edit, I can resend you this patch with
the change, or I can post a v4 of this series. Let me know
your preference.


>>  {
>> -       list_del(&rep->rr_all);
>>         rpcrdma_regbuf_free(rep->rr_rdmabuf);
>>         kfree(rep);
>>  }
>>  
>> +static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep)
>> +{
>> +       struct rpcrdma_buffer *buf = &rep->rr_rxprt->rx_buf;
>> +
>> +       spin_lock(&buf->rb_lock);
>> +       list_del(&rep->rr_all);
>> +       spin_unlock(&buf->rb_lock);
>> +
>> +       rpcrdma_rep_destroy_locked(rep);
>> +}
>> +
>>  static struct rpcrdma_rep *rpcrdma_rep_get_locked(struct
>> rpcrdma_buffer *buf)
>>  {
>>         struct llist_node *node;
>> @@ -1049,8 +1056,18 @@ static void rpcrdma_reps_destroy(struct
>> rpcrdma_buffer *buf)
>>  {
>>         struct rpcrdma_rep *rep;
>>  
>> -       while ((rep = rpcrdma_rep_get_locked(buf)) != NULL)
>> -               rpcrdma_rep_destroy(rep);
>> +       spin_lock(&buf->rb_lock);
>> +       while ((rep = list_first_entry_or_null(&buf->rb_all_reps,
>> +                                              struct rpcrdma_rep,
>> +                                              rr_all)) != NULL) {
>> +               list_del(&rep->rr_all);
>> +               spin_unlock(&buf->rb_lock);
>> +
>> +               rpcrdma_rep_destroy_locked(rep);
>> +
>> +               spin_lock(&buf->rb_lock);
>> +       }
>> +       spin_unlock(&buf->rb_lock);
>>  }
>>  
>>  /**
>> 
>> 
> 
> -- 
> Trond Myklebust
> Linux NFS client maintainer, Hammerspace
> trond.myklebust@hammerspace.com

--
Chuck Lever




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

* Re: [PATCH v3 07/26] xprtrdma: Improve locking around rpcrdma_rep destruction
  2021-04-24 17:39     ` Chuck Lever III
@ 2021-04-24 17:59       ` Trond Myklebust
  0 siblings, 0 replies; 34+ messages in thread
From: Trond Myklebust @ 2021-04-24 17:59 UTC (permalink / raw)
  To: chuck.lever; +Cc: linux-nfs, linux-rdma

On Sat, 2021-04-24 at 17:39 +0000, Chuck Lever III wrote:
> 
> 
> > On Apr 23, 2021, at 5:06 PM, Trond Myklebust < 
> > trondmy@hammerspace.com> wrote:
> > 
> > On Mon, 2021-04-19 at 14:02 -0400, Chuck Lever wrote:
> > > Currently rpcrdma_reps_destroy() assumes that, at transport
> > > tear-down, the content of the rb_free_reps list is the same as
> > > the
> > > content of the rb_all_reps list. Although that is usually true,
> > > using the rb_all_reps list should be more reliable because of
> > > the way it's managed. And, rpcrdma_reps_unmap() uses rb_all_reps;
> > > these two functions should both traverse the "all" list.
> > > 
> > > Ensure that all rpcrdma_reps are always destroyed whether they
> > > are
> > > on the rep free list or not.
> > > 
> > > Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
> > > ---
> > >  net/sunrpc/xprtrdma/verbs.c |   31 ++++++++++++++++++++++++-----
> > > --
> > >  1 file changed, 24 insertions(+), 7 deletions(-)
> > > 
> > > diff --git a/net/sunrpc/xprtrdma/verbs.c
> > > b/net/sunrpc/xprtrdma/verbs.c
> > > index 1b599a623eea..482fdc9e25c2 100644
> > > --- a/net/sunrpc/xprtrdma/verbs.c
> > > +++ b/net/sunrpc/xprtrdma/verbs.c
> > > @@ -1007,16 +1007,23 @@ struct rpcrdma_rep
> > > *rpcrdma_rep_create(struct
> > > rpcrdma_xprt *r_xprt,
> > >         return NULL;
> > >  }
> > >  
> > > -/* No locking needed here. This function is invoked only by the
> > > - * Receive completion handler, or during transport shutdown.
> > > - */
> > > -static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep)
> > > +static void rpcrdma_rep_destroy_locked(struct rpcrdma_rep *rep)
> > 
> > The name here is extremely confusing. As far as I can tell, this
> > isn't
> > called with any lock?
> 
> Fair enough.
> 
> I renamed it rpcrdma_rep_free() and it doesn't seem to have
> any consequences for downstream commits in this series.

Sounds good.

> 
> You could do a global edit, I can resend you this patch with
> the change, or I can post a v4 of this series. Let me know
> your preference.
> 

Can you please just resend this patch with the update, unless there are
repercussions for the other patches? (in which case a v4 would be
welcome)

> 
> > >  {
> > > -       list_del(&rep->rr_all);
> > >         rpcrdma_regbuf_free(rep->rr_rdmabuf);
> > >         kfree(rep);
> > >  }
> > >  
> > > +static void rpcrdma_rep_destroy(struct rpcrdma_rep *rep)
> > > +{
> > > +       struct rpcrdma_buffer *buf = &rep->rr_rxprt->rx_buf;
> > > +
> > > +       spin_lock(&buf->rb_lock);
> > > +       list_del(&rep->rr_all);
> > > +       spin_unlock(&buf->rb_lock);
> > > +
> > > +       rpcrdma_rep_destroy_locked(rep);
> > > +}
> > > +
> > >  static struct rpcrdma_rep *rpcrdma_rep_get_locked(struct
> > > rpcrdma_buffer *buf)
> > >  {
> > >         struct llist_node *node;
> > > @@ -1049,8 +1056,18 @@ static void rpcrdma_reps_destroy(struct
> > > rpcrdma_buffer *buf)
> > >  {
> > >         struct rpcrdma_rep *rep;
> > >  
> > > -       while ((rep = rpcrdma_rep_get_locked(buf)) != NULL)
> > > -               rpcrdma_rep_destroy(rep);
> > > +       spin_lock(&buf->rb_lock);
> > > +       while ((rep = list_first_entry_or_null(&buf->rb_all_reps,
> > > +                                              struct
> > > rpcrdma_rep,
> > > +                                              rr_all)) != NULL)
> > > {
> > > +               list_del(&rep->rr_all);
> > > +               spin_unlock(&buf->rb_lock);
> > > +
> > > +               rpcrdma_rep_destroy_locked(rep);
> > > +
> > > +               spin_lock(&buf->rb_lock);
> > > +       }
> > > +       spin_unlock(&buf->rb_lock);
> > >  }
> > >  
> > >  /**
> > > 
> > > 
> > 
> > -- 
> > Trond Myklebust
> > Linux NFS client maintainer, Hammerspace
> > trond.myklebust@hammerspace.com
> 
> --
> Chuck Lever
> 
> 
> 

-- 
Trond Myklebust
Linux NFS client maintainer, Hammerspace
trond.myklebust@hammerspace.com



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

* Re: [PATCH v3 15/26] xprtrdma: Do not recycle MR after FastReg/LocalInv flushes
  2021-04-19 18:03 ` [PATCH v3 15/26] xprtrdma: Do not recycle MR after FastReg/LocalInv flushes Chuck Lever
@ 2021-04-25 14:19   ` Dan Aloni
  2021-04-25 16:21     ` Chuck Lever III
  0 siblings, 1 reply; 34+ messages in thread
From: Dan Aloni @ 2021-04-25 14:19 UTC (permalink / raw)
  To: Chuck Lever; +Cc: trondmy, linux-nfs, linux-rdma

On Mon, Apr 19, 2021 at 02:03:12PM -0400, Chuck Lever wrote:
> Better not to touch MRs involved in a flush or post error until the
> Send and Receive Queues are drained and the transport is fully
> quiescent. Simply don't insert such MRs back onto the free list.
> They remain on mr_all and will be released when the connection is
> torn down.
> 
> I had thought that recycling would prevent hardware resources from
> being tied up for a long time. However, since v5.7, a transport
> disconnect destroys the QP and other hardware-owned resources. The
> MRs get cleaned up nicely at that point.
> 
> Signed-off-by: Chuck Lever <chuck.lever@oracle.com>

Is this a fix for the crash below?  I just wonder if it appeared for
others in the wild, and the fix is not just theoretical.

    WARNING: CPU: 5 PID: 20312 at lib/list_debug.c:53 __list_del_entry+0x63/0xd0
    list_del corruption, ffff9df150b06768->next is LIST_POISON1 (dead000000000100)

    Call Trace:
     [<ffffffff99764147>] dump_stack+0x19/0x1b
     [<ffffffff99098848>] __warn+0xd8/0x100
     [<ffffffff990988cf>] warn_slowpath_fmt+0x5f/0x80
     [<ffffffff9921d5f6>] ? kfree+0x106/0x140
     [<ffffffff99396953>] __list_del_entry+0x63/0xd0
     [<ffffffff993969cd>] list_del+0xd/0x30
     [<ffffffffc0bb307f>] frwr_mr_recycle+0xaf/0x150 [rpcrdma]
     [<ffffffffc0bb3264>] frwr_wc_localinv+0x94/0xa0 [rpcrdma]
     [<ffffffffc067d20e>] __ib_process_cq+0x8e/0x100 [ib_core]
     [<ffffffffc067d2f9>] ib_cq_poll_work+0x29/0x70 [ib_core]
     [<ffffffff990baf9f>] process_one_work+0x17f/0x440
     [<ffffffff990bc036>] worker_thread+0x126/0x3c0
     [<ffffffff990bbf10>] ? manage_workers.isra.25+0x2a0/0x2a0
     [<ffffffff990c2e81>] kthread+0xd1/0xe0
     [<ffffffff990c2db0>] ? insert_kthread_work+0x40/0x40
     [<ffffffff99776c37>] ret_from_fork_nospec_begin+0x21/0x21
     [<ffffffff990c2db0>] ? insert_kthread_work+0x40/0x40

-- 
Dan Aloni

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

* Re: [PATCH v3 15/26] xprtrdma: Do not recycle MR after FastReg/LocalInv flushes
  2021-04-25 14:19   ` Dan Aloni
@ 2021-04-25 16:21     ` Chuck Lever III
  2021-04-25 17:00       ` Dan Aloni
  0 siblings, 1 reply; 34+ messages in thread
From: Chuck Lever III @ 2021-04-25 16:21 UTC (permalink / raw)
  To: Dan Aloni; +Cc: Trond Myklebust, Linux NFS Mailing List, linux-rdma



> On Apr 25, 2021, at 10:19 AM, Dan Aloni <dan@kernelim.com> wrote:
> 
> On Mon, Apr 19, 2021 at 02:03:12PM -0400, Chuck Lever wrote:
>> Better not to touch MRs involved in a flush or post error until the
>> Send and Receive Queues are drained and the transport is fully
>> quiescent. Simply don't insert such MRs back onto the free list.
>> They remain on mr_all and will be released when the connection is
>> torn down.
>> 
>> I had thought that recycling would prevent hardware resources from
>> being tied up for a long time. However, since v5.7, a transport
>> disconnect destroys the QP and other hardware-owned resources. The
>> MRs get cleaned up nicely at that point.
>> 
>> Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
> 
> Is this a fix for the crash below?

Yes, it is plausible. That is a familiar backtrace.

However, it's usually because the provider called the LocalInv
completion handler twice for the same CQE. Which provider is this?


> I just wonder if it appeared for
> others in the wild, and the fix is not just theoretical.
> 
>    WARNING: CPU: 5 PID: 20312 at lib/list_debug.c:53 __list_del_entry+0x63/0xd0
>    list_del corruption, ffff9df150b06768->next is LIST_POISON1 (dead000000000100)
> 
>    Call Trace:
>     [<ffffffff99764147>] dump_stack+0x19/0x1b
>     [<ffffffff99098848>] __warn+0xd8/0x100
>     [<ffffffff990988cf>] warn_slowpath_fmt+0x5f/0x80
>     [<ffffffff9921d5f6>] ? kfree+0x106/0x140
>     [<ffffffff99396953>] __list_del_entry+0x63/0xd0
>     [<ffffffff993969cd>] list_del+0xd/0x30
>     [<ffffffffc0bb307f>] frwr_mr_recycle+0xaf/0x150 [rpcrdma]
>     [<ffffffffc0bb3264>] frwr_wc_localinv+0x94/0xa0 [rpcrdma]
>     [<ffffffffc067d20e>] __ib_process_cq+0x8e/0x100 [ib_core]
>     [<ffffffffc067d2f9>] ib_cq_poll_work+0x29/0x70 [ib_core]
>     [<ffffffff990baf9f>] process_one_work+0x17f/0x440
>     [<ffffffff990bc036>] worker_thread+0x126/0x3c0
>     [<ffffffff990bbf10>] ? manage_workers.isra.25+0x2a0/0x2a0
>     [<ffffffff990c2e81>] kthread+0xd1/0xe0
>     [<ffffffff990c2db0>] ? insert_kthread_work+0x40/0x40
>     [<ffffffff99776c37>] ret_from_fork_nospec_begin+0x21/0x21
>     [<ffffffff990c2db0>] ? insert_kthread_work+0x40/0x40
> 
> -- 
> Dan Aloni

--
Chuck Lever




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

* Re: [PATCH v3 15/26] xprtrdma: Do not recycle MR after FastReg/LocalInv flushes
  2021-04-25 16:21     ` Chuck Lever III
@ 2021-04-25 17:00       ` Dan Aloni
  0 siblings, 0 replies; 34+ messages in thread
From: Dan Aloni @ 2021-04-25 17:00 UTC (permalink / raw)
  To: Chuck Lever III; +Cc: Trond Myklebust, Linux NFS Mailing List, linux-rdma

On Sun, Apr 25, 2021 at 04:21:03PM +0000, Chuck Lever III wrote:
> 
> 
> > On Apr 25, 2021, at 10:19 AM, Dan Aloni <dan@kernelim.com> wrote:
> > 
> > On Mon, Apr 19, 2021 at 02:03:12PM -0400, Chuck Lever wrote:
> >> Better not to touch MRs involved in a flush or post error until the
> >> Send and Receive Queues are drained and the transport is fully
> >> quiescent. Simply don't insert such MRs back onto the free list.
> >> They remain on mr_all and will be released when the connection is
> >> torn down.
> >> 
> >> I had thought that recycling would prevent hardware resources from
> >> being tied up for a long time. However, since v5.7, a transport
> >> disconnect destroys the QP and other hardware-owned resources. The
> >> MRs get cleaned up nicely at that point.
> >> 
> >> Signed-off-by: Chuck Lever <chuck.lever@oracle.com>
> > 
> > Is this a fix for the crash below?
> 
> Yes, it is plausible. That is a familiar backtrace.
> 
> However, it's usually because the provider called the LocalInv
> completion handler twice for the same CQE. Which provider is this?

It's mlx5 driver, ConnectX-4 (MT27700).

-- 
Dan Aloni

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

end of thread, other threads:[~2021-04-25 17:00 UTC | newest]

Thread overview: 34+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-04-19 18:01 [PATCH v3 00/26] NFS/RDMA client patches for next Chuck Lever
2021-04-19 18:01 ` [PATCH v3 01/26] SUNRPC: Move fault injection call sites Chuck Lever
2021-04-19 18:05   ` Chuck Lever III
2021-04-19 18:01 ` [PATCH v3 02/26] SUNRPC: Remove trace_xprt_transmit_queued Chuck Lever
2021-04-19 18:01 ` [PATCH v3 03/26] SUNRPC: Add tracepoint that fires when an RPC is retransmitted Chuck Lever
2021-04-19 18:02 ` [PATCH v3 04/26] xprtrdma: Avoid Receive Queue wrapping Chuck Lever
2021-04-19 18:02 ` [PATCH v3 05/26] xprtrdma: Do not refresh Receive Queue while it is draining Chuck Lever
2021-04-19 18:02 ` [PATCH v3 06/26] xprtrdma: Put flushed Receives on free list instead of destroying them Chuck Lever
2021-04-19 18:02 ` [PATCH v3 07/26] xprtrdma: Improve locking around rpcrdma_rep destruction Chuck Lever
2021-04-23 21:06   ` Trond Myklebust
2021-04-24 17:39     ` Chuck Lever III
2021-04-24 17:59       ` Trond Myklebust
2021-04-19 18:02 ` [PATCH v3 08/26] xprtrdma: Improve commentary around rpcrdma_reps_unmap() Chuck Lever
2021-04-19 18:02 ` [PATCH v3 09/26] xprtrdma: Improve locking around rpcrdma_rep creation Chuck Lever
2021-04-19 18:02 ` [PATCH v3 10/26] xprtrdma: Fix cwnd update ordering Chuck Lever
2021-04-19 18:02 ` [PATCH v3 11/26] xprtrdma: Delete rpcrdma_recv_buffer_put() Chuck Lever
2021-04-19 18:02 ` [PATCH v3 12/26] xprtrdma: rpcrdma_mr_pop() already does list_del_init() Chuck Lever
2021-04-19 18:03 ` [PATCH v3 13/26] xprtrdma: Rename frwr_release_mr() Chuck Lever
2021-04-19 18:03 ` [PATCH v3 14/26] xprtrdma: Clarify use of barrier in frwr_wc_localinv_done() Chuck Lever
2021-04-19 18:03 ` [PATCH v3 15/26] xprtrdma: Do not recycle MR after FastReg/LocalInv flushes Chuck Lever
2021-04-25 14:19   ` Dan Aloni
2021-04-25 16:21     ` Chuck Lever III
2021-04-25 17:00       ` Dan Aloni
2021-04-19 18:03 ` [PATCH v3 16/26] xprtrdma: Do not wake RPC consumer on a failed LocalInv Chuck Lever
2021-04-19 18:03 ` [PATCH v3 17/26] xprtrdma: Avoid Send Queue wrapping Chuck Lever
2021-04-19 18:03 ` [PATCH v3 18/26] xprtrdma: Add tracepoints showing FastReg WRs and remote invalidation Chuck Lever
2021-04-19 18:03 ` [PATCH v3 19/26] xprtrdma: Add an rpcrdma_mr_completion_class Chuck Lever
2021-04-19 18:03 ` [PATCH v3 20/26] xprtrdma: Don't display r_xprt memory addresses in tracepoints Chuck Lever
2021-04-19 18:03 ` [PATCH v3 21/26] xprtrdma: Remove the RPC/RDMA QP event handler Chuck Lever
2021-04-19 18:03 ` [PATCH v3 22/26] xprtrdma: Move fr_cid to struct rpcrdma_mr Chuck Lever
2021-04-19 18:04 ` [PATCH v3 23/26] xprtrdma: Move cqe " Chuck Lever
2021-04-19 18:04 ` [PATCH v3 24/26] xprtrdma: Move fr_linv_done field " Chuck Lever
2021-04-19 18:04 ` [PATCH v3 25/26] xprtrdma: Move the Work Request union " Chuck Lever
2021-04-19 18:04 ` [PATCH v3 26/26] xprtrdma: Move fr_mr field " Chuck Lever

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).