All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 00/12] SUNRPC: various thread management improvements
@ 2023-07-31  6:48 NeilBrown
  2023-07-31  6:48 ` [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent() NeilBrown
                   ` (11 more replies)
  0 siblings, 12 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

This patch set is against topic-sunrpc-thread-scheduling.
The series contains 2 fix-ups that I suggest be merged into earlier patches 
in the topic branch with matching name.
For the first to work, patch 01/12 needs to be inserted before the first
fixed patch.
Remaining 9 patches can go at the end.

An end result of this is that waking up an idle thread no longer
searches the list of all threads.  However it does requiring taking a
spinlock, though it has a very short hold time.

If/when these successfully land in the topic branch (or earlier if you
like) I can post a further collection of patches which removes that new
locking and reduces locking for the queueing of transports and NFSv4.1
callback requests.

Thanks,
NeilBrown



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

* [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent()
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31 14:21   ` Chuck Lever
  2023-07-31 14:33   ` Jeff Layton
  2023-07-31  6:48 ` [PATCH 02/12] FIXUP: SUNRPC: Deduplicate thread wake-up code NeilBrown
                   ` (10 subsequent siblings)
  11 siblings, 2 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

Based on its name you would think that rqst_should_sleep() would be
read-only, not changing anything.  But it fact it will clear
SP_TASK_PENDING if that was set.  This is surprising, and it blurs the
line between "check for work to do" and "dequeue work to do".

So change the "test_and_clear" to simple "test" and clear the bit once
the thread has decided to wake up and return to the caller.

With this, it makes sense to *always* set SP_TASK_PENDING when asked,
rather than only to set it if no thread could be woken up.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 net/sunrpc/svc_xprt.c | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)

diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
index cd92cb54132d..380fb3caea4c 100644
--- a/net/sunrpc/svc_xprt.c
+++ b/net/sunrpc/svc_xprt.c
@@ -581,8 +581,8 @@ void svc_wake_up(struct svc_serv *serv)
 {
 	struct svc_pool *pool = &serv->sv_pools[0];
 
-	if (!svc_pool_wake_idle_thread(serv, pool))
-		set_bit(SP_TASK_PENDING, &pool->sp_flags);
+	set_bit(SP_TASK_PENDING, &pool->sp_flags);
+	svc_pool_wake_idle_thread(serv, pool);
 }
 EXPORT_SYMBOL_GPL(svc_wake_up);
 
@@ -704,7 +704,7 @@ rqst_should_sleep(struct svc_rqst *rqstp)
 	struct svc_pool		*pool = rqstp->rq_pool;
 
 	/* did someone call svc_wake_up? */
-	if (test_and_clear_bit(SP_TASK_PENDING, &pool->sp_flags))
+	if (test_bit(SP_TASK_PENDING, &pool->sp_flags))
 		return false;
 
 	/* was a socket queued? */
@@ -750,6 +750,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
 
 	set_bit(RQ_BUSY, &rqstp->rq_flags);
 	smp_mb__after_atomic();
+	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
 	rqstp->rq_xprt = svc_xprt_dequeue(pool);
 	if (rqstp->rq_xprt) {
 		trace_svc_pool_awoken(rqstp);
@@ -761,6 +762,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
 	percpu_counter_inc(&pool->sp_threads_no_work);
 	return NULL;
 out_found:
+	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
 	/* Normally we will wait up to 5 seconds for any required
 	 * cache information to be provided.
 	 */
-- 
2.40.1


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

* [PATCH 02/12] FIXUP: SUNRPC: Deduplicate thread wake-up code
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
  2023-07-31  6:48 ` [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent() NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31  6:48 ` [PATCH 03/12] FIXUP: SUNRPC: call svc_process() from svc_recv() NeilBrown
                   ` (9 subsequent siblings)
  11 siblings, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

The returned value is not used (any more), so don't return it.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 include/linux/sunrpc/svc.h | 2 +-
 net/sunrpc/svc.c           | 8 ++------
 2 files changed, 3 insertions(+), 7 deletions(-)

diff --git a/include/linux/sunrpc/svc.h b/include/linux/sunrpc/svc.h
index c3f7726fc9f2..110f4560be38 100644
--- a/include/linux/sunrpc/svc.h
+++ b/include/linux/sunrpc/svc.h
@@ -434,7 +434,7 @@ int		   svc_register(const struct svc_serv *, struct net *, const int,
 
 void		   svc_wake_up(struct svc_serv *);
 void		   svc_reserve(struct svc_rqst *rqstp, int space);
-bool		   svc_pool_wake_idle_thread(struct svc_serv *serv,
+void		   svc_pool_wake_idle_thread(struct svc_serv *serv,
 					     struct svc_pool *pool);
 struct svc_pool   *svc_pool_for_cpu(struct svc_serv *serv);
 char *		   svc_print_addr(struct svc_rqst *, char *, size_t);
diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
index deb40d527b32..cbfd4ac02a4d 100644
--- a/net/sunrpc/svc.c
+++ b/net/sunrpc/svc.c
@@ -701,11 +701,8 @@ svc_prepare_thread(struct svc_serv *serv, struct svc_pool *pool, int node)
  * service thread and marking it BUSY is atomic with respect to
  * other calls to svc_pool_wake_idle_thread().
  *
- * Return value:
- *   %true: An idle thread was awoken
- *   %false: No idle thread was found
  */
-bool svc_pool_wake_idle_thread(struct svc_serv *serv, struct svc_pool *pool)
+void svc_pool_wake_idle_thread(struct svc_serv *serv, struct svc_pool *pool)
 {
 	struct svc_rqst	*rqstp;
 
@@ -719,14 +716,13 @@ bool svc_pool_wake_idle_thread(struct svc_serv *serv, struct svc_pool *pool)
 		rcu_read_unlock();
 		percpu_counter_inc(&pool->sp_threads_woken);
 		trace_svc_wake_up(rqstp);
-		return true;
+		return;
 	}
 	rcu_read_unlock();
 
 	trace_svc_pool_starved(serv, pool);
 	percpu_counter_inc(&pool->sp_threads_starved);
 	set_bit(SP_CONGESTED, &pool->sp_flags);
-	return false;
 }
 
 static struct svc_pool *
-- 
2.40.1


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

* [PATCH 03/12] FIXUP: SUNRPC: call svc_process() from svc_recv()
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
  2023-07-31  6:48 ` [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent() NeilBrown
  2023-07-31  6:48 ` [PATCH 02/12] FIXUP: SUNRPC: Deduplicate thread wake-up code NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31 14:22   ` Chuck Lever
  2023-07-31  6:48 ` [PATCH 04/12] nfsd: Simplify code around svc_exit_thread() call in nfsd() NeilBrown
                   ` (8 subsequent siblings)
  11 siblings, 1 reply; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

Now that svc_process() is called only by svc_recv(), it doesn't need to
be exported.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 net/sunrpc/svc.c | 1 -
 1 file changed, 1 deletion(-)

diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
index cbfd4ac02a4d..f2971d94b4aa 100644
--- a/net/sunrpc/svc.c
+++ b/net/sunrpc/svc.c
@@ -1546,7 +1546,6 @@ void svc_process(struct svc_rqst *rqstp)
 out_drop:
 	svc_drop(rqstp);
 }
-EXPORT_SYMBOL_GPL(svc_process);
 
 #if defined(CONFIG_SUNRPC_BACKCHANNEL)
 /*
-- 
2.40.1


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

* [PATCH 04/12] nfsd: Simplify code around svc_exit_thread() call in nfsd()
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (2 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 03/12] FIXUP: SUNRPC: call svc_process() from svc_recv() NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31  6:48 ` [PATCH 05/12] nfsd: separate nfsd_last_thread() from nfsd_put() NeilBrown
                   ` (7 subsequent siblings)
  11 siblings, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

Previously a thread could exit asynchronously (due to a signal) so some
care was needed to hold nfsd_mutex over the last svc_put() call.  Now a
thread can only exit when svc_set_num_thread() is called, and this is
always called under nfsd_mutex.  So no care is needed.

Not only is it the mutex held when a thread exits now, but the svc
refcount is elevated, so the svc_put() in svc_exit_thread() will never
be a final put, so the mutex isn't even needed at this point in the
code.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 fs/nfsd/nfssvc.c           | 23 -----------------------
 include/linux/sunrpc/svc.h | 13 -------------
 2 files changed, 36 deletions(-)

diff --git a/fs/nfsd/nfssvc.c b/fs/nfsd/nfssvc.c
index 82e272707d2f..33a80725e14e 100644
--- a/fs/nfsd/nfssvc.c
+++ b/fs/nfsd/nfssvc.c
@@ -982,31 +982,8 @@ nfsd(void *vrqstp)
 	atomic_dec(&nfsdstats.th_cnt);
 
 out:
-	/* Take an extra ref so that the svc_put in svc_exit_thread()
-	 * doesn't call svc_destroy()
-	 */
-	svc_get(nn->nfsd_serv);
-
 	/* Release the thread */
 	svc_exit_thread(rqstp);
-
-	/* We need to drop a ref, but may not drop the last reference
-	 * without holding nfsd_mutex, and we cannot wait for nfsd_mutex as that
-	 * could deadlock with nfsd_shutdown_threads() waiting for us.
-	 * So three options are:
-	 * - drop a non-final reference,
-	 * - get the mutex without waiting
-	 * - sleep briefly andd try the above again
-	 */
-	while (!svc_put_not_last(nn->nfsd_serv)) {
-		if (mutex_trylock(&nfsd_mutex)) {
-			nfsd_put(net);
-			mutex_unlock(&nfsd_mutex);
-			break;
-		}
-		msleep(20);
-	}
-
 	return 0;
 }
 
diff --git a/include/linux/sunrpc/svc.h b/include/linux/sunrpc/svc.h
index 110f4560be38..db3de4ea33f9 100644
--- a/include/linux/sunrpc/svc.h
+++ b/include/linux/sunrpc/svc.h
@@ -125,19 +125,6 @@ static inline void svc_put(struct svc_serv *serv)
 	kref_put(&serv->sv_refcnt, svc_destroy);
 }
 
-/**
- * svc_put_not_last - decrement non-final reference count on SUNRPC serv
- * @serv:  the svc_serv to have count decremented
- *
- * Returns: %true is refcount was decremented.
- *
- * If the refcount is 1, it is not decremented and instead failure is reported.
- */
-static inline bool svc_put_not_last(struct svc_serv *serv)
-{
-	return refcount_dec_not_one(&serv->sv_refcnt.refcount);
-}
-
 /*
  * Maximum payload size supported by a kernel RPC server.
  * This is use to determine the max number of pages nfsd is
-- 
2.40.1


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

* [PATCH 05/12] nfsd: separate nfsd_last_thread() from nfsd_put()
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (3 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 04/12] nfsd: Simplify code around svc_exit_thread() call in nfsd() NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31 14:23   ` Chuck Lever
  2023-07-31  6:48 ` [PATCH 06/12] SUNRPC: rename and refactor svc_get_next_xprt() NeilBrown
                   ` (6 subsequent siblings)
  11 siblings, 1 reply; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

Now that the last nfsd thread is stopped by an explicit act of calling
svc_set_num_threads() with a count of zero, we only have a limited
number of places that can happen, and don't need to call
nfsd_last_thread() in nfsd_put()

So separate that out and call it at the two places where the number of
threads is set to zero.

Move the clearing of ->nfsd_serv and the call to svc_xprt_destroy_all()
into nfsd_last_thread(), as they are really part of the same action.

nfsd_put() is now a thin wrapper around svc_put(), so make it a static inline.

nfsd_put() cannot be called after nfsd_last_thread(), so in a couple of
places we have to use svc_put() instead.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 fs/nfsd/nfsd.h   |  7 ++++++-
 fs/nfsd/nfssvc.c | 52 ++++++++++++++++++------------------------------
 2 files changed, 25 insertions(+), 34 deletions(-)

diff --git a/fs/nfsd/nfsd.h b/fs/nfsd/nfsd.h
index d88498f8b275..11c14faa6c67 100644
--- a/fs/nfsd/nfsd.h
+++ b/fs/nfsd/nfsd.h
@@ -96,7 +96,12 @@ int		nfsd_pool_stats_open(struct inode *, struct file *);
 int		nfsd_pool_stats_release(struct inode *, struct file *);
 void		nfsd_shutdown_threads(struct net *net);
 
-void		nfsd_put(struct net *net);
+static inline void nfsd_put(struct net *net)
+{
+	struct nfsd_net *nn = net_generic(net, nfsd_net_id);
+
+	svc_put(nn->nfsd_serv);
+}
 
 bool		i_am_nfsd(void);
 
diff --git a/fs/nfsd/nfssvc.c b/fs/nfsd/nfssvc.c
index 33a80725e14e..1582af33e204 100644
--- a/fs/nfsd/nfssvc.c
+++ b/fs/nfsd/nfssvc.c
@@ -542,9 +542,14 @@ static struct notifier_block nfsd_inet6addr_notifier = {
 /* Only used under nfsd_mutex, so this atomic may be overkill: */
 static atomic_t nfsd_notifier_refcount = ATOMIC_INIT(0);
 
-static void nfsd_last_thread(struct svc_serv *serv, struct net *net)
+static void nfsd_last_thread(struct net *net)
 {
 	struct nfsd_net *nn = net_generic(net, nfsd_net_id);
+	struct svc_serv *serv = nn->nfsd_serv;
+
+	spin_lock(&nfsd_notifier_lock);
+	nn->nfsd_serv = NULL;
+	spin_unlock(&nfsd_notifier_lock);
 
 	/* check if the notifier still has clients */
 	if (atomic_dec_return(&nfsd_notifier_refcount) == 0) {
@@ -554,6 +559,8 @@ static void nfsd_last_thread(struct svc_serv *serv, struct net *net)
 #endif
 	}
 
+	svc_xprt_destroy_all(serv, net);
+
 	/*
 	 * write_ports can create the server without actually starting
 	 * any threads--if we get shut down before any threads are
@@ -644,7 +651,8 @@ void nfsd_shutdown_threads(struct net *net)
 	svc_get(serv);
 	/* Kill outstanding nfsd threads */
 	svc_set_num_threads(serv, NULL, 0);
-	nfsd_put(net);
+	nfsd_last_thread(net);
+	svc_put(serv);
 	mutex_unlock(&nfsd_mutex);
 }
 
@@ -674,9 +682,6 @@ int nfsd_create_serv(struct net *net)
 	serv->sv_maxconn = nn->max_connections;
 	error = svc_bind(serv, net);
 	if (error < 0) {
-		/* NOT nfsd_put() as notifiers (see below) haven't
-		 * been set up yet.
-		 */
 		svc_put(serv);
 		return error;
 	}
@@ -719,29 +724,6 @@ int nfsd_get_nrthreads(int n, int *nthreads, struct net *net)
 	return 0;
 }
 
