linux-nfs.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC PATCH 0/5] bh-safe lock removal for SUNRPC
@ 2019-05-03 11:18 Trond Myklebust
  2019-05-03 11:18 ` [RFC PATCH 1/5] SUNRPC: Replace the queue timer with a delayed work function Trond Myklebust
  2019-05-06 18:22 ` [RFC PATCH 0/5] bh-safe lock removal for SUNRPC Chuck Lever
  0 siblings, 2 replies; 20+ messages in thread
From: Trond Myklebust @ 2019-05-03 11:18 UTC (permalink / raw)
  To: Chuck Lever; +Cc: linux-nfs

This patchset aims to remove the bh-safe locks on the client side.
At this time it should be seen as a toy/strawman effort in order to
help the community figure out whether or not there are setups out
there that are actually seeing performance bottlenecks resulting
from taking bh-safe locks inside other spinlocks.

Trond Myklebust (5):
  SUNRPC: Replace the queue timer with a delayed work function
  SUNRPC: Replace direct task wakeups from softirq context
  SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock
  SUNRPC: Remove the bh-safe lock requirement on the
    rpc_wait_queue->lock
  SUNRPC: Reduce the priority of the xprtiod queue

 include/linux/sunrpc/sched.h               |   3 +-
 include/linux/sunrpc/xprtsock.h            |   5 +
 net/sunrpc/sched.c                         |  76 +++++++++-------
 net/sunrpc/xprt.c                          |  61 ++++++-------
 net/sunrpc/xprtrdma/rpc_rdma.c             |   4 +-
 net/sunrpc/xprtrdma/svc_rdma_backchannel.c |   4 +-
 net/sunrpc/xprtrdma/svc_rdma_transport.c   |   8 +-
 net/sunrpc/xprtsock.c                      | 101 +++++++++++++++++----
 8 files changed, 168 insertions(+), 94 deletions(-)

-- 
2.21.0


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

* [RFC PATCH 1/5] SUNRPC: Replace the queue timer with a delayed work function
  2019-05-03 11:18 [RFC PATCH 0/5] bh-safe lock removal for SUNRPC Trond Myklebust
@ 2019-05-03 11:18 ` Trond Myklebust
  2019-05-03 11:18   ` [RFC PATCH 2/5] SUNRPC: Replace direct task wakeups from softirq context Trond Myklebust
  2019-05-06 18:22 ` [RFC PATCH 0/5] bh-safe lock removal for SUNRPC Chuck Lever
  1 sibling, 1 reply; 20+ messages in thread
From: Trond Myklebust @ 2019-05-03 11:18 UTC (permalink / raw)
  To: Chuck Lever; +Cc: linux-nfs

The queue timer function, which walks the RPC queue in order to locate
candidates for waking up is one of the current constraints against
removing the bh-safe queue spin locks. Replace it with a delayed
work queue, so that we can do the actual rpc task wake ups from an
ordinary process context.

Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
---
 include/linux/sunrpc/sched.h |  3 ++-
 net/sunrpc/sched.c           | 32 ++++++++++++++++++++------------
 2 files changed, 22 insertions(+), 13 deletions(-)

diff --git a/include/linux/sunrpc/sched.h b/include/linux/sunrpc/sched.h
index d0e451868f02..7d8db5dcac04 100644
--- a/include/linux/sunrpc/sched.h
+++ b/include/linux/sunrpc/sched.h
@@ -183,8 +183,9 @@ struct rpc_task_setup {
 #define RPC_NR_PRIORITY		(1 + RPC_PRIORITY_PRIVILEGED - RPC_PRIORITY_LOW)
 
 struct rpc_timer {
-	struct timer_list timer;
 	struct list_head list;
+	unsigned long expires;
+	struct delayed_work dwork;
 };
 
 /*
diff --git a/net/sunrpc/sched.c b/net/sunrpc/sched.c
index 1a12fb03e611..e7723c2c1b1c 100644
--- a/net/sunrpc/sched.c
+++ b/net/sunrpc/sched.c
@@ -45,7 +45,7 @@ static mempool_t	*rpc_buffer_mempool __read_mostly;
 
 static void			rpc_async_schedule(struct work_struct *);
 static void			 rpc_release_task(struct rpc_task *task);
-static void __rpc_queue_timer_fn(struct timer_list *t);
+static void __rpc_queue_timer_fn(struct work_struct *);
 
 /*
  * RPC tasks sit here while waiting for conditions to improve.
@@ -86,13 +86,19 @@ __rpc_disable_timer(struct rpc_wait_queue *queue, struct rpc_task *task)
 	task->tk_timeout = 0;
 	list_del(&task->u.tk_wait.timer_list);
 	if (list_empty(&queue->timer_list.list))
-		del_timer(&queue->timer_list.timer);
+		cancel_delayed_work(&queue->timer_list.dwork);
 }
 
 static void
 rpc_set_queue_timer(struct rpc_wait_queue *queue, unsigned long expires)
 {
-	timer_reduce(&queue->timer_list.timer, expires);
+	unsigned long now = jiffies;
+	queue->timer_list.expires = expires;
+	if (time_before_eq(expires, now))
+		expires = 0;
+	else
+		expires -= now;
+	mod_delayed_work(rpciod_workqueue, &queue->timer_list.dwork, expires);
 }
 
 /*
@@ -106,7 +112,8 @@ __rpc_add_timer(struct rpc_wait_queue *queue, struct rpc_task *task,
 		task->tk_pid, jiffies_to_msecs(timeout - jiffies));
 
 	task->tk_timeout = timeout;
-	rpc_set_queue_timer(queue, timeout);
+	if (list_empty(&queue->timer_list.list) || time_before(timeout, queue->timer_list.expires))
+		rpc_set_queue_timer(queue, timeout);
 	list_add(&task->u.tk_wait.timer_list, &queue->timer_list.list);
 }
 
@@ -249,9 +256,8 @@ static void __rpc_init_priority_wait_queue(struct rpc_wait_queue *queue, const c
 	queue->maxpriority = nr_queues - 1;
 	rpc_reset_waitqueue_priority(queue);
 	queue->qlen = 0;
-	timer_setup(&queue->timer_list.timer,
-			__rpc_queue_timer_fn,
-			TIMER_DEFERRABLE);
+	queue->timer_list.expires = 0;
+	INIT_DEFERRABLE_WORK(&queue->timer_list.dwork, __rpc_queue_timer_fn);
 	INIT_LIST_HEAD(&queue->timer_list.list);
 	rpc_assign_waitqueue_name(queue, qname);
 }
@@ -270,7 +276,7 @@ EXPORT_SYMBOL_GPL(rpc_init_wait_queue);
 
 void rpc_destroy_wait_queue(struct rpc_wait_queue *queue)
 {
-	del_timer_sync(&queue->timer_list.timer);
+	cancel_delayed_work_sync(&queue->timer_list.dwork);
 }
 EXPORT_SYMBOL_GPL(rpc_destroy_wait_queue);
 
@@ -760,13 +766,15 @@ void rpc_wake_up_status(struct rpc_wait_queue *queue, int status)
 }
 EXPORT_SYMBOL_GPL(rpc_wake_up_status);
 
-static void __rpc_queue_timer_fn(struct timer_list *t)
+static void __rpc_queue_timer_fn(struct work_struct *work)
 {
-	struct rpc_wait_queue *queue = from_timer(queue, t, timer_list.timer);
+	struct rpc_wait_queue *queue = container_of(work,
+			struct rpc_wait_queue,
+			timer_list.dwork.work);
 	struct rpc_task *task, *n;
 	unsigned long expires, now, timeo;
 
-	spin_lock(&queue->lock);
+	spin_lock_bh(&queue->lock);
 	expires = now = jiffies;
 	list_for_each_entry_safe(task, n, &queue->timer_list.list, u.tk_wait.timer_list) {
 		timeo = task->tk_timeout;
@@ -781,7 +789,7 @@ static void __rpc_queue_timer_fn(struct timer_list *t)
 	}
 	if (!list_empty(&queue->timer_list.list))
 		rpc_set_queue_timer(queue, expires);
-	spin_unlock(&queue->lock);
+	spin_unlock_bh(&queue->lock);
 }
 
 static void __rpc_atrun(struct rpc_task *task)
-- 
2.21.0


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

* [RFC PATCH 2/5] SUNRPC: Replace direct task wakeups from softirq context
  2019-05-03 11:18 ` [RFC PATCH 1/5] SUNRPC: Replace the queue timer with a delayed work function Trond Myklebust
@ 2019-05-03 11:18   ` Trond Myklebust
  2019-05-03 11:18     ` [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock Trond Myklebust
  0 siblings, 1 reply; 20+ messages in thread
From: Trond Myklebust @ 2019-05-03 11:18 UTC (permalink / raw)
  To: Chuck Lever; +Cc: linux-nfs

Replace the direct task wakeups from inside a softirq context with
wakeups from a process context.

Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
---
 include/linux/sunrpc/xprtsock.h |  5 +++
 net/sunrpc/xprtsock.c           | 78 ++++++++++++++++++++++++++++++---
 2 files changed, 77 insertions(+), 6 deletions(-)

diff --git a/include/linux/sunrpc/xprtsock.h b/include/linux/sunrpc/xprtsock.h
index b81d0b3e0799..7638dbe7bc50 100644
--- a/include/linux/sunrpc/xprtsock.h
+++ b/include/linux/sunrpc/xprtsock.h
@@ -56,6 +56,7 @@ struct sock_xprt {
 	 */
 	unsigned long		sock_state;
 	struct delayed_work	connect_worker;
+	struct work_struct	error_worker;
 	struct work_struct	recv_worker;
 	struct mutex		recv_mutex;
 	struct sockaddr_storage	srcaddr;
@@ -84,6 +85,10 @@ struct sock_xprt {
 #define XPRT_SOCK_CONNECTING	1U
 #define XPRT_SOCK_DATA_READY	(2)
 #define XPRT_SOCK_UPD_TIMEOUT	(3)
+#define XPRT_SOCK_WAKE_ERROR	(4)
+#define XPRT_SOCK_WAKE_WRITE	(5)
+#define XPRT_SOCK_WAKE_PENDING	(6)
+#define XPRT_SOCK_WAKE_DISCONNECT	(7)
 
 #endif /* __KERNEL__ */
 
diff --git a/net/sunrpc/xprtsock.c b/net/sunrpc/xprtsock.c
index c69951ed2ebc..e0195b1a0c18 100644
--- a/net/sunrpc/xprtsock.c
+++ b/net/sunrpc/xprtsock.c
@@ -1211,6 +1211,15 @@ static void xs_sock_reset_state_flags(struct rpc_xprt *xprt)
 	struct sock_xprt *transport = container_of(xprt, struct sock_xprt, xprt);
 
 	clear_bit(XPRT_SOCK_DATA_READY, &transport->sock_state);
+	clear_bit(XPRT_SOCK_WAKE_ERROR, &transport->sock_state);
+	clear_bit(XPRT_SOCK_WAKE_WRITE, &transport->sock_state);
+	clear_bit(XPRT_SOCK_WAKE_DISCONNECT, &transport->sock_state);
+}
+
+static void xs_run_error_worker(struct sock_xprt *transport, unsigned int nr)
+{
+	set_bit(nr, &transport->sock_state);
+	queue_work(xprtiod_workqueue, &transport->error_worker);
 }
 
 static void xs_sock_reset_connection_flags(struct rpc_xprt *xprt)
@@ -1231,6 +1240,7 @@ static void xs_sock_reset_connection_flags(struct rpc_xprt *xprt)
  */
 static void xs_error_report(struct sock *sk)
 {
+	struct sock_xprt *transport;
 	struct rpc_xprt *xprt;
 	int err;
 
@@ -1238,13 +1248,14 @@ static void xs_error_report(struct sock *sk)
 	if (!(xprt = xprt_from_sock(sk)))
 		goto out;
 
+	transport = container_of(xprt, struct sock_xprt, xprt);
 	err = -sk->sk_err;
 	if (err == 0)
 		goto out;
 	dprintk("RPC:       xs_error_report client %p, error=%d...\n",
 			xprt, -err);
 	trace_rpc_socket_error(xprt, sk->sk_socket, err);
-	xprt_wake_pending_tasks(xprt, err);
+	xs_run_error_worker(transport, XPRT_SOCK_WAKE_ERROR);
  out:
 	read_unlock_bh(&sk->sk_callback_lock);
 }
@@ -1507,7 +1518,7 @@ static void xs_tcp_state_change(struct sock *sk)
 			xprt->stat.connect_count++;
 			xprt->stat.connect_time += (long)jiffies -
 						   xprt->stat.connect_start;
-			xprt_wake_pending_tasks(xprt, -EAGAIN);
+			xs_run_error_worker(transport, XPRT_SOCK_WAKE_PENDING);
 		}
 		spin_unlock(&xprt->transport_lock);
 		break;
@@ -1525,7 +1536,7 @@ static void xs_tcp_state_change(struct sock *sk)
 		/* The server initiated a shutdown of the socket */
 		xprt->connect_cookie++;
 		clear_bit(XPRT_CONNECTED, &xprt->state);
-		xs_tcp_force_close(xprt);
+		xs_run_error_worker(transport, XPRT_SOCK_WAKE_DISCONNECT);
 		/* fall through */
 	case TCP_CLOSING:
 		/*
@@ -1547,7 +1558,7 @@ static void xs_tcp_state_change(struct sock *sk)
 			xprt_clear_connecting(xprt);
 		clear_bit(XPRT_CLOSING, &xprt->state);
 		/* Trigger the socket release */
-		xs_tcp_force_close(xprt);
+		xs_run_error_worker(transport, XPRT_SOCK_WAKE_DISCONNECT);
 	}
  out:
 	read_unlock_bh(&sk->sk_callback_lock);
