All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCHSET] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status}
@ 2018-05-17  4:34 Tejun Heo
  2018-05-17  4:34 ` [PATCH 1/6] proc: Don't allow empty /proc/PID/cmdline for user tasks Tejun Heo
                   ` (5 more replies)
  0 siblings, 6 replies; 8+ messages in thread
From: Tejun Heo @ 2018-05-17  4:34 UTC (permalink / raw)
  To: torvalds, jiangshanlai, akpm; +Cc: linux-kernel, linux-api, kernel-team, csmall

There can be a lot of workqueue workers and they all show up with the
cryptic kworker/* names making it difficult to understand which is
doing what and how they came to be.

 # ps -ef | grep kworker
 root           4       2  0 Feb25 ?        00:00:00 [kworker/0:0H]
 root           6       2  0 Feb25 ?        00:00:00 [kworker/u112:0]
 root          19       2  0 Feb25 ?        00:00:00 [kworker/1:0H]
 root          25       2  0 Feb25 ?        00:00:00 [kworker/2:0H]
 root          31       2  0 Feb25 ?        00:00:00 [kworker/3:0H]
 ...

This patchset makes workqueue workers report the latest workqueue it
was executing for through /proc/PID/{comm,stat,status}.  The extra
information is appended to the kthread name with intervening '+' if
currently executing, otherwise '-'.

 # cat /proc/25/comm
 kworker/2:0-events_power_efficient
 # cat /proc/25/stat
 25 (kworker/2:0-events_power_efficient) I 2 0 0 0 -1 69238880 0 0...
 # grep Name /proc/25/status
 Name:   kworker/2:0-events_power_efficient

For details on the design decisions, please refer to the following
thread.

 http://lkml.kernel.org/r/20180516153939.GH2368884@devbig577.frc2.facebook.com

This patchset contains the following six patches.

 0001-proc-Don-t-allow-empty-proc-PID-cmdline-for-user-tas.patch
 0002-workqueue-Replace-pool-attach_mutex-with-global-wq_p.patch
 0003-workqueue-Make-worker_attach-detach_pool-update-work.patch
 0004-workqueue-Set-worker-desc-to-workqueue-name-by-defau.patch
 0005-proc-Consolidate-task-comm-formatting-into-proc_task.patch
 0006-workqueue-Show-the-latest-workqueue-name-in-proc-PID.patch

I'm applying the patches to wq/for-4.18.  Please let me know if the
patchset need updates (the branch doesn't have any other changes
anyway).  diffstat follows.  Thanks.

 fs/proc/array.c             |   33 +++++++-----
 fs/proc/base.c              |   27 +++++++---
 fs/proc/internal.h          |    2 
 include/linux/workqueue.h   |    1 
 kernel/workqueue.c          |  117 ++++++++++++++++++++++++++++----------------
 kernel/workqueue_internal.h |    3 -
 6 files changed, 122 insertions(+), 61 deletions(-)

--
tejun

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

* [PATCH 1/6] proc: Don't allow empty /proc/PID/cmdline for user tasks
  2018-05-17  4:34 [PATCHSET] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
@ 2018-05-17  4:34 ` Tejun Heo
  2018-05-17  4:34 ` [PATCH 2/6] workqueue: Replace pool->attach_mutex with global wq_pool_attach_mutex Tejun Heo
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 8+ messages in thread
From: Tejun Heo @ 2018-05-17  4:34 UTC (permalink / raw)
  To: torvalds, jiangshanlai, akpm
  Cc: linux-kernel, linux-api, kernel-team, csmall, Tejun Heo

Kernel threads have empty /proc/PID/cmdline and some userland tools
including ps(1) and older versions of systemd use this to detect
kernel threads.  However, any userland program can emulate the
behavior by making its argvs unavailable and trick the affected tools
into thinking that the task is a kernel thread.  Linus's reproducer
follows.

  #include <sys/prctl.h>
  #include <sys/mman.h>

  int main(void)
  {
	  char empty[16384];
	  unsigned long ptr;

	  asm volatile("" :"=r" (ptr) : "0" (empty):"memory");
	  ptr = (ptr+4095) & ~4095;
	  munmap((void *)ptr, 32768);

	  sleep(1000);
	  return 0;
  }

Compiling the above program into nullcmdline and running it on an
unpatche kernel shows the following behavior.

  $ ./nullcmdline &
  [1] 2382031
  [devbig577 ~/tmp]$ hexdump -C /proc/2382031/comm
  00000000  6e 75 6c 6c 63 6d 64 6c  69 6e 65 0a              |nullcmdline.|
  0000000c
  $ hexdump -C /proc/2382031/cmdline
  $ ps 2382031
      PID TTY      STAT   TIME COMMAND
  2382031 pts/2    S      0:00 [nullcmdline]

The empty cmdline makes ps(1) think that nullcmdline is a kernel
thread and put brackets around its name (comm), which is mostly a
nuisance but it's possible that this confusion can lead to more
harmful confusions.

This patch fixes the issue by making proc_pid_cmdline_read() never
return empty string for user tasks.  If the result is empty for
whatever reason, comm string is returned.  Even when the comm string
is empty, it still returns the null termnation character.  On a
patched kernel, running the same command as above gives us.

  $ ./nullcmdline &
  [1] 2317
  [test ~]# hexdump -C /proc/2317/comm
  00000000  6e 75 6c 6c 63 6d 64 6c  69 6e 65 0a              |nullcmdline.|
  0000000c
  $ hexdump -C /proc/2317/cmdline
  00000000  6e 75 6c 6c 63 6d 64 6c  69 6e 65 00              |nullcmdline.|
  0000000c
  $ ps 2317
    PID TTY      STAT   TIME COMMAND
   2317 pts/0    S      0:00 nullcmdline

Note that cmdline is a dup of comm and ps(1) is no longer confused.

Signed-off-by: Tejun Heo <tj@kernel.org>
Suggested-by: Linus Torvalds <torvalds@linux-foundation.org>
---
 fs/proc/base.c | 22 +++++++++++++++++++---
 1 file changed, 19 insertions(+), 3 deletions(-)

diff --git a/fs/proc/base.c b/fs/proc/base.c
index 1b2ede6..2eee4d7 100644
--- a/fs/proc/base.c
+++ b/fs/proc/base.c
@@ -224,9 +224,10 @@ static ssize_t proc_pid_cmdline_read(struct file *file, char __user *buf,
 	if (!tsk)
 		return -ESRCH;
 	mm = get_task_mm(tsk);
-	put_task_struct(tsk);
-	if (!mm)
-		return 0;
+	if (!mm) {
+		rv = 0;
+		goto out_put_task;
+	}
 	/* Check if process spawned far enough to have cmdline. */
 	if (!mm->env_end) {
 		rv = 0;
@@ -367,8 +368,23 @@ static ssize_t proc_pid_cmdline_read(struct file *file, char __user *buf,
 	free_page((unsigned long)page);
 out_mmput:
 	mmput(mm);
+out_put_task:
+	/*
+	 * Some userland tools use empty cmdline to distinguish kthreads.
+	 * Avoid empty cmdline for user tasks by returning tsk->comm with
+	 * \0 termination when empty.
+	 */
+	if (*pos == 0 && rv == 0 && !(tsk->flags & PF_KTHREAD)) {
+		char tcomm[TASK_COMM_LEN];
+
+		get_task_comm(tcomm, tsk);
+		rv = min(strlen(tcomm) + 1, count);
+		if (copy_to_user(buf, tsk->comm, rv))
+			rv = -EFAULT;
+	}
 	if (rv > 0)
 		*pos += rv;
+	put_task_struct(tsk);
 	return rv;
 }
 
-- 
2.9.5


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

* [PATCH 2/6] workqueue: Replace pool->attach_mutex with global wq_pool_attach_mutex
  2018-05-17  4:34 [PATCHSET] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
  2018-05-17  4:34 ` [PATCH 1/6] proc: Don't allow empty /proc/PID/cmdline for user tasks Tejun Heo
@ 2018-05-17  4:34 ` Tejun Heo
  2018-05-17  4:34 ` [PATCH 3/6] workqueue: Make worker_attach/detach_pool() update worker->pool Tejun Heo
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 8+ messages in thread
From: Tejun Heo @ 2018-05-17  4:34 UTC (permalink / raw)
  To: torvalds, jiangshanlai, akpm
  Cc: linux-kernel, linux-api, kernel-team, csmall, Tejun Heo

To improve workqueue visibility, we want to be able to access
workqueue information from worker tasks.  The per-pool attach mutex
makes that difficult because there's no way of stabilizing task ->
worker pool association without knowing the pool first.

Worker attach/detach is a slow path and there's no need for different
pools to be able to perform them concurrently.  This patch replaces
the per-pool attach_mutex with global wq_pool_attach_mutex to prepare
for visibility improvement changes.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c | 41 ++++++++++++++++++++---------------------
 1 file changed, 20 insertions(+), 21 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index ca7959b..91fe0a6 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -66,7 +66,7 @@ enum {
 	 * be executing on any CPU.  The pool behaves as an unbound one.
 	 *
 	 * Note that DISASSOCIATED should be flipped only while holding
-	 * attach_mutex to avoid changing binding state while
+	 * wq_pool_attach_mutex to avoid changing binding state while
 	 * worker_attach_to_pool() is in progress.
 	 */
 	POOL_MANAGER_ACTIVE	= 1 << 0,	/* being managed */
@@ -123,7 +123,7 @@ enum {
  *    cpu or grabbing pool->lock is enough for read access.  If
  *    POOL_DISASSOCIATED is set, it's identical to L.
  *
- * A: pool->attach_mutex protected.
+ * A: wq_pool_attach_mutex protected.
  *
  * PL: wq_pool_mutex protected.
  *
@@ -166,7 +166,6 @@ struct worker_pool {
 						/* L: hash of busy workers */
 
 	struct worker		*manager;	/* L: purely informational */
-	struct mutex		attach_mutex;	/* attach/detach exclusion */
 	struct list_head	workers;	/* A: attached workers */
 	struct completion	*detach_completion; /* all workers detached */
 
@@ -297,6 +296,7 @@ static bool wq_numa_enabled;		/* unbound NUMA affinity enabled */
 static struct workqueue_attrs *wq_update_unbound_numa_attrs_buf;
 
 static DEFINE_MUTEX(wq_pool_mutex);	/* protects pools and workqueues list */
+static DEFINE_MUTEX(wq_pool_attach_mutex); /* protects worker attach/detach */
 static DEFINE_SPINLOCK(wq_mayday_lock);	/* protects wq->maydays list */
 static DECLARE_WAIT_QUEUE_HEAD(wq_manager_wait); /* wait for manager to go away */
 
@@ -399,14 +399,14 @@ static void workqueue_sysfs_unregister(struct workqueue_struct *wq);
  * @worker: iteration cursor
  * @pool: worker_pool to iterate workers of
  *
- * This must be called with @pool->attach_mutex.
+ * This must be called with wq_pool_attach_mutex.
  *
  * The if/else clause exists only for the lockdep assertion and can be
  * ignored.
  */
 #define for_each_pool_worker(worker, pool)				\
 	list_for_each_entry((worker), &(pool)->workers, node)		\
-		if (({ lockdep_assert_held(&pool->attach_mutex); false; })) { } \
+		if (({ lockdep_assert_held(&wq_pool_attach_mutex); false; })) { } \
 		else
 
 /**
@@ -1724,7 +1724,7 @@ static struct worker *alloc_worker(int node)
 static void worker_attach_to_pool(struct worker *worker,
 				   struct worker_pool *pool)
 {
-	mutex_lock(&pool->attach_mutex);
+	mutex_lock(&wq_pool_attach_mutex);
 
 	/*
 	 * set_cpus_allowed_ptr() will fail if the cpumask doesn't have any
@@ -1733,16 +1733,16 @@ static void worker_attach_to_pool(struct worker *worker,
 	set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask);
 
 	/*
-	 * The pool->attach_mutex ensures %POOL_DISASSOCIATED remains
-	 * stable across this function.  See the comments above the
-	 * flag definition for details.
+	 * The wq_pool_attach_mutex ensures %POOL_DISASSOCIATED remains
+	 * stable across this function.  See the comments above the flag
+	 * definition for details.
 	 */
 	if (pool->flags & POOL_DISASSOCIATED)
 		worker->flags |= WORKER_UNBOUND;
 
 	list_add_tail(&worker->node, &pool->workers);
 
-	mutex_unlock(&pool->attach_mutex);
+	mutex_unlock(&wq_pool_attach_mutex);
 }
 
 /**
@@ -1759,11 +1759,11 @@ static void worker_detach_from_pool(struct worker *worker,
 {
 	struct completion *detach_completion = NULL;
 
-	mutex_lock(&pool->attach_mutex);
+	mutex_lock(&wq_pool_attach_mutex);
 	list_del(&worker->node);
 	if (list_empty(&pool->workers))
 		detach_completion = pool->detach_completion;
-	mutex_unlock(&pool->attach_mutex);
+	mutex_unlock(&wq_pool_attach_mutex);
 
 	/* clear leftover flags without pool->lock after it is detached */
 	worker->flags &= ~(WORKER_UNBOUND | WORKER_REBOUND);
@@ -3271,7 +3271,6 @@ static int init_worker_pool(struct worker_pool *pool)
 
 	timer_setup(&pool->mayday_timer, pool_mayday_timeout, 0);
 
-	mutex_init(&pool->attach_mutex);
 	INIT_LIST_HEAD(&pool->workers);
 
 	ida_init(&pool->worker_ida);
@@ -3354,10 +3353,10 @@ static void put_unbound_pool(struct worker_pool *pool)
 	WARN_ON(pool->nr_workers || pool->nr_idle);
 	spin_unlock_irq(&pool->lock);
 
-	mutex_lock(&pool->attach_mutex);
+	mutex_lock(&wq_pool_attach_mutex);
 	if (!list_empty(&pool->workers))
 		pool->detach_completion = &detach_completion;
-	mutex_unlock(&pool->attach_mutex);
+	mutex_unlock(&wq_pool_attach_mutex);
 
 	if (pool->detach_completion)
 		wait_for_completion(pool->detach_completion);
@@ -4600,7 +4599,7 @@ static void unbind_workers(int cpu)
 	struct worker *worker;
 
 	for_each_cpu_worker_pool(pool, cpu) {
-		mutex_lock(&pool->attach_mutex);
+		mutex_lock(&wq_pool_attach_mutex);
 		spin_lock_irq(&pool->lock);
 
 		/*
@@ -4616,7 +4615,7 @@ static void unbind_workers(int cpu)
 		pool->flags |= POOL_DISASSOCIATED;
 
 		spin_unlock_irq(&pool->lock);
-		mutex_unlock(&pool->attach_mutex);
+		mutex_unlock(&wq_pool_attach_mutex);
 
 		/*
 		 * Call schedule() so that we cross rq->lock and thus can
@@ -4657,7 +4656,7 @@ static void rebind_workers(struct worker_pool *pool)
 {
 	struct worker *worker;
 
-	lockdep_assert_held(&pool->attach_mutex);
+	lockdep_assert_held(&wq_pool_attach_mutex);
 
 	/*
 	 * Restore CPU affinity of all workers.  As all idle workers should
@@ -4727,7 +4726,7 @@ static void restore_unbound_workers_cpumask(struct worker_pool *pool, int cpu)
 	static cpumask_t cpumask;
 	struct worker *worker;
 
-	lockdep_assert_held(&pool->attach_mutex);
+	lockdep_assert_held(&wq_pool_attach_mutex);
 
 	/* is @cpu allowed for @pool? */
 	if (!cpumask_test_cpu(cpu, pool->attrs->cpumask))
@@ -4762,14 +4761,14 @@ int workqueue_online_cpu(unsigned int cpu)
 	mutex_lock(&wq_pool_mutex);
 
 	for_each_pool(pool, pi) {
-		mutex_lock(&pool->attach_mutex);
+		mutex_lock(&wq_pool_attach_mutex);
 
 		if (pool->cpu == cpu)
 			rebind_workers(pool);
 		else if (pool->cpu < 0)
 			restore_unbound_workers_cpumask(pool, cpu);
 
-		mutex_unlock(&pool->attach_mutex);
+		mutex_unlock(&wq_pool_attach_mutex);
 	}
 
 	/* update NUMA affinity of unbound workqueues */
-- 
2.9.5


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

* [PATCH 3/6] workqueue: Make worker_attach/detach_pool() update worker->pool
  2018-05-17  4:34 [PATCHSET] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
  2018-05-17  4:34 ` [PATCH 1/6] proc: Don't allow empty /proc/PID/cmdline for user tasks Tejun Heo
  2018-05-17  4:34 ` [PATCH 2/6] workqueue: Replace pool->attach_mutex with global wq_pool_attach_mutex Tejun Heo
@ 2018-05-17  4:34 ` Tejun Heo
  2018-05-19  6:44   ` Lai Jiangshan
  2018-05-17  4:34 ` [PATCH 4/6] workqueue: Set worker->desc to workqueue name by default Tejun Heo
                   ` (2 subsequent siblings)
  5 siblings, 1 reply; 8+ messages in thread
From: Tejun Heo @ 2018-05-17  4:34 UTC (permalink / raw)
  To: torvalds, jiangshanlai, akpm
  Cc: linux-kernel, linux-api, kernel-team, csmall, Tejun Heo

For historical reasons, the worker attach/detach functions don't
currently manage worker->pool and the callers are manually and
inconsistently updating it.

This patch moves worker->pool updates into the worker attach/detach
functions.  This makes worker->pool consistent and clearly defines how
worker->pool updates are synchronized.

This will help later workqueue visibility improvements by allowing
safe access to workqueue information from worker->task.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c          | 16 ++++++++--------
 kernel/workqueue_internal.h |  2 +-
 2 files changed, 9 insertions(+), 9 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 91fe0a6..2fde50f 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -1741,6 +1741,7 @@ static void worker_attach_to_pool(struct worker *worker,
 		worker->flags |= WORKER_UNBOUND;
 
 	list_add_tail(&worker->node, &pool->workers);
+	worker->pool = pool;
 
 	mutex_unlock(&wq_pool_attach_mutex);
 }
@@ -1748,19 +1749,21 @@ static void worker_attach_to_pool(struct worker *worker,
 /**
  * worker_detach_from_pool() - detach a worker from its pool
  * @worker: worker which is attached to its pool
- * @pool: the pool @worker is attached to
  *
  * Undo the attaching which had been done in worker_attach_to_pool().  The
  * caller worker shouldn't access to the pool after detached except it has
  * other reference to the pool.
  */
-static void worker_detach_from_pool(struct worker *worker,
-				    struct worker_pool *pool)
+static void worker_detach_from_pool(struct worker *worker)
 {
+	struct worker_pool *pool = worker->pool;
 	struct completion *detach_completion = NULL;
 
 	mutex_lock(&wq_pool_attach_mutex);
+
 	list_del(&worker->node);
+	worker->pool = NULL;
+
 	if (list_empty(&pool->workers))
 		detach_completion = pool->detach_completion;
 	mutex_unlock(&wq_pool_attach_mutex);
@@ -1799,7 +1802,6 @@ static struct worker *create_worker(struct worker_pool *pool)
 	if (!worker)
 		goto fail;
 
-	worker->pool = pool;
 	worker->id = id;
 
 	if (pool->cpu >= 0)
@@ -2236,7 +2238,7 @@ static int worker_thread(void *__worker)
 
 		set_task_comm(worker->task, "kworker/dying");
 		ida_simple_remove(&pool->worker_ida, worker->id);
-		worker_detach_from_pool(worker, pool);
+		worker_detach_from_pool(worker);
 		kfree(worker);
 		return 0;
 	}
@@ -2367,7 +2369,6 @@ static int rescuer_thread(void *__rescuer)
 		worker_attach_to_pool(rescuer, pool);
 
 		spin_lock_irq(&pool->lock);
-		rescuer->pool = pool;
 
 		/*
 		 * Slurp in all works issued via this workqueue and
@@ -2417,10 +2418,9 @@ static int rescuer_thread(void *__rescuer)
 		if (need_more_worker(pool))
 			wake_up_worker(pool);
 
-		rescuer->pool = NULL;
 		spin_unlock_irq(&pool->lock);
 
-		worker_detach_from_pool(rescuer, pool);
+		worker_detach_from_pool(rescuer);
 
 		spin_lock_irq(&wq_mayday_lock);
 	}
diff --git a/kernel/workqueue_internal.h b/kernel/workqueue_internal.h
index d390d1b..4a182e0 100644
--- a/kernel/workqueue_internal.h
+++ b/kernel/workqueue_internal.h
@@ -37,7 +37,7 @@ struct worker {
 	/* 64 bytes boundary on 64bit, 32 on 32bit */
 
 	struct task_struct	*task;		/* I: worker task */
-	struct worker_pool	*pool;		/* I: the associated pool */
+	struct worker_pool	*pool;		/* A: the associated pool */
 						/* L: for rescuers */
 	struct list_head	node;		/* A: anchored at pool->workers */
 						/* A: runs through worker->node */
-- 
2.9.5


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

* [PATCH 4/6] workqueue: Set worker->desc to workqueue name by default
  2018-05-17  4:34 [PATCHSET] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
                   ` (2 preceding siblings ...)
  2018-05-17  4:34 ` [PATCH 3/6] workqueue: Make worker_attach/detach_pool() update worker->pool Tejun Heo
@ 2018-05-17  4:34 ` Tejun Heo
  2018-05-17  4:34 ` [PATCH 5/6] proc: Consolidate task->comm formatting into proc_task_name() Tejun Heo
  2018-05-17  4:34 ` [PATCH 6/6] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
  5 siblings, 0 replies; 8+ messages in thread
From: Tejun Heo @ 2018-05-17  4:34 UTC (permalink / raw)
  To: torvalds, jiangshanlai, akpm
  Cc: linux-kernel, linux-api, kernel-team, csmall, Tejun Heo

Work functions can use set_worker_desc() to improve the visibility of
what the worker task is doing.  Currently, the desc field is unset at
the beginning of each execution and there is a separate field to track
the field is set during the current execution.

Instead of leaving empty till desc is set, worker->desc can be used to
remember the last workqueue the worker worked on by default and users
that use set_worker_desc() can override it to something more
informative as necessary.

This simplifies desc handling and helps tracking the last workqueue
that the worker exected on to improve visibility.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 kernel/workqueue.c          | 21 ++++++++++-----------
 kernel/workqueue_internal.h |  1 -
 2 files changed, 10 insertions(+), 12 deletions(-)

diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 2fde50f..3fbe007 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -2088,6 +2088,12 @@ __acquires(&pool->lock)
 	worker->current_pwq = pwq;
 	work_color = get_work_color(work);
 
+	/*
+	 * Record wq name for cmdline and debug reporting, may get
+	 * overridden through set_worker_desc().
+	 */
+	strscpy(worker->desc, pwq->wq->name, WORKER_DESC_LEN);
+
 	list_del_init(&work->entry);
 
 	/*
@@ -2183,7 +2189,6 @@ __acquires(&pool->lock)
 	worker->current_work = NULL;
 	worker->current_func = NULL;
 	worker->current_pwq = NULL;
-	worker->desc_valid = false;
 	pwq_dec_nr_in_flight(pwq, work_color);
 }
 
@@ -4346,7 +4351,6 @@ void set_worker_desc(const char *fmt, ...)
 		va_start(args, fmt);
 		vsnprintf(worker->desc, sizeof(worker->desc), fmt, args);
 		va_end(args);
-		worker->desc_valid = true;
 	}
 }
 
@@ -4370,7 +4374,6 @@ void print_worker_info(const char *log_lvl, struct task_struct *task)
 	char desc[WORKER_DESC_LEN] = { };
 	struct pool_workqueue *pwq = NULL;
 	struct workqueue_struct *wq = NULL;
-	bool desc_valid = false;
 	struct worker *worker;
 
 	if (!(task->flags & PF_WQ_WORKER))
@@ -4383,22 +4386,18 @@ void print_worker_info(const char *log_lvl, struct task_struct *task)
 	worker = kthread_probe_data(task);
 
 	/*
-	 * Carefully copy the associated workqueue's workfn and name.  Keep
-	 * the original last '\0' in case the original contains garbage.
+	 * Carefully copy the associated workqueue's workfn, name and desc.
+	 * Keep the original last '\0' in case the original is garbage.
 	 */
 	probe_kernel_read(&fn, &worker->current_func, sizeof(fn));
 	probe_kernel_read(&pwq, &worker->current_pwq, sizeof(pwq));
 	probe_kernel_read(&wq, &pwq->wq, sizeof(wq));
 	probe_kernel_read(name, wq->name, sizeof(name) - 1);
-
-	/* copy worker description */
-	probe_kernel_read(&desc_valid, &worker->desc_valid, sizeof(desc_valid));
-	if (desc_valid)
-		probe_kernel_read(desc, worker->desc, sizeof(desc) - 1);
+	probe_kernel_read(desc, worker->desc, sizeof(desc) - 1);
 
 	if (fn || name[0] || desc[0]) {
 		printk("%sWorkqueue: %s %pf", log_lvl, name, fn);
-		if (desc[0])
+		if (strcmp(name, desc))
 			pr_cont(" (%s)", desc);
 		pr_cont("\n");
 	}
diff --git a/kernel/workqueue_internal.h b/kernel/workqueue_internal.h
index 4a182e0..66fbb5a 100644
--- a/kernel/workqueue_internal.h
+++ b/kernel/workqueue_internal.h
@@ -31,7 +31,6 @@ struct worker {
 	struct work_struct	*current_work;	/* L: work being processed */
 	work_func_t		current_func;	/* L: current_work's fn */
 	struct pool_workqueue	*current_pwq; /* L: current_work's pwq */
-	bool			desc_valid;	/* ->desc is valid */
 	struct list_head	scheduled;	/* L: scheduled works */
 
 	/* 64 bytes boundary on 64bit, 32 on 32bit */
-- 
2.9.5


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

* [PATCH 5/6] proc: Consolidate task->comm formatting into proc_task_name()
  2018-05-17  4:34 [PATCHSET] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
                   ` (3 preceding siblings ...)
  2018-05-17  4:34 ` [PATCH 4/6] workqueue: Set worker->desc to workqueue name by default Tejun Heo
@ 2018-05-17  4:34 ` Tejun Heo
  2018-05-17  4:34 ` [PATCH 6/6] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
  5 siblings, 0 replies; 8+ messages in thread
From: Tejun Heo @ 2018-05-17  4:34 UTC (permalink / raw)
  To: torvalds, jiangshanlai, akpm
  Cc: linux-kernel, linux-api, kernel-team, csmall, Tejun Heo

proc shows task->comm in three places - comm, stat, status - and each
is fetching and formatting task->comm slighly differently.  This patch
renames task_name() to proc_task_name(), makes it more generic, and
updates all three paths to use it.

This will enable expanding comm reporting for workqueue workers.

Signed-off-by: Tejun Heo <tj@kernel.org>
---
 fs/proc/array.c    | 26 +++++++++++++++-----------
 fs/proc/base.c     |  5 ++---
 fs/proc/internal.h |  2 ++
 3 files changed, 19 insertions(+), 14 deletions(-)

diff --git a/fs/proc/array.c b/fs/proc/array.c
index ae2c807..f29221e 100644
--- a/fs/proc/array.c
+++ b/fs/proc/array.c
@@ -95,7 +95,7 @@
 #include <asm/processor.h>
 #include "internal.h"
 
-static inline void task_name(struct seq_file *m, struct task_struct *p)
+void proc_task_name(struct seq_file *m, struct task_struct *p, bool escape)
 {
 	char *buf;
 	size_t size;
@@ -104,13 +104,17 @@ static inline void task_name(struct seq_file *m, struct task_struct *p)
 
 	get_task_comm(tcomm, p);
 
-	seq_puts(m, "Name:\t");
-
 	size = seq_get_buf(m, &buf);
-	ret = string_escape_str(tcomm, buf, size, ESCAPE_SPACE | ESCAPE_SPECIAL, "\n\\");
-	seq_commit(m, ret < size ? ret : -1);
+	if (escape) {
+		ret = string_escape_str(tcomm, buf, size,
+					ESCAPE_SPACE | ESCAPE_SPECIAL, "\n\\");
+		if (ret >= size)
+			ret = -1;
+	} else {
+		ret = strscpy(buf, tcomm, size);
+	}
 
-	seq_putc(m, '\n');
+	seq_commit(m, ret);
 }
 
 /*
@@ -365,7 +369,10 @@ int proc_pid_status(struct seq_file *m, struct pid_namespace *ns,
 {
 	struct mm_struct *mm = get_task_mm(task);
 
-	task_name(m, task);
+	seq_puts(m, "Name:\t");
+	proc_task_name(m, task, true);
+	seq_putc(m, '\n');
+
 	task_state(m, ns, pid, task);
 
 	if (mm) {
@@ -400,7 +407,6 @@ static int do_task_stat(struct seq_file *m, struct pid_namespace *ns,
 	u64 cutime, cstime, utime, stime;
 	u64 cgtime, gtime;
 	unsigned long rsslim = 0;
-	char tcomm[sizeof(task->comm)];
 	unsigned long flags;
 
 	state = *get_task_state(task);
@@ -427,8 +433,6 @@ static int do_task_stat(struct seq_file *m, struct pid_namespace *ns,
 		}
 	}
 
-	get_task_comm(tcomm, task);
-
 	sigemptyset(&sigign);
 	sigemptyset(&sigcatch);
 	cutime = cstime = utime = stime = 0;
@@ -495,7 +499,7 @@ static int do_task_stat(struct seq_file *m, struct pid_namespace *ns,
 
 	seq_put_decimal_ull(m, "", pid_nr_ns(pid, ns));
 	seq_puts(m, " (");
-	seq_puts(m, tcomm);
+	proc_task_name(m, task, false);
 	seq_puts(m, ") ");
 	seq_putc(m, state);
 	seq_put_decimal_ll(m, " ", ppid);
diff --git a/fs/proc/base.c b/fs/proc/base.c
index 2eee4d7..eb17917ca 100644
--- a/fs/proc/base.c
+++ b/fs/proc/base.c
@@ -1581,9 +1581,8 @@ static int comm_show(struct seq_file *m, void *v)
 	if (!p)
 		return -ESRCH;
 
-	task_lock(p);
-	seq_printf(m, "%s\n", p->comm);
-	task_unlock(p);
+	proc_task_name(m, p, false);
+	seq_putc(m, '\n');
 
 	put_task_struct(p);
 
diff --git a/fs/proc/internal.h b/fs/proc/internal.h
index 0f1692e..b823fac62 100644
--- a/fs/proc/internal.h
+++ b/fs/proc/internal.h
@@ -131,6 +131,8 @@ unsigned name_to_int(const struct qstr *qstr);
  */
 extern const struct file_operations proc_tid_children_operations;
 
+extern void proc_task_name(struct seq_file *m, struct task_struct *p,
+			   bool escape);
 extern int proc_tid_stat(struct seq_file *, struct pid_namespace *,
 			 struct pid *, struct task_struct *);
 extern int proc_tgid_stat(struct seq_file *, struct pid_namespace *,
-- 
2.9.5


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

* [PATCH 6/6] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status}
  2018-05-17  4:34 [PATCHSET] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
                   ` (4 preceding siblings ...)
  2018-05-17  4:34 ` [PATCH 5/6] proc: Consolidate task->comm formatting into proc_task_name() Tejun Heo
@ 2018-05-17  4:34 ` Tejun Heo
  5 siblings, 0 replies; 8+ messages in thread
From: Tejun Heo @ 2018-05-17  4:34 UTC (permalink / raw)
  To: torvalds, jiangshanlai, akpm
  Cc: linux-kernel, linux-api, kernel-team, csmall, Tejun Heo

There can be a lot of workqueue workers and they all show up with the
cryptic kworker/* names making it difficult to understand which is
doing what and how they came to be.

  # ps -ef | grep kworker
  root           4       2  0 Feb25 ?        00:00:00 [kworker/0:0H]
  root           6       2  0 Feb25 ?        00:00:00 [kworker/u112:0]
  root          19       2  0 Feb25 ?        00:00:00 [kworker/1:0H]
  root          25       2  0 Feb25 ?        00:00:00 [kworker/2:0H]
  root          31       2  0 Feb25 ?        00:00:00 [kworker/3:0H]
  ...

This patch makes workqueue workers report the latest workqueue it was
executing for through /proc/PID/{comm,stat,status}.  The extra
information is appended to the kthread name with intervening '+' if
currently executing, otherwise '-'.

  # cat /proc/25/comm
  kworker/2:0-events_power_efficient
  # cat /proc/25/stat
  25 (kworker/2:0-events_power_efficient) I 2 0 0 0 -1 69238880 0 0...
  # grep Name /proc/25/status
  Name:   kworker/2:0-events_power_efficient

Unfortunately, ps(1) truncates comm to 15 characters,

  # ps 25
    PID TTY      STAT   TIME COMMAND
     25 ?        I      0:00 [kworker/2:0-eve]

making it a lot less useful; however, this should be an easy fix from
ps(1) side.

Signed-off-by: Tejun Heo <tj@kernel.org>
Suggested-by: Linus Torvalds <torvalds@linux-foundation.org>
Cc: Craig Small <csmall@enc.com.au>
---
 fs/proc/array.c           |  7 +++++--
 include/linux/workqueue.h |  1 +
 kernel/workqueue.c        | 39 +++++++++++++++++++++++++++++++++++++++
 3 files changed, 45 insertions(+), 2 deletions(-)

diff --git a/fs/proc/array.c b/fs/proc/array.c
index f29221e..bb1d361 100644
--- a/fs/proc/array.c
+++ b/fs/proc/array.c
@@ -99,10 +99,13 @@ void proc_task_name(struct seq_file *m, struct task_struct *p, bool escape)
 {
 	char *buf;
 	size_t size;
-	char tcomm[sizeof(p->comm)];
+	char tcomm[64];
 	int ret;
 
-	get_task_comm(tcomm, p);
+	if (p->flags & PF_WQ_WORKER)
+		wq_worker_comm(tcomm, sizeof(tcomm), p);
+	else
+		__get_task_comm(tcomm, sizeof(tcomm), p);
 
 	size = seq_get_buf(m, &buf);
 	if (escape) {
diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index 39a0e21..60d673e 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -494,6 +494,7 @@ extern unsigned int work_busy(struct work_struct *work);
 extern __printf(1, 2) void set_worker_desc(const char *fmt, ...);
 extern void print_worker_info(const char *log_lvl, struct task_struct *task);
 extern void show_workqueue_state(void);
+extern void wq_worker_comm(char *buf, size_t size, struct task_struct *task);
 
 /**
  * queue_work - queue work on a workqueue
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 3fbe007..b4a39a1 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -4577,6 +4577,45 @@ void show_workqueue_state(void)
 	rcu_read_unlock_sched();
 }
 
+/* used to show worker information through /proc/PID/{comm,stat,status} */
+void wq_worker_comm(char *buf, size_t size, struct task_struct *task)
+{
+	struct worker *worker;
+	struct worker_pool *pool;
+	int off;
+
+	/* always show the actual comm */
+	off = strscpy(buf, task->comm, size);
+	if (off < 0)
+		return;
+
+	/* stabilize worker pool association */
+	mutex_lock(&wq_pool_attach_mutex);
+
+	worker = kthread_data(task);
+	pool = worker->pool;
+
+	if (pool) {
+		spin_lock_irq(&pool->lock);
+		/*
+		 * ->desc tracks information (wq name or set_worker_desc())
+		 * for the latest execution.  If current, prepend '+',
+		 * otherwise '-'.
+		 */
+		if (worker->desc[0] != '\0') {
+			if (worker->current_work)
+				scnprintf(buf + off, size - off, "+%s",
+					  worker->desc);
+			else
+				scnprintf(buf + off, size - off, "-%s",
+					  worker->desc);
+		}
+		spin_unlock_irq(&pool->lock);
+	}
+
+	mutex_unlock(&wq_pool_attach_mutex);
+}
+
 /*
  * CPU hotplug.
  *
-- 
2.9.5


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

* Re: [PATCH 3/6] workqueue: Make worker_attach/detach_pool() update worker->pool
  2018-05-17  4:34 ` [PATCH 3/6] workqueue: Make worker_attach/detach_pool() update worker->pool Tejun Heo
@ 2018-05-19  6:44   ` Lai Jiangshan
  0 siblings, 0 replies; 8+ messages in thread
From: Lai Jiangshan @ 2018-05-19  6:44 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Linus Torvalds, Andrew Morton, LKML, Linux API, kernel-team, Craig Small

On Thu, May 17, 2018 at 12:34 PM, Tejun Heo <tj@kernel.org> wrote:
> For historical reasons, the worker attach/detach functions don't
> currently manage worker->pool and the callers are manually and
> inconsistently updating it.
>
> This patch moves worker->pool updates into the worker attach/detach
> functions.  This makes worker->pool consistent and clearly defines how
> worker->pool updates are synchronized.
>
> This will help later workqueue visibility improvements by allowing
> safe access to workqueue information from worker->task.
>
> Signed-off-by: Tejun Heo <tj@kernel.org>
> ---
>  kernel/workqueue.c          | 16 ++++++++--------
>  kernel/workqueue_internal.h |  2 +-
>  2 files changed, 9 insertions(+), 9 deletions(-)
>
> diff --git a/kernel/workqueue.c b/kernel/workqueue.c
> index 91fe0a6..2fde50f 100644
> --- a/kernel/workqueue.c
> +++ b/kernel/workqueue.c
> @@ -1741,6 +1741,7 @@ static void worker_attach_to_pool(struct worker *worker,
>                 worker->flags |= WORKER_UNBOUND;
>
>         list_add_tail(&worker->node, &pool->workers);
> +       worker->pool = pool;
>
>         mutex_unlock(&wq_pool_attach_mutex);
>  }
> @@ -1748,19 +1749,21 @@ static void worker_attach_to_pool(struct worker *worker,
>  /**
>   * worker_detach_from_pool() - detach a worker from its pool
>   * @worker: worker which is attached to its pool
> - * @pool: the pool @worker is attached to
>   *
>   * Undo the attaching which had been done in worker_attach_to_pool().  The
>   * caller worker shouldn't access to the pool after detached except it has
>   * other reference to the pool.
>   */
> -static void worker_detach_from_pool(struct worker *worker,
> -                                   struct worker_pool *pool)
> +static void worker_detach_from_pool(struct worker *worker)
>  {
> +       struct worker_pool *pool = worker->pool;
>         struct completion *detach_completion = NULL;
>
>         mutex_lock(&wq_pool_attach_mutex);
> +
>         list_del(&worker->node);
> +       worker->pool = NULL;
> +
>         if (list_empty(&pool->workers))
>                 detach_completion = pool->detach_completion;
>         mutex_unlock(&wq_pool_attach_mutex);
> @@ -1799,7 +1802,6 @@ static struct worker *create_worker(struct worker_pool *pool)
>         if (!worker)
>                 goto fail;
>
> -       worker->pool = pool;
>         worker->id = id;
>
>         if (pool->cpu >= 0)
> @@ -2236,7 +2238,7 @@ static int worker_thread(void *__worker)
>
>                 set_task_comm(worker->task, "kworker/dying");
>                 ida_simple_remove(&pool->worker_ida, worker->id);
> -               worker_detach_from_pool(worker, pool);
> +               worker_detach_from_pool(worker);
>                 kfree(worker);
>                 return 0;
>         }
> @@ -2367,7 +2369,6 @@ static int rescuer_thread(void *__rescuer)
>                 worker_attach_to_pool(rescuer, pool);
>
>                 spin_lock_irq(&pool->lock);
> -               rescuer->pool = pool;
>
>                 /*
>                  * Slurp in all works issued via this workqueue and
> @@ -2417,10 +2418,9 @@ static int rescuer_thread(void *__rescuer)
>                 if (need_more_worker(pool))
>                         wake_up_worker(pool);
>
> -               rescuer->pool = NULL;
>                 spin_unlock_irq(&pool->lock);
>
> -               worker_detach_from_pool(rescuer, pool);
> +               worker_detach_from_pool(rescuer);
>
>                 spin_lock_irq(&wq_mayday_lock);
>         }
> diff --git a/kernel/workqueue_internal.h b/kernel/workqueue_internal.h
> index d390d1b..4a182e0 100644
> --- a/kernel/workqueue_internal.h
> +++ b/kernel/workqueue_internal.h
> @@ -37,7 +37,7 @@ struct worker {
>         /* 64 bytes boundary on 64bit, 32 on 32bit */
>
>         struct task_struct      *task;          /* I: worker task */
> -       struct worker_pool      *pool;          /* I: the associated pool */
> +       struct worker_pool      *pool;          /* A: the associated pool */
>                                                 /* L: for rescuers */

I guess ` /* L: for rescuers */ ` needed to be removed.

And there are many usages of worker->pool without attached_mutex
heled. The only locking annotation "A:" is not enough. Additional
comment is needed too.

1) called from the worker(including rescuer) task: it is safe because only
    the worker task itself can call the attach()/detach().

2) called from the destroy_worker() (only normal worker): it is safe
    because it is immutable before the worker detaches itself.

>         struct list_head        node;           /* A: anchored at pool->workers */
>                                                 /* A: runs through worker->node */
> --
> 2.9.5
>

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

end of thread, other threads:[~2018-05-19  6:44 UTC | newest]

Thread overview: 8+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-05-17  4:34 [PATCHSET] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo
2018-05-17  4:34 ` [PATCH 1/6] proc: Don't allow empty /proc/PID/cmdline for user tasks Tejun Heo
2018-05-17  4:34 ` [PATCH 2/6] workqueue: Replace pool->attach_mutex with global wq_pool_attach_mutex Tejun Heo
2018-05-17  4:34 ` [PATCH 3/6] workqueue: Make worker_attach/detach_pool() update worker->pool Tejun Heo
2018-05-19  6:44   ` Lai Jiangshan
2018-05-17  4:34 ` [PATCH 4/6] workqueue: Set worker->desc to workqueue name by default Tejun Heo
2018-05-17  4:34 ` [PATCH 5/6] proc: Consolidate task->comm formatting into proc_task_name() Tejun Heo
2018-05-17  4:34 ` [PATCH 6/6] workqueue: Show the latest workqueue name in /proc/PID/{comm,stat,status} Tejun Heo

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.