-/* This is the callback for kref_put() below.
- * There is no code here as the first thing to be done is
- * call svc_shutdown_net(), but we cannot get the 'net' from
- * the kref.  So do all the work when kref_put returns true.
- */
-static void nfsd_noop(struct kref *ref)
-{
-}
-
-void nfsd_put(struct net *net)
-{
-	struct nfsd_net *nn = net_generic(net, nfsd_net_id);
-
-	if (kref_put(&nn->nfsd_serv->sv_refcnt, nfsd_noop)) {
-		svc_xprt_destroy_all(nn->nfsd_serv, net);
-		nfsd_last_thread(nn->nfsd_serv, net);
-		svc_destroy(&nn->nfsd_serv->sv_refcnt);
-		spin_lock(&nfsd_notifier_lock);
-		nn->nfsd_serv = NULL;
-		spin_unlock(&nfsd_notifier_lock);
-	}
-}
-
 int nfsd_set_nrthreads(int n, int *nthreads, struct net *net)
 {
 	int i = 0;
@@ -792,7 +774,7 @@ int nfsd_set_nrthreads(int n, int *nthreads, struct net *net)
 		if (err)
 			break;
 	}
-	nfsd_put(net);
+	svc_put(nn->nfsd_serv);
 	return err;
 }
 
@@ -807,6 +789,7 @@ nfsd_svc(int nrservs, struct net *net, const struct cred *cred)
 	int	error;
 	bool	nfsd_up_before;
 	struct nfsd_net *nn = net_generic(net, nfsd_net_id);
+	struct svc_serv *serv;
 
 	mutex_lock(&nfsd_mutex);
 	dprintk("nfsd: creating service\n");
@@ -826,22 +809,25 @@ nfsd_svc(int nrservs, struct net *net, const struct cred *cred)
 		goto out;
 
 	nfsd_up_before = nn->nfsd_net_up;
+	serv = nn->nfsd_serv;
 
 	error = nfsd_startup_net(net, cred);
 	if (error)
 		goto out_put;
-	error = svc_set_num_threads(nn->nfsd_serv, NULL, nrservs);
+	error = svc_set_num_threads(serv, NULL, nrservs);
 	if (error)
 		goto out_shutdown;
-	error = nn->nfsd_serv->sv_nrthreads;
+	error = serv->sv_nrthreads;
+	if (error == 0)
+		nfsd_last_thread(net);
 out_shutdown:
 	if (error < 0 && !nfsd_up_before)
 		nfsd_shutdown_net(net);
 out_put:
 	/* Threads now hold service active */
 	if (xchg(&nn->keep_active, 0))
-		nfsd_put(net);
-	nfsd_put(net);
+		svc_put(serv);
+	svc_put(serv);
 out:
 	mutex_unlock(&nfsd_mutex);
 	return error;
-- 
2.40.1


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