@@ -1556,6 +1567,7 @@ static void xs_tcp_state_change(struct sock *sk)
 static void xs_write_space(struct sock *sk)
 {
 	struct socket_wq *wq;
+	struct sock_xprt *transport;
 	struct rpc_xprt *xprt;
 
 	if (!sk->sk_socket)
@@ -1564,13 +1576,14 @@ static void xs_write_space(struct sock *sk)
 
 	if (unlikely(!(xprt = xprt_from_sock(sk))))
 		return;
+	transport = container_of(xprt, struct sock_xprt, xprt);
 	rcu_read_lock();
 	wq = rcu_dereference(sk->sk_wq);
 	if (!wq || test_and_clear_bit(SOCKWQ_ASYNC_NOSPACE, &wq->flags) == 0)
 		goto out;
 
-	if (xprt_write_space(xprt))
-		sk->sk_write_pending--;
+	xs_run_error_worker(transport, XPRT_SOCK_WAKE_WRITE);
+	sk->sk_write_pending--;
 out:
 	rcu_read_unlock();
 }
@@ -2461,6 +2474,56 @@ static void xs_connect(struct rpc_xprt *xprt, struct rpc_task *task)
 			delay);
 }
 
+static void xs_wake_disconnect(struct sock_xprt *transport)
+{
+	if (test_and_clear_bit(XPRT_SOCK_WAKE_DISCONNECT, &transport->sock_state))
+		xs_tcp_force_close(&transport->xprt);
+}
+
+static void xs_wake_write(struct sock_xprt *transport)
+{
+	if (test_and_clear_bit(XPRT_SOCK_WAKE_WRITE, &transport->sock_state))
+		xprt_write_space(&transport->xprt);
+}
+
+static void xs_wake_error(struct sock_xprt *transport)
+{
+	int sockerr;
+	int sockerr_len = sizeof(sockerr);
+
+	if (!test_bit(XPRT_SOCK_WAKE_ERROR, &transport->sock_state))
+		return;
+	mutex_lock(&transport->recv_mutex);
+	if (transport->sock == NULL)
+		goto out;
+	if (!test_and_clear_bit(XPRT_SOCK_WAKE_ERROR, &transport->sock_state))
+		goto out;
+	if (kernel_getsockopt(transport->sock, SOL_SOCKET, SO_ERROR,
+				(char *)&sockerr, &sockerr_len) != 0)
+		goto out;
+	if (sockerr < 0)
+		xprt_wake_pending_tasks(&transport->xprt, sockerr);
+out:
+	mutex_unlock(&transport->recv_mutex);
+}
+
+static void xs_wake_pending(struct sock_xprt *transport)
+{
+	if (test_and_clear_bit(XPRT_SOCK_WAKE_PENDING, &transport->sock_state))
+		xprt_wake_pending_tasks(&transport->xprt, -EAGAIN);
+}
+
+static void xs_error_handle(struct work_struct *work)
+{
+	struct sock_xprt *transport = container_of(work,
+			struct sock_xprt, error_worker);
+
+	xs_wake_disconnect(transport);
+	xs_wake_write(transport);
+	xs_wake_error(transport);
+	xs_wake_pending(transport);
+}
+
 /**
  * xs_local_print_stats - display AF_LOCAL socket-specifc stats
  * @xprt: rpc_xprt struct containing statistics
@@ -2873,6 +2936,7 @@ static struct rpc_xprt *xs_setup_local(struct xprt_create *args)
 	xprt->timeout = &xs_local_default_timeout;
 
 	INIT_WORK(&transport->recv_worker, xs_stream_data_receive_workfn);
+	INIT_WORK(&transport->error_worker, xs_error_handle);
 	INIT_DELAYED_WORK(&transport->connect_worker, xs_dummy_setup_socket);
 
 	switch (sun->sun_family) {
@@ -2943,6 +3007,7 @@ static struct rpc_xprt *xs_setup_udp(struct xprt_create *args)
 	xprt->timeout = &xs_udp_default_timeout;
 
 	INIT_WORK(&transport->recv_worker, xs_udp_data_receive_workfn);
+	INIT_WORK(&transport->error_worker, xs_error_handle);
 	INIT_DELAYED_WORK(&transport->connect_worker, xs_udp_setup_socket);
 
 	switch (addr->sa_family) {
@@ -3024,6 +3089,7 @@ static struct rpc_xprt *xs_setup_tcp(struct xprt_create *args)
 		(xprt->timeout->to_retries + 1);
 
 	INIT_WORK(&transport->recv_worker, xs_stream_data_receive_workfn);
+	INIT_WORK(&transport->error_worker, xs_error_handle);
 	INIT_DELAYED_WORK(&transport->connect_worker, xs_tcp_setup_socket);
 
 	switch (addr->sa_family) {
-- 
2.21.0


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

* [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock
  2019-05-03 11:18   ` [RFC PATCH 2/5] SUNRPC: Replace direct task wakeups from softirq context Trond Myklebust
@ 2019-05-03 11:18     ` Trond Myklebust
  2019-05-03 11:18       ` [RFC PATCH 4/5] SUNRPC: Remove the bh-safe lock requirement on the rpc_wait_queue->lock Trond Myklebust
  2019-05-03 14:21       ` [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock Chuck Lever
  0 siblings, 2 replies; 20+ messages in thread
From: Trond Myklebust @ 2019-05-03 11:18 UTC (permalink / raw)
  To: Chuck Lever; +Cc: linux-nfs

Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
---
 net/sunrpc/xprt.c                          | 61 ++++++++++------------
 net/sunrpc/xprtrdma/rpc_rdma.c             |  4 +-
 net/sunrpc/xprtrdma/svc_rdma_backchannel.c |  4 +-
 net/sunrpc/xprtrdma/svc_rdma_transport.c   |  8 +--
 net/sunrpc/xprtsock.c                      | 23 ++++----
 5 files changed, 47 insertions(+), 53 deletions(-)

diff --git a/net/sunrpc/xprt.c b/net/sunrpc/xprt.c
index bc1c8247750d..b87d185cf010 100644
--- a/net/sunrpc/xprt.c
+++ b/net/sunrpc/xprt.c
@@ -301,9 +301,9 @@ static inline int xprt_lock_write(struct rpc_xprt *xprt, struct rpc_task *task)
 
 	if (test_bit(XPRT_LOCKED, &xprt->state) && xprt->snd_task == task)
 		return 1;
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	retval = xprt->ops->reserve_xprt(xprt, task);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 	return retval;
 }
 
@@ -380,9 +380,9 @@ static inline void xprt_release_write(struct rpc_xprt *xprt, struct rpc_task *ta
 {
 	if (xprt->snd_task != task)
 		return;
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	xprt->ops->release_xprt(xprt, task);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 }
 
 /*
@@ -434,9 +434,9 @@ xprt_request_get_cong(struct rpc_xprt *xprt, struct rpc_rqst *req)
 
 	if (req->rq_cong)
 		return true;
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	ret = __xprt_get_cong(xprt, req) != 0;
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 	return ret;
 }
 EXPORT_SYMBOL_GPL(xprt_request_get_cong);
@@ -463,9 +463,9 @@ static void
 xprt_clear_congestion_window_wait(struct rpc_xprt *xprt)
 {
 	if (test_and_clear_bit(XPRT_CWND_WAIT, &xprt->state)) {
-		spin_lock_bh(&xprt->transport_lock);
+		spin_lock(&xprt->transport_lock);
 		__xprt_lock_write_next_cong(xprt);
-		spin_unlock_bh(&xprt->transport_lock);
+		spin_unlock(&xprt->transport_lock);
 	}
 }
 
@@ -562,9 +562,9 @@ bool xprt_write_space(struct rpc_xprt *xprt)
 
 	if (!test_bit(XPRT_WRITE_SPACE, &xprt->state))
 		return false;
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	ret = xprt_clear_write_space_locked(xprt);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 	return ret;
 }
 EXPORT_SYMBOL_GPL(xprt_write_space);
@@ -633,9 +633,9 @@ int xprt_adjust_timeout(struct rpc_rqst *req)
 		req->rq_retries = 0;
 		xprt_reset_majortimeo(req);
 		/* Reset the RTT counters == "slow start" */
-		spin_lock_bh(&xprt->transport_lock);
+		spin_lock(&xprt->transport_lock);
 		rpc_init_rtt(req->rq_task->tk_client->cl_rtt, to->to_initval);
-		spin_unlock_bh(&xprt->transport_lock);
+		spin_unlock(&xprt->transport_lock);
 		status = -ETIMEDOUT;
 	}
 
@@ -667,11 +667,11 @@ static void xprt_autoclose(struct work_struct *work)
 void xprt_disconnect_done(struct rpc_xprt *xprt)
 {
 	dprintk("RPC:       disconnected transport %p\n", xprt);
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	xprt_clear_connected(xprt);
 	xprt_clear_write_space_locked(xprt);
 	xprt_wake_pending_tasks(xprt, -ENOTCONN);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 }
 EXPORT_SYMBOL_GPL(xprt_disconnect_done);
 
@@ -683,7 +683,7 @@ EXPORT_SYMBOL_GPL(xprt_disconnect_done);
 void xprt_force_disconnect(struct rpc_xprt *xprt)
 {
 	/* Don't race with the test_bit() in xprt_clear_locked() */
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	set_bit(XPRT_CLOSE_WAIT, &xprt->state);
 	/* Try to schedule an autoclose RPC call */
 	if (test_and_set_bit(XPRT_LOCKED, &xprt->state) == 0)
@@ -691,7 +691,7 @@ void xprt_force_disconnect(struct rpc_xprt *xprt)
 	else if (xprt->snd_task)
 		rpc_wake_up_queued_task_set_status(&xprt->pending,
 				xprt->snd_task, -ENOTCONN);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 }
 EXPORT_SYMBOL_GPL(xprt_force_disconnect);
 
@@ -725,7 +725,7 @@ xprt_request_retransmit_after_disconnect(struct rpc_task *task)
 void xprt_conditional_disconnect(struct rpc_xprt *xprt, unsigned int cookie)
 {
 	/* Don't race with the test_bit() in xprt_clear_locked() */
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	if (cookie != xprt->connect_cookie)
 		goto out;
 	if (test_bit(XPRT_CLOSING, &xprt->state))
@@ -736,7 +736,7 @@ void xprt_conditional_disconnect(struct rpc_xprt *xprt, unsigned int cookie)
 		queue_work(xprtiod_workqueue, &xprt->task_cleanup);
 	xprt_wake_pending_tasks(xprt, -EAGAIN);
 out:
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 }
 
 static bool