* [PATCH 06/12] SUNRPC: rename and refactor svc_get_next_xprt().
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (4 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 05/12] nfsd: separate nfsd_last_thread() from nfsd_put() NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31 23:16   ` Chuck Lever
  2023-07-31  6:48 ` [PATCH 07/12] SUNRPC: move all of xprt handling into svc_xprt_handle() NeilBrown
                   ` (5 subsequent siblings)
  11 siblings, 1 reply; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

svc_get_next_xprt() does a lot more than just get an xprt.  It also
decides if it needs to sleep, depending not only on the availability of
xprts, but also on the need to exit or handle external work
(SP_TASK_PENDING).

So rename it to svc_rqst_wait_and_dequeue_work(), don't return the xprt
(which can easily be found in rqstp->rq_xprt), and restructure to make a
clear separation between waiting and dequeueing.

All the scheduling-related code like try_to_freeze() and
kthread_should_stop() is moved into svc_rqst_wait_and_dequeue_work().

Rather than calling svc_xprt_dequeue() twice (before and after deciding
to wait), it now calls rqst_should_sleep() twice.  If the first fails,
we skip all the waiting code completely.  In the waiting code we call
again after setting the task state in case we missed a wake-up.

We now only have one call to try_to_freeze() and one call to
svc_xprt_dequeue().  We still have two calls to kthread_should_stop() -
one in rqst_should_sleep() to avoid sleeping, and one afterwards to
avoid dequeueing any work (it previously came after dequeueing which
doesn't seem right).

Signed-off-by: NeilBrown <neilb@suse.de>
---
 net/sunrpc/svc_xprt.c | 62 +++++++++++++++++++++----------------------
 1 file changed, 31 insertions(+), 31 deletions(-)

diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
index 380fb3caea4c..67f2b34cb8e4 100644
--- a/net/sunrpc/svc_xprt.c
+++ b/net/sunrpc/svc_xprt.c
@@ -722,47 +722,51 @@ rqst_should_sleep(struct svc_rqst *rqstp)
 	return true;
 }
 
-static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
+static void svc_rqst_wait_and_dequeue_work(struct svc_rqst *rqstp)
 {
 	struct svc_pool		*pool = rqstp->rq_pool;
+	bool slept = false;
 
 	/* rq_xprt should be clear on entry */
 	WARN_ON_ONCE(rqstp->rq_xprt);
 
-	rqstp->rq_xprt = svc_xprt_dequeue(pool);
-	if (rqstp->rq_xprt) {
-		trace_svc_pool_polled(rqstp);
-		goto out_found;
+	if (rqst_should_sleep(rqstp)) {
+		set_current_state(TASK_IDLE);
+		smp_mb__before_atomic();
+		clear_bit(SP_CONGESTED, &pool->sp_flags);
+		clear_bit(RQ_BUSY, &rqstp->rq_flags);
+		smp_mb__after_atomic();
+
+		/* Need to test again after setting task state */
+		if (likely(rqst_should_sleep(rqstp))) {
+			schedule();
+			slept = true;
+		} else {
+			__set_current_state(TASK_RUNNING);
+			cond_resched();
+		}
+		set_bit(RQ_BUSY, &rqstp->rq_flags);
+		smp_mb__after_atomic();
 	}
-
-	set_current_state(TASK_IDLE);
-	smp_mb__before_atomic();
-	clear_bit(SP_CONGESTED, &pool->sp_flags);
-	clear_bit(RQ_BUSY, &rqstp->rq_flags);
-	smp_mb__after_atomic();
-
-	if (likely(rqst_should_sleep(rqstp)))
-		schedule();
-	else
-		__set_current_state(TASK_RUNNING);
-
 	try_to_freeze();
 
-	set_bit(RQ_BUSY, &rqstp->rq_flags);
-	smp_mb__after_atomic();
+	if (kthread_should_stop())
+		return;
+
 	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
 	rqstp->rq_xprt = svc_xprt_dequeue(pool);
 	if (rqstp->rq_xprt) {
-		trace_svc_pool_awoken(rqstp);
+		if (slept)
+			trace_svc_pool_awoken(rqstp);
+		else
+			trace_svc_pool_polled(rqstp);
 		goto out_found;
 	}
 
-	if (kthread_should_stop())
-		return NULL;
-	percpu_counter_inc(&pool->sp_threads_no_work);
-	return NULL;
+	if (slept)
+		percpu_counter_inc(&pool->sp_threads_no_work);
+	return;
 out_found:
-	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
 	/* Normally we will wait up to 5 seconds for any required
 	 * cache information to be provided.
 	 */
@@ -770,7 +774,6 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
 		rqstp->rq_chandle.thread_wait = 5*HZ;
 	else
 		rqstp->rq_chandle.thread_wait = 1*HZ;
-	return rqstp->rq_xprt;
 }
 
 static void svc_add_new_temp_xprt(struct svc_serv *serv, struct svc_xprt *newxpt)
@@ -854,12 +857,9 @@ void svc_recv(struct svc_rqst *rqstp)
 	if (!svc_alloc_arg(rqstp))
 		goto out;
 
-	try_to_freeze();
-	cond_resched();
-	if (kthread_should_stop())
-		goto out;
+	svc_rqst_wait_and_dequeue_work(rqstp);
 
-	xprt = svc_get_next_xprt(rqstp);
+	xprt = rqstp->rq_xprt;
 	if (!xprt)
 		goto out;
 
-- 
2.40.1


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

* [PATCH 07/12] SUNRPC: move all of xprt handling into svc_xprt_handle()
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (5 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 06/12] SUNRPC: rename and refactor svc_get_next_xprt() NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31  6:48 ` [PATCH 08/12] SUNRPC: move task-dequeueing code into svc_recv() NeilBrown
                   ` (4 subsequent siblings)
  11 siblings, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

svc_xprt_handle() does lots of things itself, but leaves some to the
caller - svc_recv().  This isn't elegant.

Move that code out of svc_recv() into svc_xprt_handle()

Remove the calls to svc_xprt_release() from svc_send() and svc_drop()
(the two possible final steps in svc_process()) and from svc_recv() (in
the case where svc_process() wasn't called) into svc_xprt_handle().

Signed-off-by: NeilBrown <neilb@suse.de>
---
 net/sunrpc/svc_xprt.c | 53 ++++++++++++++++---------------------------
 1 file changed, 20 insertions(+), 33 deletions(-)

diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
index 67f2b34cb8e4..604c486c8576 100644
--- a/net/sunrpc/svc_xprt.c
+++ b/net/sunrpc/svc_xprt.c
@@ -792,7 +792,7 @@ static void svc_add_new_temp_xprt(struct svc_serv *serv, struct svc_xprt *newxpt
 	svc_xprt_received(newxpt);
 }
 
-static int svc_handle_xprt(struct svc_rqst *rqstp, struct svc_xprt *xprt)
+static void svc_handle_xprt(struct svc_rqst *rqstp, struct svc_xprt *xprt)
 {
 	struct svc_serv *serv = rqstp->rq_server;
 	int len = 0;
@@ -833,11 +833,26 @@ static int svc_handle_xprt(struct svc_rqst *rqstp, struct svc_xprt *xprt)
 			len = xprt->xpt_ops->xpo_recvfrom(rqstp);
 		rqstp->rq_reserved = serv->sv_max_mesg;
 		atomic_add(rqstp->rq_reserved, &xprt->xpt_reserved);
+		if (len <= 0)
+			goto out;
+
+		trace_svc_xdr_recvfrom(&rqstp->rq_arg);
+
+		clear_bit(XPT_OLD, &xprt->xpt_flags);
+
+		rqstp->rq_chandle.defer = svc_defer;
+
+		if (serv->sv_stats)
+			serv->sv_stats->netcnt++;
+		percpu_counter_inc(&rqstp->rq_pool->sp_messages_arrived);
+		rqstp->rq_stime = ktime_get();
+		svc_process(rqstp);
 	} else
 		svc_xprt_received(xprt);
 
 out:
-	return len;
+	rqstp->rq_res.len = 0;
+	svc_xprt_release(rqstp);
 }
 
 /**
@@ -851,40 +866,16 @@ static int svc_handle_xprt(struct svc_rqst *rqstp, struct svc_xprt *xprt)
 void svc_recv(struct svc_rqst *rqstp)
 {
 	struct svc_xprt		*xprt = NULL;
-	struct svc_serv		*serv = rqstp->rq_server;
-	int			len;
 
 	if (!svc_alloc_arg(rqstp))
-		goto out;
+		return;
 
 	svc_rqst_wait_and_dequeue_work(rqstp);
 
 	xprt = rqstp->rq_xprt;
-	if (!xprt)
-		goto out;
-
-	len = svc_handle_xprt(rqstp, xprt);
-
-	/* No data, incomplete (TCP) read, or accept() */
-	if (len <= 0)
-		goto out_release;
-
-	trace_svc_xdr_recvfrom(&rqstp->rq_arg);
-
-	clear_bit(XPT_OLD, &xprt->xpt_flags);
-
-	rqstp->rq_chandle.defer = svc_defer;
-
-	if (serv->sv_stats)
-		serv->sv_stats->netcnt++;
-	percpu_counter_inc(&rqstp->rq_pool->sp_messages_arrived);
-	rqstp->rq_stime = ktime_get();
-	svc_process(rqstp);
+	if (xprt)
+		svc_handle_xprt(rqstp, xprt);
 out:
-	return;
-out_release:
-	rqstp->rq_res.len = 0;
-	svc_xprt_release(rqstp);
 }
 EXPORT_SYMBOL_GPL(svc_recv);
 
@@ -894,7 +885,6 @@ EXPORT_SYMBOL_GPL(svc_recv);
 void svc_drop(struct svc_rqst *rqstp)
 {
 	trace_svc_drop(rqstp);
-	svc_xprt_release(rqstp);
 }
 EXPORT_SYMBOL_GPL(svc_drop);
 
@@ -910,8 +900,6 @@ void svc_send(struct svc_rqst *rqstp)
 	int status;
 
 	xprt = rqstp->rq_xprt;
-	if (!xprt)
-		return;
 
 	/* calculate over-all length */
 	xb = &rqstp->rq_res;
@@ -924,7 +912,6 @@ void svc_send(struct svc_rqst *rqstp)
 	status = xprt->xpt_ops->xpo_sendto(rqstp);
 
 	trace_svc_send(rqstp, status);
-	svc_xprt_release(rqstp);
 }
 
 /*
-- 
2.40.1


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

* [PATCH 08/12] SUNRPC: move task-dequeueing code into svc_recv()
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (6 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 07/12] SUNRPC: move all of xprt handling into svc_xprt_handle() NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31  6:48 ` [PATCH 09/12] SUNRPC: integrate back-channel processing with svc_recv() NeilBrown
                   ` (3 subsequent siblings)
  11 siblings, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

svc_recv() has become rather small, and svc_rqst_wait_and_dequeue_work()
performs two different tasks.

So move the "dequeue" part out of svc_rqst_wait_and_dequeue_work()
into svc_recv().  This balances code between the two.

svc_rqst_wait_and_dequeue_work() is now svc_rqst_wait_for_work() and
returns bool if it actually waited.  This is used to guide tracing and
some statistics gathering.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 net/sunrpc/svc_xprt.c | 67 +++++++++++++++++++++----------------------
 1 file changed, 32 insertions(+), 35 deletions(-)

diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
index 604c486c8576..45a76313b7e1 100644
--- a/net/sunrpc/svc_xprt.c
+++ b/net/sunrpc/svc_xprt.c
@@ -722,14 +722,11 @@ rqst_should_sleep(struct svc_rqst *rqstp)
 	return true;
 }
 
-static void svc_rqst_wait_and_dequeue_work(struct svc_rqst *rqstp)
+static bool svc_rqst_wait_for_work(struct svc_rqst *rqstp)
 {
-	struct svc_pool		*pool = rqstp->rq_pool;
+	struct svc_pool *pool = rqstp->rq_pool;
 	bool slept = false;
 
-	/* rq_xprt should be clear on entry */
-	WARN_ON_ONCE(rqstp->rq_xprt);
-
 	if (rqst_should_sleep(rqstp)) {
 		set_current_state(TASK_IDLE);
 		smp_mb__before_atomic();
@@ -749,31 +746,7 @@ static void svc_rqst_wait_and_dequeue_work(struct svc_rqst *rqstp)
 		smp_mb__after_atomic();
 	}
 	try_to_freeze();
-
-	if (kthread_should_stop())
-		return;
-
-	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
-	rqstp->rq_xprt = svc_xprt_dequeue(pool);
-	if (rqstp->rq_xprt) {
-		if (slept)
-			trace_svc_pool_awoken(rqstp);
-		else
-			trace_svc_pool_polled(rqstp);
-		goto out_found;
-	}
-
-	if (slept)
-		percpu_counter_inc(&pool->sp_threads_no_work);
-	return;
-out_found:
-	/* Normally we will wait up to 5 seconds for any required
-	 * cache information to be provided.
-	 */
-	if (!test_bit(SP_CONGESTED, &pool->sp_flags))
-		rqstp->rq_chandle.thread_wait = 5*HZ;
-	else
-		rqstp->rq_chandle.thread_wait = 1*HZ;
+	return slept;
 }
 
 static void svc_add_new_temp_xprt(struct svc_serv *serv, struct svc_xprt *newxpt)
@@ -865,17 +838,41 @@ static void svc_handle_xprt(struct svc_rqst *rqstp, struct svc_xprt *xprt)
  */
 void svc_recv(struct svc_rqst *rqstp)
 {
-	struct svc_xprt		*xprt = NULL;
+	struct svc_pool *pool = rqstp->rq_pool;
+	bool slept;
 
 	if (!svc_alloc_arg(rqstp))
 		return;
 
-	svc_rqst_wait_and_dequeue_work(rqstp);
+	slept = svc_rqst_wait_for_work(rqstp);
 
-	xprt = rqstp->rq_xprt;
-	if (xprt)
+	if (kthread_should_stop())
+		return;
+
+	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
+
+	rqstp->rq_xprt = svc_xprt_dequeue(pool);
+	if (rqstp->rq_xprt) {
+		struct svc_xprt *xprt = rqstp->rq_xprt;
+
+		if (slept)
+			trace_svc_pool_awoken(rqstp);
+		else
+			trace_svc_pool_polled(rqstp);
+
+		/* Normally we will wait up to 5 seconds for any required
+		 * cache information to be provided.
+		 */
+		if (test_bit(SP_CONGESTED, &pool->sp_flags))
+			rqstp->rq_chandle.thread_wait = 5 * HZ;
+		else
+			rqstp->rq_chandle.thread_wait = 1 * HZ;
 		svc_handle_xprt(rqstp, xprt);
-out:
+		return;
+	}
+
+	if (slept)
+		percpu_counter_inc(&pool->sp_threads_no_work);
 }
 EXPORT_SYMBOL_GPL(svc_recv);
 
-- 
2.40.1


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

* [PATCH 09/12] SUNRPC: integrate back-channel processing with svc_recv()
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (7 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 08/12] SUNRPC: move task-dequeueing code into svc_recv() NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31  6:48 ` [PATCH 10/12] SUNRPC: change how svc threads are asked to exit NeilBrown
                   ` (2 subsequent siblings)
  11 siblings, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

Using svc_recv() for (NFSv4.1) back-channel handling means we have just
one mechanism for waking threads.

Also change kthread_freezable_should_stop() in nfs4_callback_svc() to
kthread_should_stop() as used elsewhere.
kthread_freezable_should_stop() effectively adds a try_to_freeze() call,
and svc_recv() already contains that at an appropriate place.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 fs/nfs/callback.c                 | 46 ++-----------------------------
 include/linux/sunrpc/svc.h        |  2 --
 net/sunrpc/backchannel_rqst.c     |  8 ++----
 net/sunrpc/svc.c                  |  2 +-
 net/sunrpc/svc_xprt.c             | 32 +++++++++++++++++++++
 net/sunrpc/xprtrdma/backchannel.c |  2 +-
 6 files changed, 39 insertions(+), 53 deletions(-)

diff --git a/fs/nfs/callback.c b/fs/nfs/callback.c
index 466ebf1d41b2..42a0c2f1e785 100644
--- a/fs/nfs/callback.c
+++ b/fs/nfs/callback.c
@@ -78,7 +78,7 @@ nfs4_callback_svc(void *vrqstp)
 
 	set_freezable();
 
-	while (!kthread_freezable_should_stop(NULL))
+	while (!kthread_should_stop())
 		svc_recv(rqstp);
 
 	svc_exit_thread(rqstp);
@@ -86,45 +86,6 @@ nfs4_callback_svc(void *vrqstp)
 }
 
 #if defined(CONFIG_NFS_V4_1)
-/*
- * The callback service for NFSv4.1 callbacks
- */
-static int
-nfs41_callback_svc(void *vrqstp)
-{
-	struct svc_rqst *rqstp = vrqstp;
-	struct svc_serv *serv = rqstp->rq_server;
-	struct rpc_rqst *req;
-	int error;
-	DEFINE_WAIT(wq);
-
-	set_freezable();
-
-	while (!kthread_freezable_should_stop(NULL)) {
-		prepare_to_wait(&serv->sv_cb_waitq, &wq, TASK_IDLE);
-		spin_lock_bh(&serv->sv_cb_lock);
-		if (!list_empty(&serv->sv_cb_list)) {
-			req = list_first_entry(&serv->sv_cb_list,
-					struct rpc_rqst, rq_bc_list);
-			list_del(&req->rq_bc_list);
-			spin_unlock_bh(&serv->sv_cb_lock);
-			finish_wait(&serv->sv_cb_waitq, &wq);
-			dprintk("Invoking bc_svc_process()\n");
-			error = bc_svc_process(serv, req, rqstp);
-			dprintk("bc_svc_process() returned w/ error code= %d\n",
-				error);
-		} else {
-			spin_unlock_bh(&serv->sv_cb_lock);
-			if (!kthread_should_stop())
-				schedule();
-			finish_wait(&serv->sv_cb_waitq, &wq);
-		}
-	}
-
-	svc_exit_thread(rqstp);
-	return 0;
-}
-
 static inline void nfs_callback_bc_serv(u32 minorversion, struct rpc_xprt *xprt,
 		struct svc_serv *serv)
 {
@@ -237,10 +198,7 @@ static struct svc_serv *nfs_callback_create_svc(int minorversion)
 			cb_info->users);
 
 	threadfn = nfs4_callback_svc;
-#if defined(CONFIG_NFS_V4_1)
-	if (minorversion)
-		threadfn = nfs41_callback_svc;
-#else
+#if !defined(CONFIG_NFS_V4_1)
 	if (minorversion)
 		return ERR_PTR(-ENOTSUPP);
 #endif
diff --git a/include/linux/sunrpc/svc.h b/include/linux/sunrpc/svc.h
index db3de4ea33f9..a3f1916937b4 100644
--- a/include/linux/sunrpc/svc.h
+++ b/include/linux/sunrpc/svc.h
@@ -93,8 +93,6 @@ struct svc_serv {
 						 * that arrive over the same
 						 * connection */
 	spinlock_t		sv_cb_lock;	/* protects the svc_cb_list */
-	wait_queue_head_t	sv_cb_waitq;	/* sleep here if there are no
-						 * entries in the svc_cb_list */
 	bool			sv_bc_enabled;	/* service uses backchannel */
 #endif /* CONFIG_SUNRPC_BACKCHANNEL */
 };
diff --git a/net/sunrpc/backchannel_rqst.c b/net/sunrpc/backchannel_rqst.c
index 65a6c6429a53..60b8d310bb27 100644
--- a/net/sunrpc/backchannel_rqst.c
+++ b/net/sunrpc/backchannel_rqst.c
@@ -349,10 +349,8 @@ struct rpc_rqst *xprt_lookup_bc_request(struct rpc_xprt *xprt, __be32 xid)
 }
 
 /*
- * Add callback request to callback list.  The callback
- * service sleeps on the sv_cb_waitq waiting for new
- * requests.  Wake it up after adding enqueing the
- * request.
+ * Add callback request to callback list.  Wake a thread
+ * on the first pool (usually the only pool) to handle it.
  */
 void xprt_complete_bc_request(struct rpc_rqst *req, uint32_t copied)
 {
@@ -371,6 +369,6 @@ void xprt_complete_bc_request(struct rpc_rqst *req, uint32_t copied)
 	xprt_get(xprt);
 	spin_lock(&bc_serv->sv_cb_lock);
 	list_add(&req->rq_bc_list, &bc_serv->sv_cb_list);
-	wake_up(&bc_serv->sv_cb_waitq);
 	spin_unlock(&bc_serv->sv_cb_lock);
+	svc_pool_wake_idle_thread(bc_serv, &bc_serv->sv_pools[0]);
 }
diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
index f2971d94b4aa..bdb64651679f 100644
--- a/net/sunrpc/svc.c
+++ b/net/sunrpc/svc.c
@@ -440,7 +440,6 @@ __svc_init_bc(struct svc_serv *serv)
 {
 	INIT_LIST_HEAD(&serv->sv_cb_list);
 	spin_lock_init(&serv->sv_cb_lock);
-	init_waitqueue_head(&serv->sv_cb_waitq);
 }
 #else
 static void
@@ -724,6 +723,7 @@ void svc_pool_wake_idle_thread(struct svc_serv *serv, struct svc_pool *pool)
 	percpu_counter_inc(&pool->sp_threads_starved);
 	set_bit(SP_CONGESTED, &pool->sp_flags);
 }
+EXPORT_SYMBOL_GPL(svc_pool_wake_idle_thread);
 
 static struct svc_pool *
 svc_pool_next(struct svc_serv *serv, struct svc_pool *pool, unsigned int *state)
diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
index 45a76313b7e1..6543e7fac264 100644
--- a/net/sunrpc/svc_xprt.c
+++ b/net/sunrpc/svc_xprt.c
@@ -17,6 +17,7 @@
 #include <linux/sunrpc/svc_xprt.h>
 #include <linux/sunrpc/svcsock.h>
 #include <linux/sunrpc/xprt.h>
+#include <linux/sunrpc/bc_xprt.h>
 #include <linux/module.h>
 #include <linux/netdevice.h>
 #include <trace/events/sunrpc.h>
@@ -719,6 +720,13 @@ rqst_should_sleep(struct svc_rqst *rqstp)
 	if (freezing(current))
 		return false;
 
+#if defined(CONFIG_SUNRPC_BACKCHANNEL)
+	if (svc_is_backchannel(rqstp)) {
+		if (!list_empty(&rqstp->rq_server->sv_cb_list))
+			return false;
+	}
+#endif
+
 	return true;
 }
 
@@ -871,6 +879,30 @@ void svc_recv(struct svc_rqst *rqstp)
 		return;
 	}
 
+#if defined(CONFIG_SUNRPC_BACKCHANNEL)
+	if (svc_is_backchannel(rqstp)) {
+		struct svc_serv *serv = rqstp->rq_server;
+		struct rpc_rqst *req;
+
+		spin_lock_bh(&serv->sv_cb_lock);
+		req = list_first_entry_or_null(&serv->sv_cb_list,
+					       struct rpc_rqst, rq_bc_list);
+		if (req) {
+			int error;
+
+			list_del(&req->rq_bc_list);
+			spin_unlock_bh(&serv->sv_cb_lock);
+
+			dprintk("Invoking bc_svc_process()\n");
+			error = bc_svc_process(rqstp->rq_server, req, rqstp);
+			dprintk("bc_svc_process() returned w/ error code= %d\n",
+				error);
+			return;
+		}
+		spin_unlock_bh(&serv->sv_cb_lock);
+	}
+#endif
+
 	if (slept)
 		percpu_counter_inc(&pool->sp_threads_no_work);
 }
diff --git a/net/sunrpc/xprtrdma/backchannel.c b/net/sunrpc/xprtrdma/backchannel.c
index e4d84a13c566..f1e1d4909434 100644
--- a/net/sunrpc/xprtrdma/backchannel.c
+++ b/net/sunrpc/xprtrdma/backchannel.c
@@ -267,7 +267,7 @@ void rpcrdma_bc_receive_call(struct rpcrdma_xprt *r_xprt,
 	list_add(&rqst->rq_bc_list, &bc_serv->sv_cb_list);
 	spin_unlock(&bc_serv->sv_cb_lock);
 
-	wake_up(&bc_serv->sv_cb_waitq);
+	svc_pool_wake_idle_thread(bc_serv, &bc_serv->sv_pools[0]);
 
 	r_xprt->rx_stats.bcall_count++;
 	return;
-- 
2.40.1


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

* [PATCH 10/12] SUNRPC: change how svc threads are asked to exit.
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (8 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 09/12] SUNRPC: integrate back-channel processing with svc_recv() NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31  6:48 ` [PATCH 11/12] SUNRPC: add list of idle threads NeilBrown
  2023-07-31  6:48 ` [PATCH 12/12] SUNRPC: discard SP_CONGESTED NeilBrown
  11 siblings, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

svc threads are currently stopped using kthread_stop().  This requires
identifying a specific thread.  However we don't care which thread
stops, just as long as one does.

So instead, set a flag in the svc_pool to say that a thread needs to
die, and have each thread check this flag instead of calling
kthread_should_stop().  The first to find it clear the flag and moves
towards shutting down.

This removes an explicit dependency on sp_all_threads which will make a
future patch simpler.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 fs/lockd/svc.c                |  5 ++--
 fs/lockd/svclock.c            |  5 ++--
 fs/nfs/callback.c             |  2 +-
 fs/nfsd/nfs4proc.c            |  8 ++++---
 fs/nfsd/nfssvc.c              |  2 +-
 include/linux/lockd/lockd.h   |  2 +-
 include/linux/sunrpc/svc.h    | 22 +++++++++++++++++-
 include/trace/events/sunrpc.h |  7 ++++--
 net/sunrpc/svc.c              | 43 +++++++++++++++++------------------
 net/sunrpc/svc_xprt.c         |  7 +++---
 10 files changed, 62 insertions(+), 41 deletions(-)

diff --git a/fs/lockd/svc.c b/fs/lockd/svc.c
index 6579948070a4..b441c706c2b8 100644
--- a/fs/lockd/svc.c
+++ b/fs/lockd/svc.c
@@ -24,7 +24,6 @@
 #include <linux/uio.h>
 #include <linux/smp.h>
 #include <linux/mutex.h>
-#include <linux/kthread.h>
 #include <linux/freezer.h>
 #include <linux/inetdevice.h>
 
@@ -135,11 +134,11 @@ lockd(void *vrqstp)
 	 * The main request loop. We don't terminate until the last
 	 * NFS mount or NFS daemon has gone away.
 	 */
-	while (!kthread_should_stop()) {
+	while (!svc_thread_should_stop(rqstp)) {
 		/* update sv_maxconn if it has changed */
 		rqstp->rq_server->sv_maxconn = nlm_max_connections;
 
-		nlmsvc_retry_blocked();
+		nlmsvc_retry_blocked(rqstp);
 		svc_recv(rqstp);
 	}
 	if (nlmsvc_ops)
diff --git a/fs/lockd/svclock.c b/fs/lockd/svclock.c
index 43aeba9de55c..5fea06555f42 100644
--- a/fs/lockd/svclock.c
+++ b/fs/lockd/svclock.c
@@ -30,7 +30,6 @@
 #include <linux/sunrpc/svc_xprt.h>
 #include <linux/lockd/nlm.h>
 #include <linux/lockd/lockd.h>
-#include <linux/kthread.h>
 #include <linux/exportfs.h>
 
 #define NLMDBG_FACILITY		NLMDBG_SVCLOCK
@@ -1020,13 +1019,13 @@ retry_deferred_block(struct nlm_block *block)
  * be retransmitted.
  */
 void
-nlmsvc_retry_blocked(void)
+nlmsvc_retry_blocked(struct svc_rqst *rqstp)
 {
 	unsigned long	timeout = MAX_SCHEDULE_TIMEOUT;
 	struct nlm_block *block;
 
 	spin_lock(&nlm_blocked_lock);
-	while (!list_empty(&nlm_blocked) && !kthread_should_stop()) {
+	while (!list_empty(&nlm_blocked) && !svc_thread_should_stop(rqstp)) {
 		block = list_entry(nlm_blocked.next, struct nlm_block, b_list);
 
 		if (block->b_when == NLM_NEVER)
diff --git a/fs/nfs/callback.c b/fs/nfs/callback.c
index 42a0c2f1e785..4ffa1f469e90 100644
--- a/fs/nfs/callback.c
+++ b/fs/nfs/callback.c
@@ -78,7 +78,7 @@ nfs4_callback_svc(void *vrqstp)
 
 	set_freezable();
 
-	while (!kthread_should_stop())
+	while (!svc_thread_should_stop(rqstp))
 		svc_recv(rqstp);
 
 	svc_exit_thread(rqstp);
diff --git a/fs/nfsd/nfs4proc.c b/fs/nfsd/nfs4proc.c
index aa4f21f92deb..669b16348571 100644
--- a/fs/nfsd/nfs4proc.c
+++ b/fs/nfsd/nfs4proc.c
@@ -1340,7 +1340,8 @@ extern void nfs_sb_deactive(struct super_block *sb);
  * setup a work entry in the ssc delayed unmount list.
  */
 static __be32 nfsd4_ssc_setup_dul(struct nfsd_net *nn, char *ipaddr,
-				  struct nfsd4_ssc_umount_item **nsui)
+				  struct nfsd4_ssc_umount_item **nsui,
+				  struct svc_rqst *rqstp)
 {
 	struct nfsd4_ssc_umount_item *ni = NULL;
 	struct nfsd4_ssc_umount_item *work = NULL;
@@ -1362,7 +1363,7 @@ static __be32 nfsd4_ssc_setup_dul(struct nfsd_net *nn, char *ipaddr,
 			spin_unlock(&nn->nfsd_ssc_lock);
 
 			/* allow 20secs for mount/unmount for now - revisit */
-			if (kthread_should_stop() ||
+			if (svc_thread_should_stop(rqstp) ||
 					(schedule_timeout(20*HZ) == 0)) {
 				finish_wait(&nn->nfsd_ssc_waitq, &wait);
 				kfree(work);
@@ -1478,7 +1479,7 @@ nfsd4_interssc_connect(struct nl4_server *nss, struct svc_rqst *rqstp,
 		goto out_free_rawdata;
 	snprintf(dev_name, len + 5, "%s%s%s:/", startsep, ipaddr, endsep);
 
-	status = nfsd4_ssc_setup_dul(nn, ipaddr, nsui);
+	status = nfsd4_ssc_setup_dul(nn, ipaddr, nsui, rqstp);
 	if (status)
 		goto out_free_devname;
 	if ((*nsui)->nsui_vfsmount)
@@ -1653,6 +1654,7 @@ static ssize_t _nfsd_copy_file_range(struct nfsd4_copy *copy,
 	if (bytes_total == 0)
 		bytes_total = ULLONG_MAX;
 	do {
+		/* Only async copies can be stopped here */
 		if (kthread_should_stop())
 			break;
 		bytes_copied = nfsd_copy_file_range(src, src_pos, dst, dst_pos,
diff --git a/fs/nfsd/nfssvc.c b/fs/nfsd/nfssvc.c
index 1582af33e204..062f51fe4dfb 100644
--- a/fs/nfsd/nfssvc.c
+++ b/fs/nfsd/nfssvc.c
@@ -957,7 +957,7 @@ nfsd(void *vrqstp)
 	/*
 	 * The main request loop
 	 */
-	while (!kthread_should_stop()) {
+	while (!svc_thread_should_stop(rqstp)) {
 		/* Update sv_maxconn if it has changed */
 		rqstp->rq_server->sv_maxconn = nn->max_connections;
 
diff --git a/include/linux/lockd/lockd.h b/include/linux/lockd/lockd.h
index 0f016d69c996..9f565416d186 100644
--- a/include/linux/lockd/lockd.h
+++ b/include/linux/lockd/lockd.h
@@ -282,7 +282,7 @@ __be32		  nlmsvc_testlock(struct svc_rqst *, struct nlm_file *,
 			struct nlm_host *, struct nlm_lock *,
 			struct nlm_lock *, struct nlm_cookie *);
 __be32		  nlmsvc_cancel_blocked(struct net *net, struct nlm_file *, struct nlm_lock *);
-void		  nlmsvc_retry_blocked(void);
+void		  nlmsvc_retry_blocked(struct svc_rqst *rqstp);
 void		  nlmsvc_traverse_blocks(struct nlm_host *, struct nlm_file *,
 					nlm_host_match_fn_t match);
 void		  nlmsvc_grant_reply(struct nlm_cookie *, __be32);
diff --git a/include/linux/sunrpc/svc.h b/include/linux/sunrpc/svc.h
index a3f1916937b4..a11b6bb42c17 100644
--- a/include/linux/sunrpc/svc.h
+++ b/include/linux/sunrpc/svc.h
@@ -51,6 +51,8 @@ struct svc_pool {
 enum {
 	SP_TASK_PENDING,	/* still work to do even if no xprt is queued */
 	SP_CONGESTED,		/* all threads are busy, none idle */
+	SP_NEED_VICTIM,		/* One thread needs to agree to exit */
+	SP_VICTIM_REMAINS,	/* One thread needs to actually exit */
 };
 
 
@@ -260,7 +262,7 @@ enum {
 	RQ_DROPME,		/* drop current reply */
 	RQ_SPLICE_OK,		/* turned off in gss privacy to prevent
 				 * encrypting page cache pages */
-	RQ_VICTIM,		/* about to be shut down */
+	RQ_VICTIM,		/* Have agreed to shut down */
 	RQ_BUSY,		/* request is busy */
 	RQ_DATA,		/* request has data */
 };
@@ -300,6 +302,24 @@ static inline struct sockaddr *svc_daddr(const struct svc_rqst *rqst)
 	return (struct sockaddr *) &rqst->rq_daddr;
 }
 
+/**
+ * svc_thread_should_stop - check if this thread should stop
+ * @rqstp: the thread that might need to stop
+ *
+ * To stop an svc thread, the pool flags SP_NEED_VICTIM and SP_VICTIM_REMAINS
+ * are set.  The firs thread which sees SP_NEED_VICTIM clear it becoming
+ * the victim using this function.  It should then promptly call
+ * svc_exit_thread() which completes the process, clearing SP_VICTIM_REMAINS
+ * so the task waiting for a thread to exit can wake and continue.
+ */
+static inline bool svc_thread_should_stop(struct svc_rqst *rqstp)
+{
+	if (test_and_clear_bit(SP_NEED_VICTIM, &rqstp->rq_pool->sp_flags))
+		set_bit(RQ_VICTIM, &rqstp->rq_flags);
+
+	return test_bit(RQ_VICTIM, &rqstp->rq_flags);
+}
+
 struct svc_deferred_req {
 	u32			prot;	/* protocol (UDP or TCP) */
 	struct svc_xprt		*xprt;
diff --git a/include/trace/events/sunrpc.h b/include/trace/events/sunrpc.h
index b6cb93f22720..488c3ccfb6dc 100644
--- a/include/trace/events/sunrpc.h
+++ b/include/trace/events/sunrpc.h
@@ -2060,12 +2060,15 @@ TRACE_EVENT(svc_xprt_enqueue,
 
 TRACE_DEFINE_ENUM(SP_TASK_PENDING);
 TRACE_DEFINE_ENUM(SP_CONGESTED);
+TRACE_DEFINE_ENUM(SP_NEED_VICTIM);
+TRACE_DEFINE_ENUM(SP_VICTIM_REMAINS);
 
 #define show_svc_pool_flags(x)						\
 	__print_flags(x, "|",						\
 		{ BIT(SP_TASK_PENDING),		"TASK_PENDING" },	\
-		{ BIT(SP_CONGESTED),		"CONGESTED" })
-
+		{ BIT(SP_CONGESTED),		"CONGESTED" },		\
+		{ BIT(SP_NEED_VICTIM),		"NEED_VICTIM" },	\
+		{ BIT(SP_VICTIM_REMAINS),	"VICTIM_REMAINS" })
 DECLARE_EVENT_CLASS(svc_pool_scheduler_class,
 	TP_PROTO(
 		const struct svc_rqst *rqstp
diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
index bdb64651679f..2420d6a09368 100644
--- a/net/sunrpc/svc.c
+++ b/net/sunrpc/svc.c
@@ -731,19 +731,22 @@ svc_pool_next(struct svc_serv *serv, struct svc_pool *pool, unsigned int *state)
 	return pool ? pool : &serv->sv_pools[(*state)++ % serv->sv_nrpools];
 }
 
-static struct task_struct *
+static struct svc_pool *
 svc_pool_victim(struct svc_serv *serv, struct svc_pool *pool, unsigned int *state)
 {
 	unsigned int i;
-	struct task_struct *task = NULL;
 
 	if (pool != NULL) {
 		spin_lock_bh(&pool->sp_lock);
+		if (pool->sp_nrthreads)
+			goto found_pool;
+		spin_unlock_bh(&pool->sp_lock);
+		return NULL;
 	} else {
 		for (i = 0; i < serv->sv_nrpools; i++) {
 			pool = &serv->sv_pools[--(*state) % serv->sv_nrpools];
 			spin_lock_bh(&pool->sp_lock);
-			if (!list_empty(&pool->sp_all_threads))
+			if (pool->sp_nrthreads)
 				goto found_pool;
 			spin_unlock_bh(&pool->sp_lock);
 		}
@@ -751,16 +754,10 @@ svc_pool_victim(struct svc_serv *serv, struct svc_pool *pool, unsigned int *stat
 	}
 
 found_pool:
-	if (!list_empty(&pool->sp_all_threads)) {
-		struct svc_rqst *rqstp;
-
-		rqstp = list_entry(pool->sp_all_threads.next, struct svc_rqst, rq_all);
-		set_bit(RQ_VICTIM, &rqstp->rq_flags);
-		list_del_rcu(&rqstp->rq_all);
-		task = rqstp->rq_task;
-	}
+	set_bit(SP_VICTIM_REMAINS, &pool->sp_flags);
+	set_bit(SP_NEED_VICTIM, &pool->sp_flags);
 	spin_unlock_bh(&pool->sp_lock);
-	return task;
+	return pool;
 }
 
 static int
@@ -801,18 +798,16 @@ svc_start_kthreads(struct svc_serv *serv, struct svc_pool *pool, int nrservs)
 static int
 svc_stop_kthreads(struct svc_serv *serv, struct svc_pool *pool, int nrservs)
 {
-	struct svc_rqst	*rqstp;
-	struct task_struct *task;
 	unsigned int state = serv->sv_nrthreads-1;
+	struct svc_pool *victim;
 
 	do {
-		task = svc_pool_victim(serv, pool, &state);
-		if (task == NULL)
+		victim = svc_pool_victim(serv, pool, &state);
+		if (!victim)
 			break;
-		rqstp = kthread_data(task);
-		/* Did we lose a race to svo_function threadfn? */
-		if (kthread_stop(task) == -EINTR)
-			svc_exit_thread(rqstp);
+		svc_pool_wake_idle_thread(serv, victim);
+		wait_on_bit(&victim->sp_flags, SP_VICTIM_REMAINS,
+			    TASK_IDLE);
 		nrservs++;
 	} while (nrservs < 0);
 	return 0;
@@ -932,8 +927,7 @@ svc_exit_thread(struct svc_rqst *rqstp)
 
 	spin_lock_bh(&pool->sp_lock);
 	pool->sp_nrthreads--;
-	if (!test_and_set_bit(RQ_VICTIM, &rqstp->rq_flags))
-		list_del_rcu(&rqstp->rq_all);
+	list_del_rcu(&rqstp->rq_all);
 	spin_unlock_bh(&pool->sp_lock);
 
 	spin_lock_bh(&serv->sv_lock);
@@ -944,6 +938,11 @@ svc_exit_thread(struct svc_rqst *rqstp)
 	svc_rqst_free(rqstp);
 
 	svc_put(serv);
+	/* That svc_put() cannot be the last, because the thread
+	 * waiting for SP_VICTIM_REMAINS to clear must hold
+	 * a reference. So it is still safe to access pool.
+	 */
+	clear_and_wake_up_bit(SP_VICTIM_REMAINS, &pool->sp_flags);
 }
 EXPORT_SYMBOL_GPL(svc_exit_thread);
 
diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
index 6543e7fac264..32469a8c5ba7 100644
--- a/net/sunrpc/svc_xprt.c
+++ b/net/sunrpc/svc_xprt.c
@@ -9,7 +9,6 @@
 #include <linux/sched/mm.h>
 #include <linux/errno.h>
 #include <linux/freezer.h>
-#include <linux/kthread.h>
 #include <linux/slab.h>
 #include <net/sock.h>
 #include <linux/sunrpc/addr.h>
@@ -675,7 +674,7 @@ static bool svc_alloc_arg(struct svc_rqst *rqstp)
 			continue;
 
 		set_current_state(TASK_IDLE);
-		if (kthread_should_stop()) {
+		if (svc_thread_should_stop(rqstp)) {
 			set_current_state(TASK_RUNNING);
 			return false;
 		}
@@ -713,7 +712,7 @@ rqst_should_sleep(struct svc_rqst *rqstp)
 		return false;
 
 	/* are we shutting down? */
-	if (kthread_should_stop())
+	if (svc_thread_should_stop(rqstp))
 		return false;
 
 	/* are we freezing? */
@@ -854,7 +853,7 @@ void svc_recv(struct svc_rqst *rqstp)
 
 	slept = svc_rqst_wait_for_work(rqstp);
 
-	if (kthread_should_stop())
+	if (svc_thread_should_stop(rqstp))
 		return;
 
 	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
-- 
2.40.1


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

* [PATCH 11/12] SUNRPC: add list of idle threads
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (9 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 10/12] SUNRPC: change how svc threads are asked to exit NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  2023-07-31  6:48 ` [PATCH 12/12] SUNRPC: discard SP_CONGESTED NeilBrown
  11 siblings, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

Rather than searching a list of threads to find an idle one, having a
list of idle threads allows an idle thread to be found immediately.

This adds some spin_lock calls which is not ideal, but as the hold-time
is tiny it is still faster than searching a list.  A future patch will
remove them using llist.h.  This involves some subtlety and so is left
to a separate patch.

This removes the need for the RQ_BUSY flag.  The rqst is "busy"
precisely when it is not on the "idle" list.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 include/linux/sunrpc/svc.h    | 25 ++++++++++++++++++++++++-
 include/trace/events/sunrpc.h |  1 -
 net/sunrpc/svc.c              | 13 ++++++++-----
 net/sunrpc/svc_xprt.c         | 15 +++++++++++----
 4 files changed, 43 insertions(+), 11 deletions(-)

diff --git a/include/linux/sunrpc/svc.h b/include/linux/sunrpc/svc.h
index a11b6bb42c17..c2111bc8a7a1 100644
--- a/include/linux/sunrpc/svc.h
+++ b/include/linux/sunrpc/svc.h
@@ -36,6 +36,7 @@ struct svc_pool {
 	struct list_head	sp_sockets;	/* pending sockets */
 	unsigned int		sp_nrthreads;	/* # of threads in pool */
 	struct list_head	sp_all_threads;	/* all server threads */
+	struct list_head	sp_idle_threads; /* idle server threads */
 
 	/* statistics on pool operation */
 	struct percpu_counter	sp_messages_arrived;
@@ -187,6 +188,7 @@ extern u32 svc_max_payload(const struct svc_rqst *rqstp);
  */
 struct svc_rqst {
 	struct list_head	rq_all;		/* all threads list */
+	struct list_head	rq_idle;	/* On the idle list */
 	struct rcu_head		rq_rcu_head;	/* for RCU deferred kfree */
 	struct svc_xprt *	rq_xprt;	/* transport ptr */
 
@@ -263,10 +265,31 @@ enum {
 	RQ_SPLICE_OK,		/* turned off in gss privacy to prevent
 				 * encrypting page cache pages */
 	RQ_VICTIM,		/* Have agreed to shut down */
-	RQ_BUSY,		/* request is busy */
 	RQ_DATA,		/* request has data */
 };
 
+/**
+ * svc_thread_set_busy - mark a thread as busy
+ * @rqstp: the thread which is now busy
+ *
+ * If rq_idle is "empty", the thread must be busy.
+ */
+static inline void svc_thread_set_busy(struct svc_rqst *rqstp)
+{
+	INIT_LIST_HEAD(&rqstp->rq_idle);
+}
+
+/**
+ * svc_thread_busy - check if a thread as busy
+ * @rqstp: the thread which might be busy
+ *
+ * If rq_idle is "empty", the thread must be busy.
+ */
+static inline bool svc_thread_busy(struct svc_rqst *rqstp)
+{
+	return list_empty(&rqstp->rq_idle);
+}
+
 #define SVC_NET(rqst) (rqst->rq_xprt ? rqst->rq_xprt->xpt_net : rqst->rq_bc_net)
 
 /*
diff --git a/include/trace/events/sunrpc.h b/include/trace/events/sunrpc.h
index 488c3ccfb6dc..d00a1a6b9616 100644
--- a/include/trace/events/sunrpc.h
+++ b/include/trace/events/sunrpc.h
@@ -1677,7 +1677,6 @@ DEFINE_SVCXDRBUF_EVENT(sendto);
 	svc_rqst_flag(DROPME)						\
 	svc_rqst_flag(SPLICE_OK)					\
 	svc_rqst_flag(VICTIM)						\
-	svc_rqst_flag(BUSY)						\
 	svc_rqst_flag_end(DATA)
 
 #undef svc_rqst_flag
diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
index 2420d6a09368..44a614d96d8d 100644
--- a/net/sunrpc/svc.c
+++ b/net/sunrpc/svc.c
@@ -644,7 +644,7 @@ svc_rqst_alloc(struct svc_serv *serv, struct svc_pool *pool, int node)
 
 	folio_batch_init(&rqstp->rq_fbatch);
 
-	__set_bit(RQ_BUSY, &rqstp->rq_flags);
+	svc_thread_set_busy(rqstp);
 	rqstp->rq_server = serv;
 	rqstp->rq_pool = pool;
 
@@ -706,10 +706,13 @@ void svc_pool_wake_idle_thread(struct svc_serv *serv, struct svc_pool *pool)
 	struct svc_rqst	*rqstp;
 
 	rcu_read_lock();
-	list_for_each_entry_rcu(rqstp, &pool->sp_all_threads, rq_all) {
-		if (test_and_set_bit(RQ_BUSY, &rqstp->rq_flags))
-			continue;
-
+	spin_lock_bh(&pool->sp_lock);
+	rqstp = list_first_entry_or_null(&pool->sp_idle_threads,
+					 struct svc_rqst, rq_idle);
+	if (rqstp)
+		list_del_init(&rqstp->rq_idle);
+	spin_unlock_bh(&pool->sp_lock);
+	if (rqstp) {
 		WRITE_ONCE(rqstp->rq_qtime, ktime_get());
 		wake_up_process(rqstp->rq_task);
 		rcu_read_unlock();
diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
index 32469a8c5ba7..0ba16cbb998b 100644
--- a/net/sunrpc/svc_xprt.c
+++ b/net/sunrpc/svc_xprt.c
@@ -738,8 +738,9 @@ static bool svc_rqst_wait_for_work(struct svc_rqst *rqstp)
 		set_current_state(TASK_IDLE);
 		smp_mb__before_atomic();
 		clear_bit(SP_CONGESTED, &pool->sp_flags);
-		clear_bit(RQ_BUSY, &rqstp->rq_flags);
-		smp_mb__after_atomic();
+		spin_lock_bh(&pool->sp_lock);
+		list_add(&rqstp->rq_idle, &pool->sp_idle_threads);
+		spin_unlock_bh(&pool->sp_lock);
 
 		/* Need to test again after setting task state */
 		if (likely(rqst_should_sleep(rqstp))) {
@@ -749,8 +750,14 @@ static bool svc_rqst_wait_for_work(struct svc_rqst *rqstp)
 			__set_current_state(TASK_RUNNING);
 			cond_resched();
 		}
-		set_bit(RQ_BUSY, &rqstp->rq_flags);
-		smp_mb__after_atomic();
+		/* We *must* be removed from the list before we can continue.
+		 * If we were woken, this is already done
+		 */
+		if (!svc_thread_busy(rqstp)) {
+			spin_lock_bh(&pool->sp_lock);
+			list_del_init(&rqstp->rq_idle);
+			spin_unlock_bh(&pool->sp_lock);
+		}
 	}
 	try_to_freeze();
 	return slept;
-- 
2.40.1


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

* [PATCH 12/12] SUNRPC: discard SP_CONGESTED
  2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
                   ` (10 preceding siblings ...)
  2023-07-31  6:48 ` [PATCH 11/12] SUNRPC: add list of idle threads NeilBrown
@ 2023-07-31  6:48 ` NeilBrown
  11 siblings, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-07-31  6:48 UTC (permalink / raw)
  To: Chuck Lever, Jeff Layton; +Cc: linux-nfs

We can tell if a pool is congested by checking if the idle list is
empty.  We don't need a separate flag.

Signed-off-by: NeilBrown <neilb@suse.de>
---
 include/linux/sunrpc/svc.h    | 1 -
 include/trace/events/sunrpc.h | 2 --
 net/sunrpc/svc.c              | 1 -
 net/sunrpc/svc_xprt.c         | 4 +---
 4 files changed, 1 insertion(+), 7 deletions(-)

diff --git a/include/linux/sunrpc/svc.h b/include/linux/sunrpc/svc.h
index c2111bc8a7a1..b100ca16a25f 100644
--- a/include/linux/sunrpc/svc.h
+++ b/include/linux/sunrpc/svc.h
@@ -51,7 +51,6 @@ struct svc_pool {
 /* bits for sp_flags */
 enum {
 	SP_TASK_PENDING,	/* still work to do even if no xprt is queued */
-	SP_CONGESTED,		/* all threads are busy, none idle */
 	SP_NEED_VICTIM,		/* One thread needs to agree to exit */
 	SP_VICTIM_REMAINS,	/* One thread needs to actually exit */
 };
diff --git a/include/trace/events/sunrpc.h b/include/trace/events/sunrpc.h
index d00a1a6b9616..6101c1e38eb0 100644
--- a/include/trace/events/sunrpc.h
+++ b/include/trace/events/sunrpc.h
@@ -2058,14 +2058,12 @@ TRACE_EVENT(svc_xprt_enqueue,
 );
 
 TRACE_DEFINE_ENUM(SP_TASK_PENDING);
-TRACE_DEFINE_ENUM(SP_CONGESTED);
 TRACE_DEFINE_ENUM(SP_NEED_VICTIM);
 TRACE_DEFINE_ENUM(SP_VICTIM_REMAINS);
 
 #define show_svc_pool_flags(x)						\
 	__print_flags(x, "|",						\
 		{ BIT(SP_TASK_PENDING),		"TASK_PENDING" },	\
-		{ BIT(SP_CONGESTED),		"CONGESTED" },		\
 		{ BIT(SP_NEED_VICTIM),		"NEED_VICTIM" },	\
 		{ BIT(SP_VICTIM_REMAINS),	"VICTIM_REMAINS" })
 DECLARE_EVENT_CLASS(svc_pool_scheduler_class,
diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
index 44a614d96d8d..9102cbd3976c 100644
--- a/net/sunrpc/svc.c
+++ b/net/sunrpc/svc.c
@@ -724,7 +724,6 @@ void svc_pool_wake_idle_thread(struct svc_serv *serv, struct svc_pool *pool)
 
 	trace_svc_pool_starved(serv, pool);
 	percpu_counter_inc(&pool->sp_threads_starved);
-	set_bit(SP_CONGESTED, &pool->sp_flags);
 }
 EXPORT_SYMBOL_GPL(svc_pool_wake_idle_thread);
 
diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
index 0ba16cbb998b..f294523595fa 100644
--- a/net/sunrpc/svc_xprt.c
+++ b/net/sunrpc/svc_xprt.c
@@ -736,8 +736,6 @@ static bool svc_rqst_wait_for_work(struct svc_rqst *rqstp)
 
 	if (rqst_should_sleep(rqstp)) {
 		set_current_state(TASK_IDLE);
-		smp_mb__before_atomic();
-		clear_bit(SP_CONGESTED, &pool->sp_flags);
 		spin_lock_bh(&pool->sp_lock);
 		list_add(&rqstp->rq_idle, &pool->sp_idle_threads);
 		spin_unlock_bh(&pool->sp_lock);
@@ -877,7 +875,7 @@ void svc_recv(struct svc_rqst *rqstp)
 		/* Normally we will wait up to 5 seconds for any required
 		 * cache information to be provided.
 		 */
-		if (test_bit(SP_CONGESTED, &pool->sp_flags))
+		if (list_empty(&pool->sp_idle_threads))
 			rqstp->rq_chandle.thread_wait = 5 * HZ;
 		else
 			rqstp->rq_chandle.thread_wait = 1 * HZ;
-- 
2.40.1


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

* Re: [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent()
  2023-07-31  6:48 ` [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent() NeilBrown
@ 2023-07-31 14:21   ` Chuck Lever
  2023-07-31 22:05     ` NeilBrown
  2023-07-31 14:33   ` Jeff Layton
  1 sibling, 1 reply; 22+ messages in thread
From: Chuck Lever @ 2023-07-31 14:21 UTC (permalink / raw)
  To: NeilBrown; +Cc: Jeff Layton, linux-nfs

On Mon, Jul 31, 2023 at 04:48:28PM +1000, NeilBrown wrote:
> Based on its name you would think that rqst_should_sleep() would be
> read-only, not changing anything.  But it fact it will clear
> SP_TASK_PENDING if that was set.  This is surprising, and it blurs the
> line between "check for work to do" and "dequeue work to do".

I agree that rqst_should_sleep() sounds like it should be a
predicate without side effects.


> So change the "test_and_clear" to simple "test" and clear the bit once
> the thread has decided to wake up and return to the caller.
> 
> With this, it makes sense to *always* set SP_TASK_PENDING when asked,
> rather than only to set it if no thread could be woken up.

I'm lost here. Why does always setting TASK_PENDING now make sense?
If there's no task pending, won't this trigger a wake up when there
is nothing to do?


> Signed-off-by: NeilBrown <neilb@suse.de>
> ---
>  net/sunrpc/svc_xprt.c | 8 +++++---
>  1 file changed, 5 insertions(+), 3 deletions(-)
> 
> diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
> index cd92cb54132d..380fb3caea4c 100644
> --- a/net/sunrpc/svc_xprt.c
> +++ b/net/sunrpc/svc_xprt.c
> @@ -581,8 +581,8 @@ void svc_wake_up(struct svc_serv *serv)
>  {
>  	struct svc_pool *pool = &serv->sv_pools[0];
>  
> -	if (!svc_pool_wake_idle_thread(serv, pool))
> -		set_bit(SP_TASK_PENDING, &pool->sp_flags);
> +	set_bit(SP_TASK_PENDING, &pool->sp_flags);
> +	svc_pool_wake_idle_thread(serv, pool);
>  }
>  EXPORT_SYMBOL_GPL(svc_wake_up);
>  
> @@ -704,7 +704,7 @@ rqst_should_sleep(struct svc_rqst *rqstp)
>  	struct svc_pool		*pool = rqstp->rq_pool;
>  
>  	/* did someone call svc_wake_up? */
> -	if (test_and_clear_bit(SP_TASK_PENDING, &pool->sp_flags))
> +	if (test_bit(SP_TASK_PENDING, &pool->sp_flags))
>  		return false;
>  
>  	/* was a socket queued? */
> @@ -750,6 +750,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
>  
>  	set_bit(RQ_BUSY, &rqstp->rq_flags);
>  	smp_mb__after_atomic();
> +	clear_bit(SP_TASK_PENDING, &pool->sp_flags);

Why wouldn't this go before the smp_mb__after_atomic()?


>  	rqstp->rq_xprt = svc_xprt_dequeue(pool);
>  	if (rqstp->rq_xprt) {
>  		trace_svc_pool_awoken(rqstp);
> @@ -761,6 +762,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
>  	percpu_counter_inc(&pool->sp_threads_no_work);
>  	return NULL;
>  out_found:
> +	clear_bit(SP_TASK_PENDING, &pool->sp_flags);

clear_bit_unlock ?

>  	/* Normally we will wait up to 5 seconds for any required
>  	 * cache information to be provided.
>  	 */
> -- 
> 2.40.1
> 

-- 
Chuck Lever

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

* Re: [PATCH 03/12] FIXUP: SUNRPC: call svc_process() from svc_recv()
  2023-07-31  6:48 ` [PATCH 03/12] FIXUP: SUNRPC: call svc_process() from svc_recv() NeilBrown
@ 2023-07-31 14:22   ` Chuck Lever
  0 siblings, 0 replies; 22+ messages in thread
From: Chuck Lever @ 2023-07-31 14:22 UTC (permalink / raw)
  To: NeilBrown; +Cc: Jeff Layton, linux-nfs

On Mon, Jul 31, 2023 at 04:48:30PM +1000, NeilBrown wrote:
> Now that svc_process() is called only by svc_recv(), it doesn't need to
> be exported.
> 
> Signed-off-by: NeilBrown <neilb@suse.de>

Squashed.


> ---
>  net/sunrpc/svc.c | 1 -
>  1 file changed, 1 deletion(-)
> 
> diff --git a/net/sunrpc/svc.c b/net/sunrpc/svc.c
> index cbfd4ac02a4d..f2971d94b4aa 100644
> --- a/net/sunrpc/svc.c
> +++ b/net/sunrpc/svc.c
> @@ -1546,7 +1546,6 @@ void svc_process(struct svc_rqst *rqstp)
>  out_drop:
>  	svc_drop(rqstp);
>  }
> -EXPORT_SYMBOL_GPL(svc_process);
>  
>  #if defined(CONFIG_SUNRPC_BACKCHANNEL)
>  /*
> -- 
> 2.40.1
> 

-- 
Chuck Lever

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

* Re: [PATCH 05/12] nfsd: separate nfsd_last_thread() from nfsd_put()
  2023-07-31  6:48 ` [PATCH 05/12] nfsd: separate nfsd_last_thread() from nfsd_put() NeilBrown
@ 2023-07-31 14:23   ` Chuck Lever
  0 siblings, 0 replies; 22+ messages in thread
From: Chuck Lever @ 2023-07-31 14:23 UTC (permalink / raw)
  To: NeilBrown; +Cc: Jeff Layton, linux-nfs

On Mon, Jul 31, 2023 at 04:48:32PM +1000, NeilBrown wrote:
> Now that the last nfsd thread is stopped by an explicit act of calling
> svc_set_num_threads() with a count of zero, we only have a limited
> number of places that can happen, and don't need to call
> nfsd_last_thread() in nfsd_put()
> 
> So separate that out and call it at the two places where the number of
> threads is set to zero.
> 
> Move the clearing of ->nfsd_serv and the call to svc_xprt_destroy_all()
> into nfsd_last_thread(), as they are really part of the same action.
> 
> nfsd_put() is now a thin wrapper around svc_put(), so make it a static inline.
> 
> nfsd_put() cannot be called after nfsd_last_thread(), so in a couple of
> places we have to use svc_put() instead.
> 
> Signed-off-by: NeilBrown <neilb@suse.de>

I've applied 4/12 and 5/12.

I'll push these out later today.


> ---
>  fs/nfsd/nfsd.h   |  7 ++++++-
>  fs/nfsd/nfssvc.c | 52 ++++++++++++++++++------------------------------
>  2 files changed, 25 insertions(+), 34 deletions(-)
> 
> diff --git a/fs/nfsd/nfsd.h b/fs/nfsd/nfsd.h
> index d88498f8b275..11c14faa6c67 100644
> --- a/fs/nfsd/nfsd.h
> +++ b/fs/nfsd/nfsd.h
> @@ -96,7 +96,12 @@ int		nfsd_pool_stats_open(struct inode *, struct file *);
>  int		nfsd_pool_stats_release(struct inode *, struct file *);
>  void		nfsd_shutdown_threads(struct net *net);
>  
> -void		nfsd_put(struct net *net);
> +static inline void nfsd_put(struct net *net)
> +{
> +	struct nfsd_net *nn = net_generic(net, nfsd_net_id);
> +
> +	svc_put(nn->nfsd_serv);
> +}
>  
>  bool		i_am_nfsd(void);
>  
> diff --git a/fs/nfsd/nfssvc.c b/fs/nfsd/nfssvc.c
> index 33a80725e14e..1582af33e204 100644
> --- a/fs/nfsd/nfssvc.c
> +++ b/fs/nfsd/nfssvc.c
> @@ -542,9 +542,14 @@ static struct notifier_block nfsd_inet6addr_notifier = {
>  /* Only used under nfsd_mutex, so this atomic may be overkill: */
>  static atomic_t nfsd_notifier_refcount = ATOMIC_INIT(0);
>  
> -static void nfsd_last_thread(struct svc_serv *serv, struct net *net)
> +static void nfsd_last_thread(struct net *net)
>  {
>  	struct nfsd_net *nn = net_generic(net, nfsd_net_id);
> +	struct svc_serv *serv = nn->nfsd_serv;
> +
> +	spin_lock(&nfsd_notifier_lock);
> +	nn->nfsd_serv = NULL;
> +	spin_unlock(&nfsd_notifier_lock);
>  
>  	/* check if the notifier still has clients */
>  	if (atomic_dec_return(&nfsd_notifier_refcount) == 0) {
> @@ -554,6 +559,8 @@ static void nfsd_last_thread(struct svc_serv *serv, struct net *net)
>  #endif
>  	}
>  
> +	svc_xprt_destroy_all(serv, net);
> +
>  	/*
>  	 * write_ports can create the server without actually starting
>  	 * any threads--if we get shut down before any threads are
> @@ -644,7 +651,8 @@ void nfsd_shutdown_threads(struct net *net)
>  	svc_get(serv);
>  	/* Kill outstanding nfsd threads */
>  	svc_set_num_threads(serv, NULL, 0);
> -	nfsd_put(net);
> +	nfsd_last_thread(net);
> +	svc_put(serv);
>  	mutex_unlock(&nfsd_mutex);
>  }
>  
> @@ -674,9 +682,6 @@ int nfsd_create_serv(struct net *net)
>  	serv->sv_maxconn = nn->max_connections;
>  	error = svc_bind(serv, net);
>  	if (error < 0) {
> -		/* NOT nfsd_put() as notifiers (see below) haven't
> -		 * been set up yet.
> -		 */
>  		svc_put(serv);
>  		return error;
>  	}
> @@ -719,29 +724,6 @@ int nfsd_get_nrthreads(int n, int *nthreads, struct net *net)
>  	return 0;
>  }
>  
> -/* This is the callback for kref_put() below.
> - * There is no code here as the first thing to be done is
> - * call svc_shutdown_net(), but we cannot get the 'net' from
> - * the kref.  So do all the work when kref_put returns true.
> - */
> -static void nfsd_noop(struct kref *ref)
> -{
> -}
> -
> -void nfsd_put(struct net *net)
> -{
> -	struct nfsd_net *nn = net_generic(net, nfsd_net_id);
> -
> -	if (kref_put(&nn->nfsd_serv->sv_refcnt, nfsd_noop)) {
> -		svc_xprt_destroy_all(nn->nfsd_serv, net);
> -		nfsd_last_thread(nn->nfsd_serv, net);
> -		svc_destroy(&nn->nfsd_serv->sv_refcnt);
> -		spin_lock(&nfsd_notifier_lock);
> -		nn->nfsd_serv = NULL;
> -		spin_unlock(&nfsd_notifier_lock);
> -	}
> -}
> -
>  int nfsd_set_nrthreads(int n, int *nthreads, struct net *net)
>  {
>  	int i = 0;
> @@ -792,7 +774,7 @@ int nfsd_set_nrthreads(int n, int *nthreads, struct net *net)
>  		if (err)
>  			break;
>  	}
> -	nfsd_put(net);
> +	svc_put(nn->nfsd_serv);
>  	return err;
>  }
>  
> @@ -807,6 +789,7 @@ nfsd_svc(int nrservs, struct net *net, const struct cred *cred)
>  	int	error;
>  	bool	nfsd_up_before;
>  	struct nfsd_net *nn = net_generic(net, nfsd_net_id);
> +	struct svc_serv *serv;
>  
>  	mutex_lock(&nfsd_mutex);
>  	dprintk("nfsd: creating service\n");
> @@ -826,22 +809,25 @@ nfsd_svc(int nrservs, struct net *net, const struct cred *cred)
>  		goto out;
>  
>  	nfsd_up_before = nn->nfsd_net_up;
> +	serv = nn->nfsd_serv;
>  
>  	error = nfsd_startup_net(net, cred);
>  	if (error)
>  		goto out_put;
> -	error = svc_set_num_threads(nn->nfsd_serv, NULL, nrservs);
> +	error = svc_set_num_threads(serv, NULL, nrservs);
>  	if (error)
>  		goto out_shutdown;
> -	error = nn->nfsd_serv->sv_nrthreads;
> +	error = serv->sv_nrthreads;
> +	if (error == 0)
> +		nfsd_last_thread(net);
>  out_shutdown:
>  	if (error < 0 && !nfsd_up_before)
>  		nfsd_shutdown_net(net);
>  out_put:
>  	/* Threads now hold service active */
>  	if (xchg(&nn->keep_active, 0))
> -		nfsd_put(net);
> -	nfsd_put(net);
> +		svc_put(serv);
> +	svc_put(serv);
>  out:
>  	mutex_unlock(&nfsd_mutex);
>  	return error;
> -- 
> 2.40.1
> 

-- 
Chuck Lever

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

* Re: [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent()
  2023-07-31  6:48 ` [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent() NeilBrown
  2023-07-31 14:21   ` Chuck Lever
@ 2023-07-31 14:33   ` Jeff Layton
  1 sibling, 0 replies; 22+ messages in thread
From: Jeff Layton @ 2023-07-31 14:33 UTC (permalink / raw)
  To: NeilBrown, Chuck Lever; +Cc: linux-nfs

On Mon, 2023-07-31 at 16:48 +1000, NeilBrown wrote:
> Based on its name you would think that rqst_should_sleep() would be
> read-only, not changing anything.  But it fact it will clear
> SP_TASK_PENDING if that was set.  This is surprising, and it blurs the
> line between "check for work to do" and "dequeue work to do".
> 
> So change the "test_and_clear" to simple "test" and clear the bit once
> the thread has decided to wake up and return to the caller.
> 
> With this, it makes sense to *always* set SP_TASK_PENDING when asked,
> rather than only to set it if no thread could be woken up.
> 
> Signed-off-by: NeilBrown <neilb@suse.de>
> ---
>  net/sunrpc/svc_xprt.c | 8 +++++---
>  1 file changed, 5 insertions(+), 3 deletions(-)
> 
> diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
> index cd92cb54132d..380fb3caea4c 100644
> --- a/net/sunrpc/svc_xprt.c
> +++ b/net/sunrpc/svc_xprt.c
> @@ -581,8 +581,8 @@ void svc_wake_up(struct svc_serv *serv)
>  {
>  	struct svc_pool *pool = &serv->sv_pools[0];
>  
> -	if (!svc_pool_wake_idle_thread(serv, pool))
> -		set_bit(SP_TASK_PENDING, &pool->sp_flags);
> +	set_bit(SP_TASK_PENDING, &pool->sp_flags);
> +	svc_pool_wake_idle_thread(serv, pool);
>  }
>  EXPORT_SYMBOL_GPL(svc_wake_up);
>  
> @@ -704,7 +704,7 @@ rqst_should_sleep(struct svc_rqst *rqstp)
>  	struct svc_pool		*pool = rqstp->rq_pool;
>  
>  	/* did someone call svc_wake_up? */
> -	if (test_and_clear_bit(SP_TASK_PENDING, &pool->sp_flags))
> +	if (test_bit(SP_TASK_PENDING, &pool->sp_flags))
>  		return false;
>  
>  	/* was a socket queued? */
> @@ -750,6 +750,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
>  
>  	set_bit(RQ_BUSY, &rqstp->rq_flags);
>  	smp_mb__after_atomic();
> +	clear_bit(SP_TASK_PENDING, &pool->sp_flags);

Took me a few mins to decide that splitting up the test_and_clear_bit
didn't open a ToC/ToU race. I think we're saved by the fact that only
nfsd thread itself clears the bit, so we're guaranteed not to race with
another clear (whew).
 
>  	rqstp->rq_xprt = svc_xprt_dequeue(pool);
>  	if (rqstp->rq_xprt) {
>  		trace_svc_pool_awoken(rqstp);
> @@ -761,6 +762,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
>  	percpu_counter_inc(&pool->sp_threads_no_work);
>  	return NULL;
>  out_found:
> +	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
>  	/* Normally we will wait up to 5 seconds for any required
>  	 * cache information to be provided.
>  	 */

Reviewed-by: Jeff Layton <jlayton@kernel.org>

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

* Re: [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent()
  2023-07-31 14:21   ` Chuck Lever
@ 2023-07-31 22:05     ` NeilBrown
  2023-07-31 22:31       ` Chuck Lever III
  0 siblings, 1 reply; 22+ messages in thread
From: NeilBrown @ 2023-07-31 22:05 UTC (permalink / raw)
  To: Chuck Lever; +Cc: Jeff Layton, linux-nfs

On Tue, 01 Aug 2023, Chuck Lever wrote:
> On Mon, Jul 31, 2023 at 04:48:28PM +1000, NeilBrown wrote:
> > Based on its name you would think that rqst_should_sleep() would be
> > read-only, not changing anything.  But it fact it will clear
> > SP_TASK_PENDING if that was set.  This is surprising, and it blurs the
> > line between "check for work to do" and "dequeue work to do".
> 
> I agree that rqst_should_sleep() sounds like it should be a
> predicate without side effects.
> 
> 
> > So change the "test_and_clear" to simple "test" and clear the bit once
> > the thread has decided to wake up and return to the caller.
> > 
> > With this, it makes sense to *always* set SP_TASK_PENDING when asked,
> > rather than only to set it if no thread could be woken up.
> 
> I'm lost here. Why does always setting TASK_PENDING now make sense?
> If there's no task pending, won't this trigger a wake up when there
> is nothing to do?

Clearly Jedi mind tricks don't work on you...  I'll have to try logic
instead.

 This separation of "test" and "clear" is a first step in re-organising
 the queueing of tasks around a clear pattern of "client queues a task",
 "service checks if any tasks are queued" and "service dequeues and
 performs a task".  The first step for TASK_PENDING doesn't quite follow
 a clear pattern as the flag is only set (the work is only queued) if
 no thread could be immediately woken.  This imposes on the
 implementation of the service.  For example, whenever a service is
 woken it *must* return to the caller of svc_recv(), just in case it was
 woken by svc_wake_up().  It cannot test if there is work to do, and if
 not - go back to sleep.  It provides a cleaner implementation of the
 pattern to *always* queue the work.  i.e. *always* set the flag.  Which
 ever thread first sees and clears the flag will return to caller and
 perform the required work.  If the woken thread doesn't find anything
 to do, it could go back to sleep (though currently it doesn't).

If that more convincing?

Thanks,
NeilBrown


> 
> 
> > Signed-off-by: NeilBrown <neilb@suse.de>
> > ---
> >  net/sunrpc/svc_xprt.c | 8 +++++---
> >  1 file changed, 5 insertions(+), 3 deletions(-)
> > 
> > diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
> > index cd92cb54132d..380fb3caea4c 100644
> > --- a/net/sunrpc/svc_xprt.c
> > +++ b/net/sunrpc/svc_xprt.c
> > @@ -581,8 +581,8 @@ void svc_wake_up(struct svc_serv *serv)
> >  {
> >  	struct svc_pool *pool = &serv->sv_pools[0];
> >  
> > -	if (!svc_pool_wake_idle_thread(serv, pool))
> > -		set_bit(SP_TASK_PENDING, &pool->sp_flags);
> > +	set_bit(SP_TASK_PENDING, &pool->sp_flags);
> > +	svc_pool_wake_idle_thread(serv, pool);
> >  }
> >  EXPORT_SYMBOL_GPL(svc_wake_up);
> >  
> > @@ -704,7 +704,7 @@ rqst_should_sleep(struct svc_rqst *rqstp)
> >  	struct svc_pool		*pool = rqstp->rq_pool;
> >  
> >  	/* did someone call svc_wake_up? */
> > -	if (test_and_clear_bit(SP_TASK_PENDING, &pool->sp_flags))
> > +	if (test_bit(SP_TASK_PENDING, &pool->sp_flags))
> >  		return false;
> >  
> >  	/* was a socket queued? */
> > @@ -750,6 +750,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
> >  
> >  	set_bit(RQ_BUSY, &rqstp->rq_flags);
> >  	smp_mb__after_atomic();
> > +	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
> 
> Why wouldn't this go before the smp_mb__after_atomic()?
> 
> 
> >  	rqstp->rq_xprt = svc_xprt_dequeue(pool);
> >  	if (rqstp->rq_xprt) {
> >  		trace_svc_pool_awoken(rqstp);
> > @@ -761,6 +762,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
> >  	percpu_counter_inc(&pool->sp_threads_no_work);
> >  	return NULL;
> >  out_found:
> > +	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
> 
> clear_bit_unlock ?
> 
> >  	/* Normally we will wait up to 5 seconds for any required
> >  	 * cache information to be provided.
> >  	 */
> > -- 
> > 2.40.1
> > 
> 
> -- 
> Chuck Lever
> 


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

* Re: [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent()
  2023-07-31 22:05     ` NeilBrown
@ 2023-07-31 22:31       ` Chuck Lever III
  0 siblings, 0 replies; 22+ messages in thread
From: Chuck Lever III @ 2023-07-31 22:31 UTC (permalink / raw)
  To: Neil Brown; +Cc: Jeff Layton, Linux NFS Mailing List



> On Jul 31, 2023, at 6:05 PM, NeilBrown <neilb@suse.de> wrote:
> 
> On Tue, 01 Aug 2023, Chuck Lever wrote:
>> On Mon, Jul 31, 2023 at 04:48:28PM +1000, NeilBrown wrote:
>>> Based on its name you would think that rqst_should_sleep() would be
>>> read-only, not changing anything.  But it fact it will clear
>>> SP_TASK_PENDING if that was set.  This is surprising, and it blurs the
>>> line between "check for work to do" and "dequeue work to do".
>> 
>> I agree that rqst_should_sleep() sounds like it should be a
>> predicate without side effects.
>> 
>> 
>>> So change the "test_and_clear" to simple "test" and clear the bit once
>>> the thread has decided to wake up and return to the caller.
>>> 
>>> With this, it makes sense to *always* set SP_TASK_PENDING when asked,
>>> rather than only to set it if no thread could be woken up.
>> 
>> I'm lost here. Why does always setting TASK_PENDING now make sense?
>> If there's no task pending, won't this trigger a wake up when there
>> is nothing to do?
> 
> Clearly Jedi mind tricks don't work on you...  I'll have to try logic
> instead.
> 
> This separation of "test" and "clear" is a first step in re-organising
> the queueing of tasks around a clear pattern of "client queues a task",
> "service checks if any tasks are queued" and "service dequeues and
> performs a task".  The first step for TASK_PENDING doesn't quite follow
> a clear pattern as the flag is only set (the work is only queued) if
> no thread could be immediately woken.  This imposes on the
> implementation of the service.  For example, whenever a service is
> woken it *must* return to the caller of svc_recv(), just in case it was
> woken by svc_wake_up().  It cannot test if there is work to do, and if
> not - go back to sleep.  It provides a cleaner implementation of the
> pattern to *always* queue the work.  i.e. *always* set the flag.  Which
> ever thread first sees and clears the flag will return to caller and
> perform the required work.  If the woken thread doesn't find anything
> to do, it could go back to sleep (though currently it doesn't).
> 
> If that more convincing?

Well, that makes sense if TASK_PENDING means the same as XPT_BUSY
except it marks the whole pool busy instead of only a particular
transport.

We're always setting XPT_BUSY this way, for instance -- if there's
work to do, whether or not a wake-up was successful, this bit is
set.

The meaning of TASK_PENDING before this patch was "the wake-up
didn't succeed, so make sure to go back and look for the work".
Your patch changes that.

So, perhaps the patch description needs to document that
particular change in semantics rather than describing only the
code changes. Or we could rename the bit SP_BUSY or something a
little more self-explanatory.

(A more crazy idea would be to add a phony transport to each
pool that acts as the locus for these transportless tasks).

I do not object to the patch. It's just hard to square up the
current description with the patch itself. Speaking, of course,
as someone who will no doubt have to read this description
in a year or two and ask "what were we thinking?"


> Thanks,
> NeilBrown
> 
> 
>> 
>> 
>>> Signed-off-by: NeilBrown <neilb@suse.de>
>>> ---
>>> net/sunrpc/svc_xprt.c | 8 +++++---
>>> 1 file changed, 5 insertions(+), 3 deletions(-)
>>> 
>>> diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
>>> index cd92cb54132d..380fb3caea4c 100644
>>> --- a/net/sunrpc/svc_xprt.c
>>> +++ b/net/sunrpc/svc_xprt.c
>>> @@ -581,8 +581,8 @@ void svc_wake_up(struct svc_serv *serv)
>>> {
>>> struct svc_pool *pool = &serv->sv_pools[0];
>>> 
>>> - if (!svc_pool_wake_idle_thread(serv, pool))
>>> - set_bit(SP_TASK_PENDING, &pool->sp_flags);
>>> + set_bit(SP_TASK_PENDING, &pool->sp_flags);
>>> + svc_pool_wake_idle_thread(serv, pool);
>>> }
>>> EXPORT_SYMBOL_GPL(svc_wake_up);
>>> 
>>> @@ -704,7 +704,7 @@ rqst_should_sleep(struct svc_rqst *rqstp)
>>> struct svc_pool *pool = rqstp->rq_pool;
>>> 
>>> /* did someone call svc_wake_up? */
>>> - if (test_and_clear_bit(SP_TASK_PENDING, &pool->sp_flags))
>>> + if (test_bit(SP_TASK_PENDING, &pool->sp_flags))
>>> return false;
>>> 
>>> /* was a socket queued? */
>>> @@ -750,6 +750,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
>>> 
>>> set_bit(RQ_BUSY, &rqstp->rq_flags);
>>> smp_mb__after_atomic();
>>> + clear_bit(SP_TASK_PENDING, &pool->sp_flags);
>> 
>> Why wouldn't this go before the smp_mb__after_atomic()?
>> 
>> 
>>> rqstp->rq_xprt = svc_xprt_dequeue(pool);
>>> if (rqstp->rq_xprt) {
>>> trace_svc_pool_awoken(rqstp);
>>> @@ -761,6 +762,7 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
>>> percpu_counter_inc(&pool->sp_threads_no_work);
>>> return NULL;
>>> out_found:
>>> + clear_bit(SP_TASK_PENDING, &pool->sp_flags);
>> 
>> clear_bit_unlock ?
>> 
>>> /* Normally we will wait up to 5 seconds for any required
>>>  * cache information to be provided.
>>>  */
>>> -- 
>>> 2.40.1
>>> 
>> 
>> -- 
>> Chuck Lever


--
Chuck Lever



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

* Re: [PATCH 06/12] SUNRPC: rename and refactor svc_get_next_xprt().
  2023-07-31  6:48 ` [PATCH 06/12] SUNRPC: rename and refactor svc_get_next_xprt() NeilBrown
@ 2023-07-31 23:16   ` Chuck Lever
  2023-08-01 22:46     ` Chuck Lever
  2023-08-02  5:00     ` NeilBrown
  0 siblings, 2 replies; 22+ messages in thread
From: Chuck Lever @ 2023-07-31 23:16 UTC (permalink / raw)
  To: NeilBrown; +Cc: Jeff Layton, linux-nfs

On Mon, Jul 31, 2023 at 04:48:33PM +1000, NeilBrown wrote:
> svc_get_next_xprt() does a lot more than just get an xprt.  It also
> decides if it needs to sleep, depending not only on the availability of
> xprts, but also on the need to exit or handle external work
> (SP_TASK_PENDING).
> 
> So rename it to svc_rqst_wait_and_dequeue_work(), don't return the xprt
> (which can easily be found in rqstp->rq_xprt), and restructure to make a
> clear separation between waiting and dequeueing.

For me, the most valuable part of this patch is the last part here:
refactoring the dequeue and the wait, and deduplicating the dequeue.


> All the scheduling-related code like try_to_freeze() and
> kthread_should_stop() is moved into svc_rqst_wait_and_dequeue_work().
> 
> Rather than calling svc_xprt_dequeue() twice (before and after deciding
> to wait), it now calls rqst_should_sleep() twice.  If the first fails,
> we skip all the waiting code completely.  In the waiting code we call
> again after setting the task state in case we missed a wake-up.
> 
> We now only have one call to try_to_freeze() and one call to
> svc_xprt_dequeue().  We still have two calls to kthread_should_stop() -
> one in rqst_should_sleep() to avoid sleeping, and one afterwards to
> avoid dequeueing any work (it previously came after dequeueing which
> doesn't seem right).
> 
> Signed-off-by: NeilBrown <neilb@suse.de>
> ---
>  net/sunrpc/svc_xprt.c | 62 +++++++++++++++++++++----------------------
>  1 file changed, 31 insertions(+), 31 deletions(-)
> 
> diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
> index 380fb3caea4c..67f2b34cb8e4 100644
> --- a/net/sunrpc/svc_xprt.c
> +++ b/net/sunrpc/svc_xprt.c
> @@ -722,47 +722,51 @@ rqst_should_sleep(struct svc_rqst *rqstp)
>  	return true;
>  }
>  
> -static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
> +static void svc_rqst_wait_and_dequeue_work(struct svc_rqst *rqstp)

It would be simpler to follow if you renamed this function once
(here), and changed directly from returning struct svc_xprt to
returning bool.


>  {
>  	struct svc_pool		*pool = rqstp->rq_pool;
> +	bool slept = false;
>  
>  	/* rq_xprt should be clear on entry */
>  	WARN_ON_ONCE(rqstp->rq_xprt);
>  
> -	rqstp->rq_xprt = svc_xprt_dequeue(pool);
> -	if (rqstp->rq_xprt) {
> -		trace_svc_pool_polled(rqstp);
> -		goto out_found;
> +	if (rqst_should_sleep(rqstp)) {
> +		set_current_state(TASK_IDLE);
> +		smp_mb__before_atomic();
> +		clear_bit(SP_CONGESTED, &pool->sp_flags);
> +		clear_bit(RQ_BUSY, &rqstp->rq_flags);
> +		smp_mb__after_atomic();
> +
> +		/* Need to test again after setting task state */

This comment isn't illuminating. It needs to explain the "need to
test again".


> +		if (likely(rqst_should_sleep(rqstp))) {

Is likely() still needed here?


> +			schedule();
> +			slept = true;
> +		} else {
> +			__set_current_state(TASK_RUNNING);
> +			cond_resched();

This makes me happy. Only call cond_resched() if we didn't sleep.


> +		}
> +		set_bit(RQ_BUSY, &rqstp->rq_flags);
> +		smp_mb__after_atomic();
>  	}
> -
> -	set_current_state(TASK_IDLE);
> -	smp_mb__before_atomic();
> -	clear_bit(SP_CONGESTED, &pool->sp_flags);
> -	clear_bit(RQ_BUSY, &rqstp->rq_flags);
> -	smp_mb__after_atomic();
> -
> -	if (likely(rqst_should_sleep(rqstp)))
> -		schedule();
> -	else
> -		__set_current_state(TASK_RUNNING);
> -
>  	try_to_freeze();
>  
> -	set_bit(RQ_BUSY, &rqstp->rq_flags);
> -	smp_mb__after_atomic();
> +	if (kthread_should_stop())
> +		return;
> +
>  	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
>  	rqstp->rq_xprt = svc_xprt_dequeue(pool);
>  	if (rqstp->rq_xprt) {
> -		trace_svc_pool_awoken(rqstp);
> +		if (slept)
> +			trace_svc_pool_awoken(rqstp);
> +		else
> +			trace_svc_pool_polled(rqstp);

Again, it would perhaps be better if we rearranged this code first,
and then added tracepoints. This is ... well, ugly.


>  		goto out_found;
>  	}
>  
> -	if (kthread_should_stop())
> -		return NULL;
> -	percpu_counter_inc(&pool->sp_threads_no_work);
> -	return NULL;
> +	if (slept)
> +		percpu_counter_inc(&pool->sp_threads_no_work);
> +	return;
>  out_found:
> -	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
>  	/* Normally we will wait up to 5 seconds for any required
>  	 * cache information to be provided.
>  	 */
> @@ -770,7 +774,6 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
>  		rqstp->rq_chandle.thread_wait = 5*HZ;
>  	else
>  		rqstp->rq_chandle.thread_wait = 1*HZ;
> -	return rqstp->rq_xprt;
>  }
>  
>  static void svc_add_new_temp_xprt(struct svc_serv *serv, struct svc_xprt *newxpt)
> @@ -854,12 +857,9 @@ void svc_recv(struct svc_rqst *rqstp)
>  	if (!svc_alloc_arg(rqstp))
>  		goto out;
>  
> -	try_to_freeze();
> -	cond_resched();
> -	if (kthread_should_stop())
> -		goto out;
> +	svc_rqst_wait_and_dequeue_work(rqstp);
>  
> -	xprt = svc_get_next_xprt(rqstp);
> +	xprt = rqstp->rq_xprt;
>  	if (!xprt)
>  		goto out;
>  
> -- 
> 2.40.1
> 

-- 
Chuck Lever

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

* Re: [PATCH 06/12] SUNRPC: rename and refactor svc_get_next_xprt().
  2023-07-31 23:16   ` Chuck Lever
@ 2023-08-01 22:46     ` Chuck Lever
  2023-08-02  5:00     ` NeilBrown
  1 sibling, 0 replies; 22+ messages in thread
From: Chuck Lever @ 2023-08-01 22:46 UTC (permalink / raw)
  To: NeilBrown; +Cc: Jeff Layton, linux-nfs

On Mon, Jul 31, 2023 at 07:16:01PM -0400, Chuck Lever wrote:
> On Mon, Jul 31, 2023 at 04:48:33PM +1000, NeilBrown wrote:
> > svc_get_next_xprt() does a lot more than just get an xprt.  It also
> > decides if it needs to sleep, depending not only on the availability of
> > xprts, but also on the need to exit or handle external work
> > (SP_TASK_PENDING).
> > 
> > So rename it to svc_rqst_wait_and_dequeue_work(), don't return the xprt
> > (which can easily be found in rqstp->rq_xprt), and restructure to make a
> > clear separation between waiting and dequeueing.
> 
> For me, the most valuable part of this patch is the last part here:
> refactoring the dequeue and the wait, and deduplicating the dequeue.
> 
> 
> > All the scheduling-related code like try_to_freeze() and
> > kthread_should_stop() is moved into svc_rqst_wait_and_dequeue_work().
> > 
> > Rather than calling svc_xprt_dequeue() twice (before and after deciding
> > to wait), it now calls rqst_should_sleep() twice.  If the first fails,
> > we skip all the waiting code completely.  In the waiting code we call
> > again after setting the task state in case we missed a wake-up.
> > 
> > We now only have one call to try_to_freeze() and one call to
> > svc_xprt_dequeue().  We still have two calls to kthread_should_stop() -
> > one in rqst_should_sleep() to avoid sleeping, and one afterwards to
> > avoid dequeueing any work (it previously came after dequeueing which
> > doesn't seem right).
> > 
> > Signed-off-by: NeilBrown <neilb@suse.de>
> > ---
> >  net/sunrpc/svc_xprt.c | 62 +++++++++++++++++++++----------------------
> >  1 file changed, 31 insertions(+), 31 deletions(-)
> > 
> > diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
> > index 380fb3caea4c..67f2b34cb8e4 100644
> > --- a/net/sunrpc/svc_xprt.c
> > +++ b/net/sunrpc/svc_xprt.c
> > @@ -722,47 +722,51 @@ rqst_should_sleep(struct svc_rqst *rqstp)
> >  	return true;
> >  }
> >  
> > -static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
> > +static void svc_rqst_wait_and_dequeue_work(struct svc_rqst *rqstp)
> 
> It would be simpler to follow if you renamed this function once
> (here), and changed directly from returning struct svc_xprt to
> returning bool.
> 
> 
> >  {
> >  	struct svc_pool		*pool = rqstp->rq_pool;
> > +	bool slept = false;
> >  
> >  	/* rq_xprt should be clear on entry */
> >  	WARN_ON_ONCE(rqstp->rq_xprt);
> >  
> > -	rqstp->rq_xprt = svc_xprt_dequeue(pool);
> > -	if (rqstp->rq_xprt) {
> > -		trace_svc_pool_polled(rqstp);
> > -		goto out_found;
> > +	if (rqst_should_sleep(rqstp)) {
> > +		set_current_state(TASK_IDLE);
> > +		smp_mb__before_atomic();
> > +		clear_bit(SP_CONGESTED, &pool->sp_flags);
> > +		clear_bit(RQ_BUSY, &rqstp->rq_flags);
> > +		smp_mb__after_atomic();
> > +
> > +		/* Need to test again after setting task state */
> 
> This comment isn't illuminating. It needs to explain the "need to
> test again".
> 
> 
> > +		if (likely(rqst_should_sleep(rqstp))) {
> 
> Is likely() still needed here?
> 
> 
> > +			schedule();
> > +			slept = true;
> > +		} else {
> > +			__set_current_state(TASK_RUNNING);
> > +			cond_resched();
> 
> This makes me happy. Only call cond_resched() if we didn't sleep.
> 
> 
> > +		}
> > +		set_bit(RQ_BUSY, &rqstp->rq_flags);
> > +		smp_mb__after_atomic();
> >  	}
> > -
> > -	set_current_state(TASK_IDLE);
> > -	smp_mb__before_atomic();
> > -	clear_bit(SP_CONGESTED, &pool->sp_flags);
> > -	clear_bit(RQ_BUSY, &rqstp->rq_flags);
> > -	smp_mb__after_atomic();
> > -
> > -	if (likely(rqst_should_sleep(rqstp)))
> > -		schedule();
> > -	else
> > -		__set_current_state(TASK_RUNNING);
> > -
> >  	try_to_freeze();
> >  
> > -	set_bit(RQ_BUSY, &rqstp->rq_flags);
> > -	smp_mb__after_atomic();
> > +	if (kthread_should_stop())
> > +		return;
> > +
> >  	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
> >  	rqstp->rq_xprt = svc_xprt_dequeue(pool);
> >  	if (rqstp->rq_xprt) {
> > -		trace_svc_pool_awoken(rqstp);
> > +		if (slept)
> > +			trace_svc_pool_awoken(rqstp);
> > +		else
> > +			trace_svc_pool_polled(rqstp);
> 
> Again, it would perhaps be better if we rearranged this code first,
> and then added tracepoints. This is ... well, ugly.

I've dropped the three tracepoint patches and pushed out the changes
to topic-sunrpc-thread-scheduling . We can circle back to adding
tracepoints once this code has settled.


> >  		goto out_found;
> >  	}
> >  
> > -	if (kthread_should_stop())
> > -		return NULL;
> > -	percpu_counter_inc(&pool->sp_threads_no_work);
> > -	return NULL;
> > +	if (slept)
> > +		percpu_counter_inc(&pool->sp_threads_no_work);
> > +	return;
> >  out_found:
> > -	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
> >  	/* Normally we will wait up to 5 seconds for any required
> >  	 * cache information to be provided.
> >  	 */
> > @@ -770,7 +774,6 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
> >  		rqstp->rq_chandle.thread_wait = 5*HZ;
> >  	else
> >  		rqstp->rq_chandle.thread_wait = 1*HZ;
> > -	return rqstp->rq_xprt;
> >  }
> >  
> >  static void svc_add_new_temp_xprt(struct svc_serv *serv, struct svc_xprt *newxpt)
> > @@ -854,12 +857,9 @@ void svc_recv(struct svc_rqst *rqstp)
> >  	if (!svc_alloc_arg(rqstp))
> >  		goto out;
> >  
> > -	try_to_freeze();
> > -	cond_resched();
> > -	if (kthread_should_stop())
> > -		goto out;
> > +	svc_rqst_wait_and_dequeue_work(rqstp);
> >  
> > -	xprt = svc_get_next_xprt(rqstp);
> > +	xprt = rqstp->rq_xprt;
> >  	if (!xprt)
> >  		goto out;
> >  
> > -- 
> > 2.40.1
> > 
> 
> -- 
> Chuck Lever

-- 
Chuck Lever

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

* Re: [PATCH 06/12] SUNRPC: rename and refactor svc_get_next_xprt().
  2023-07-31 23:16   ` Chuck Lever
  2023-08-01 22:46     ` Chuck Lever
@ 2023-08-02  5:00     ` NeilBrown
  1 sibling, 0 replies; 22+ messages in thread
From: NeilBrown @ 2023-08-02  5:00 UTC (permalink / raw)
  To: Chuck Lever; +Cc: Jeff Layton, linux-nfs

On Tue, 01 Aug 2023, Chuck Lever wrote:
> On Mon, Jul 31, 2023 at 04:48:33PM +1000, NeilBrown wrote:
> > svc_get_next_xprt() does a lot more than just get an xprt.  It also
> > decides if it needs to sleep, depending not only on the availability of
> > xprts, but also on the need to exit or handle external work
> > (SP_TASK_PENDING).
> > 
> > So rename it to svc_rqst_wait_and_dequeue_work(), don't return the xprt
> > (which can easily be found in rqstp->rq_xprt), and restructure to make a
> > clear separation between waiting and dequeueing.
> 
> For me, the most valuable part of this patch is the last part here:
> refactoring the dequeue and the wait, and deduplicating the dequeue.
> 
> 
> > All the scheduling-related code like try_to_freeze() and
> > kthread_should_stop() is moved into svc_rqst_wait_and_dequeue_work().
> > 
> > Rather than calling svc_xprt_dequeue() twice (before and after deciding
> > to wait), it now calls rqst_should_sleep() twice.  If the first fails,
> > we skip all the waiting code completely.  In the waiting code we call
> > again after setting the task state in case we missed a wake-up.
> > 
> > We now only have one call to try_to_freeze() and one call to
> > svc_xprt_dequeue().  We still have two calls to kthread_should_stop() -
> > one in rqst_should_sleep() to avoid sleeping, and one afterwards to
> > avoid dequeueing any work (it previously came after dequeueing which
> > doesn't seem right).
> > 
> > Signed-off-by: NeilBrown <neilb@suse.de>
> > ---
> >  net/sunrpc/svc_xprt.c | 62 +++++++++++++++++++++----------------------
> >  1 file changed, 31 insertions(+), 31 deletions(-)
> > 
> > diff --git a/net/sunrpc/svc_xprt.c b/net/sunrpc/svc_xprt.c
> > index 380fb3caea4c..67f2b34cb8e4 100644
> > --- a/net/sunrpc/svc_xprt.c
> > +++ b/net/sunrpc/svc_xprt.c
> > @@ -722,47 +722,51 @@ rqst_should_sleep(struct svc_rqst *rqstp)
> >  	return true;
> >  }
> >  
> > -static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
> > +static void svc_rqst_wait_and_dequeue_work(struct svc_rqst *rqstp)
> 
> It would be simpler to follow if you renamed this function once
> (here), and changed directly from returning struct svc_xprt to
> returning bool.

It isn't clear to me why it would be simpler, or exactly what you are
suggesting.
Should U just squash
   SUNRPC: rename and refactor svc_get_next_xprt().
and
   SUNRPC: move task-dequeueing code into svc_recv()

together?  I can see that it would make sense to move
   SUNRPC: move all of xprt handling into svc_xprt_handle()
earlier.

> 
> 
> >  {
> >  	struct svc_pool		*pool = rqstp->rq_pool;
> > +	bool slept = false;
> >  
> >  	/* rq_xprt should be clear on entry */
> >  	WARN_ON_ONCE(rqstp->rq_xprt);
> >  
> > -	rqstp->rq_xprt = svc_xprt_dequeue(pool);
> > -	if (rqstp->rq_xprt) {
> > -		trace_svc_pool_polled(rqstp);
> > -		goto out_found;
> > +	if (rqst_should_sleep(rqstp)) {
> > +		set_current_state(TASK_IDLE);
> > +		smp_mb__before_atomic();
> > +		clear_bit(SP_CONGESTED, &pool->sp_flags);
> > +		clear_bit(RQ_BUSY, &rqstp->rq_flags);
> > +		smp_mb__after_atomic();
> > +
> > +		/* Need to test again after setting task state */
> 
> This comment isn't illuminating. It needs to explain the "need to
> test again".

"after setting task state" was meant to be the explanation, but I guess
more words wouldn't hurt.

> 
> 
> > +		if (likely(rqst_should_sleep(rqstp))) {
> 
> Is likely() still needed here?

It is ever needed?  Let's drop it.

Thanks,
NeilBrown


> 
> 
> > +			schedule();
> > +			slept = true;
> > +		} else {
> > +			__set_current_state(TASK_RUNNING);
> > +			cond_resched();
> 
> This makes me happy. Only call cond_resched() if we didn't sleep.
> 
> 
> > +		}
> > +		set_bit(RQ_BUSY, &rqstp->rq_flags);
> > +		smp_mb__after_atomic();
> >  	}
> > -
> > -	set_current_state(TASK_IDLE);
> > -	smp_mb__before_atomic();
> > -	clear_bit(SP_CONGESTED, &pool->sp_flags);
> > -	clear_bit(RQ_BUSY, &rqstp->rq_flags);
> > -	smp_mb__after_atomic();
> > -
> > -	if (likely(rqst_should_sleep(rqstp)))
> > -		schedule();
> > -	else
> > -		__set_current_state(TASK_RUNNING);
> > -
> >  	try_to_freeze();
> >  
> > -	set_bit(RQ_BUSY, &rqstp->rq_flags);
> > -	smp_mb__after_atomic();
> > +	if (kthread_should_stop())
> > +		return;
> > +
> >  	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
> >  	rqstp->rq_xprt = svc_xprt_dequeue(pool);
> >  	if (rqstp->rq_xprt) {
> > -		trace_svc_pool_awoken(rqstp);
> > +		if (slept)
> > +			trace_svc_pool_awoken(rqstp);
> > +		else
> > +			trace_svc_pool_polled(rqstp);
> 
> Again, it would perhaps be better if we rearranged this code first,
> and then added tracepoints. This is ... well, ugly.
> 
> 
> >  		goto out_found;
> >  	}
> >  
> > -	if (kthread_should_stop())
> > -		return NULL;
> > -	percpu_counter_inc(&pool->sp_threads_no_work);
> > -	return NULL;
> > +	if (slept)
> > +		percpu_counter_inc(&pool->sp_threads_no_work);
> > +	return;
> >  out_found:
> > -	clear_bit(SP_TASK_PENDING, &pool->sp_flags);
> >  	/* Normally we will wait up to 5 seconds for any required
> >  	 * cache information to be provided.
> >  	 */
> > @@ -770,7 +774,6 @@ static struct svc_xprt *svc_get_next_xprt(struct svc_rqst *rqstp)
> >  		rqstp->rq_chandle.thread_wait = 5*HZ;
> >  	else
> >  		rqstp->rq_chandle.thread_wait = 1*HZ;
> > -	return rqstp->rq_xprt;
> >  }
> >  
> >  static void svc_add_new_temp_xprt(struct svc_serv *serv, struct svc_xprt *newxpt)
> > @@ -854,12 +857,9 @@ void svc_recv(struct svc_rqst *rqstp)
> >  	if (!svc_alloc_arg(rqstp))
> >  		goto out;
> >  
> > -	try_to_freeze();
> > -	cond_resched();
> > -	if (kthread_should_stop())
> > -		goto out;
> > +	svc_rqst_wait_and_dequeue_work(rqstp);
> >  
> > -	xprt = svc_get_next_xprt(rqstp);
> > +	xprt = rqstp->rq_xprt;
> >  	if (!xprt)
> >  		goto out;
> >  
> > -- 
> > 2.40.1
> > 
> 
> -- 
> Chuck Lever
> 


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

end of thread, other threads:[~2023-08-02  5:01 UTC | newest]

Thread overview: 22+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2023-07-31  6:48 [PATCH 00/12] SUNRPC: various thread management improvements NeilBrown
2023-07-31  6:48 ` [PATCH 01/12] SUNRPC: make rqst_should_sleep() idempotent() NeilBrown
2023-07-31 14:21   ` Chuck Lever
2023-07-31 22:05     ` NeilBrown
2023-07-31 22:31       ` Chuck Lever III
2023-07-31 14:33   ` Jeff Layton
2023-07-31  6:48 ` [PATCH 02/12] FIXUP: SUNRPC: Deduplicate thread wake-up code NeilBrown
2023-07-31  6:48 ` [PATCH 03/12] FIXUP: SUNRPC: call svc_process() from svc_recv() NeilBrown
2023-07-31 14:22   ` Chuck Lever
2023-07-31  6:48 ` [PATCH 04/12] nfsd: Simplify code around svc_exit_thread() call in nfsd() NeilBrown
2023-07-31  6:48 ` [PATCH 05/12] nfsd: separate nfsd_last_thread() from nfsd_put() NeilBrown
2023-07-31 14:23   ` Chuck Lever
2023-07-31  6:48 ` [PATCH 06/12] SUNRPC: rename and refactor svc_get_next_xprt() NeilBrown
2023-07-31 23:16   ` Chuck Lever
2023-08-01 22:46     ` Chuck Lever
2023-08-02  5:00     ` NeilBrown
2023-07-31  6:48 ` [PATCH 07/12] SUNRPC: move all of xprt handling into svc_xprt_handle() NeilBrown
2023-07-31  6:48 ` [PATCH 08/12] SUNRPC: move task-dequeueing code into svc_recv() NeilBrown
2023-07-31  6:48 ` [PATCH 09/12] SUNRPC: integrate back-channel processing with svc_recv() NeilBrown
2023-07-31  6:48 ` [PATCH 10/12] SUNRPC: change how svc threads are asked to exit NeilBrown
2023-07-31  6:48 ` [PATCH 11/12] SUNRPC: add list of idle threads NeilBrown
2023-07-31  6:48 ` [PATCH 12/12] SUNRPC: discard SP_CONGESTED NeilBrown

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.