@@ -758,18 +758,13 @@ xprt_init_autodisconnect(struct timer_list *t)
 {
 	struct rpc_xprt *xprt = from_timer(xprt, t, timer);
 
-	spin_lock(&xprt->transport_lock);
 	if (!RB_EMPTY_ROOT(&xprt->recv_queue))
-		goto out_abort;
+		return;
 	/* Reset xprt->last_used to avoid connect/autodisconnect cycling */
 	xprt->last_used = jiffies;
 	if (test_and_set_bit(XPRT_LOCKED, &xprt->state))
-		goto out_abort;
-	spin_unlock(&xprt->transport_lock);
+		return;
 	queue_work(xprtiod_workqueue, &xprt->task_cleanup);
-	return;
-out_abort:
-	spin_unlock(&xprt->transport_lock);
 }
 
 bool xprt_lock_connect(struct rpc_xprt *xprt,
@@ -778,7 +773,7 @@ bool xprt_lock_connect(struct rpc_xprt *xprt,
 {
 	bool ret = false;
 
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	if (!test_bit(XPRT_LOCKED, &xprt->state))
 		goto out;
 	if (xprt->snd_task != task)
@@ -786,13 +781,13 @@ bool xprt_lock_connect(struct rpc_xprt *xprt,
 	xprt->snd_task = cookie;
 	ret = true;
 out:
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 	return ret;
 }
 
 void xprt_unlock_connect(struct rpc_xprt *xprt, void *cookie)
 {
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	if (xprt->snd_task != cookie)
 		goto out;
 	if (!test_bit(XPRT_LOCKED, &xprt->state))
@@ -801,7 +796,7 @@ void xprt_unlock_connect(struct rpc_xprt *xprt, void *cookie)
 	xprt->ops->release_xprt(xprt, NULL);
 	xprt_schedule_autodisconnect(xprt);
 out:
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 	wake_up_bit(&xprt->state, XPRT_LOCKED);
 }
 
@@ -1411,14 +1406,14 @@ xprt_request_transmit(struct rpc_rqst *req, struct rpc_task *snd_task)
 	xprt_inject_disconnect(xprt);
 
 	task->tk_flags |= RPC_TASK_SENT;
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 
 	xprt->stat.sends++;
 	xprt->stat.req_u += xprt->stat.sends - xprt->stat.recvs;
 	xprt->stat.bklog_u += xprt->backlog.qlen;
 	xprt->stat.sending_u += xprt->sending.qlen;
 	xprt->stat.pending_u += xprt->pending.qlen;
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 
 	req->rq_connect_cookie = connect_cookie;
 out_dequeue:
@@ -1769,13 +1764,13 @@ void xprt_release(struct rpc_task *task)
 	else if (task->tk_client)
 		rpc_count_iostats(task, task->tk_client->cl_metrics);
 	xprt_request_dequeue_all(task, req);
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	xprt->ops->release_xprt(xprt, task);
 	if (xprt->ops->release_request)
 		xprt->ops->release_request(task);
 	xprt->last_used = jiffies;
 	xprt_schedule_autodisconnect(xprt);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 	if (req->rq_buffer)
 		xprt->ops->buf_free(task);
 	xprt_inject_disconnect(xprt);
diff --git a/net/sunrpc/xprtrdma/rpc_rdma.c b/net/sunrpc/xprtrdma/rpc_rdma.c
index 6c1fb270f127..26419be782d0 100644
--- a/net/sunrpc/xprtrdma/rpc_rdma.c
+++ b/net/sunrpc/xprtrdma/rpc_rdma.c
@@ -1359,10 +1359,10 @@ void rpcrdma_reply_handler(struct rpcrdma_rep *rep)
 	else if (credits > buf->rb_max_requests)
 		credits = buf->rb_max_requests;
 	if (buf->rb_credits != credits) {
-		spin_lock_bh(&xprt->transport_lock);
+		spin_lock(&xprt->transport_lock);
 		buf->rb_credits = credits;
 		xprt->cwnd = credits << RPC_CWNDSHIFT;
-		spin_unlock_bh(&xprt->transport_lock);
+		spin_unlock(&xprt->transport_lock);
 	}
 
 	req = rpcr_to_rdmar(rqst);
diff --git a/net/sunrpc/xprtrdma/svc_rdma_backchannel.c b/net/sunrpc/xprtrdma/svc_rdma_backchannel.c
index bed57d8b5c19..d1fcc41d5eb5 100644
--- a/net/sunrpc/xprtrdma/svc_rdma_backchannel.c
+++ b/net/sunrpc/xprtrdma/svc_rdma_backchannel.c
@@ -72,9 +72,9 @@ int svc_rdma_handle_bc_reply(struct rpc_xprt *xprt, __be32 *rdma_resp,
 	else if (credits > r_xprt->rx_buf.rb_bc_max_requests)
 		credits = r_xprt->rx_buf.rb_bc_max_requests;
 
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	xprt->cwnd = credits << RPC_CWNDSHIFT;
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 
 	spin_lock(&xprt->queue_lock);
 	ret = 0;
diff --git a/net/sunrpc/xprtrdma/svc_rdma_transport.c b/net/sunrpc/xprtrdma/svc_rdma_transport.c
index 027a3b07d329..18ffc6190ea9 100644
--- a/net/sunrpc/xprtrdma/svc_rdma_transport.c
+++ b/net/sunrpc/xprtrdma/svc_rdma_transport.c
@@ -221,9 +221,9 @@ static void handle_connect_req(struct rdma_cm_id *new_cma_id,
 	 * Enqueue the new transport on the accept queue of the listening
 	 * transport
 	 */
-	spin_lock_bh(&listen_xprt->sc_lock);
+	spin_lock(&listen_xprt->sc_lock);
 	list_add_tail(&newxprt->sc_accept_q, &listen_xprt->sc_accept_q);
-	spin_unlock_bh(&listen_xprt->sc_lock);
+	spin_unlock(&listen_xprt->sc_lock);
 
 	set_bit(XPT_CONN, &listen_xprt->sc_xprt.xpt_flags);
 	svc_xprt_enqueue(&listen_xprt->sc_xprt);
@@ -396,7 +396,7 @@ static struct svc_xprt *svc_rdma_accept(struct svc_xprt *xprt)
 	listen_rdma = container_of(xprt, struct svcxprt_rdma, sc_xprt);
 	clear_bit(XPT_CONN, &xprt->xpt_flags);
 	/* Get the next entry off the accept list */
-	spin_lock_bh(&listen_rdma->sc_lock);
+	spin_lock(&listen_rdma->sc_lock);
 	if (!list_empty(&listen_rdma->sc_accept_q)) {
 		newxprt = list_entry(listen_rdma->sc_accept_q.next,
 				     struct svcxprt_rdma, sc_accept_q);
@@ -404,7 +404,7 @@ static struct svc_xprt *svc_rdma_accept(struct svc_xprt *xprt)
 	}
 	if (!list_empty(&listen_rdma->sc_accept_q))
 		set_bit(XPT_CONN, &listen_rdma->sc_xprt.xpt_flags);
-	spin_unlock_bh(&listen_rdma->sc_lock);
+	spin_unlock(&listen_rdma->sc_lock);
 	if (!newxprt)
 		return NULL;
 
diff --git a/net/sunrpc/xprtsock.c b/net/sunrpc/xprtsock.c
index e0195b1a0c18..d7b8e95a61c8 100644
--- a/net/sunrpc/xprtsock.c
+++ b/net/sunrpc/xprtsock.c
@@ -880,7 +880,7 @@ static int xs_nospace(struct rpc_rqst *req)
 			req->rq_slen);
 
 	/* Protect against races with write_space */
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 
 	/* Don't race with disconnect */
 	if (xprt_connected(xprt)) {
@@ -890,7 +890,7 @@ static int xs_nospace(struct rpc_rqst *req)
 	} else
 		ret = -ENOTCONN;
 
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 
 	/* Race breaker in case memory is freed before above code is called */
 	if (ret == -EAGAIN) {
@@ -1344,6 +1344,7 @@ static void xs_destroy(struct rpc_xprt *xprt)
 	cancel_delayed_work_sync(&transport->connect_worker);
 	xs_close(xprt);
 	cancel_work_sync(&transport->recv_worker);
+	cancel_work_sync(&transport->error_worker);
 	xs_xprt_free(xprt);
 	module_put(THIS_MODULE);
 }
@@ -1397,9 +1398,9 @@ static void xs_udp_data_read_skb(struct rpc_xprt *xprt,
 	}
 
 
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	xprt_adjust_cwnd(xprt, task, copied);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 	spin_lock(&xprt->queue_lock);
 	xprt_complete_rqst(task, copied);
 	__UDPX_INC_STATS(sk, UDP_MIB_INDATAGRAMS);
@@ -1509,7 +1510,6 @@ static void xs_tcp_state_change(struct sock *sk)
 	trace_rpc_socket_state_change(xprt, sk->sk_socket);
 	switch (sk->sk_state) {
 	case TCP_ESTABLISHED:
-		spin_lock(&xprt->transport_lock);
 		if (!xprt_test_and_set_connected(xprt)) {
 			xprt->connect_cookie++;
 			clear_bit(XPRT_SOCK_CONNECTING, &transport->sock_state);
@@ -1520,7 +1520,6 @@ static void xs_tcp_state_change(struct sock *sk)
 						   xprt->stat.connect_start;
 			xs_run_error_worker(transport, XPRT_SOCK_WAKE_PENDING);
 		}
-		spin_unlock(&xprt->transport_lock);
 		break;
 	case TCP_FIN_WAIT1:
 		/* The client initiated a shutdown of the socket */
@@ -1677,9 +1676,9 @@ static void xs_udp_set_buffer_size(struct rpc_xprt *xprt, size_t sndsize, size_t
  */
 static void xs_udp_timer(struct rpc_xprt *xprt, struct rpc_task *task)
 {
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	xprt_adjust_cwnd(xprt, task, -ETIMEDOUT);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 }
 
 static int xs_get_random_port(void)
@@ -2214,13 +2213,13 @@ static void xs_tcp_set_socket_timeouts(struct rpc_xprt *xprt,
 	unsigned int opt_on = 1;
 	unsigned int timeo;
 
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	keepidle = DIV_ROUND_UP(xprt->timeout->to_initval, HZ);
 	keepcnt = xprt->timeout->to_retries + 1;
 	timeo = jiffies_to_msecs(xprt->timeout->to_initval) *
 		(xprt->timeout->to_retries + 1);
 	clear_bit(XPRT_SOCK_UPD_TIMEOUT, &transport->sock_state);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 
 	/* TCP Keepalive options */
 	kernel_setsockopt(sock, SOL_SOCKET, SO_KEEPALIVE,
@@ -2245,7 +2244,7 @@ static void xs_tcp_set_connect_timeout(struct rpc_xprt *xprt,
 	struct rpc_timeout to;
 	unsigned long initval;
 
-	spin_lock_bh(&xprt->transport_lock);
+	spin_lock(&xprt->transport_lock);
 	if (reconnect_timeout < xprt->max_reconnect_timeout)
 		xprt->max_reconnect_timeout = reconnect_timeout;
 	if (connect_timeout < xprt->connect_timeout) {
@@ -2262,7 +2261,7 @@ static void xs_tcp_set_connect_timeout(struct rpc_xprt *xprt,
 		xprt->connect_timeout = connect_timeout;
 	}
 	set_bit(XPRT_SOCK_UPD_TIMEOUT, &transport->sock_state);
-	spin_unlock_bh(&xprt->transport_lock);
+	spin_unlock(&xprt->transport_lock);
 }
 
 static int xs_tcp_finish_connecting(struct rpc_xprt *xprt, struct socket *sock)
-- 
2.21.0


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

* [RFC PATCH 4/5] SUNRPC: Remove the bh-safe lock requirement on the rpc_wait_queue->lock
  2019-05-03 11:18     ` [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock Trond Myklebust
@ 2019-05-03 11:18       ` Trond Myklebust
  2019-05-03 11:18         ` [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue Trond Myklebust
  2019-05-03 14:21       ` [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock Chuck Lever
  1 sibling, 1 reply; 20+ messages in thread
From: Trond Myklebust @ 2019-05-03 11:18 UTC (permalink / raw)
  To: Chuck Lever; +Cc: linux-nfs

Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
---
 net/sunrpc/sched.c | 50 +++++++++++++++++++++++-----------------------
 1 file changed, 25 insertions(+), 25 deletions(-)

diff --git a/net/sunrpc/sched.c b/net/sunrpc/sched.c
index e7723c2c1b1c..c7e81336620c 100644
--- a/net/sunrpc/sched.c
+++ b/net/sunrpc/sched.c
@@ -431,9 +431,9 @@ void rpc_sleep_on_timeout(struct rpc_wait_queue *q, struct rpc_task *task,
 	/*
 	 * Protect the queue operations.
 	 */
-	spin_lock_bh(&q->lock);
+	spin_lock(&q->lock);
 	__rpc_sleep_on_priority_timeout(q, task, timeout, task->tk_priority);
-	spin_unlock_bh(&q->lock);
+	spin_unlock(&q->lock);
 }
 EXPORT_SYMBOL_GPL(rpc_sleep_on_timeout);
 
@@ -449,9 +449,9 @@ void rpc_sleep_on(struct rpc_wait_queue *q, struct rpc_task *task,
 	/*
 	 * Protect the queue operations.
 	 */
-	spin_lock_bh(&q->lock);
+	spin_lock(&q->lock);
 	__rpc_sleep_on_priority(q, task, task->tk_priority);
-	spin_unlock_bh(&q->lock);
+	spin_unlock(&q->lock);
 }
 EXPORT_SYMBOL_GPL(rpc_sleep_on);
 
@@ -465,9 +465,9 @@ void rpc_sleep_on_priority_timeout(struct rpc_wait_queue *q,
 	/*
 	 * Protect the queue operations.
 	 */
-	spin_lock_bh(&q->lock);
+	spin_lock(&q->lock);
 	__rpc_sleep_on_priority_timeout(q, task, timeout, priority);
-	spin_unlock_bh(&q->lock);
+	spin_unlock(&q->lock);
 }
 EXPORT_SYMBOL_GPL(rpc_sleep_on_priority_timeout);
 
@@ -482,9 +482,9 @@ void rpc_sleep_on_priority(struct rpc_wait_queue *q, struct rpc_task *task,
 	/*
 	 * Protect the queue operations.
 	 */
-	spin_lock_bh(&q->lock);
+	spin_lock(&q->lock);
 	__rpc_sleep_on_priority(q, task, priority);
-	spin_unlock_bh(&q->lock);
+	spin_unlock(&q->lock);
 }
 EXPORT_SYMBOL_GPL(rpc_sleep_on_priority);
 
@@ -562,9 +562,9 @@ void rpc_wake_up_queued_task_on_wq(struct workqueue_struct *wq,
 {
 	if (!RPC_IS_QUEUED(task))
 		return;
-	spin_lock_bh(&queue->lock);
+	spin_lock(&queue->lock);
 	rpc_wake_up_task_on_wq_queue_locked(wq, queue, task);
-	spin_unlock_bh(&queue->lock);
+	spin_unlock(&queue->lock);
 }
 
 /*
@@ -574,9 +574,9 @@ void rpc_wake_up_queued_task(struct rpc_wait_queue *queue, struct rpc_task *task
 {
 	if (!RPC_IS_QUEUED(task))
 		return;
-	spin_lock_bh(&queue->lock);
+	spin_lock(&queue->lock);
 	rpc_wake_up_task_queue_locked(queue, task);
-	spin_unlock_bh(&queue->lock);
+	spin_unlock(&queue->lock);
 }
 EXPORT_SYMBOL_GPL(rpc_wake_up_queued_task);
 
@@ -609,9 +609,9 @@ rpc_wake_up_queued_task_set_status(struct rpc_wait_queue *queue,
 {
 	if (!RPC_IS_QUEUED(task))
 		return;
-	spin_lock_bh(&queue->lock);
+	spin_lock(&queue->lock);
 	rpc_wake_up_task_queue_set_status_locked(queue, task, status);
-	spin_unlock_bh(&queue->lock);
+	spin_unlock(&queue->lock);
 }
 
 /*
@@ -674,12 +674,12 @@ struct rpc_task *rpc_wake_up_first_on_wq(struct workqueue_struct *wq,
 
 	dprintk("RPC:       wake_up_first(%p \"%s\")\n",
 			queue, rpc_qname(queue));
-	spin_lock_bh(&queue->lock);
+	spin_lock(&queue->lock);
 	task = __rpc_find_next_queued(queue);
 	if (task != NULL)
 		task = rpc_wake_up_task_on_wq_queue_action_locked(wq, queue,
 				task, func, data);
-	spin_unlock_bh(&queue->lock);
+	spin_unlock(&queue->lock);
 
 	return task;
 }
@@ -718,7 +718,7 @@ void rpc_wake_up(struct rpc_wait_queue *queue)
 {
 	struct list_head *head;
 
-	spin_lock_bh(&queue->lock);
+	spin_lock(&queue->lock);
 	head = &queue->tasks[queue->maxpriority];
 	for (;;) {
 		while (!list_empty(head)) {
@@ -732,7 +732,7 @@ void rpc_wake_up(struct rpc_wait_queue *queue)
 			break;
 		head--;
 	}
-	spin_unlock_bh(&queue->lock);
+	spin_unlock(&queue->lock);
 }
 EXPORT_SYMBOL_GPL(rpc_wake_up);
 
@@ -747,7 +747,7 @@ void rpc_wake_up_status(struct rpc_wait_queue *queue, int status)
 {
 	struct list_head *head;
 
-	spin_lock_bh(&queue->lock);
+	spin_lock(&queue->lock);
 	head = &queue->tasks[queue->maxpriority];
 	for (;;) {
 		while (!list_empty(head)) {
@@ -762,7 +762,7 @@ void rpc_wake_up_status(struct rpc_wait_queue *queue, int status)
 			break;
 		head--;
 	}
-	spin_unlock_bh(&queue->lock);
+	spin_unlock(&queue->lock);
 }
 EXPORT_SYMBOL_GPL(rpc_wake_up_status);
 
@@ -774,7 +774,7 @@ static void __rpc_queue_timer_fn(struct work_struct *work)
 	struct rpc_task *task, *n;
 	unsigned long expires, now, timeo;
 
-	spin_lock_bh(&queue->lock);
+	spin_lock(&queue->lock);
 	expires = now = jiffies;
 	list_for_each_entry_safe(task, n, &queue->timer_list.list, u.tk_wait.timer_list) {
 		timeo = task->tk_timeout;
@@ -789,7 +789,7 @@ static void __rpc_queue_timer_fn(struct work_struct *work)
 	}
 	if (!list_empty(&queue->timer_list.list))
 		rpc_set_queue_timer(queue, expires);
-	spin_unlock_bh(&queue->lock);
+	spin_unlock(&queue->lock);
 }
 
 static void __rpc_atrun(struct rpc_task *task)
@@ -936,13 +936,13 @@ static void __rpc_execute(struct rpc_task *task)
 		 * rpc_task pointer may still be dereferenced.
 		 */
 		queue = task->tk_waitqueue;
-		spin_lock_bh(&queue->lock);
+		spin_lock(&queue->lock);
 		if (!RPC_IS_QUEUED(task)) {
-			spin_unlock_bh(&queue->lock);
+			spin_unlock(&queue->lock);
 			continue;
 		}
 		rpc_clear_running(task);
-		spin_unlock_bh(&queue->lock);
+		spin_unlock(&queue->lock);
 		if (task_is_async)
 			return;
 
-- 
2.21.0


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

* [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-03 11:18       ` [RFC PATCH 4/5] SUNRPC: Remove the bh-safe lock requirement on the rpc_wait_queue->lock Trond Myklebust
@ 2019-05-03 11:18         ` Trond Myklebust
  2019-05-06 20:41           ` Chuck Lever
  2019-05-28 20:10           ` Olga Kornievskaia
  0 siblings, 2 replies; 20+ messages in thread
From: Trond Myklebust @ 2019-05-03 11:18 UTC (permalink / raw)
  To: Chuck Lever; +Cc: linux-nfs

Allow more time for softirqd

Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
---
 net/sunrpc/sched.c | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/net/sunrpc/sched.c b/net/sunrpc/sched.c
index c7e81336620c..6b37c9a4b48f 100644
--- a/net/sunrpc/sched.c
+++ b/net/sunrpc/sched.c
@@ -1253,7 +1253,7 @@ static int rpciod_start(void)
 		goto out_failed;
 	rpciod_workqueue = wq;
 	/* Note: highpri because network receive is latency sensitive */
-	wq = alloc_workqueue("xprtiod", WQ_UNBOUND|WQ_MEM_RECLAIM|WQ_HIGHPRI, 0);
+	wq = alloc_workqueue("xprtiod", WQ_MEM_RECLAIM | WQ_UNBOUND, 0);
 	if (!wq)
 		goto free_rpciod;
 	xprtiod_workqueue = wq;
-- 
2.21.0


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

* Re: [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock
  2019-05-03 11:18     ` [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock Trond Myklebust
  2019-05-03 11:18       ` [RFC PATCH 4/5] SUNRPC: Remove the bh-safe lock requirement on the rpc_wait_queue->lock Trond Myklebust
@ 2019-05-03 14:21       ` Chuck Lever
  2019-05-03 15:28         ` Trond Myklebust
  1 sibling, 1 reply; 20+ messages in thread
From: Chuck Lever @ 2019-05-03 14:21 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List



> On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com> wrote:
> 
> Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
> ---
> net/sunrpc/xprt.c                          | 61 ++++++++++------------
> net/sunrpc/xprtrdma/rpc_rdma.c             |  4 +-
> net/sunrpc/xprtrdma/svc_rdma_backchannel.c |  4 +-
> net/sunrpc/xprtrdma/svc_rdma_transport.c   |  8 +--
> net/sunrpc/xprtsock.c                      | 23 ++++----
> 5 files changed, 47 insertions(+), 53 deletions(-)

For rpc_rdma.c and svc_rdma_backchannel.c:

   Reviewed-by: Chuck Lever <chuck.lever@oracle.com>

For svc_rdma_transport.c:

These locks are server-side only. AFAICS it's not safe
to leave BH's enabled here. Can you drop these hunks?


> diff --git a/net/sunrpc/xprt.c b/net/sunrpc/xprt.c
> index bc1c8247750d..b87d185cf010 100644
> --- a/net/sunrpc/xprt.c
> +++ b/net/sunrpc/xprt.c
> @@ -301,9 +301,9 @@ static inline int xprt_lock_write(struct rpc_xprt *xprt, struct rpc_task *task)
> 
> 	if (test_bit(XPRT_LOCKED, &xprt->state) && xprt->snd_task == task)
> 		return 1;
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	retval = xprt->ops->reserve_xprt(xprt, task);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 	return retval;
> }
> 
> @@ -380,9 +380,9 @@ static inline void xprt_release_write(struct rpc_xprt *xprt, struct rpc_task *ta
> {
> 	if (xprt->snd_task != task)
> 		return;
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	xprt->ops->release_xprt(xprt, task);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> }
> 
> /*
> @@ -434,9 +434,9 @@ xprt_request_get_cong(struct rpc_xprt *xprt, struct rpc_rqst *req)
> 
> 	if (req->rq_cong)
> 		return true;
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	ret = __xprt_get_cong(xprt, req) != 0;
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 	return ret;
> }
> EXPORT_SYMBOL_GPL(xprt_request_get_cong);
> @@ -463,9 +463,9 @@ static void
> xprt_clear_congestion_window_wait(struct rpc_xprt *xprt)
> {
> 	if (test_and_clear_bit(XPRT_CWND_WAIT, &xprt->state)) {
> -		spin_lock_bh(&xprt->transport_lock);
> +		spin_lock(&xprt->transport_lock);
> 		__xprt_lock_write_next_cong(xprt);
> -		spin_unlock_bh(&xprt->transport_lock);
> +		spin_unlock(&xprt->transport_lock);
> 	}
> }
> 
> @@ -562,9 +562,9 @@ bool xprt_write_space(struct rpc_xprt *xprt)
> 
> 	if (!test_bit(XPRT_WRITE_SPACE, &xprt->state))
> 		return false;
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	ret = xprt_clear_write_space_locked(xprt);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 	return ret;
> }
> EXPORT_SYMBOL_GPL(xprt_write_space);
> @@ -633,9 +633,9 @@ int xprt_adjust_timeout(struct rpc_rqst *req)
> 		req->rq_retries = 0;
> 		xprt_reset_majortimeo(req);
> 		/* Reset the RTT counters == "slow start" */
> -		spin_lock_bh(&xprt->transport_lock);
> +		spin_lock(&xprt->transport_lock);
> 		rpc_init_rtt(req->rq_task->tk_client->cl_rtt, to->to_initval);
> -		spin_unlock_bh(&xprt->transport_lock);
> +		spin_unlock(&xprt->transport_lock);
> 		status = -ETIMEDOUT;
> 	}
> 
> @@ -667,11 +667,11 @@ static void xprt_autoclose(struct work_struct *work)
> void xprt_disconnect_done(struct rpc_xprt *xprt)
> {
> 	dprintk("RPC:       disconnected transport %p\n", xprt);
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	xprt_clear_connected(xprt);
> 	xprt_clear_write_space_locked(xprt);
> 	xprt_wake_pending_tasks(xprt, -ENOTCONN);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> }
> EXPORT_SYMBOL_GPL(xprt_disconnect_done);
> 
> @@ -683,7 +683,7 @@ EXPORT_SYMBOL_GPL(xprt_disconnect_done);
> void xprt_force_disconnect(struct rpc_xprt *xprt)
> {
> 	/* Don't race with the test_bit() in xprt_clear_locked() */
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	set_bit(XPRT_CLOSE_WAIT, &xprt->state);
> 	/* Try to schedule an autoclose RPC call */
> 	if (test_and_set_bit(XPRT_LOCKED, &xprt->state) == 0)
> @@ -691,7 +691,7 @@ void xprt_force_disconnect(struct rpc_xprt *xprt)
> 	else if (xprt->snd_task)
> 		rpc_wake_up_queued_task_set_status(&xprt->pending,
> 				xprt->snd_task, -ENOTCONN);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> }
> EXPORT_SYMBOL_GPL(xprt_force_disconnect);
> 
> @@ -725,7 +725,7 @@ xprt_request_retransmit_after_disconnect(struct rpc_task *task)
> void xprt_conditional_disconnect(struct rpc_xprt *xprt, unsigned int cookie)
> {
> 	/* Don't race with the test_bit() in xprt_clear_locked() */
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	if (cookie != xprt->connect_cookie)
> 		goto out;
> 	if (test_bit(XPRT_CLOSING, &xprt->state))
> @@ -736,7 +736,7 @@ void xprt_conditional_disconnect(struct rpc_xprt *xprt, unsigned int cookie)
> 		queue_work(xprtiod_workqueue, &xprt->task_cleanup);
> 	xprt_wake_pending_tasks(xprt, -EAGAIN);
> out:
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> }
> 
> static bool
> @@ -758,18 +758,13 @@ xprt_init_autodisconnect(struct timer_list *t)
> {
> 	struct rpc_xprt *xprt = from_timer(xprt, t, timer);
> 
> -	spin_lock(&xprt->transport_lock);
> 	if (!RB_EMPTY_ROOT(&xprt->recv_queue))
> -		goto out_abort;
> +		return;
> 	/* Reset xprt->last_used to avoid connect/autodisconnect cycling */
> 	xprt->last_used = jiffies;
> 	if (test_and_set_bit(XPRT_LOCKED, &xprt->state))
> -		goto out_abort;
> -	spin_unlock(&xprt->transport_lock);
> +		return;
> 	queue_work(xprtiod_workqueue, &xprt->task_cleanup);
> -	return;
> -out_abort:
> -	spin_unlock(&xprt->transport_lock);
> }
> 
> bool xprt_lock_connect(struct rpc_xprt *xprt,
> @@ -778,7 +773,7 @@ bool xprt_lock_connect(struct rpc_xprt *xprt,
> {
> 	bool ret = false;
> 
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	if (!test_bit(XPRT_LOCKED, &xprt->state))
> 		goto out;
> 	if (xprt->snd_task != task)
> @@ -786,13 +781,13 @@ bool xprt_lock_connect(struct rpc_xprt *xprt,
> 	xprt->snd_task = cookie;
> 	ret = true;
> out:
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 	return ret;
> }
> 
> void xprt_unlock_connect(struct rpc_xprt *xprt, void *cookie)
> {
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	if (xprt->snd_task != cookie)
> 		goto out;
> 	if (!test_bit(XPRT_LOCKED, &xprt->state))
> @@ -801,7 +796,7 @@ void xprt_unlock_connect(struct rpc_xprt *xprt, void *cookie)
> 	xprt->ops->release_xprt(xprt, NULL);
> 	xprt_schedule_autodisconnect(xprt);
> out:
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 	wake_up_bit(&xprt->state, XPRT_LOCKED);
> }
> 
> @@ -1411,14 +1406,14 @@ xprt_request_transmit(struct rpc_rqst *req, struct rpc_task *snd_task)
> 	xprt_inject_disconnect(xprt);
> 
> 	task->tk_flags |= RPC_TASK_SENT;
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 
> 	xprt->stat.sends++;
> 	xprt->stat.req_u += xprt->stat.sends - xprt->stat.recvs;
> 	xprt->stat.bklog_u += xprt->backlog.qlen;
> 	xprt->stat.sending_u += xprt->sending.qlen;
> 	xprt->stat.pending_u += xprt->pending.qlen;
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 
> 	req->rq_connect_cookie = connect_cookie;
> out_dequeue:
> @@ -1769,13 +1764,13 @@ void xprt_release(struct rpc_task *task)
> 	else if (task->tk_client)
> 		rpc_count_iostats(task, task->tk_client->cl_metrics);
> 	xprt_request_dequeue_all(task, req);
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	xprt->ops->release_xprt(xprt, task);
> 	if (xprt->ops->release_request)
> 		xprt->ops->release_request(task);
> 	xprt->last_used = jiffies;
> 	xprt_schedule_autodisconnect(xprt);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 	if (req->rq_buffer)
> 		xprt->ops->buf_free(task);
> 	xprt_inject_disconnect(xprt);
> diff --git a/net/sunrpc/xprtrdma/rpc_rdma.c b/net/sunrpc/xprtrdma/rpc_rdma.c
> index 6c1fb270f127..26419be782d0 100644
> --- a/net/sunrpc/xprtrdma/rpc_rdma.c
> +++ b/net/sunrpc/xprtrdma/rpc_rdma.c
> @@ -1359,10 +1359,10 @@ void rpcrdma_reply_handler(struct rpcrdma_rep *rep)
> 	else if (credits > buf->rb_max_requests)
> 		credits = buf->rb_max_requests;
> 	if (buf->rb_credits != credits) {
> -		spin_lock_bh(&xprt->transport_lock);
> +		spin_lock(&xprt->transport_lock);
> 		buf->rb_credits = credits;
> 		xprt->cwnd = credits << RPC_CWNDSHIFT;
> -		spin_unlock_bh(&xprt->transport_lock);
> +		spin_unlock(&xprt->transport_lock);
> 	}
> 
> 	req = rpcr_to_rdmar(rqst);
> diff --git a/net/sunrpc/xprtrdma/svc_rdma_backchannel.c b/net/sunrpc/xprtrdma/svc_rdma_backchannel.c
> index bed57d8b5c19..d1fcc41d5eb5 100644
> --- a/net/sunrpc/xprtrdma/svc_rdma_backchannel.c
> +++ b/net/sunrpc/xprtrdma/svc_rdma_backchannel.c
> @@ -72,9 +72,9 @@ int svc_rdma_handle_bc_reply(struct rpc_xprt *xprt, __be32 *rdma_resp,
> 	else if (credits > r_xprt->rx_buf.rb_bc_max_requests)
> 		credits = r_xprt->rx_buf.rb_bc_max_requests;
> 
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	xprt->cwnd = credits << RPC_CWNDSHIFT;
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 
> 	spin_lock(&xprt->queue_lock);
> 	ret = 0;
> diff --git a/net/sunrpc/xprtrdma/svc_rdma_transport.c b/net/sunrpc/xprtrdma/svc_rdma_transport.c
> index 027a3b07d329..18ffc6190ea9 100644
> --- a/net/sunrpc/xprtrdma/svc_rdma_transport.c
> +++ b/net/sunrpc/xprtrdma/svc_rdma_transport.c
> @@ -221,9 +221,9 @@ static void handle_connect_req(struct rdma_cm_id *new_cma_id,
> 	 * Enqueue the new transport on the accept queue of the listening
> 	 * transport
> 	 */
> -	spin_lock_bh(&listen_xprt->sc_lock);
> +	spin_lock(&listen_xprt->sc_lock);
> 	list_add_tail(&newxprt->sc_accept_q, &listen_xprt->sc_accept_q);
> -	spin_unlock_bh(&listen_xprt->sc_lock);
> +	spin_unlock(&listen_xprt->sc_lock);
> 
> 	set_bit(XPT_CONN, &listen_xprt->sc_xprt.xpt_flags);
> 	svc_xprt_enqueue(&listen_xprt->sc_xprt);
> @@ -396,7 +396,7 @@ static struct svc_xprt *svc_rdma_accept(struct svc_xprt *xprt)
> 	listen_rdma = container_of(xprt, struct svcxprt_rdma, sc_xprt);
> 	clear_bit(XPT_CONN, &xprt->xpt_flags);
> 	/* Get the next entry off the accept list */
> -	spin_lock_bh(&listen_rdma->sc_lock);
> +	spin_lock(&listen_rdma->sc_lock);
> 	if (!list_empty(&listen_rdma->sc_accept_q)) {
> 		newxprt = list_entry(listen_rdma->sc_accept_q.next,
> 				     struct svcxprt_rdma, sc_accept_q);
> @@ -404,7 +404,7 @@ static struct svc_xprt *svc_rdma_accept(struct svc_xprt *xprt)
> 	}
> 	if (!list_empty(&listen_rdma->sc_accept_q))
> 		set_bit(XPT_CONN, &listen_rdma->sc_xprt.xpt_flags);
> -	spin_unlock_bh(&listen_rdma->sc_lock);
> +	spin_unlock(&listen_rdma->sc_lock);
> 	if (!newxprt)
> 		return NULL;
> 
> diff --git a/net/sunrpc/xprtsock.c b/net/sunrpc/xprtsock.c
> index e0195b1a0c18..d7b8e95a61c8 100644
> --- a/net/sunrpc/xprtsock.c
> +++ b/net/sunrpc/xprtsock.c
> @@ -880,7 +880,7 @@ static int xs_nospace(struct rpc_rqst *req)
> 			req->rq_slen);
> 
> 	/* Protect against races with write_space */
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 
> 	/* Don't race with disconnect */
> 	if (xprt_connected(xprt)) {
> @@ -890,7 +890,7 @@ static int xs_nospace(struct rpc_rqst *req)
> 	} else
> 		ret = -ENOTCONN;
> 
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 
> 	/* Race breaker in case memory is freed before above code is called */
> 	if (ret == -EAGAIN) {
> @@ -1344,6 +1344,7 @@ static void xs_destroy(struct rpc_xprt *xprt)
> 	cancel_delayed_work_sync(&transport->connect_worker);
> 	xs_close(xprt);
> 	cancel_work_sync(&transport->recv_worker);
> +	cancel_work_sync(&transport->error_worker);
> 	xs_xprt_free(xprt);
> 	module_put(THIS_MODULE);
> }
> @@ -1397,9 +1398,9 @@ static void xs_udp_data_read_skb(struct rpc_xprt *xprt,
> 	}
> 
> 
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	xprt_adjust_cwnd(xprt, task, copied);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 	spin_lock(&xprt->queue_lock);
> 	xprt_complete_rqst(task, copied);
> 	__UDPX_INC_STATS(sk, UDP_MIB_INDATAGRAMS);
> @@ -1509,7 +1510,6 @@ static void xs_tcp_state_change(struct sock *sk)
> 	trace_rpc_socket_state_change(xprt, sk->sk_socket);
> 	switch (sk->sk_state) {
> 	case TCP_ESTABLISHED:
> -		spin_lock(&xprt->transport_lock);
> 		if (!xprt_test_and_set_connected(xprt)) {
> 			xprt->connect_cookie++;
> 			clear_bit(XPRT_SOCK_CONNECTING, &transport->sock_state);
> @@ -1520,7 +1520,6 @@ static void xs_tcp_state_change(struct sock *sk)
> 						   xprt->stat.connect_start;
> 			xs_run_error_worker(transport, XPRT_SOCK_WAKE_PENDING);
> 		}
> -		spin_unlock(&xprt->transport_lock);
> 		break;
> 	case TCP_FIN_WAIT1:
> 		/* The client initiated a shutdown of the socket */
> @@ -1677,9 +1676,9 @@ static void xs_udp_set_buffer_size(struct rpc_xprt *xprt, size_t sndsize, size_t
>  */
> static void xs_udp_timer(struct rpc_xprt *xprt, struct rpc_task *task)
> {
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	xprt_adjust_cwnd(xprt, task, -ETIMEDOUT);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> }
> 
> static int xs_get_random_port(void)
> @@ -2214,13 +2213,13 @@ static void xs_tcp_set_socket_timeouts(struct rpc_xprt *xprt,
> 	unsigned int opt_on = 1;
> 	unsigned int timeo;
> 
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	keepidle = DIV_ROUND_UP(xprt->timeout->to_initval, HZ);
> 	keepcnt = xprt->timeout->to_retries + 1;
> 	timeo = jiffies_to_msecs(xprt->timeout->to_initval) *
> 		(xprt->timeout->to_retries + 1);
> 	clear_bit(XPRT_SOCK_UPD_TIMEOUT, &transport->sock_state);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> 
> 	/* TCP Keepalive options */
> 	kernel_setsockopt(sock, SOL_SOCKET, SO_KEEPALIVE,
> @@ -2245,7 +2244,7 @@ static void xs_tcp_set_connect_timeout(struct rpc_xprt *xprt,
> 	struct rpc_timeout to;
> 	unsigned long initval;
> 
> -	spin_lock_bh(&xprt->transport_lock);
> +	spin_lock(&xprt->transport_lock);
> 	if (reconnect_timeout < xprt->max_reconnect_timeout)
> 		xprt->max_reconnect_timeout = reconnect_timeout;
> 	if (connect_timeout < xprt->connect_timeout) {
> @@ -2262,7 +2261,7 @@ static void xs_tcp_set_connect_timeout(struct rpc_xprt *xprt,
> 		xprt->connect_timeout = connect_timeout;
> 	}
> 	set_bit(XPRT_SOCK_UPD_TIMEOUT, &transport->sock_state);
> -	spin_unlock_bh(&xprt->transport_lock);
> +	spin_unlock(&xprt->transport_lock);
> }
> 
> static int xs_tcp_finish_connecting(struct rpc_xprt *xprt, struct socket *sock)
> -- 
> 2.21.0
> 

--
Chuck Lever




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

* Re: [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock
  2019-05-03 14:21       ` [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock Chuck Lever
@ 2019-05-03 15:28         ` Trond Myklebust
  2019-05-03 15:43           ` Chuck Lever
  0 siblings, 1 reply; 20+ messages in thread
From: Trond Myklebust @ 2019-05-03 15:28 UTC (permalink / raw)
  To: Chuck Lever; +Cc: Linux NFS Mailing List

On Fri, 2019-05-03 at 10:21 -0400, Chuck Lever wrote:
> > On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com>
> > wrote:
> > 
> > Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
> > ---
> > net/sunrpc/xprt.c                          | 61 ++++++++++---------
> > ---
> > net/sunrpc/xprtrdma/rpc_rdma.c             |  4 +-
> > net/sunrpc/xprtrdma/svc_rdma_backchannel.c |  4 +-
> > net/sunrpc/xprtrdma/svc_rdma_transport.c   |  8 +--
> > net/sunrpc/xprtsock.c                      | 23 ++++----
> > 5 files changed, 47 insertions(+), 53 deletions(-)
> 
> For rpc_rdma.c and svc_rdma_backchannel.c:
> 
>    Reviewed-by: Chuck Lever <chuck.lever@oracle.com>
> 
> For svc_rdma_transport.c:
> 
> These locks are server-side only. AFAICS it's not safe
> to leave BH's enabled here. Can you drop these hunks?

Oops... Yes, I don't know why I mistook that for the xprt-
>transport_lock...

You mean these 3 hunks, right?

> > diff --git a/net/sunrpc/xprtrdma/svc_rdma_transport.c
> > b/net/sunrpc/xprtrdma/svc_rdma_transport.c
> > index 027a3b07d329..18ffc6190ea9 100644
> > --- a/net/sunrpc/xprtrdma/svc_rdma_transport.c
> > +++ b/net/sunrpc/xprtrdma/svc_rdma_transport.c
> > @@ -221,9 +221,9 @@ static void handle_connect_req(struct
> > rdma_cm_id *new_cma_id,
> > 	 * Enqueue the new transport on the accept queue of the
> > listening
> > 	 * transport
> > 	 */
> > -	spin_lock_bh(&listen_xprt->sc_lock);
> > +	spin_lock(&listen_xprt->sc_lock);
> > 	list_add_tail(&newxprt->sc_accept_q, &listen_xprt-
> > >sc_accept_q);
> > -	spin_unlock_bh(&listen_xprt->sc_lock);
> > +	spin_unlock(&listen_xprt->sc_lock);
> > 
> > 	set_bit(XPT_CONN, &listen_xprt->sc_xprt.xpt_flags);
> > 	svc_xprt_enqueue(&listen_xprt->sc_xprt);
> > @@ -396,7 +396,7 @@ static struct svc_xprt *svc_rdma_accept(struct
> > svc_xprt *xprt)
> > 	listen_rdma = container_of(xprt, struct svcxprt_rdma, sc_xprt);
> > 	clear_bit(XPT_CONN, &xprt->xpt_flags);
> > 	/* Get the next entry off the accept list */
> > -	spin_lock_bh(&listen_rdma->sc_lock);
> > +	spin_lock(&listen_rdma->sc_lock);
> > 	if (!list_empty(&listen_rdma->sc_accept_q)) {
> > 		newxprt = list_entry(listen_rdma->sc_accept_q.next,
> > 				     struct svcxprt_rdma, sc_accept_q);
> > @@ -404,7 +404,7 @@ static struct svc_xprt *svc_rdma_accept(struct
> > svc_xprt *xprt)
> > 	}
> > 	if (!list_empty(&listen_rdma->sc_accept_q))
> > 		set_bit(XPT_CONN, &listen_rdma->sc_xprt.xpt_flags);
> > -	spin_unlock_bh(&listen_rdma->sc_lock);
> > +	spin_unlock(&listen_rdma->sc_lock);
> > 	if (!newxprt)
> > 		return NULL;
> > 
> > 

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




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

* Re: [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock
  2019-05-03 15:28         ` Trond Myklebust
@ 2019-05-03 15:43           ` Chuck Lever
  0 siblings, 0 replies; 20+ messages in thread
From: Chuck Lever @ 2019-05-03 15:43 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List


> On May 3, 2019, at 11:28 AM, Trond Myklebust <trondmy@gmail.com> wrote:
> 
> On Fri, 2019-05-03 at 10:21 -0400, Chuck Lever wrote:
>>> On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com>
>>> wrote:
>>> 
>>> Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
>>> ---
>>> net/sunrpc/xprt.c                          | 61 ++++++++++---------
>>> ---
>>> net/sunrpc/xprtrdma/rpc_rdma.c             |  4 +-
>>> net/sunrpc/xprtrdma/svc_rdma_backchannel.c |  4 +-
>>> net/sunrpc/xprtrdma/svc_rdma_transport.c   |  8 +--
>>> net/sunrpc/xprtsock.c                      | 23 ++++----
>>> 5 files changed, 47 insertions(+), 53 deletions(-)
>> 
>> For rpc_rdma.c and svc_rdma_backchannel.c:
>> 
>>   Reviewed-by: Chuck Lever <chuck.lever@oracle.com>
>> 
>> For svc_rdma_transport.c:
>> 
>> These locks are server-side only. AFAICS it's not safe
>> to leave BH's enabled here. Can you drop these hunks?
> 
> Oops... Yes, I don't know why I mistook that for the xprt-
>> transport_lock...
> 
> You mean these 3 hunks, right?

Sí.

> 
>>> diff --git a/net/sunrpc/xprtrdma/svc_rdma_transport.c
>>> b/net/sunrpc/xprtrdma/svc_rdma_transport.c
>>> index 027a3b07d329..18ffc6190ea9 100644
>>> --- a/net/sunrpc/xprtrdma/svc_rdma_transport.c
>>> +++ b/net/sunrpc/xprtrdma/svc_rdma_transport.c
>>> @@ -221,9 +221,9 @@ static void handle_connect_req(struct
>>> rdma_cm_id *new_cma_id,
>>>     * Enqueue the new transport on the accept queue of the
>>> listening
>>>     * transport
>>>     */
>>> -    spin_lock_bh(&listen_xprt->sc_lock);
>>> +    spin_lock(&listen_xprt->sc_lock);
>>>    list_add_tail(&newxprt->sc_accept_q, &listen_xprt-
>>>> sc_accept_q);
>>> -    spin_unlock_bh(&listen_xprt->sc_lock);
>>> +    spin_unlock(&listen_xprt->sc_lock);
>>> 
>>>    set_bit(XPT_CONN, &listen_xprt->sc_xprt.xpt_flags);
>>>    svc_xprt_enqueue(&listen_xprt->sc_xprt);
>>> @@ -396,7 +396,7 @@ static struct svc_xprt *svc_rdma_accept(struct
>>> svc_xprt *xprt)
>>>    listen_rdma = container_of(xprt, struct svcxprt_rdma, sc_xprt);
>>>    clear_bit(XPT_CONN, &xprt->xpt_flags);
>>>    /* Get the next entry off the accept list */
>>> -    spin_lock_bh(&listen_rdma->sc_lock);
>>> +    spin_lock(&listen_rdma->sc_lock);
>>>    if (!list_empty(&listen_rdma->sc_accept_q)) {
>>>        newxprt = list_entry(listen_rdma->sc_accept_q.next,
>>>                     struct svcxprt_rdma, sc_accept_q);
>>> @@ -404,7 +404,7 @@ static struct svc_xprt *svc_rdma_accept(struct
>>> svc_xprt *xprt)
>>>    }
>>>    if (!list_empty(&listen_rdma->sc_accept_q))
>>>        set_bit(XPT_CONN, &listen_rdma->sc_xprt.xpt_flags);
>>> -    spin_unlock_bh(&listen_rdma->sc_lock);
>>> +    spin_unlock(&listen_rdma->sc_lock);
>>>    if (!newxprt)
>>>        return NULL;
>>> 
>>> 
> 
> -- 
> Trond Myklebust
> Linux NFS client maintainer, Hammerspace
> trond.myklebust@hammerspace.com
> 
> 
> 


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

* Re: [RFC PATCH 0/5] bh-safe lock removal for SUNRPC
  2019-05-03 11:18 [RFC PATCH 0/5] bh-safe lock removal for SUNRPC Trond Myklebust
  2019-05-03 11:18 ` [RFC PATCH 1/5] SUNRPC: Replace the queue timer with a delayed work function Trond Myklebust
@ 2019-05-06 18:22 ` Chuck Lever
  2019-05-06 18:37   ` Trond Myklebust
  1 sibling, 1 reply; 20+ messages in thread
From: Chuck Lever @ 2019-05-06 18:22 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List

Hi Trond-

> On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com> wrote:
> 
> This patchset aims to remove the bh-safe locks on the client side.
> At this time it should be seen as a toy/strawman effort in order to
> help the community figure out whether or not there are setups out
> there that are actually seeing performance bottlenecks resulting
> from taking bh-safe locks inside other spinlocks.

What kernel does this patch set apply to? I've tried both v5.0 and
v5.1, but there appear to be some changes that I'm missing. The
first patch does not apply cleanly.


> Trond Myklebust (5):
>  SUNRPC: Replace the queue timer with a delayed work function
>  SUNRPC: Replace direct task wakeups from softirq context
>  SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock
>  SUNRPC: Remove the bh-safe lock requirement on the
>    rpc_wait_queue->lock
>  SUNRPC: Reduce the priority of the xprtiod queue
> 
> include/linux/sunrpc/sched.h               |   3 +-
> include/linux/sunrpc/xprtsock.h            |   5 +
> net/sunrpc/sched.c                         |  76 +++++++++-------
> net/sunrpc/xprt.c                          |  61 ++++++-------
> net/sunrpc/xprtrdma/rpc_rdma.c             |   4 +-
> net/sunrpc/xprtrdma/svc_rdma_backchannel.c |   4 +-
> net/sunrpc/xprtrdma/svc_rdma_transport.c   |   8 +-
> net/sunrpc/xprtsock.c                      | 101 +++++++++++++++++----
> 8 files changed, 168 insertions(+), 94 deletions(-)
> 
> -- 
> 2.21.0
> 

--
Chuck Lever




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

* Re: [RFC PATCH 0/5] bh-safe lock removal for SUNRPC
  2019-05-06 18:22 ` [RFC PATCH 0/5] bh-safe lock removal for SUNRPC Chuck Lever
@ 2019-05-06 18:37   ` Trond Myklebust
  2019-05-06 20:02     ` Chuck Lever
  0 siblings, 1 reply; 20+ messages in thread
From: Trond Myklebust @ 2019-05-06 18:37 UTC (permalink / raw)
  To: chuck.lever; +Cc: linux-nfs

On Mon, 2019-05-06 at 14:22 -0400, Chuck Lever wrote:
> Hi Trond-
> 
> > On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com>
> > wrote:
> > 
> > This patchset aims to remove the bh-safe locks on the client side.
> > At this time it should be seen as a toy/strawman effort in order to
> > help the community figure out whether or not there are setups out
> > there that are actually seeing performance bottlenecks resulting
> > from taking bh-safe locks inside other spinlocks.
> 
> What kernel does this patch set apply to? I've tried both v5.0 and
> v5.1, but there appear to be some changes that I'm missing. The
> first patch does not apply cleanly.
> 

It should hopefully apply on top of Anna's linux-next branch.

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



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

* Re: [RFC PATCH 0/5] bh-safe lock removal for SUNRPC
  2019-05-06 18:37   ` Trond Myklebust
@ 2019-05-06 20:02     ` Chuck Lever
  0 siblings, 0 replies; 20+ messages in thread
From: Chuck Lever @ 2019-05-06 20:02 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List



> On May 6, 2019, at 2:37 PM, Trond Myklebust <trondmy@hammerspace.com> wrote:
> 
> On Mon, 2019-05-06 at 14:22 -0400, Chuck Lever wrote:
>> Hi Trond-
>> 
>>> On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com>
>>> wrote:
>>> 
>>> This patchset aims to remove the bh-safe locks on the client side.
>>> At this time it should be seen as a toy/strawman effort in order to
>>> help the community figure out whether or not there are setups out
>>> there that are actually seeing performance bottlenecks resulting
>>> from taking bh-safe locks inside other spinlocks.
>> 
>> What kernel does this patch set apply to? I've tried both v5.0 and
>> v5.1, but there appear to be some changes that I'm missing. The
>> first patch does not apply cleanly.
>> 
> 
> It should hopefully apply on top of Anna's linux-next branch.

OK, you did mention that to me last week. Sorry for the noise.

--
Chuck Lever




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

* Re: [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-03 11:18         ` [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue Trond Myklebust
@ 2019-05-06 20:41           ` Chuck Lever
  2019-05-28 19:03             ` Chuck Lever
  2019-05-28 20:10           ` Olga Kornievskaia
  1 sibling, 1 reply; 20+ messages in thread
From: Chuck Lever @ 2019-05-06 20:41 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List


> On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com> wrote:
> 
> Allow more time for softirqd

Have you thought about performance tests for this one?


> Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
> ---
> net/sunrpc/sched.c | 2 +-
> 1 file changed, 1 insertion(+), 1 deletion(-)
> 
> diff --git a/net/sunrpc/sched.c b/net/sunrpc/sched.c
> index c7e81336620c..6b37c9a4b48f 100644
> --- a/net/sunrpc/sched.c
> +++ b/net/sunrpc/sched.c
> @@ -1253,7 +1253,7 @@ static int rpciod_start(void)
> 		goto out_failed;
> 	rpciod_workqueue = wq;
> 	/* Note: highpri because network receive is latency sensitive */

The above comment should be deleted as well.


> -	wq = alloc_workqueue("xprtiod", WQ_UNBOUND|WQ_MEM_RECLAIM|WQ_HIGHPRI, 0);
> +	wq = alloc_workqueue("xprtiod", WQ_MEM_RECLAIM | WQ_UNBOUND, 0);
> 	if (!wq)
> 		goto free_rpciod;
> 	xprtiod_workqueue = wq;

--
Chuck Lever




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

* Re: [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-06 20:41           ` Chuck Lever
@ 2019-05-28 19:03             ` Chuck Lever
  2019-05-28 19:33               ` Trond Myklebust
  0 siblings, 1 reply; 20+ messages in thread
From: Chuck Lever @ 2019-05-28 19:03 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List

Following up on this. Now with even more data!

> On May 6, 2019, at 4:41 PM, Chuck Lever <chuck.lever@oracle.com> wrote:
> 
> 
>> On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com> wrote:
>> 
>> Allow more time for softirqd
> 
> Have you thought about performance tests for this one?

I tested this series on my 12-core two-socket client using a variety
of tests including iozone, fio, and fstests. The network under test
is 56Gb InfiniBand (TCP uses IPoIB). I tested both TCP and RDMA.

With lock debugging and memory leak testing enabled, I did not see
any functional regressions or new leaks or crashes. Thus IMO this
series is "safe to apply."

With TCP, I saw no change in performance between a "stock" kernel
and one with all five patches in this series applied, as, IIRC,
you predicted.

The following discussion is based on testing with NFS/RDMA.

With RDMA, I saw an improvement of 5-10% in IOPS rate between the
"stock" kernel and a kernel with the first four patches applied. When
the fifth patch is applied, I saw IOPS throughput significantly worse
than "stock" -- like 20% worse.

I also studied average RPC execution time (the "execute" metric) with
the "stock" kernel, the one with four patches applied, and with the
one where all five are applied. The workload is 100% 4KB READs with
an iodepth of 1024 in order to saturate the transmit queue.

With four patches, the execute time is about 2.5 msec faster (average
execution time is around 75 msec due to the large backlog this test
generates). With five patches, it's slower than "stock" by 12 msec.

I also saw a 30 usec improvement in the average latency of
xprt_complete_rqst with the four patch series.

As far as I can tell, the benefit of this series comes mostly from
the third patch, which changes spin_lock_bh(&xprt->transport_lock) to
spin_lock(&xprt->transport_lock). When the xprtiod work queue is
lowered in priority in 5/5, that benefit vanishes.

I am still confused about why 5/5 is needed. I did not see any soft
lockups without this patch applied when using RDMA. Is the issue
with xprtsock's use of xprtiod for handling incoming TCP receives?

I still have some things I'd like to look at. One thing I haven't
yet tried is looking at lock_stat, which would confirm or refute
my theory that this is all about the transport_lock, for instance.


>> Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
>> ---
>> net/sunrpc/sched.c | 2 +-
>> 1 file changed, 1 insertion(+), 1 deletion(-)
>> 
>> diff --git a/net/sunrpc/sched.c b/net/sunrpc/sched.c
>> index c7e81336620c..6b37c9a4b48f 100644
>> --- a/net/sunrpc/sched.c
>> +++ b/net/sunrpc/sched.c
>> @@ -1253,7 +1253,7 @@ static int rpciod_start(void)
>> 		goto out_failed;
>> 	rpciod_workqueue = wq;
>> 	/* Note: highpri because network receive is latency sensitive */
> 
> The above comment should be deleted as well.
> 
> 
>> -	wq = alloc_workqueue("xprtiod", WQ_UNBOUND|WQ_MEM_RECLAIM|WQ_HIGHPRI, 0);
>> +	wq = alloc_workqueue("xprtiod", WQ_MEM_RECLAIM | WQ_UNBOUND, 0);
>> 	if (!wq)
>> 		goto free_rpciod;
>> 	xprtiod_workqueue = wq;
> 
> --
> Chuck Lever
> 
> 
> 

--
Chuck Lever




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

* Re: [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-28 19:03             ` Chuck Lever
@ 2019-05-28 19:33               ` Trond Myklebust
  2019-05-28 19:52                 ` Chuck Lever
  0 siblings, 1 reply; 20+ messages in thread
From: Trond Myklebust @ 2019-05-28 19:33 UTC (permalink / raw)
  To: chuck.lever; +Cc: linux-nfs

On Tue, 2019-05-28 at 15:03 -0400, Chuck Lever wrote:
> Following up on this. Now with even more data!
> 
> > On May 6, 2019, at 4:41 PM, Chuck Lever <chuck.lever@oracle.com>
> > wrote:
> > 
> > 
> > > On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com>
> > > wrote:
> > > 
> > > Allow more time for softirqd
> > 
> > Have you thought about performance tests for this one?
> 
> I tested this series on my 12-core two-socket client using a variety
> of tests including iozone, fio, and fstests. The network under test
> is 56Gb InfiniBand (TCP uses IPoIB). I tested both TCP and RDMA.
> 
> With lock debugging and memory leak testing enabled, I did not see
> any functional regressions or new leaks or crashes. Thus IMO this
> series is "safe to apply."
> 
> With TCP, I saw no change in performance between a "stock" kernel
> and one with all five patches in this series applied, as, IIRC,
> you predicted.
> 
> The following discussion is based on testing with NFS/RDMA.
> 
> With RDMA, I saw an improvement of 5-10% in IOPS rate between the
> "stock" kernel and a kernel with the first four patches applied. When
> the fifth patch is applied, I saw IOPS throughput significantly worse
> than "stock" -- like 20% worse.
> 
> I also studied average RPC execution time (the "execute" metric) with
> the "stock" kernel, the one with four patches applied, and with the
> one where all five are applied. The workload is 100% 4KB READs with
> an iodepth of 1024 in order to saturate the transmit queue.
> 
> With four patches, the execute time is about 2.5 msec faster (average
> execution time is around 75 msec due to the large backlog this test
> generates). With five patches, it's slower than "stock" by 12 msec.
> 
> I also saw a 30 usec improvement in the average latency of
> xprt_complete_rqst with the four patch series.
> 
> As far as I can tell, the benefit of this series comes mostly from
> the third patch, which changes spin_lock_bh(&xprt->transport_lock) to
> spin_lock(&xprt->transport_lock). When the xprtiod work queue is
> lowered in priority in 5/5, that benefit vanishes.
> 
> I am still confused about why 5/5 is needed. I did not see any soft
> lockups without this patch applied when using RDMA. Is the issue
> with xprtsock's use of xprtiod for handling incoming TCP receives?
> 
> I still have some things I'd like to look at. One thing I haven't
> yet tried is looking at lock_stat, which would confirm or refute
> my theory that this is all about the transport_lock, for instance.
> 

OK. I can drop 5/5.

The issue there was not about soft lockups. However since we were
previously running most soft irqs as part of spin_unlock_bh(), the
question was whether or not we would see more of them needing to move
to softirqd. As far as I can see, your answer to that question is 'no'
(at least for your system).

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



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

* Re: [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-28 19:33               ` Trond Myklebust
@ 2019-05-28 19:52                 ` Chuck Lever
  2019-05-29 17:13                   ` Chuck Lever
  0 siblings, 1 reply; 20+ messages in thread
From: Chuck Lever @ 2019-05-28 19:52 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List



> On May 28, 2019, at 3:33 PM, Trond Myklebust <trondmy@hammerspace.com> wrote:
> 
> On Tue, 2019-05-28 at 15:03 -0400, Chuck Lever wrote:
>> Following up on this. Now with even more data!
>> 
>>> On May 6, 2019, at 4:41 PM, Chuck Lever <chuck.lever@oracle.com>
>>> wrote:
>>> 
>>> 
>>>> On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com>
>>>> wrote:
>>>> 
>>>> Allow more time for softirqd
>>> 
>>> Have you thought about performance tests for this one?
>> 
>> I tested this series on my 12-core two-socket client using a variety
>> of tests including iozone, fio, and fstests. The network under test
>> is 56Gb InfiniBand (TCP uses IPoIB). I tested both TCP and RDMA.
>> 
>> With lock debugging and memory leak testing enabled, I did not see
>> any functional regressions or new leaks or crashes. Thus IMO this
>> series is "safe to apply."
>> 
>> With TCP, I saw no change in performance between a "stock" kernel
>> and one with all five patches in this series applied, as, IIRC,
>> you predicted.
>> 
>> The following discussion is based on testing with NFS/RDMA.
>> 
>> With RDMA, I saw an improvement of 5-10% in IOPS rate between the
>> "stock" kernel and a kernel with the first four patches applied. When
>> the fifth patch is applied, I saw IOPS throughput significantly worse
>> than "stock" -- like 20% worse.
>> 
>> I also studied average RPC execution time (the "execute" metric) with
>> the "stock" kernel, the one with four patches applied, and with the
>> one where all five are applied. The workload is 100% 4KB READs with
>> an iodepth of 1024 in order to saturate the transmit queue.
>> 
>> With four patches, the execute time is about 2.5 msec faster (average
>> execution time is around 75 msec due to the large backlog this test
>> generates). With five patches, it's slower than "stock" by 12 msec.
>> 
>> I also saw a 30 usec improvement in the average latency of
>> xprt_complete_rqst with the four patch series.
>> 
>> As far as I can tell, the benefit of this series comes mostly from
>> the third patch, which changes spin_lock_bh(&xprt->transport_lock) to
>> spin_lock(&xprt->transport_lock). When the xprtiod work queue is
>> lowered in priority in 5/5, that benefit vanishes.
>> 
>> I am still confused about why 5/5 is needed. I did not see any soft
>> lockups without this patch applied when using RDMA. Is the issue
>> with xprtsock's use of xprtiod for handling incoming TCP receives?
>> 
>> I still have some things I'd like to look at. One thing I haven't
>> yet tried is looking at lock_stat, which would confirm or refute
>> my theory that this is all about the transport_lock, for instance.
>> 
> 
> OK. I can drop 5/5.
> 
> The issue there was not about soft lockups. However since we were
> previously running most soft irqs as part of spin_unlock_bh(), the
> question was whether or not we would see more of them needing to move
> to softirqd. As far as I can see, your answer to that question is 'no'
> (at least for your system).

The top contended lock now is the work queue lock. I believe that's a
full irqsave lock. Someone should try testing on a single core system.

I also plan to try this series on my mlx5_en system. The mlx5 Ethernet
driver does a lot more work in soft IRQ than mlx4/IB does.


--
Chuck Lever




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

* Re: [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-03 11:18         ` [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue Trond Myklebust
  2019-05-06 20:41           ` Chuck Lever
@ 2019-05-28 20:10           ` Olga Kornievskaia
  2019-05-29 18:38             ` Olga Kornievskaia
  1 sibling, 1 reply; 20+ messages in thread
From: Olga Kornievskaia @ 2019-05-28 20:10 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Chuck Lever, linux-nfs

On Fri, May 3, 2019 at 7:24 AM Trond Myklebust <trondmy@gmail.com> wrote:
>
> Allow more time for softirqd
>
> Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
> ---
>  net/sunrpc/sched.c | 2 +-
>  1 file changed, 1 insertion(+), 1 deletion(-)
>
> diff --git a/net/sunrpc/sched.c b/net/sunrpc/sched.c
> index c7e81336620c..6b37c9a4b48f 100644
> --- a/net/sunrpc/sched.c
> +++ b/net/sunrpc/sched.c
> @@ -1253,7 +1253,7 @@ static int rpciod_start(void)
>                 goto out_failed;
>         rpciod_workqueue = wq;
>         /* Note: highpri because network receive is latency sensitive */
> -       wq = alloc_workqueue("xprtiod", WQ_UNBOUND|WQ_MEM_RECLAIM|WQ_HIGHPRI, 0);

I thought we needed UNBOUND otherwise there was performance
degradation for read IO.

> +       wq = alloc_workqueue("xprtiod", WQ_MEM_RECLAIM | WQ_UNBOUND, 0);
>         if (!wq)
>                 goto free_rpciod;
>         xprtiod_workqueue = wq;
> --
> 2.21.0
>

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

* Re: [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-28 19:52                 ` Chuck Lever
@ 2019-05-29 17:13                   ` Chuck Lever
  0 siblings, 0 replies; 20+ messages in thread
From: Chuck Lever @ 2019-05-29 17:13 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Linux NFS Mailing List



> On May 28, 2019, at 3:52 PM, Chuck Lever <chuck.lever@oracle.com> wrote:
> 
> 
> 
>> On May 28, 2019, at 3:33 PM, Trond Myklebust <trondmy@hammerspace.com> wrote:
>> 
>> On Tue, 2019-05-28 at 15:03 -0400, Chuck Lever wrote:
>>> Following up on this. Now with even more data!
>>> 
>>>> On May 6, 2019, at 4:41 PM, Chuck Lever <chuck.lever@oracle.com>
>>>> wrote:
>>>> 
>>>> 
>>>>> On May 3, 2019, at 7:18 AM, Trond Myklebust <trondmy@gmail.com>
>>>>> wrote:
>>>>> 
>>>>> Allow more time for softirqd
>>>> 
>>>> Have you thought about performance tests for this one?
>>> 
>>> I tested this series on my 12-core two-socket client using a variety
>>> of tests including iozone, fio, and fstests. The network under test
>>> is 56Gb InfiniBand (TCP uses IPoIB). I tested both TCP and RDMA.
>>> 
>>> With lock debugging and memory leak testing enabled, I did not see
>>> any functional regressions or new leaks or crashes. Thus IMO this
>>> series is "safe to apply."
>>> 
>>> With TCP, I saw no change in performance between a "stock" kernel
>>> and one with all five patches in this series applied, as, IIRC,
>>> you predicted.
>>> 
>>> The following discussion is based on testing with NFS/RDMA.
>>> 
>>> With RDMA, I saw an improvement of 5-10% in IOPS rate between the
>>> "stock" kernel and a kernel with the first four patches applied. When
>>> the fifth patch is applied, I saw IOPS throughput significantly worse
>>> than "stock" -- like 20% worse.
>>> 
>>> I also studied average RPC execution time (the "execute" metric) with
>>> the "stock" kernel, the one with four patches applied, and with the
>>> one where all five are applied. The workload is 100% 4KB READs with
>>> an iodepth of 1024 in order to saturate the transmit queue.
>>> 
>>> With four patches, the execute time is about 2.5 msec faster (average
>>> execution time is around 75 msec due to the large backlog this test
>>> generates). With five patches, it's slower than "stock" by 12 msec.
>>> 
>>> I also saw a 30 usec improvement in the average latency of
>>> xprt_complete_rqst with the four patch series.
>>> 
>>> As far as I can tell, the benefit of this series comes mostly from
>>> the third patch, which changes spin_lock_bh(&xprt->transport_lock) to
>>> spin_lock(&xprt->transport_lock). When the xprtiod work queue is
>>> lowered in priority in 5/5, that benefit vanishes.
>>> 
>>> I am still confused about why 5/5 is needed. I did not see any soft
>>> lockups without this patch applied when using RDMA. Is the issue
>>> with xprtsock's use of xprtiod for handling incoming TCP receives?
>>> 
>>> I still have some things I'd like to look at. One thing I haven't
>>> yet tried is looking at lock_stat, which would confirm or refute
>>> my theory that this is all about the transport_lock, for instance.
>>> 
>> 
>> OK. I can drop 5/5.
>> 
>> The issue there was not about soft lockups. However since we were
>> previously running most soft irqs as part of spin_unlock_bh(), the
>> question was whether or not we would see more of them needing to move
>> to softirqd. As far as I can see, your answer to that question is 'no'
>> (at least for your system).
> 
> The top contended lock now is the work queue lock. I believe that's a
> full irqsave lock. Someone should try testing on a single core system.
> 
> I also plan to try this series on my mlx5_en system. The mlx5 Ethernet
> driver does a lot more work in soft IRQ than mlx4/IB does.

I tested with CX-5 RoCE on 100GbE. I don't see any obvious signs of
soft IRQ starvation. With 8 threads on a 4-core client, I was able to
push the 4KB random read fio workload past 300KIOPS.

--
Chuck Lever




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

* Re: [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-28 20:10           ` Olga Kornievskaia
@ 2019-05-29 18:38             ` Olga Kornievskaia
  2019-05-29 18:45               ` Trond Myklebust
  0 siblings, 1 reply; 20+ messages in thread
From: Olga Kornievskaia @ 2019-05-29 18:38 UTC (permalink / raw)
  To: Trond Myklebust; +Cc: Chuck Lever, linux-nfs

On Tue, May 28, 2019 at 4:10 PM Olga Kornievskaia <aglo@umich.edu> wrote:
>
> On Fri, May 3, 2019 at 7:24 AM Trond Myklebust <trondmy@gmail.com> wrote:
> >
> > Allow more time for softirqd
> >
> > Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
> > ---
> >  net/sunrpc/sched.c | 2 +-
> >  1 file changed, 1 insertion(+), 1 deletion(-)
> >
> > diff --git a/net/sunrpc/sched.c b/net/sunrpc/sched.c
> > index c7e81336620c..6b37c9a4b48f 100644
> > --- a/net/sunrpc/sched.c
> > +++ b/net/sunrpc/sched.c
> > @@ -1253,7 +1253,7 @@ static int rpciod_start(void)
> >                 goto out_failed;
> >         rpciod_workqueue = wq;
> >         /* Note: highpri because network receive is latency sensitive */
> > -       wq = alloc_workqueue("xprtiod", WQ_UNBOUND|WQ_MEM_RECLAIM|WQ_HIGHPRI, 0);
>
> I thought we needed UNBOUND otherwise there was performance
> degradation for read IO.

I remove my objection as this is for the xprtiod queue and not the
rpciod queue. The latter is the one when removing WQ_UNBOUND would
only use a single rpciod thread for doing all the crypto and thus
impact performance.

>
> > +       wq = alloc_workqueue("xprtiod", WQ_MEM_RECLAIM | WQ_UNBOUND, 0);
> >         if (!wq)
> >                 goto free_rpciod;
> >         xprtiod_workqueue = wq;
> > --
> > 2.21.0
> >

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

* Re: [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue
  2019-05-29 18:38             ` Olga Kornievskaia
@ 2019-05-29 18:45               ` Trond Myklebust
  0 siblings, 0 replies; 20+ messages in thread
From: Trond Myklebust @ 2019-05-29 18:45 UTC (permalink / raw)
  To: aglo; +Cc: linux-nfs, chuck.lever

On Wed, 2019-05-29 at 14:38 -0400, Olga Kornievskaia wrote:
> On Tue, May 28, 2019 at 4:10 PM Olga Kornievskaia <aglo@umich.edu>
> wrote:
> > On Fri, May 3, 2019 at 7:24 AM Trond Myklebust <trondmy@gmail.com>
> > wrote:
> > > Allow more time for softirqd
> > > 
> > > Signed-off-by: Trond Myklebust <trond.myklebust@hammerspace.com>
> > > ---
> > >  net/sunrpc/sched.c | 2 +-
> > >  1 file changed, 1 insertion(+), 1 deletion(-)
> > > 
> > > diff --git a/net/sunrpc/sched.c b/net/sunrpc/sched.c
> > > index c7e81336620c..6b37c9a4b48f 100644
> > > --- a/net/sunrpc/sched.c
> > > +++ b/net/sunrpc/sched.c
> > > @@ -1253,7 +1253,7 @@ static int rpciod_start(void)
> > >                 goto out_failed;
> > >         rpciod_workqueue = wq;
> > >         /* Note: highpri because network receive is latency
> > > sensitive */
> > > -       wq = alloc_workqueue("xprtiod",
> > > WQ_UNBOUND|WQ_MEM_RECLAIM|WQ_HIGHPRI, 0);
> > 
> > I thought we needed UNBOUND otherwise there was performance
> > degradation for read IO.
> 
> I remove my objection as this is for the xprtiod queue and not the
> rpciod queue. The latter is the one when removing WQ_UNBOUND would
> only use a single rpciod thread for doing all the crypto and thus
> impact performance.
> 
> > > +       wq = alloc_workqueue("xprtiod", WQ_MEM_RECLAIM |
> > > WQ_UNBOUND, 0);
> > >         if (!wq)
> > >                 goto free_rpciod;
> > >         xprtiod_workqueue = wq;
> > > --
> > > 2.21.0
> > > 

It was only removing the WQ_HIGHPRI flag, not the WQ_UNBOUND. However
the point it moot, as I'm dropping the patch.

Cheers
 Trond

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



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

end of thread, other threads:[~2019-05-29 18:45 UTC | newest]

Thread overview: 20+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-05-03 11:18 [RFC PATCH 0/5] bh-safe lock removal for SUNRPC Trond Myklebust
2019-05-03 11:18 ` [RFC PATCH 1/5] SUNRPC: Replace the queue timer with a delayed work function Trond Myklebust
2019-05-03 11:18   ` [RFC PATCH 2/5] SUNRPC: Replace direct task wakeups from softirq context Trond Myklebust
2019-05-03 11:18     ` [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock Trond Myklebust
2019-05-03 11:18       ` [RFC PATCH 4/5] SUNRPC: Remove the bh-safe lock requirement on the rpc_wait_queue->lock Trond Myklebust
2019-05-03 11:18         ` [RFC PATCH 5/5] SUNRPC: Reduce the priority of the xprtiod queue Trond Myklebust
2019-05-06 20:41           ` Chuck Lever
2019-05-28 19:03             ` Chuck Lever
2019-05-28 19:33               ` Trond Myklebust
2019-05-28 19:52                 ` Chuck Lever
2019-05-29 17:13                   ` Chuck Lever
2019-05-28 20:10           ` Olga Kornievskaia
2019-05-29 18:38             ` Olga Kornievskaia
2019-05-29 18:45               ` Trond Myklebust
2019-05-03 14:21       ` [RFC PATCH 3/5] SUNRPC: Remove the bh-safe lock requirement on xprt->transport_lock Chuck Lever
2019-05-03 15:28         ` Trond Myklebust
2019-05-03 15:43           ` Chuck Lever
2019-05-06 18:22 ` [RFC PATCH 0/5] bh-safe lock removal for SUNRPC Chuck Lever
2019-05-06 18:37   ` Trond Myklebust
2019-05-06 20:02     ` 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).