All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes.
@ 2017-08-25  9:33 Martijn Coenen
  2017-08-25  9:33 ` [PATCH 01/13] ANDROID: binder: remove proc waitqueue Martijn Coenen
                   ` (12 more replies)
  0 siblings, 13 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

The first six patches in this set introduce support for priority inheritance
of real-time scheduling policies in binder. With the introduction of Android
Treble, functionality that used to be in a single process is now split over
two or more processes, which communicate using binder IPC. For latency
sensitive operations such as sensor events, Bluetooth audio and rendering,
inheriting the (real-time) priority of the caller is crucial to meet
requirements.

Colin's patch adds a debug ioctl that allows us to more accurately track
memory leaks, as it allows us to identify objects to which only remote
processes have a reference.

The subsequent patches are mostly small fixes and (hopefully) well
explained in the commit messages.

All patches except 'Add tracing for binder priority inheritance' have
already been reviewed by Android engineers and are merged in Android's
common kernel trees.

Colin Cross (1):
  Add BINDER_GET_NODE_DEBUG_INFO ioctl

Martijn Coenen (12):
  ANDROID: binder: remove proc waitqueue
  ANDROID: binder: push new transactions to waiting threads.
  ANDROID: binder: add support for RT prio inheritance.
  ANDROID: binder: add min sched_policy to node.
  ANDROID: binder: improve priority inheritance.
  ANDROID: binder: add RT inheritance flag to node.
  ANDROID: binder: don't check prio permissions on restore.
  ANDROID: binder: Don't BUG_ON(!spin_is_locked()).
  ANDROID: binder: call poll_wait() unconditionally.
  ANDROID: binder: don't enqueue death notifications to thread todo.
  ANDROID: binder: don't queue async transactions to thread.
  ANDROID: binder: Add tracing for binder priority inheritance.

-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 01/13] ANDROID: binder: remove proc waitqueue
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 02/13] ANDROID: binder: push new transactions to waiting threads Martijn Coenen
                   ` (11 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

Removes the process waitqueue, so that threads
can only wait on the thread waitqueue. Whenever
there is process work to do, pick a thread and
wake it up.

This also fixes an issue with using epoll(),
since we no longer have to block on different
waitqueues.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 255 +++++++++++++++++++++++++++++++++--------------
 1 file changed, 181 insertions(+), 74 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index 947eb7056fa7..6cc2d96be5d4 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -28,10 +28,10 @@
  *    binder_node_lock() and binder_node_unlock() are
  *    used to acq/rel
  * 3) proc->inner_lock : protects the thread and node lists
- *    (proc->threads, proc->nodes) and all todo lists associated
- *    with the binder_proc (proc->todo, thread->todo,
- *    proc->delivered_death and node->async_todo), as well as
- *    thread->transaction_stack
+ *    (proc->threads, proc->waiting_threads, proc->nodes)
+ *    and all todo lists associated with the binder_proc
+ *    (proc->todo, thread->todo, proc->delivered_death and
+ *    node->async_todo), as well as thread->transaction_stack
  *    binder_inner_proc_lock() and binder_inner_proc_unlock()
  *    are used to acq/rel
  *
@@ -475,6 +475,8 @@ enum binder_deferred_state {
  *                        (protected by @outer_lock)
  * @refs_by_node:         rbtree of refs ordered by ref->node
  *                        (protected by @outer_lock)
+ * @waiting_threads:      threads currently waiting for proc work
+ *                        (protected by @inner_lock)
  * @pid                   PID of group_leader of process
  *                        (invariant after initialized)
  * @tsk                   task_struct for group_leader of process
@@ -504,8 +506,6 @@ enum binder_deferred_state {
  *                        (protected by @inner_lock)
  * @requested_threads_started: number binder threads started
  *                        (protected by @inner_lock)
- * @ready_threads:        number of threads waiting for proc work
- *                        (protected by @inner_lock)
  * @tmp_ref:              temporary reference to indicate proc is in use
  *                        (protected by @inner_lock)
  * @default_priority:     default scheduler priority
@@ -526,6 +526,7 @@ struct binder_proc {
 	struct rb_root nodes;
 	struct rb_root refs_by_desc;
 	struct rb_root refs_by_node;
+	struct list_head waiting_threads;
 	int pid;
 	struct task_struct *tsk;
 	struct files_struct *files;
@@ -540,7 +541,6 @@ struct binder_proc {
 	int max_threads;
 	int requested_threads;
 	int requested_threads_started;
-	int ready_threads;
 	int tmp_ref;
 	long default_priority;
 	struct dentry *debugfs_entry;
@@ -556,6 +556,7 @@ enum {
 	BINDER_LOOPER_STATE_EXITED      = 0x04,
 	BINDER_LOOPER_STATE_INVALID     = 0x08,
 	BINDER_LOOPER_STATE_WAITING     = 0x10,
+	BINDER_LOOPER_STATE_POLL        = 0x20,
 };
 
 /**
@@ -564,6 +565,8 @@ enum {
  *                        (invariant after initialization)
  * @rb_node:              element for proc->threads rbtree
  *                        (protected by @proc->inner_lock)
+ * @waiting_thread_node:  element for @proc->waiting_threads list
+ *                        (protected by @proc->inner_lock)
  * @pid:                  PID for this thread
  *                        (invariant after initialization)
  * @looper:               bitmap of looping state
@@ -593,6 +596,7 @@ enum {
 struct binder_thread {
 	struct binder_proc *proc;
 	struct rb_node rb_node;
+	struct list_head waiting_thread_node;
 	int pid;
 	int looper;              /* only modified by this thread */
 	bool looper_need_return; /* can be written by other thread */
@@ -920,6 +924,86 @@ static long task_close_fd(struct binder_proc *proc, unsigned int fd)
 	return retval;
 }
 
+static bool binder_has_work_ilocked(struct binder_thread *thread,
+				    bool do_proc_work)
+{
+	return !binder_worklist_empty_ilocked(&thread->todo) ||
+		thread->looper_need_return ||
+		(do_proc_work &&
+		 !binder_worklist_empty_ilocked(&thread->proc->todo));
+}
+
+static bool binder_has_work(struct binder_thread *thread, bool do_proc_work)
+{
+	bool has_work;
+
+	binder_inner_proc_lock(thread->proc);
+	has_work = binder_has_work_ilocked(thread, do_proc_work);
+	binder_inner_proc_unlock(thread->proc);
+
+	return has_work;
+}
+
+static bool binder_available_for_proc_work_ilocked(struct binder_thread *thread)
+{
+	return !thread->transaction_stack &&
+		binder_worklist_empty_ilocked(&thread->todo) &&
+		(thread->looper & (BINDER_LOOPER_STATE_ENTERED |
+				   BINDER_LOOPER_STATE_REGISTERED));
+}
+
+static void binder_wakeup_poll_threads_ilocked(struct binder_proc *proc,
+					       bool sync)
+{
+	struct rb_node *n;
+	struct binder_thread *thread;
+
+	for (n = rb_first(&proc->threads); n != NULL; n = rb_next(n)) {
+		thread = rb_entry(n, struct binder_thread, rb_node);
+		if (thread->looper & BINDER_LOOPER_STATE_POLL &&
+		    binder_available_for_proc_work_ilocked(thread)) {
+			if (sync)
+				wake_up_interruptible_sync(&thread->wait);
+			else
+				wake_up_interruptible(&thread->wait);
+		}
+	}
+}
+
+static void binder_wakeup_proc_ilocked(struct binder_proc *proc, bool sync)
+{
+	struct binder_thread *thread;
+
+	BUG_ON(!spin_is_locked(&proc->inner_lock));
+	thread = list_first_entry_or_null(&proc->waiting_threads,
+					  struct binder_thread,
+					  waiting_thread_node);
+
+	if (thread) {
+		list_del_init(&thread->waiting_thread_node);
+		if (sync)
+			wake_up_interruptible_sync(&thread->wait);
+		else
+			wake_up_interruptible(&thread->wait);
+		return;
+	}
+
+	/* Didn't find a thread waiting for proc work; this can happen
+	 * in two scenarios:
+	 * 1. All threads are busy handling transactions
+	 *    In that case, one of those threads should call back into
+	 *    the kernel driver soon and pick up this work.
+	 * 2. Threads are using the (e)poll interface, in which case
+	 *    they may be blocked on the waitqueue without having been
+	 *    added to waiting_threads. For this case, we just iterate
+	 *    over all threads not handling transaction work, and
+	 *    wake them all up. We wake all because we don't know whether
+	 *    a thread that called into (e)poll is handling non-binder
+	 *    work currently.
+	 */
+	binder_wakeup_poll_threads_ilocked(proc, sync);
+}
+
 static void binder_set_nice(long nice)
 {
 	long min_nice;
@@ -1138,7 +1222,7 @@ static bool binder_dec_node_nilocked(struct binder_node *node,
 	if (proc && (node->has_strong_ref || node->has_weak_ref)) {
 		if (list_empty(&node->work.entry)) {
 			binder_enqueue_work_ilocked(&node->work, &proc->todo);
-			wake_up_interruptible(&node->proc->wait);
+			binder_wakeup_proc_ilocked(proc, false);
 		}
 	} else {
 		if (hlist_empty(&node->refs) && !node->local_strong_refs &&
@@ -2399,7 +2483,6 @@ static void binder_transaction(struct binder_proc *proc,
 	struct binder_thread *target_thread = NULL;
 	struct binder_node *target_node = NULL;
 	struct list_head *target_list;
-	wait_queue_head_t *target_wait;
 	struct binder_transaction *in_reply_to = NULL;
 	struct binder_transaction_log_entry *e;
 	uint32_t return_error = 0;
@@ -2409,6 +2492,7 @@ static void binder_transaction(struct binder_proc *proc,
 	binder_size_t last_fixup_min_off = 0;
 	struct binder_context *context = proc->context;
 	int t_debug_id = atomic_inc_return(&binder_last_id);
+	bool wakeup_for_proc_work = false;
 
 	e = binder_transaction_log_add(&binder_transaction_log);
 	e->debug_id = t_debug_id;
@@ -2572,10 +2656,9 @@ static void binder_transaction(struct binder_proc *proc,
 	if (target_thread) {
 		e->to_thread = target_thread->pid;
 		target_list = &target_thread->todo;
-		target_wait = &target_thread->wait;
 	} else {
 		target_list = &target_proc->todo;
-		target_wait = &target_proc->wait;
+		wakeup_for_proc_work = true;
 	}
 	e->to_proc = target_proc->pid;
 
@@ -2882,7 +2965,7 @@ static void binder_transaction(struct binder_proc *proc,
 		binder_node_lock(target_node);
 		if (target_node->has_async_transaction) {
 			target_list = &target_node->async_todo;
-			target_wait = NULL;
+			wakeup_for_proc_work = false;
 		} else
 			target_node->has_async_transaction = 1;
 		/*
@@ -2901,11 +2984,13 @@ static void binder_transaction(struct binder_proc *proc,
 		binder_inner_proc_unlock(target_proc);
 		binder_node_unlock(target_node);
 	}
-	if (target_wait) {
-		if (reply || !(tr->flags & TF_ONE_WAY))
-			wake_up_interruptible_sync(target_wait);
-		else
-			wake_up_interruptible(target_wait);
+	if (target_thread) {
+		wake_up_interruptible_sync(&target_thread->wait);
+	} else if (wakeup_for_proc_work) {
+		binder_inner_proc_lock(target_proc);
+		binder_wakeup_proc_ilocked(target_proc,
+					   !(tr->flags & TF_ONE_WAY));
+		binder_inner_proc_unlock(target_proc);
 	}
 	if (target_thread)
 		binder_thread_dec_tmpref(target_thread);
@@ -3345,12 +3430,14 @@ static int binder_thread_write(struct binder_proc *proc,
 							&ref->death->work,
 							&thread->todo);
 					else {
-						binder_enqueue_work(
-							proc,
+						binder_inner_proc_lock(proc);
+						binder_enqueue_work_ilocked(
 							&ref->death->work,
 							&proc->todo);
-						wake_up_interruptible(
-								&proc->wait);
+						binder_wakeup_proc_ilocked(
+							proc,
+							false);
+						binder_inner_proc_unlock(proc);
 					}
 				}
 			} else {
@@ -3385,8 +3472,9 @@ static int binder_thread_write(struct binder_proc *proc,
 						binder_enqueue_work_ilocked(
 								&death->work,
 								&proc->todo);
-						wake_up_interruptible(
-								&proc->wait);
+						binder_wakeup_proc_ilocked(
+								proc,
+								false);
 					}
 				} else {
 					BUG_ON(death->work.type != BINDER_WORK_DEAD_BINDER);
@@ -3441,7 +3529,8 @@ static int binder_thread_write(struct binder_proc *proc,
 					binder_enqueue_work_ilocked(
 							&death->work,
 							&proc->todo);
-					wake_up_interruptible(&proc->wait);
+					binder_wakeup_proc_ilocked(
+							proc, false);
 				}
 			}
 			binder_inner_proc_unlock(proc);
@@ -3468,13 +3557,6 @@ static void binder_stat_br(struct binder_proc *proc,
 	}
 }
 
-static int binder_has_proc_work(struct binder_proc *proc,
-				struct binder_thread *thread)
-{
-	return !binder_worklist_empty(proc, &proc->todo) ||
-		thread->looper_need_return;
-}
-
 static int binder_has_thread_work(struct binder_thread *thread)
 {
 	return !binder_worklist_empty(thread->proc, &thread->todo) ||
@@ -3512,6 +3594,38 @@ static int binder_put_node_cmd(struct binder_proc *proc,
 	return 0;
 }
 
+static int binder_wait_for_work(struct binder_thread *thread,
+				bool do_proc_work)
+{
+	DEFINE_WAIT(wait);
+	struct binder_proc *proc = thread->proc;
+	int ret = 0;
+
+	freezer_do_not_count();
+	binder_inner_proc_lock(proc);
+	for (;;) {
+		prepare_to_wait(&thread->wait, &wait, TASK_INTERRUPTIBLE);
+		if (binder_has_work_ilocked(thread, do_proc_work))
+			break;
+		if (do_proc_work)
+			list_add(&thread->waiting_thread_node,
+				 &proc->waiting_threads);
+		binder_inner_proc_unlock(proc);
+		schedule();
+		binder_inner_proc_lock(proc);
+		list_del_init(&thread->waiting_thread_node);
+		if (signal_pending(current)) {
+			ret = -ERESTARTSYS;
+			break;
+		}
+	}
+	finish_wait(&thread->wait, &wait);
+	binder_inner_proc_unlock(proc);
+	freezer_count();
+
+	return ret;
+}
+
 static int binder_thread_read(struct binder_proc *proc,
 			      struct binder_thread *thread,
 			      binder_uintptr_t binder_buffer, size_t size,
@@ -3532,10 +3646,7 @@ static int binder_thread_read(struct binder_proc *proc,
 
 retry:
 	binder_inner_proc_lock(proc);
-	wait_for_proc_work = thread->transaction_stack == NULL &&
-		binder_worklist_empty_ilocked(&thread->todo);
-	if (wait_for_proc_work)
-		proc->ready_threads++;
+	wait_for_proc_work = binder_available_for_proc_work_ilocked(thread);
 	binder_inner_proc_unlock(proc);
 
 	thread->looper |= BINDER_LOOPER_STATE_WAITING;
@@ -3552,23 +3663,15 @@ static int binder_thread_read(struct binder_proc *proc,
 						 binder_stop_on_user_error < 2);
 		}
 		binder_set_nice(proc->default_priority);
-		if (non_block) {
-			if (!binder_has_proc_work(proc, thread))
-				ret = -EAGAIN;
-		} else
-			ret = wait_event_freezable_exclusive(proc->wait, binder_has_proc_work(proc, thread));
+	}
+
+	if (non_block) {
+		if (!binder_has_work(thread, wait_for_proc_work))
+			ret = -EAGAIN;
 	} else {
-		if (non_block) {
-			if (!binder_has_thread_work(thread))
-				ret = -EAGAIN;
-		} else
-			ret = wait_event_freezable(thread->wait, binder_has_thread_work(thread));
+		ret = binder_wait_for_work(thread, wait_for_proc_work);
 	}
 
-	binder_inner_proc_lock(proc);
-	if (wait_for_proc_work)
-		proc->ready_threads--;
-	binder_inner_proc_unlock(proc);
 	thread->looper &= ~BINDER_LOOPER_STATE_WAITING;
 
 	if (ret)
@@ -3854,7 +3957,8 @@ static int binder_thread_read(struct binder_proc *proc,
 
 	*consumed = ptr - buffer;
 	binder_inner_proc_lock(proc);
-	if (proc->requested_threads + proc->ready_threads == 0 &&
+	if (proc->requested_threads == 0 &&
+	    list_empty(&thread->proc->waiting_threads) &&
 	    proc->requested_threads_started < proc->max_threads &&
 	    (thread->looper & (BINDER_LOOPER_STATE_REGISTERED |
 	     BINDER_LOOPER_STATE_ENTERED)) /* the user-space code fails to */
@@ -3965,7 +4069,7 @@ static struct binder_thread *binder_get_thread_ilocked(
 	thread->return_error.cmd = BR_OK;
 	thread->reply_error.work.type = BINDER_WORK_RETURN_ERROR;
 	thread->reply_error.cmd = BR_OK;
-
+	INIT_LIST_HEAD(&new_thread->waiting_thread_node);
 	return thread;
 }
 
@@ -4078,28 +4182,24 @@ static unsigned int binder_poll(struct file *filp,
 {
 	struct binder_proc *proc = filp->private_data;
 	struct binder_thread *thread = NULL;
-	int wait_for_proc_work;
+	bool wait_for_proc_work;
 
 	thread = binder_get_thread(proc);
 
 	binder_inner_proc_lock(thread->proc);
-	wait_for_proc_work = thread->transaction_stack == NULL &&
-		binder_worklist_empty_ilocked(&thread->todo);
+	thread->looper |= BINDER_LOOPER_STATE_POLL;
+	wait_for_proc_work = binder_available_for_proc_work_ilocked(thread);
+
 	binder_inner_proc_unlock(thread->proc);
 
-	if (wait_for_proc_work) {
-		if (binder_has_proc_work(proc, thread))
-			return POLLIN;
-		poll_wait(filp, &proc->wait, wait);
-		if (binder_has_proc_work(proc, thread))
-			return POLLIN;
-	} else {
-		if (binder_has_thread_work(thread))
-			return POLLIN;
-		poll_wait(filp, &thread->wait, wait);
-		if (binder_has_thread_work(thread))
-			return POLLIN;
-	}
+	if (binder_has_work(thread, wait_for_proc_work))
+		return POLLIN;
+
+	poll_wait(filp, &thread->wait, wait);
+
+	if (binder_has_thread_work(thread))
+		return POLLIN;
+
 	return 0;
 }
 
@@ -4146,8 +4246,10 @@ static int binder_ioctl_write_read(struct file *filp,
 					 &bwr.read_consumed,
 					 filp->f_flags & O_NONBLOCK);
 		trace_binder_read_done(ret);
-		if (!binder_worklist_empty(proc, &proc->todo))
-			wake_up_interruptible(&proc->wait);
+		binder_inner_proc_lock(proc);
+		if (!binder_worklist_empty_ilocked(&proc->todo))
+			binder_wakeup_proc_ilocked(proc, false);
+		binder_inner_proc_unlock(proc);
 		if (ret < 0) {
 			if (copy_to_user(ubuf, &bwr, sizeof(bwr)))
 				ret = -EFAULT;
@@ -4387,7 +4489,6 @@ static int binder_open(struct inode *nodp, struct file *filp)
 	get_task_struct(current->group_leader);
 	proc->tsk = current->group_leader;
 	INIT_LIST_HEAD(&proc->todo);
-	init_waitqueue_head(&proc->wait);
 	proc->default_priority = task_nice(current);
 	binder_dev = container_of(filp->private_data, struct binder_device,
 				  miscdev);
@@ -4397,6 +4498,7 @@ static int binder_open(struct inode *nodp, struct file *filp)
 	binder_stats_created(BINDER_STAT_PROC);
 	proc->pid = current->group_leader->pid;
 	INIT_LIST_HEAD(&proc->delivered_death);
+	INIT_LIST_HEAD(&proc->waiting_threads);
 	filp->private_data = proc;
 
 	mutex_lock(&binder_procs_lock);
@@ -4448,7 +4550,6 @@ static void binder_deferred_flush(struct binder_proc *proc)
 		}
 	}
 	binder_inner_proc_unlock(proc);
-	wake_up_interruptible_all(&proc->wait);
 
 	binder_debug(BINDER_DEBUG_OPEN_CLOSE,
 		     "binder_flush: %d woke %d threads\n", proc->pid,
@@ -4517,7 +4618,7 @@ static int binder_node_release(struct binder_node *node, int refs)
 		ref->death->work.type = BINDER_WORK_DEAD_BINDER;
 		binder_enqueue_work_ilocked(&ref->death->work,
 					    &ref->proc->todo);
-		wake_up_interruptible(&ref->proc->wait);
+		binder_wakeup_proc_ilocked(ref->proc, false);
 		binder_inner_proc_unlock(ref->proc);
 	}
 
@@ -5005,23 +5106,29 @@ static void print_binder_proc_stats(struct seq_file *m,
 				    struct binder_proc *proc)
 {
 	struct binder_work *w;
+	struct binder_thread *thread;
 	struct rb_node *n;
-	int count, strong, weak;
+	int count, strong, weak, ready_threads;
 	size_t free_async_space =
 		binder_alloc_get_free_async_space(&proc->alloc);
 
 	seq_printf(m, "proc %d\n", proc->pid);
 	seq_printf(m, "context %s\n", proc->context->name);
 	count = 0;
+	ready_threads = 0;
 	binder_inner_proc_lock(proc);
 	for (n = rb_first(&proc->threads); n != NULL; n = rb_next(n))
 		count++;
+
+	list_for_each_entry(thread, &proc->waiting_threads, waiting_thread_node)
+		ready_threads++;
+
 	seq_printf(m, "  threads: %d\n", count);
 	seq_printf(m, "  requested threads: %d+%d/%d\n"
 			"  ready threads %d\n"
 			"  free async space %zd\n", proc->requested_threads,
 			proc->requested_threads_started, proc->max_threads,
-			proc->ready_threads,
+			ready_threads,
 			free_async_space);
 	count = 0;
 	for (n = rb_first(&proc->nodes); n != NULL; n = rb_next(n))
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 02/13] ANDROID: binder: push new transactions to waiting threads.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
  2017-08-25  9:33 ` [PATCH 01/13] ANDROID: binder: remove proc waitqueue Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 03/13] ANDROID: binder: add support for RT prio inheritance Martijn Coenen
                   ` (10 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

Instead of pushing new transactions to the process
waitqueue, select a thread that is waiting on proc
work to handle the transaction. This will make it
easier to improve priority inheritance in future
patches, by setting the priority before we wake up
a thread.

If we can't find a waiting thread, submit the work
to the proc waitqueue instead as we did previously.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 181 +++++++++++++++++++++++++++++++++--------------
 1 file changed, 127 insertions(+), 54 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index 6cc2d96be5d4..2f27bce31baf 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -970,7 +970,20 @@ static void binder_wakeup_poll_threads_ilocked(struct binder_proc *proc,
 	}
 }
 
-static void binder_wakeup_proc_ilocked(struct binder_proc *proc, bool sync)
+/**
+ * binder_select_thread_ilocked() - selects a thread for doing proc work.
+ * @proc:	process to select a thread from
+ *
+ * Note that calling this function moves the thread off the waiting_threads
+ * list, so it can only be woken up by the caller of this function, or a
+ * signal. Therefore, callers *should* always wake up the thread this function
+ * returns.
+ *
+ * Return:	If there's a thread currently waiting for process work,
+ *		returns that thread. Otherwise returns NULL.
+ */
+static struct binder_thread *
+binder_select_thread_ilocked(struct binder_proc *proc)
 {
 	struct binder_thread *thread;
 
@@ -979,8 +992,35 @@ static void binder_wakeup_proc_ilocked(struct binder_proc *proc, bool sync)
 					  struct binder_thread,
 					  waiting_thread_node);
 
-	if (thread) {
+	if (thread)
 		list_del_init(&thread->waiting_thread_node);
+
+	return thread;
+}
+
+/**
+ * binder_wakeup_thread_ilocked() - wakes up a thread for doing proc work.
+ * @proc:	process to wake up a thread in
+ * @thread:	specific thread to wake-up (may be NULL)
+ * @sync:	whether to do a synchronous wake-up
+ *
+ * This function wakes up a thread in the @proc process.
+ * The caller may provide a specific thread to wake-up in
+ * the @thread parameter. If @thread is NULL, this function
+ * will wake up threads that have called poll().
+ *
+ * Note that for this function to work as expected, callers
+ * should first call binder_select_thread() to find a thread
+ * to handle the work (if they don't have a thread already),
+ * and pass the result into the @thread parameter.
+ */
+static void binder_wakeup_thread_ilocked(struct binder_proc *proc,
+					 struct binder_thread *thread,
+					 bool sync)
+{
+	BUG_ON(!spin_is_locked(&proc->inner_lock));
+
+	if (thread) {
 		if (sync)
 			wake_up_interruptible_sync(&thread->wait);
 		else
@@ -1004,6 +1044,13 @@ static void binder_wakeup_proc_ilocked(struct binder_proc *proc, bool sync)
 	binder_wakeup_poll_threads_ilocked(proc, sync);
 }
 
+static void binder_wakeup_proc_ilocked(struct binder_proc *proc)
+{
+	struct binder_thread *thread = binder_select_thread_ilocked(proc);
+
+	binder_wakeup_thread_ilocked(proc, thread, /* sync = */false);
+}
+
 static void binder_set_nice(long nice)
 {
 	long min_nice;
@@ -1222,7 +1269,7 @@ static bool binder_dec_node_nilocked(struct binder_node *node,
 	if (proc && (node->has_strong_ref || node->has_weak_ref)) {
 		if (list_empty(&node->work.entry)) {
 			binder_enqueue_work_ilocked(&node->work, &proc->todo);
-			binder_wakeup_proc_ilocked(proc, false);
+			binder_wakeup_proc_ilocked(proc);
 		}
 	} else {
 		if (hlist_empty(&node->refs) && !node->local_strong_refs &&
@@ -2468,6 +2515,73 @@ static int binder_fixup_parent(struct binder_transaction *t,
 	return 0;
 }
 
+/**
+ * binder_proc_transaction() - sends a transaction to a process and wakes it up
+ * @t:		transaction to send
+ * @proc:	process to send the transaction to
+ * @thread:	thread in @proc to send the transaction to (may be NULL)
+ *
+ * This function queues a transaction to the specified process. It will try
+ * to find a thread in the target process to handle the transaction and
+ * wake it up. If no thread is found, the work is queued to the proc
+ * waitqueue.
+ *
+ * If the @thread parameter is not NULL, the transaction is always queued
+ * to the waitlist of that specific thread.
+ *
+ * Return:	true if the transactions was successfully queued
+ *		false if the target process or thread is dead
+ */
+static bool binder_proc_transaction(struct binder_transaction *t,
+				    struct binder_proc *proc,
+				    struct binder_thread *thread)
+{
+	struct list_head *target_list = NULL;
+	struct binder_node *node = t->buffer->target_node;
+	bool oneway = !!(t->flags & TF_ONE_WAY);
+	bool wakeup = true;
+
+	BUG_ON(!node);
+	binder_node_lock(node);
+	if (oneway) {
+		BUG_ON(thread);
+		if (node->has_async_transaction) {
+			target_list = &node->async_todo;
+			wakeup = false;
+		} else {
+			node->has_async_transaction = 1;
+		}
+	}
+
+	binder_inner_proc_lock(proc);
+
+	if (proc->is_dead || (thread && thread->is_dead)) {
+		binder_inner_proc_unlock(proc);
+		binder_node_unlock(node);
+		return false;
+	}
+
+	if (!thread && !target_list)
+		thread = binder_select_thread_ilocked(proc);
+
+	if (thread)
+		target_list = &thread->todo;
+	else if (!target_list)
+		target_list = &proc->todo;
+	else
+		BUG_ON(target_list != &node->async_todo);
+
+	binder_enqueue_work_ilocked(&t->work, target_list);
+
+	if (wakeup)
+		binder_wakeup_thread_ilocked(proc, thread, !oneway /* sync */);
+
+	binder_inner_proc_unlock(proc);
+	binder_node_unlock(node);
+
+	return true;
+}
+
 static void binder_transaction(struct binder_proc *proc,
 			       struct binder_thread *thread,
 			       struct binder_transaction_data *tr, int reply,
@@ -2482,7 +2596,6 @@ static void binder_transaction(struct binder_proc *proc,
 	struct binder_proc *target_proc = NULL;
 	struct binder_thread *target_thread = NULL;
 	struct binder_node *target_node = NULL;
-	struct list_head *target_list;
 	struct binder_transaction *in_reply_to = NULL;
 	struct binder_transaction_log_entry *e;
 	uint32_t return_error = 0;
@@ -2492,7 +2605,6 @@ static void binder_transaction(struct binder_proc *proc,
 	binder_size_t last_fixup_min_off = 0;
 	struct binder_context *context = proc->context;
 	int t_debug_id = atomic_inc_return(&binder_last_id);
-	bool wakeup_for_proc_work = false;
 
 	e = binder_transaction_log_add(&binder_transaction_log);
 	e->debug_id = t_debug_id;
@@ -2653,13 +2765,8 @@ static void binder_transaction(struct binder_proc *proc,
 		}
 		binder_inner_proc_unlock(proc);
 	}
-	if (target_thread) {
+	if (target_thread)
 		e->to_thread = target_thread->pid;
-		target_list = &target_thread->todo;
-	} else {
-		target_list = &target_proc->todo;
-		wakeup_for_proc_work = true;
-	}
 	e->to_proc = target_proc->pid;
 
 	/* TODO: reuse incoming transaction for reply */
@@ -2938,8 +3045,9 @@ static void binder_transaction(struct binder_proc *proc,
 		}
 		BUG_ON(t->buffer->async_transaction != 0);
 		binder_pop_transaction_ilocked(target_thread, in_reply_to);
-		binder_enqueue_work_ilocked(&t->work, target_list);
+		binder_enqueue_work_ilocked(&t->work, &target_thread->todo);
 		binder_inner_proc_unlock(target_proc);
+		wake_up_interruptible_sync(&target_thread->wait);
 		binder_free_transaction(in_reply_to);
 	} else if (!(t->flags & TF_ONE_WAY)) {
 		BUG_ON(t->buffer->async_transaction != 0);
@@ -2948,49 +3056,17 @@ static void binder_transaction(struct binder_proc *proc,
 		t->from_parent = thread->transaction_stack;
 		thread->transaction_stack = t;
 		binder_inner_proc_unlock(proc);
-		binder_inner_proc_lock(target_proc);
-		if (target_proc->is_dead ||
-				(target_thread && target_thread->is_dead)) {
-			binder_inner_proc_unlock(target_proc);
+		if (!binder_proc_transaction(t, target_proc, target_thread)) {
 			binder_inner_proc_lock(proc);
 			binder_pop_transaction_ilocked(thread, t);
 			binder_inner_proc_unlock(proc);
 			goto err_dead_proc_or_thread;
 		}
-		binder_enqueue_work_ilocked(&t->work, target_list);
-		binder_inner_proc_unlock(target_proc);
 	} else {
 		BUG_ON(target_node == NULL);
 		BUG_ON(t->buffer->async_transaction != 1);
-		binder_node_lock(target_node);
-		if (target_node->has_async_transaction) {
-			target_list = &target_node->async_todo;
-			wakeup_for_proc_work = false;
-		} else
-			target_node->has_async_transaction = 1;
-		/*
-		 * Test/set of has_async_transaction
-		 * must be atomic with enqueue on
-		 * async_todo
-		 */
-		binder_inner_proc_lock(target_proc);
-		if (target_proc->is_dead ||
-				(target_thread && target_thread->is_dead)) {
-			binder_inner_proc_unlock(target_proc);
-			binder_node_unlock(target_node);
+		if (!binder_proc_transaction(t, target_proc, NULL))
 			goto err_dead_proc_or_thread;
-		}
-		binder_enqueue_work_ilocked(&t->work, target_list);
-		binder_inner_proc_unlock(target_proc);
-		binder_node_unlock(target_node);
-	}
-	if (target_thread) {
-		wake_up_interruptible_sync(&target_thread->wait);
-	} else if (wakeup_for_proc_work) {
-		binder_inner_proc_lock(target_proc);
-		binder_wakeup_proc_ilocked(target_proc,
-					   !(tr->flags & TF_ONE_WAY));
-		binder_inner_proc_unlock(target_proc);
 	}
 	if (target_thread)
 		binder_thread_dec_tmpref(target_thread);
@@ -3435,8 +3511,7 @@ static int binder_thread_write(struct binder_proc *proc,
 							&ref->death->work,
 							&proc->todo);
 						binder_wakeup_proc_ilocked(
-							proc,
-							false);
+							proc);
 						binder_inner_proc_unlock(proc);
 					}
 				}
@@ -3473,8 +3548,7 @@ static int binder_thread_write(struct binder_proc *proc,
 								&death->work,
 								&proc->todo);
 						binder_wakeup_proc_ilocked(
-								proc,
-								false);
+								proc);
 					}
 				} else {
 					BUG_ON(death->work.type != BINDER_WORK_DEAD_BINDER);
@@ -3529,8 +3603,7 @@ static int binder_thread_write(struct binder_proc *proc,
 					binder_enqueue_work_ilocked(
 							&death->work,
 							&proc->todo);
-					binder_wakeup_proc_ilocked(
-							proc, false);
+					binder_wakeup_proc_ilocked(proc);
 				}
 			}
 			binder_inner_proc_unlock(proc);
@@ -4248,7 +4321,7 @@ static int binder_ioctl_write_read(struct file *filp,
 		trace_binder_read_done(ret);
 		binder_inner_proc_lock(proc);
 		if (!binder_worklist_empty_ilocked(&proc->todo))
-			binder_wakeup_proc_ilocked(proc, false);
+			binder_wakeup_proc_ilocked(proc);
 		binder_inner_proc_unlock(proc);
 		if (ret < 0) {
 			if (copy_to_user(ubuf, &bwr, sizeof(bwr)))
@@ -4618,7 +4691,7 @@ static int binder_node_release(struct binder_node *node, int refs)
 		ref->death->work.type = BINDER_WORK_DEAD_BINDER;
 		binder_enqueue_work_ilocked(&ref->death->work,
 					    &ref->proc->todo);
-		binder_wakeup_proc_ilocked(ref->proc, false);
+		binder_wakeup_proc_ilocked(ref->proc);
 		binder_inner_proc_unlock(ref->proc);
 	}
 
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 03/13] ANDROID: binder: add support for RT prio inheritance.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
  2017-08-25  9:33 ` [PATCH 01/13] ANDROID: binder: remove proc waitqueue Martijn Coenen
  2017-08-25  9:33 ` [PATCH 02/13] ANDROID: binder: push new transactions to waiting threads Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25 15:08   ` Thomas Gleixner
  2017-08-25  9:33 ` [PATCH 04/13] ANDROID: binder: add min sched_policy to node Martijn Coenen
                   ` (9 subsequent siblings)
  12 siblings, 1 reply; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

Adds support for SCHED_BATCH/SCHED_FIFO/SCHED_RR
priority inheritance.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 164 ++++++++++++++++++++++++++++++++++++++---------
 1 file changed, 135 insertions(+), 29 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index 2f27bce31baf..bb61116d1c0c 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -77,6 +77,7 @@
 #endif
 
 #include <uapi/linux/android/binder.h>
+#include <uapi/linux/sched/types.h>
 #include "binder_alloc.h"
 #include "binder_trace.h"
 
@@ -463,6 +464,22 @@ enum binder_deferred_state {
 	BINDER_DEFERRED_RELEASE      = 0x04,
 };
 
+/**
+ * struct binder_priority - scheduler policy and priority
+ * @sched_policy            scheduler policy
+ * @prio                    [100..139] for SCHED_NORMAL, [0..99] for FIFO/RT
+ *
+ * The binder driver supports inheriting the following scheduler policies:
+ * SCHED_NORMAL
+ * SCHED_BATCH
+ * SCHED_FIFO
+ * SCHED_RR
+ */
+struct binder_priority {
+	unsigned int sched_policy;
+	int prio;
+};
+
 /**
  * struct binder_proc - binder process bookkeeping
  * @proc_node:            element for binder_procs list
@@ -542,7 +559,7 @@ struct binder_proc {
 	int requested_threads;
 	int requested_threads_started;
 	int tmp_ref;
-	long default_priority;
+	struct binder_priority default_priority;
 	struct dentry *debugfs_entry;
 	struct binder_alloc alloc;
 	struct binder_context *context;
@@ -624,8 +641,8 @@ struct binder_transaction {
 	struct binder_buffer *buffer;
 	unsigned int	code;
 	unsigned int	flags;
-	long	priority;
-	long	saved_priority;
+	struct binder_priority	priority;
+	struct binder_priority	saved_priority;
 	kuid_t	sender_euid;
 	/**
 	 * @lock:  protects @from, @to_proc, and @to_thread
@@ -1051,22 +1068,93 @@ static void binder_wakeup_proc_ilocked(struct binder_proc *proc)
 	binder_wakeup_thread_ilocked(proc, thread, /* sync = */false);
 }
 
-static void binder_set_nice(long nice)
+static bool is_rt_policy(int policy)
+{
+	return policy == SCHED_FIFO || policy == SCHED_RR;
+}
+
+static bool is_fair_policy(int policy)
+{
+	return policy == SCHED_NORMAL || policy == SCHED_BATCH;
+}
+
+static bool binder_supported_policy(int policy)
+{
+	return is_fair_policy(policy) || is_rt_policy(policy);
+}
+
+static int to_userspace_prio(int policy, int kernel_priority)
+{
+	if (is_fair_policy(policy))
+		return PRIO_TO_NICE(kernel_priority);
+	else
+		return MAX_USER_RT_PRIO - 1 - kernel_priority;
+}
+
+static int to_kernel_prio(int policy, int user_priority)
+{
+	if (is_fair_policy(policy))
+		return NICE_TO_PRIO(user_priority);
+	else
+		return MAX_USER_RT_PRIO - 1 - user_priority;
+}
+
+static void binder_set_priority(struct task_struct *task,
+				struct binder_priority desired)
 {
-	long min_nice;
+	int priority; /* user-space prio value */
+	bool has_cap_nice;
+	unsigned int policy = desired.sched_policy;
 
-	if (can_nice(current, nice)) {
-		set_user_nice(current, nice);
+	if (task->policy == policy && task->normal_prio == desired.prio)
 		return;
+
+	has_cap_nice = has_capability_noaudit(task, CAP_SYS_NICE);
+
+	priority = to_userspace_prio(policy, desired.prio);
+
+	if (is_rt_policy(policy) && !has_cap_nice) {
+		long max_rtprio = task_rlimit(task, RLIMIT_RTPRIO);
+
+		if (max_rtprio == 0) {
+			policy = SCHED_NORMAL;
+			priority = MIN_NICE;
+		} else if (priority > max_rtprio) {
+			priority = max_rtprio;
+		}
 	}
-	min_nice = rlimit_to_nice(rlimit(RLIMIT_NICE));
-	binder_debug(BINDER_DEBUG_PRIORITY_CAP,
-		     "%d: nice value %ld not allowed use %ld instead\n",
-		      current->pid, nice, min_nice);
-	set_user_nice(current, min_nice);
-	if (min_nice <= MAX_NICE)
-		return;
-	binder_user_error("%d RLIMIT_NICE not set\n", current->pid);
+
+	if (is_fair_policy(policy) && !has_cap_nice) {
+		long min_nice = rlimit_to_nice(task_rlimit(task, RLIMIT_NICE));
+
+		if (min_nice > MAX_NICE) {
+			binder_user_error("%d RLIMIT_NICE not set\n",
+					  task->pid);
+			return;
+		} else if (priority < min_nice) {
+			priority = min_nice;
+		}
+	}
+
+	if (policy != desired.sched_policy ||
+	    to_kernel_prio(policy, priority) != desired.prio)
+		binder_debug(BINDER_DEBUG_PRIORITY_CAP,
+			     "%d: priority %d not allowed, using %d instead\n",
+			      task->pid, desired.prio,
+			      to_kernel_prio(policy, priority));
+
+	/* Set the actual priority */
+	if (task->policy != policy || is_rt_policy(policy)) {
+		struct sched_param params;
+
+		params.sched_priority = is_rt_policy(policy) ? priority : 0;
+
+		sched_setscheduler_nocheck(task,
+					   policy | SCHED_RESET_ON_FORK,
+					   &params);
+	}
+	if (is_fair_policy(policy))
+		set_user_nice(task, priority);
 }
 
 static struct binder_node *binder_get_node_ilocked(struct binder_proc *proc,
@@ -1150,7 +1238,8 @@ static struct binder_node *binder_init_node_ilocked(
 	node->ptr = ptr;
 	node->cookie = cookie;
 	node->work.type = BINDER_WORK_NODE;
-	node->min_priority = flags & FLAT_BINDER_FLAG_PRIORITY_MASK;
+	node->min_priority = NICE_TO_PRIO(
+			flags & FLAT_BINDER_FLAG_PRIORITY_MASK);
 	node->accept_fds = !!(flags & FLAT_BINDER_FLAG_ACCEPTS_FDS);
 	spin_lock_init(&node->lock);
 	INIT_LIST_HEAD(&node->work.entry);
@@ -2646,7 +2735,7 @@ static void binder_transaction(struct binder_proc *proc,
 		}
 		thread->transaction_stack = in_reply_to->to_parent;
 		binder_inner_proc_unlock(proc);
-		binder_set_nice(in_reply_to->saved_priority);
+		binder_set_priority(current, in_reply_to->saved_priority);
 		target_thread = binder_get_txn_from_and_acq_inner(in_reply_to);
 		if (target_thread == NULL) {
 			return_error = BR_DEAD_REPLY;
@@ -2819,7 +2908,15 @@ static void binder_transaction(struct binder_proc *proc,
 	t->to_thread = target_thread;
 	t->code = tr->code;
 	t->flags = tr->flags;
-	t->priority = task_nice(current);
+	if (!(t->flags & TF_ONE_WAY) &&
+	    binder_supported_policy(current->policy)) {
+		/* Inherit supported policies for synchronous transactions */
+		t->priority.sched_policy = current->policy;
+		t->priority.prio = current->normal_prio;
+	} else {
+		/* Otherwise, fall back to the default priority */
+		t->priority = target_proc->default_priority;
+	}
 
 	trace_binder_transaction(reply, t, target_node);
 
@@ -3735,7 +3832,7 @@ static int binder_thread_read(struct binder_proc *proc,
 			wait_event_interruptible(binder_user_error_wait,
 						 binder_stop_on_user_error < 2);
 		}
-		binder_set_nice(proc->default_priority);
+		binder_set_priority(current, proc->default_priority);
 	}
 
 	if (non_block) {
@@ -3947,16 +4044,17 @@ static int binder_thread_read(struct binder_proc *proc,
 		BUG_ON(t->buffer == NULL);
 		if (t->buffer->target_node) {
 			struct binder_node *target_node = t->buffer->target_node;
+			struct binder_priority prio = t->priority;
 
 			tr.target.ptr = target_node->ptr;
 			tr.cookie =  target_node->cookie;
-			t->saved_priority = task_nice(current);
-			if (t->priority < target_node->min_priority &&
-			    !(t->flags & TF_ONE_WAY))
-				binder_set_nice(t->priority);
-			else if (!(t->flags & TF_ONE_WAY) ||
-				 t->saved_priority > target_node->min_priority)
-				binder_set_nice(target_node->min_priority);
+			t->saved_priority.sched_policy = current->policy;
+			t->saved_priority.prio = current->normal_prio;
+			if (target_node->min_priority < t->priority.prio) {
+				prio.sched_policy = SCHED_NORMAL;
+				prio.prio = target_node->min_priority;
+			}
+			binder_set_priority(current, prio);
 			cmd = BR_TRANSACTION;
 		} else {
 			tr.target.ptr = 0;
@@ -4562,7 +4660,14 @@ static int binder_open(struct inode *nodp, struct file *filp)
 	get_task_struct(current->group_leader);
 	proc->tsk = current->group_leader;
 	INIT_LIST_HEAD(&proc->todo);
-	proc->default_priority = task_nice(current);
+	if (binder_supported_policy(current->policy)) {
+		proc->default_priority.sched_policy = current->policy;
+		proc->default_priority.prio = current->normal_prio;
+	} else {
+		proc->default_priority.sched_policy = SCHED_NORMAL;
+		proc->default_priority.prio = NICE_TO_PRIO(0);
+	}
+
 	binder_dev = container_of(filp->private_data, struct binder_device,
 				  miscdev);
 	proc->context = &binder_dev->context;
@@ -4855,13 +4960,14 @@ static void print_binder_transaction_ilocked(struct seq_file *m,
 	spin_lock(&t->lock);
 	to_proc = t->to_proc;
 	seq_printf(m,
-		   "%s %d: %p from %d:%d to %d:%d code %x flags %x pri %ld r%d",
+		   "%s %d: %p from %d:%d to %d:%d code %x flags %x pri %d:%d r%d",
 		   prefix, t->debug_id, t,
 		   t->from ? t->from->proc->pid : 0,
 		   t->from ? t->from->pid : 0,
 		   to_proc ? to_proc->pid : 0,
 		   t->to_thread ? t->to_thread->pid : 0,
-		   t->code, t->flags, t->priority, t->need_reply);
+		   t->code, t->flags, t->priority.sched_policy,
+		   t->priority.prio, t->need_reply);
 	spin_unlock(&t->lock);
 
 	if (proc != to_proc) {
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 04/13] ANDROID: binder: add min sched_policy to node.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (2 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 03/13] ANDROID: binder: add support for RT prio inheritance Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 05/13] ANDROID: binder: improve priority inheritance Martijn Coenen
                   ` (8 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

This change adds flags to flat_binder_object.flags
to allow indicating a minimum scheduling policy for
the node. It also clarifies the valid value range
for the priority bits in the flags.

Internally, we use the priority map that the kernel
uses, e.g. [0..99] for real-time policies and [100..139]
for the SCHED_NORMAL/SCHED_BATCH policies.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c            | 26 ++++++++++++++++++-----
 include/uapi/linux/android/binder.h | 41 ++++++++++++++++++++++++++++++++++++-
 2 files changed, 61 insertions(+), 6 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index bb61116d1c0c..247e913a6e6a 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -352,6 +352,8 @@ struct binder_error {
  *                        and by @lock)
  * @has_async_transaction: async transaction to node in progress
  *                        (protected by @lock)
+ * @sched_policy:         minimum scheduling policy for node
+ *                        (invariant after initialized)
  * @accept_fds:           file descriptor operations supported for node
  *                        (invariant after initialized)
  * @min_priority:         minimum scheduling priority
@@ -391,6 +393,7 @@ struct binder_node {
 		/*
 		 * invariant after initialization
 		 */
+		u8 sched_policy:2;
 		u8 accept_fds:1;
 		u8 min_priority;
 	};
@@ -1207,6 +1210,7 @@ static struct binder_node *binder_init_node_ilocked(
 	binder_uintptr_t ptr = fp ? fp->binder : 0;
 	binder_uintptr_t cookie = fp ? fp->cookie : 0;
 	__u32 flags = fp ? fp->flags : 0;
+	s8 priority;
 
 	BUG_ON(!spin_is_locked(&proc->inner_lock));
 	while (*p) {
@@ -1238,8 +1242,10 @@ static struct binder_node *binder_init_node_ilocked(
 	node->ptr = ptr;
 	node->cookie = cookie;
 	node->work.type = BINDER_WORK_NODE;
-	node->min_priority = NICE_TO_PRIO(
-			flags & FLAT_BINDER_FLAG_PRIORITY_MASK);
+	priority = flags & FLAT_BINDER_FLAG_PRIORITY_MASK;
+	node->sched_policy = (flags & FLAT_BINDER_FLAG_PRIORITY_MASK) >>
+		FLAT_BINDER_FLAG_SCHED_POLICY_SHIFT;
+	node->min_priority = to_kernel_prio(node->sched_policy, priority);
 	node->accept_fds = !!(flags & FLAT_BINDER_FLAG_ACCEPTS_FDS);
 	spin_lock_init(&node->lock);
 	INIT_LIST_HEAD(&node->work.entry);
@@ -4050,8 +4056,17 @@ static int binder_thread_read(struct binder_proc *proc,
 			tr.cookie =  target_node->cookie;
 			t->saved_priority.sched_policy = current->policy;
 			t->saved_priority.prio = current->normal_prio;
-			if (target_node->min_priority < t->priority.prio) {
-				prio.sched_policy = SCHED_NORMAL;
+			if (target_node->min_priority < t->priority.prio ||
+			    (target_node->min_priority == t->priority.prio &&
+			     target_node->sched_policy == SCHED_FIFO)) {
+				/*
+				 * In case the minimum priority on the node is
+				 * higher (lower value), use that priority. If
+				 * the priority is the same, but the node uses
+				 * SCHED_FIFO, prefer SCHED_FIFO, since it can
+				 * run unbounded, unlike SCHED_RR.
+				 */
+				prio.sched_policy = target_node->sched_policy;
 				prio.prio = target_node->min_priority;
 			}
 			binder_set_priority(current, prio);
@@ -5090,8 +5105,9 @@ static void print_binder_node_nilocked(struct seq_file *m,
 	hlist_for_each_entry(ref, &node->refs, node_entry)
 		count++;
 
-	seq_printf(m, "  node %d: u%016llx c%016llx hs %d hw %d ls %d lw %d is %d iw %d tr %d",
+	seq_printf(m, "  node %d: u%016llx c%016llx pri %d:%d hs %d hw %d ls %d lw %d is %d iw %d tr %d",
 		   node->debug_id, (u64)node->ptr, (u64)node->cookie,
+		   node->sched_policy, node->min_priority,
 		   node->has_strong_ref, node->has_weak_ref,
 		   node->local_strong_refs, node->local_weak_refs,
 		   node->internal_strong_refs, count, node->tmp_refs);
diff --git a/include/uapi/linux/android/binder.h b/include/uapi/linux/android/binder.h
index 7668b5791c91..026558ac254d 100644
--- a/include/uapi/linux/android/binder.h
+++ b/include/uapi/linux/android/binder.h
@@ -37,9 +37,48 @@ enum {
 	BINDER_TYPE_PTR		= B_PACK_CHARS('p', 't', '*', B_TYPE_LARGE),
 };
 
-enum {
+/**
+ * enum flat_binder_object_shifts: shift values for flat_binder_object_flags
+ * @FLAT_BINDER_FLAG_SCHED_POLICY_SHIFT: shift for getting scheduler policy.
+ *
+ */
+enum flat_binder_object_shifts {
+	FLAT_BINDER_FLAG_SCHED_POLICY_SHIFT = 9,
+};
+
+/**
+ * enum flat_binder_object_flags - flags for use in flat_binder_object.flags
+ */
+enum flat_binder_object_flags {
+	/**
+	 * @FLAT_BINDER_FLAG_PRIORITY_MASK: bit-mask for min scheduler priority
+	 *
+	 * These bits can be used to set the minimum scheduler priority
+	 * at which transactions into this node should run. Valid values
+	 * in these bits depend on the scheduler policy encoded in
+	 * @FLAT_BINDER_FLAG_SCHED_POLICY_MASK.
+	 *
+	 * For SCHED_NORMAL/SCHED_BATCH, the valid range is between [-20..19]
+	 * For SCHED_FIFO/SCHED_RR, the value can run between [1..99]
+	 */
 	FLAT_BINDER_FLAG_PRIORITY_MASK = 0xff,
+	/**
+	 * @FLAT_BINDER_FLAG_ACCEPTS_FDS: whether the node accepts fds.
+	 */
 	FLAT_BINDER_FLAG_ACCEPTS_FDS = 0x100,
+	/**
+	 * @FLAT_BINDER_FLAG_SCHED_POLICY_MASK: bit-mask for scheduling policy
+	 *
+	 * These two bits can be used to set the min scheduling policy at which
+	 * transactions on this node should run. These match the UAPI
+	 * scheduler policy values, eg:
+	 * 00b: SCHED_NORMAL
+	 * 01b: SCHED_FIFO
+	 * 10b: SCHED_RR
+	 * 11b: SCHED_BATCH
+	 */
+	FLAT_BINDER_FLAG_SCHED_POLICY_MASK =
+		3U << FLAT_BINDER_FLAG_SCHED_POLICY_SHIFT,
 };
 
 #ifdef BINDER_IPC_32BIT
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 05/13] ANDROID: binder: improve priority inheritance.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (3 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 04/13] ANDROID: binder: add min sched_policy to node Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 06/13] ANDROID: binder: add RT inheritance flag to node Martijn Coenen
                   ` (7 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

By raising the priority of a thread selected for
a transaction *before* we wake it up.

Delay restoring the priority when doing a reply
until after we wake-up the process receiving
the reply.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 74 ++++++++++++++++++++++++++++++++++--------------
 1 file changed, 53 insertions(+), 21 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index 247e913a6e6a..76d7544120a2 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -610,6 +610,7 @@ enum {
  * @is_dead:              thread is dead and awaiting free
  *                        when outstanding transactions are cleaned up
  *                        (protected by @proc->inner_lock)
+ * @task:                 struct task_struct for this thread
  *
  * Bookkeeping structure for binder threads.
  */
@@ -628,6 +629,7 @@ struct binder_thread {
 	struct binder_stats stats;
 	atomic_t tmp_ref;
 	bool is_dead;
+	struct task_struct *task;
 };
 
 struct binder_transaction {
@@ -646,6 +648,7 @@ struct binder_transaction {
 	unsigned int	flags;
 	struct binder_priority	priority;
 	struct binder_priority	saved_priority;
+	bool    set_priority_called;
 	kuid_t	sender_euid;
 	/**
 	 * @lock:  protects @from, @to_proc, and @to_thread
@@ -1160,6 +1163,38 @@ static void binder_set_priority(struct task_struct *task,
 		set_user_nice(task, priority);
 }
 
+static void binder_transaction_priority(struct task_struct *task,
+					struct binder_transaction *t,
+					struct binder_priority node_prio)
+{
+	struct binder_priority desired_prio;
+
+	if (t->set_priority_called)
+		return;
+
+	t->set_priority_called = true;
+	t->saved_priority.sched_policy = task->policy;
+	t->saved_priority.prio = task->normal_prio;
+
+	desired_prio.prio = t->priority.prio;
+	desired_prio.sched_policy = t->priority.sched_policy;
+
+	if (node_prio.prio < t->priority.prio ||
+	    (node_prio.prio == t->priority.prio &&
+	     node_prio.sched_policy == SCHED_FIFO)) {
+		/*
+		 * In case the minimum priority on the node is
+		 * higher (lower value), use that priority. If
+		 * the priority is the same, but the node uses
+		 * SCHED_FIFO, prefer SCHED_FIFO, since it can
+		 * run unbounded, unlike SCHED_RR.
+		 */
+		desired_prio = node_prio;
+	}
+
+	binder_set_priority(task, desired_prio);
+}
+
 static struct binder_node *binder_get_node_ilocked(struct binder_proc *proc,
 						   binder_uintptr_t ptr)
 {
@@ -2633,11 +2668,15 @@ static bool binder_proc_transaction(struct binder_transaction *t,
 {
 	struct list_head *target_list = NULL;
 	struct binder_node *node = t->buffer->target_node;
+	struct binder_priority node_prio;
 	bool oneway = !!(t->flags & TF_ONE_WAY);
 	bool wakeup = true;
 
 	BUG_ON(!node);
 	binder_node_lock(node);
+	node_prio.prio = node->min_priority;
+	node_prio.sched_policy = node->sched_policy;
+
 	if (oneway) {
 		BUG_ON(thread);
 		if (node->has_async_transaction) {
@@ -2659,12 +2698,14 @@ static bool binder_proc_transaction(struct binder_transaction *t,
 	if (!thread && !target_list)
 		thread = binder_select_thread_ilocked(proc);
 
-	if (thread)
+	if (thread) {
 		target_list = &thread->todo;
-	else if (!target_list)
+		binder_transaction_priority(thread->task, t, node_prio);
+	} else if (!target_list) {
 		target_list = &proc->todo;
-	else
+	} else {
 		BUG_ON(target_list != &node->async_todo);
+	}
 
 	binder_enqueue_work_ilocked(&t->work, target_list);
 
@@ -2741,7 +2782,6 @@ static void binder_transaction(struct binder_proc *proc,
 		}
 		thread->transaction_stack = in_reply_to->to_parent;
 		binder_inner_proc_unlock(proc);
-		binder_set_priority(current, in_reply_to->saved_priority);
 		target_thread = binder_get_txn_from_and_acq_inner(in_reply_to);
 		if (target_thread == NULL) {
 			return_error = BR_DEAD_REPLY;
@@ -3151,6 +3191,7 @@ static void binder_transaction(struct binder_proc *proc,
 		binder_enqueue_work_ilocked(&t->work, &target_thread->todo);
 		binder_inner_proc_unlock(target_proc);
 		wake_up_interruptible_sync(&target_thread->wait);
+		binder_set_priority(current, in_reply_to->saved_priority);
 		binder_free_transaction(in_reply_to);
 	} else if (!(t->flags & TF_ONE_WAY)) {
 		BUG_ON(t->buffer->async_transaction != 0);
@@ -3239,6 +3280,7 @@ static void binder_transaction(struct binder_proc *proc,
 
 	BUG_ON(thread->return_error.cmd != BR_OK);
 	if (in_reply_to) {
+		binder_set_priority(current, in_reply_to->saved_priority);
 		thread->return_error.cmd = BR_TRANSACTION_COMPLETE;
 		binder_enqueue_work(thread->proc,
 				    &thread->return_error.work,
@@ -4050,26 +4092,13 @@ static int binder_thread_read(struct binder_proc *proc,
 		BUG_ON(t->buffer == NULL);
 		if (t->buffer->target_node) {
 			struct binder_node *target_node = t->buffer->target_node;
-			struct binder_priority prio = t->priority;
+			struct binder_priority node_prio;
 
 			tr.target.ptr = target_node->ptr;
 			tr.cookie =  target_node->cookie;
-			t->saved_priority.sched_policy = current->policy;
-			t->saved_priority.prio = current->normal_prio;
-			if (target_node->min_priority < t->priority.prio ||
-			    (target_node->min_priority == t->priority.prio &&
-			     target_node->sched_policy == SCHED_FIFO)) {
-				/*
-				 * In case the minimum priority on the node is
-				 * higher (lower value), use that priority. If
-				 * the priority is the same, but the node uses
-				 * SCHED_FIFO, prefer SCHED_FIFO, since it can
-				 * run unbounded, unlike SCHED_RR.
-				 */
-				prio.sched_policy = target_node->sched_policy;
-				prio.prio = target_node->min_priority;
-			}
-			binder_set_priority(current, prio);
+			node_prio.sched_policy = target_node->sched_policy;
+			node_prio.prio = target_node->min_priority;
+			binder_transaction_priority(current, t, node_prio);
 			cmd = BR_TRANSACTION;
 		} else {
 			tr.target.ptr = 0;
@@ -4245,6 +4274,8 @@ static struct binder_thread *binder_get_thread_ilocked(
 	binder_stats_created(BINDER_STAT_THREAD);
 	thread->proc = proc;
 	thread->pid = current->pid;
+	get_task_struct(current);
+	thread->task = current;
 	atomic_set(&thread->tmp_ref, 0);
 	init_waitqueue_head(&thread->wait);
 	INIT_LIST_HEAD(&thread->todo);
@@ -4295,6 +4326,7 @@ static void binder_free_thread(struct binder_thread *thread)
 	BUG_ON(!list_empty(&thread->todo));
 	binder_stats_deleted(BINDER_STAT_THREAD);
 	binder_proc_dec_tmpref(thread->proc);
+	put_task_struct(thread->task);
 	kfree(thread);
 }
 
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 06/13] ANDROID: binder: add RT inheritance flag to node.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (4 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 05/13] ANDROID: binder: improve priority inheritance Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 07/13] Add BINDER_GET_NODE_DEBUG_INFO ioctl Martijn Coenen
                   ` (6 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

Allows a binder node to specify whether it wants to
inherit real-time scheduling policy from a caller.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c            | 22 +++++++++++++++++-----
 include/uapi/linux/android/binder.h |  8 ++++++++
 2 files changed, 25 insertions(+), 5 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index 76d7544120a2..e6778c696942 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -358,6 +358,8 @@ struct binder_error {
  *                        (invariant after initialized)
  * @min_priority:         minimum scheduling priority
  *                        (invariant after initialized)
+ * @inherit_rt:           inherit RT scheduling policy from caller
+ *                        (invariant after initialized)
  * @async_todo:           list of async work items
  *                        (protected by @proc->inner_lock)
  *
@@ -394,6 +396,7 @@ struct binder_node {
 		 * invariant after initialization
 		 */
 		u8 sched_policy:2;
+		u8 inherit_rt:1;
 		u8 accept_fds:1;
 		u8 min_priority;
 	};
@@ -1165,7 +1168,8 @@ static void binder_set_priority(struct task_struct *task,
 
 static void binder_transaction_priority(struct task_struct *task,
 					struct binder_transaction *t,
-					struct binder_priority node_prio)
+					struct binder_priority node_prio,
+					bool inherit_rt)
 {
 	struct binder_priority desired_prio;
 
@@ -1176,8 +1180,13 @@ static void binder_transaction_priority(struct task_struct *task,
 	t->saved_priority.sched_policy = task->policy;
 	t->saved_priority.prio = task->normal_prio;
 
-	desired_prio.prio = t->priority.prio;
-	desired_prio.sched_policy = t->priority.sched_policy;
+	if (!inherit_rt && is_rt_policy(desired_prio.sched_policy)) {
+		desired_prio.prio = NICE_TO_PRIO(0);
+		desired_prio.sched_policy = SCHED_NORMAL;
+	} else {
+		desired_prio.prio = t->priority.prio;
+		desired_prio.sched_policy = t->priority.sched_policy;
+	}
 
 	if (node_prio.prio < t->priority.prio ||
 	    (node_prio.prio == t->priority.prio &&
@@ -1282,6 +1291,7 @@ static struct binder_node *binder_init_node_ilocked(
 		FLAT_BINDER_FLAG_SCHED_POLICY_SHIFT;
 	node->min_priority = to_kernel_prio(node->sched_policy, priority);
 	node->accept_fds = !!(flags & FLAT_BINDER_FLAG_ACCEPTS_FDS);
+	node->inherit_rt = !!(flags & FLAT_BINDER_FLAG_INHERIT_RT);
 	spin_lock_init(&node->lock);
 	INIT_LIST_HEAD(&node->work.entry);
 	INIT_LIST_HEAD(&node->async_todo);
@@ -2700,7 +2710,8 @@ static bool binder_proc_transaction(struct binder_transaction *t,
 
 	if (thread) {
 		target_list = &thread->todo;
-		binder_transaction_priority(thread->task, t, node_prio);
+		binder_transaction_priority(thread->task, t, node_prio,
+					    node->inherit_rt);
 	} else if (!target_list) {
 		target_list = &proc->todo;
 	} else {
@@ -4098,7 +4109,8 @@ static int binder_thread_read(struct binder_proc *proc,
 			tr.cookie =  target_node->cookie;
 			node_prio.sched_policy = target_node->sched_policy;
 			node_prio.prio = target_node->min_priority;
-			binder_transaction_priority(current, t, node_prio);
+			binder_transaction_priority(current, t, node_prio,
+						    target_node->inherit_rt);
 			cmd = BR_TRANSACTION;
 		} else {
 			tr.target.ptr = 0;
diff --git a/include/uapi/linux/android/binder.h b/include/uapi/linux/android/binder.h
index 026558ac254d..70e252bf0be0 100644
--- a/include/uapi/linux/android/binder.h
+++ b/include/uapi/linux/android/binder.h
@@ -79,6 +79,14 @@ enum flat_binder_object_flags {
 	 */
 	FLAT_BINDER_FLAG_SCHED_POLICY_MASK =
 		3U << FLAT_BINDER_FLAG_SCHED_POLICY_SHIFT,
+
+	/**
+	 * @FLAT_BINDER_FLAG_INHERIT_RT: whether the node inherits RT policy
+	 *
+	 * Only when set, calls into this node will inherit a real-time
+	 * scheduling policy from the caller (for synchronous transactions).
+	 */
+	FLAT_BINDER_FLAG_INHERIT_RT = 0x800,
 };
 
 #ifdef BINDER_IPC_32BIT
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 07/13] Add BINDER_GET_NODE_DEBUG_INFO ioctl
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (5 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 06/13] ANDROID: binder: add RT inheritance flag to node Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 08/13] ANDROID: binder: don't check prio permissions on restore Martijn Coenen
                   ` (5 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

From: Colin Cross <ccross@android.com>

The BINDER_GET_NODE_DEBUG_INFO ioctl will return debug info on
a node.  Each successive call reusing the previous return value
will return the next node.  The data will be used by
libmemunreachable to mark the pointers with kernel references
as reachable.

Signed-off-by: Colin Cross <ccross@android.com>
Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c            | 43 +++++++++++++++++++++++++++++++++++++
 include/uapi/linux/android/binder.h | 14 ++++++++++++
 2 files changed, 57 insertions(+)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index e6778c696942..8eee4716e777 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -4546,6 +4546,31 @@ static int binder_ioctl_set_ctx_mgr(struct file *filp)
 	return ret;
 }
 
+static int binder_ioctl_get_node_debug_info(struct binder_proc *proc,
+				struct binder_node_debug_info *info)
+{
+	struct rb_node *n;
+	binder_uintptr_t ptr = info->ptr;
+
+	memset(info, 0, sizeof(*info));
+
+	binder_inner_proc_lock(proc);
+	for (n = rb_first(&proc->nodes); n != NULL; n = rb_next(n)) {
+		struct binder_node *node = rb_entry(n, struct binder_node,
+						    rb_node);
+		if (node->ptr > ptr) {
+			info->ptr = node->ptr;
+			info->cookie = node->cookie;
+			info->has_strong_ref = node->has_strong_ref;
+			info->has_weak_ref = node->has_weak_ref;
+			break;
+		}
+	}
+	binder_inner_proc_unlock(proc);
+
+	return 0;
+}
+
 static long binder_ioctl(struct file *filp, unsigned int cmd, unsigned long arg)
 {
 	int ret;
@@ -4613,6 +4638,24 @@ static long binder_ioctl(struct file *filp, unsigned int cmd, unsigned long arg)
 		}
 		break;
 	}
+	case BINDER_GET_NODE_DEBUG_INFO: {
+		struct binder_node_debug_info info;
+
+		if (copy_from_user(&info, ubuf, sizeof(info))) {
+			ret = -EFAULT;
+			goto err;
+		}
+
+		ret = binder_ioctl_get_node_debug_info(proc, &info);
+		if (ret < 0)
+			goto err;
+
+		if (copy_to_user(ubuf, &info, sizeof(info))) {
+			ret = -EFAULT;
+			goto err;
+		}
+		break;
+	}
 	default:
 		ret = -EINVAL;
 		goto err;
diff --git a/include/uapi/linux/android/binder.h b/include/uapi/linux/android/binder.h
index 70e252bf0be0..5539933b3491 100644
--- a/include/uapi/linux/android/binder.h
+++ b/include/uapi/linux/android/binder.h
@@ -233,6 +233,19 @@ struct binder_version {
 #define BINDER_CURRENT_PROTOCOL_VERSION 8
 #endif
 
+/*
+ * Use with BINDER_GET_NODE_DEBUG_INFO, driver reads ptr, writes to all fields.
+ * Set ptr to NULL for the first call to get the info for the first node, and
+ * then repeat the call passing the previously returned value to get the next
+ * nodes.  ptr will be 0 when there are no more nodes.
+ */
+struct binder_node_debug_info {
+	binder_uintptr_t ptr;
+	binder_uintptr_t cookie;
+	__u32            has_strong_ref;
+	__u32            has_weak_ref;
+};
+
 #define BINDER_WRITE_READ		_IOWR('b', 1, struct binder_write_read)
 #define BINDER_SET_IDLE_TIMEOUT		_IOW('b', 3, __s64)
 #define BINDER_SET_MAX_THREADS		_IOW('b', 5, __u32)
@@ -240,6 +253,7 @@ struct binder_version {
 #define BINDER_SET_CONTEXT_MGR		_IOW('b', 7, __s32)
 #define BINDER_THREAD_EXIT		_IOW('b', 8, __s32)
 #define BINDER_VERSION			_IOWR('b', 9, struct binder_version)
+#define BINDER_GET_NODE_DEBUG_INFO	_IOWR('b', 11, struct binder_node_debug_info)
 
 /*
  * NOTE: Two special error codes you should check for when calling
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 08/13] ANDROID: binder: don't check prio permissions on restore.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (6 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 07/13] Add BINDER_GET_NODE_DEBUG_INFO ioctl Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 09/13] ANDROID: binder: Don't BUG_ON(!spin_is_locked()) Martijn Coenen
                   ` (4 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

Because we have disabled RT priority inheritance for
the regular binder domain, the following can happen:

1) thread A (prio 98) calls into thread B
2) because RT prio inheritance is disabled, thread B
   runs at the lowest nice (prio 100) instead
3) thread B calls back into A; A will run at prio 100
   for the duration of the transaction
4) When thread A is done with the call from B, we will
   try to restore the prio back to 98. But, we fail
   because the process doesn't hold CAP_SYS_NICE,
   neither is RLIMIT_RT_PRIO set.

While the proper fix going forward will be to
correctly apply CAP_SYS_NICE or RLIMIT_RT_PRIO,
for now it seems reasonable to not check permissions
on the restore path.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 27 ++++++++++++++++++++-------
 1 file changed, 20 insertions(+), 7 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index 8eee4716e777..d6648b5f5988 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -1108,8 +1108,9 @@ static int to_kernel_prio(int policy, int user_priority)
 		return MAX_USER_RT_PRIO - 1 - user_priority;
 }
 
-static void binder_set_priority(struct task_struct *task,
-				struct binder_priority desired)
+static void binder_do_set_priority(struct task_struct *task,
+				   struct binder_priority desired,
+				   bool verify)
 {
 	int priority; /* user-space prio value */
 	bool has_cap_nice;
@@ -1122,7 +1123,7 @@ static void binder_set_priority(struct task_struct *task,
 
 	priority = to_userspace_prio(policy, desired.prio);
 
-	if (is_rt_policy(policy) && !has_cap_nice) {
+	if (verify && is_rt_policy(policy) && !has_cap_nice) {
 		long max_rtprio = task_rlimit(task, RLIMIT_RTPRIO);
 
 		if (max_rtprio == 0) {
@@ -1133,7 +1134,7 @@ static void binder_set_priority(struct task_struct *task,
 		}
 	}
 
-	if (is_fair_policy(policy) && !has_cap_nice) {
+	if (verify && is_fair_policy(policy) && !has_cap_nice) {
 		long min_nice = rlimit_to_nice(task_rlimit(task, RLIMIT_NICE));
 
 		if (min_nice > MAX_NICE) {
@@ -1166,6 +1167,18 @@ static void binder_set_priority(struct task_struct *task,
 		set_user_nice(task, priority);
 }
 
+static void binder_set_priority(struct task_struct *task,
+				struct binder_priority desired)
+{
+	binder_do_set_priority(task, desired, /* verify = */ true);
+}
+
+static void binder_restore_priority(struct task_struct *task,
+				    struct binder_priority desired)
+{
+	binder_do_set_priority(task, desired, /* verify = */ false);
+}
+
 static void binder_transaction_priority(struct task_struct *task,
 					struct binder_transaction *t,
 					struct binder_priority node_prio,
@@ -3202,7 +3215,7 @@ static void binder_transaction(struct binder_proc *proc,
 		binder_enqueue_work_ilocked(&t->work, &target_thread->todo);
 		binder_inner_proc_unlock(target_proc);
 		wake_up_interruptible_sync(&target_thread->wait);
-		binder_set_priority(current, in_reply_to->saved_priority);
+		binder_restore_priority(current, in_reply_to->saved_priority);
 		binder_free_transaction(in_reply_to);
 	} else if (!(t->flags & TF_ONE_WAY)) {
 		BUG_ON(t->buffer->async_transaction != 0);
@@ -3291,7 +3304,7 @@ static void binder_transaction(struct binder_proc *proc,
 
 	BUG_ON(thread->return_error.cmd != BR_OK);
 	if (in_reply_to) {
-		binder_set_priority(current, in_reply_to->saved_priority);
+		binder_restore_priority(current, in_reply_to->saved_priority);
 		thread->return_error.cmd = BR_TRANSACTION_COMPLETE;
 		binder_enqueue_work(thread->proc,
 				    &thread->return_error.work,
@@ -3891,7 +3904,7 @@ static int binder_thread_read(struct binder_proc *proc,
 			wait_event_interruptible(binder_user_error_wait,
 						 binder_stop_on_user_error < 2);
 		}
-		binder_set_priority(current, proc->default_priority);
+		binder_restore_priority(current, proc->default_priority);
 	}
 
 	if (non_block) {
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 09/13] ANDROID: binder: Don't BUG_ON(!spin_is_locked()).
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (7 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 08/13] ANDROID: binder: don't check prio permissions on restore Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 10/13] ANDROID: binder: call poll_wait() unconditionally Martijn Coenen
                   ` (3 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

Because is_spin_locked() always returns false on UP
systems.

Use assert_spin_locked() instead, and remove the
WARN_ON() instances, since those were easy to verify.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 26 ++++++++++----------------
 1 file changed, 10 insertions(+), 16 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index d6648b5f5988..eb82226be098 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -1013,7 +1013,7 @@ binder_select_thread_ilocked(struct binder_proc *proc)
 {
 	struct binder_thread *thread;
 
-	BUG_ON(!spin_is_locked(&proc->inner_lock));
+	assert_spin_locked(&proc->inner_lock);
 	thread = list_first_entry_or_null(&proc->waiting_threads,
 					  struct binder_thread,
 					  waiting_thread_node);
@@ -1044,7 +1044,7 @@ static void binder_wakeup_thread_ilocked(struct binder_proc *proc,
 					 struct binder_thread *thread,
 					 bool sync)
 {
-	BUG_ON(!spin_is_locked(&proc->inner_lock));
+	assert_spin_locked(&proc->inner_lock);
 
 	if (thread) {
 		if (sync)
@@ -1223,7 +1223,7 @@ static struct binder_node *binder_get_node_ilocked(struct binder_proc *proc,
 	struct rb_node *n = proc->nodes.rb_node;
 	struct binder_node *node;
 
-	BUG_ON(!spin_is_locked(&proc->inner_lock));
+	assert_spin_locked(&proc->inner_lock);
 
 	while (n) {
 		node = rb_entry(n, struct binder_node, rb_node);
@@ -1269,7 +1269,8 @@ static struct binder_node *binder_init_node_ilocked(
 	__u32 flags = fp ? fp->flags : 0;
 	s8 priority;
 
-	BUG_ON(!spin_is_locked(&proc->inner_lock));
+	assert_spin_locked(&proc->inner_lock);
+
 	while (*p) {
 
 		parent = *p;
@@ -1348,9 +1349,9 @@ static int binder_inc_node_nilocked(struct binder_node *node, int strong,
 {
 	struct binder_proc *proc = node->proc;
 
-	BUG_ON(!spin_is_locked(&node->lock));
+	assert_spin_locked(&node->lock);
 	if (proc)
-		BUG_ON(!spin_is_locked(&proc->inner_lock));
+		assert_spin_locked(&proc->inner_lock);
 	if (strong) {
 		if (internal) {
 			if (target_list == NULL &&
@@ -1401,9 +1402,9 @@ static bool binder_dec_node_nilocked(struct binder_node *node,
 {
 	struct binder_proc *proc = node->proc;
 
-	BUG_ON(!spin_is_locked(&node->lock));
+	assert_spin_locked(&node->lock);
 	if (proc)
-		BUG_ON(!spin_is_locked(&proc->inner_lock));
+		assert_spin_locked(&proc->inner_lock);
 	if (strong) {
 		if (internal)
 			node->internal_strong_refs--;
@@ -1927,7 +1928,7 @@ static void binder_pop_transaction_ilocked(struct binder_thread *target_thread,
 					   struct binder_transaction *t)
 {
 	BUG_ON(!target_thread);
-	BUG_ON(!spin_is_locked(&target_thread->proc->inner_lock));
+	assert_spin_locked(&target_thread->proc->inner_lock);
 	BUG_ON(target_thread->transaction_stack != t);
 	BUG_ON(target_thread->transaction_stack->from != target_thread);
 	target_thread->transaction_stack =
@@ -5071,7 +5072,6 @@ static void print_binder_transaction_ilocked(struct seq_file *m,
 	struct binder_proc *to_proc;
 	struct binder_buffer *buffer = t->buffer;
 
-	WARN_ON(!spin_is_locked(&proc->inner_lock));
 	spin_lock(&t->lock);
 	to_proc = t->to_proc;
 	seq_printf(m,
@@ -5160,7 +5160,6 @@ static void print_binder_thread_ilocked(struct seq_file *m,
 	size_t start_pos = m->count;
 	size_t header_pos;
 
-	WARN_ON(!spin_is_locked(&thread->proc->inner_lock));
 	seq_printf(m, "  thread %d: l %02x need_return %d tr %d\n",
 			thread->pid, thread->looper,
 			thread->looper_need_return,
@@ -5197,10 +5196,6 @@ static void print_binder_node_nilocked(struct seq_file *m,
 	struct binder_work *w;
 	int count;
 
-	WARN_ON(!spin_is_locked(&node->lock));
-	if (node->proc)
-		WARN_ON(!spin_is_locked(&node->proc->inner_lock));
-
 	count = 0;
 	hlist_for_each_entry(ref, &node->refs, node_entry)
 		count++;
@@ -5227,7 +5222,6 @@ static void print_binder_node_nilocked(struct seq_file *m,
 static void print_binder_ref_olocked(struct seq_file *m,
 				     struct binder_ref *ref)
 {
-	WARN_ON(!spin_is_locked(&ref->proc->outer_lock));
 	binder_node_lock(ref->node);
 	seq_printf(m, "  ref %d: desc %d %snode %d s %d w %d d %pK\n",
 		   ref->data.debug_id, ref->data.desc,
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 10/13] ANDROID: binder: call poll_wait() unconditionally.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (8 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 09/13] ANDROID: binder: Don't BUG_ON(!spin_is_locked()) Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 11/13] ANDROID: binder: don't enqueue death notifications to thread todo Martijn Coenen
                   ` (2 subsequent siblings)
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

Because we're not guaranteed that subsequent calls
to poll() will have a poll_table_struct parameter
with _qproc set. When _qproc is not set, poll_wait()
is a noop, and we won't be woken up correctly.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 11 +----------
 1 file changed, 1 insertion(+), 10 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index eb82226be098..e8a70a7c789c 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -3800,12 +3800,6 @@ static void binder_stat_br(struct binder_proc *proc,
 	}
 }
 
-static int binder_has_thread_work(struct binder_thread *thread)
-{
-	return !binder_worklist_empty(thread->proc, &thread->todo) ||
-		thread->looper_need_return;
-}
-
 static int binder_put_node_cmd(struct binder_proc *proc,
 			       struct binder_thread *thread,
 			       void __user **ptrp,
@@ -4436,12 +4430,9 @@ static unsigned int binder_poll(struct file *filp,
 
 	binder_inner_proc_unlock(thread->proc);
 
-	if (binder_has_work(thread, wait_for_proc_work))
-		return POLLIN;
-
 	poll_wait(filp, &thread->wait, wait);
 
-	if (binder_has_thread_work(thread))
+	if (binder_has_work(thread, wait_for_proc_work))
 		return POLLIN;
 
 	return 0;
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 11/13] ANDROID: binder: don't enqueue death notifications to thread todo.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (9 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 10/13] ANDROID: binder: call poll_wait() unconditionally Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 12/13] ANDROID: binder: don't queue async transactions to thread Martijn Coenen
  2017-08-25  9:33 ` [PATCH 13/13] ANDROID: binder: Add tracing for binder priority inheritance Martijn Coenen
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

This allows userspace to request death notifications without
having to worry about getting an immediate callback on the same
thread; one scenario where this would be problematic is if the
death recipient handler grabs a lock that was already taken
earlier (eg as part of a nested transaction).

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 22 ++++++----------------
 1 file changed, 6 insertions(+), 16 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index e8a70a7c789c..12ab16bb676c 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -3668,22 +3668,12 @@ static int binder_thread_write(struct binder_proc *proc,
 				ref->death = death;
 				if (ref->node->proc == NULL) {
 					ref->death->work.type = BINDER_WORK_DEAD_BINDER;
-					if (thread->looper &
-					    (BINDER_LOOPER_STATE_REGISTERED |
-					     BINDER_LOOPER_STATE_ENTERED))
-						binder_enqueue_work(
-							proc,
-							&ref->death->work,
-							&thread->todo);
-					else {
-						binder_inner_proc_lock(proc);
-						binder_enqueue_work_ilocked(
-							&ref->death->work,
-							&proc->todo);
-						binder_wakeup_proc_ilocked(
-							proc);
-						binder_inner_proc_unlock(proc);
-					}
+
+					binder_inner_proc_lock(proc);
+					binder_enqueue_work_ilocked(
+						&ref->death->work, &proc->todo);
+					binder_wakeup_proc_ilocked(proc);
+					binder_inner_proc_unlock(proc);
 				}
 			} else {
 				if (ref->death == NULL) {
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 12/13] ANDROID: binder: don't queue async transactions to thread.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (10 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 11/13] ANDROID: binder: don't enqueue death notifications to thread todo Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  2017-08-25  9:33 ` [PATCH 13/13] ANDROID: binder: Add tracing for binder priority inheritance Martijn Coenen
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

This can cause issues with processes using the poll()
interface:

1) client sends two oneway transactions
2) the second one gets queued on async_todo
   (because the server didn't handle the first one
    yet)
3) server returns from poll(), picks up the
   first transaction and does transaction work
4) server is done with the transaction, sends
   BC_FREE_BUFFER, and the second transaction gets
   moved to thread->todo
5) libbinder's handlePolledCommands() only handles
   the commands in the current data buffer, so
   doesn't see the new transaction
6) the server continues running and issues a new
   outgoing transaction. Now, it suddenly finds
   the incoming oneway transaction on its thread
   todo, and returns that to userspace.
7) userspace does not expect this to happen; it
   may be holding a lock while making the outgoing
   transaction, and if handling the incoming
   trasnaction requires taking the same lock,
   userspace will deadlock.

By queueing the async transaction to the proc
workqueue, we make sure it's only picked up when
a thread is ready for proc work.

Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index 12ab16bb676c..337c88fd675d 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -3520,11 +3520,13 @@ static int binder_thread_write(struct binder_proc *proc,
 				BUG_ON(buf_node->proc != proc);
 				w = binder_dequeue_work_head_ilocked(
 						&buf_node->async_todo);
-				if (!w)
+				if (!w) {
 					buf_node->has_async_transaction = 0;
-				else
+				} else {
 					binder_enqueue_work_ilocked(
-							w, &thread->todo);
+							w, &proc->todo);
+					binder_wakeup_proc_ilocked(proc);
+				}
 				binder_node_inner_unlock(buf_node);
 			}
 			trace_binder_transaction_buffer_release(buffer);
-- 
2.14.1.480.gb18f417b89-goog

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

* [PATCH 13/13] ANDROID: binder: Add tracing for binder priority inheritance.
  2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
                   ` (11 preceding siblings ...)
  2017-08-25  9:33 ` [PATCH 12/13] ANDROID: binder: don't queue async transactions to thread Martijn Coenen
@ 2017-08-25  9:33 ` Martijn Coenen
  12 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25  9:33 UTC (permalink / raw)
  To: gregkh, john.stultz, tkjos, arve, amit.pundir
  Cc: linux-kernel, devel, maco, malchev, ccross, Martijn Coenen

This allows to easily trace and visualize priority inheritance
in the binder driver.

Change-Id: I8449ae4b002e55c5e9517a47f3581e05eef051d8
Signed-off-by: Martijn Coenen <maco@android.com>
---
 drivers/android/binder.c       |  4 ++++
 drivers/android/binder_trace.h | 24 ++++++++++++++++++++++++
 2 files changed, 28 insertions(+)

diff --git a/drivers/android/binder.c b/drivers/android/binder.c
index 337c88fd675d..b29af3de6c34 100644
--- a/drivers/android/binder.c
+++ b/drivers/android/binder.c
@@ -1153,6 +1153,10 @@ static void binder_do_set_priority(struct task_struct *task,
 			      task->pid, desired.prio,
 			      to_kernel_prio(policy, priority));
 
+	trace_binder_set_priority(task->tgid, task->pid, task->normal_prio,
+				  to_kernel_prio(policy, priority),
+				  desired.prio);
+
 	/* Set the actual priority */
 	if (task->policy != policy || is_rt_policy(policy)) {
 		struct sched_param params;
diff --git a/drivers/android/binder_trace.h b/drivers/android/binder_trace.h
index 7967db16ba5a..1e0169796d81 100644
--- a/drivers/android/binder_trace.h
+++ b/drivers/android/binder_trace.h
@@ -85,6 +85,30 @@ DEFINE_BINDER_FUNCTION_RETURN_EVENT(binder_ioctl_done);
 DEFINE_BINDER_FUNCTION_RETURN_EVENT(binder_write_done);
 DEFINE_BINDER_FUNCTION_RETURN_EVENT(binder_read_done);
 
+TRACE_EVENT(binder_set_priority,
+	TP_PROTO(int proc, int thread, unsigned int old_prio,
+		 unsigned int desired_prio, unsigned int new_prio),
+	TP_ARGS(proc, thread, old_prio, new_prio, desired_prio),
+
+	TP_STRUCT__entry(
+		__field(int, proc)
+		__field(int, thread)
+		__field(unsigned int, old_prio)
+		__field(unsigned int, new_prio)
+		__field(unsigned int, desired_prio)
+	),
+	TP_fast_assign(
+		__entry->proc = proc;
+		__entry->thread = thread;
+		__entry->old_prio = old_prio;
+		__entry->new_prio = new_prio;
+		__entry->desired_prio = desired_prio;
+	),
+	TP_printk("proc=%d thread=%d old=%d => new=%d desired=%d",
+		  __entry->proc, __entry->thread, __entry->old_prio,
+		  __entry->new_prio, __entry->desired_prio)
+);
+
 TRACE_EVENT(binder_wait_for_work,
 	TP_PROTO(bool proc_work, bool transaction_stack, bool thread_todo),
 	TP_ARGS(proc_work, transaction_stack, thread_todo),
-- 
2.14.1.480.gb18f417b89-goog

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

* Re: [PATCH 03/13] ANDROID: binder: add support for RT prio inheritance.
  2017-08-25  9:33 ` [PATCH 03/13] ANDROID: binder: add support for RT prio inheritance Martijn Coenen
@ 2017-08-25 15:08   ` Thomas Gleixner
  2017-08-25 18:47     ` Martijn Coenen
  0 siblings, 1 reply; 16+ messages in thread
From: Thomas Gleixner @ 2017-08-25 15:08 UTC (permalink / raw)
  To: Martijn Coenen
  Cc: gregkh, John Stultz, tkjos, arve, amit.pundir, LKML, devel, maco,
	malchev, ccross, Peter Zijlstra, Christoph Hellwig

On Fri, 25 Aug 2017, Martijn Coenen wrote:

> Adds support for SCHED_BATCH/SCHED_FIFO/SCHED_RR
> priority inheritance.

Sorry, but this has not much to do with real priority inheritance.

It's a poor mans pseudo PI implementation. What I can't see from the sparse
changelog is how all of this is supposed to work.

My interpretation of it is, that you need to make sure that the other
thread in that IPC mechanism gets boosted enough to not block the thread
which needs a reply or action.

Therefor you create a unreadable maze of capability checks and other things
which do not make any sense to me due to lack of comments and something
which explains the big picture.

The whole thing looks wrong and engineered sideways circumventing the
existing facilities and making weird assumptions about priority settings.

As fair as I can see this is pretty much similar to stuff like RCU boosting
or other mechanisms which temporarily elevate the priority of a different
task.

I have no idea about the inner workings of all this, so let me describe my
understanding and ask a few questions.

Current state:

  1) thread queues work to worker via binder

  2) thread waits for the work to complete (I deduced that from a stray
     comment about sync work)

If the waiting thread is a high priority thread it might wait for a long
time if the worker thread is low priority or SCHED_OTHER.

Desired state:

  1) thread queues work to worker thread via binder
  
  1a) thread boosts the worker to its own priority

  2) thread waits for the work to complete

  2a) worker completes and drops priority boost

Is that about right?

If yes, then this whole thing can be simplified via the existing rtmutex
mechanism:

  1) thread creates work

  1a) thread proxy locks an rtmutex embedded in the work on behalf of the
      worker

  1b) thread queues work

  2) thread locks the rtmutex embedded in the work and blocks. This
     automatically includes the existing priority inheritance machinery
     without further voodoo.

  2a) worker runs with the inherited priority and completes the work

  2b) worker unlocks the rtmutex embedded in the work and drops priority

I probably miss something here, but I'm happy to read some proper
explanation how all of this is supposed to work.

Thanks,

	tglx

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

* Re: [PATCH 03/13] ANDROID: binder: add support for RT prio inheritance.
  2017-08-25 15:08   ` Thomas Gleixner
@ 2017-08-25 18:47     ` Martijn Coenen
  0 siblings, 0 replies; 16+ messages in thread
From: Martijn Coenen @ 2017-08-25 18:47 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Greg KH, John Stultz, Todd Kjos, Arve Hjønnevåg,
	Amit Pundir, LKML, devel, Martijn Coenen, Iliyan Malchev,
	Colin Cross, Peter Zijlstra, Christoph Hellwig

Hi Thomas,

On Fri, Aug 25, 2017 at 5:08 PM, Thomas Gleixner <tglx@linutronix.de> wrote:
> Sorry, but this has not much to do with real priority inheritance.

Can you clarify what "real priority inheritance" is, or are you more
concerned about this particular implementation of it?

>
> It's a poor mans pseudo PI implementation. What I can't see from the sparse
> changelog is how all of this is supposed to work.

Ok, guess the latter :-) I agree with you I should have included a
more verbose description of how this works, and I should also have
mentioned I did look at rtmutex and didn't think it could be easily
used for our purposes (more below).

>
> My interpretation of it is, that you need to make sure that the other
> thread in that IPC mechanism gets boosted enough to not block the thread
> which needs a reply or action.

This is true, but it's not the whole picture - more below.

>
> Therefor you create a unreadable maze of capability checks and other things
> which do not make any sense to me due to lack of comments and something
> which explains the big picture.

I thought the code itself was reasonably self-explanatory, but I'm
happy to comment it more. The binder driver has done "priority
inheritance" with nice values for a long time, and this is an
extension of that that more or less works in the same way.

>
> The whole thing looks wrong and engineered sideways circumventing the
> existing facilities and making weird assumptions about priority settings.

I'd be happy to use existing facilities if they can do what we need.
Can you describe the "weird assumptions" in more detail?


> Current state:
>
>   1) thread queues work to worker via binder
>
>   2) thread waits for the work to complete (I deduced that from a stray
>      comment about sync work)
>
> If the waiting thread is a high priority thread it might wait for a long
> time if the worker thread is low priority or SCHED_OTHER.
>
> Desired state:
>
>   1) thread queues work to worker thread via binder
>
>   1a) thread boosts the worker to its own priority
>
>   2) thread waits for the work to complete
>
>   2a) worker completes and drops priority boost
>
> Is that about right?

It is correct for synchronous transactions. Synchronous transactions
are transactions for which the caller blocks until they are completed
(eg a reply has been received). The receiving process has a threadpool
of one or more threads waiting for transactions. Typically those
receiving threads call into the binder driver with an ioctl and are
then waiting for work on a waitqueue. Before this patch series, all
threads (available for new transactions) would wait on the same
waitqueue, and so it was hard to do any sort of PI, because we didn't
know which thread was going to wake up. The first set in this series
changes this behavior by getting rid of this process-wide wait-queue -
the caller itself picks a thread to wake up. Anyway, after that things
more or less go as you describe: the thread picks up the work, returns
back to userspace to have it do the work, and eventually a reply comes
back, for which we unblock the caller (which is blocked on its own
waitqueue).

One place where using an rtmutex becomes tricky is that there may be
no threads waiting for work: all threads could be busy handling
transactions, so we can't pick one to wake up and do the work. In that
case, we push the work to a process-wide workqueue, and the first
thread to become available will pick up the work and do it. In that
case a "proxy rtmutex" doesn't really seem to work, because the proxy
doesn't know the owner when the work is queued. The current
implementation in this patchset just has the thread boost its own
priority in this scenario.

Another reason rtmutex is not straightforward is that we support
something called "node priority inheritance". A node is binder
terminology for an object that you can make binder transactions to.
For some nodes, we like to set a minimum scheduling priority. An
example of this are all the nodes in our system_server process. The
reason for this is that binder calls into the system server process
often take critical userspace locks. Say a thread calls into
system_server with priority 130; the system_server binder thread
inherits, runs at 130, takes a lock in userspace and gets preempted;
now, when somebody else tries to take the same lock, it can get
blocked for a long time. So, we set a minimum scheduling policy for
system_server at prio 120. This makes using rtmutex APIs hard, because
we don't necessarily want the binder thread to have the priority of
the caller. I guess you could say the proper way to fix this is that
our userspace mutexes should also support priority inheritance; I
don't think they do today, though I don't know the exact reason behind
it.

The final reason using rtmutex is not straightforward is asynchronous
transactions. Those are transactions for which the caller does not
block until completion. We push the work, wake up a thread, and then
the caller returns to userspace immediately. In that case we don't
inherit the priority from the caller, but we still want to run at the
minimum node priority. So there really is no caller that can be
"blocked" on a lock.

I will look into the rtmutex code a bit more - to be honest I hadn't
seen the "proxy" mechanism earlier, which is why I thought rtmutex
wouldn't even work for synchronous transactions. But if you have
suggestions for how to deal with these other scenarios, I'd be happy
to see if I can rework this.

Thanks,
Martijn

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

end of thread, other threads:[~2017-08-25 18:47 UTC | newest]

Thread overview: 16+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2017-08-25  9:33 [PATCH 00/13] ANDROID: binder: RT priority inheritance and small fixes Martijn Coenen
2017-08-25  9:33 ` [PATCH 01/13] ANDROID: binder: remove proc waitqueue Martijn Coenen
2017-08-25  9:33 ` [PATCH 02/13] ANDROID: binder: push new transactions to waiting threads Martijn Coenen
2017-08-25  9:33 ` [PATCH 03/13] ANDROID: binder: add support for RT prio inheritance Martijn Coenen
2017-08-25 15:08   ` Thomas Gleixner
2017-08-25 18:47     ` Martijn Coenen
2017-08-25  9:33 ` [PATCH 04/13] ANDROID: binder: add min sched_policy to node Martijn Coenen
2017-08-25  9:33 ` [PATCH 05/13] ANDROID: binder: improve priority inheritance Martijn Coenen
2017-08-25  9:33 ` [PATCH 06/13] ANDROID: binder: add RT inheritance flag to node Martijn Coenen
2017-08-25  9:33 ` [PATCH 07/13] Add BINDER_GET_NODE_DEBUG_INFO ioctl Martijn Coenen
2017-08-25  9:33 ` [PATCH 08/13] ANDROID: binder: don't check prio permissions on restore Martijn Coenen
2017-08-25  9:33 ` [PATCH 09/13] ANDROID: binder: Don't BUG_ON(!spin_is_locked()) Martijn Coenen
2017-08-25  9:33 ` [PATCH 10/13] ANDROID: binder: call poll_wait() unconditionally Martijn Coenen
2017-08-25  9:33 ` [PATCH 11/13] ANDROID: binder: don't enqueue death notifications to thread todo Martijn Coenen
2017-08-25  9:33 ` [PATCH 12/13] ANDROID: binder: don't queue async transactions to thread Martijn Coenen
2017-08-25  9:33 ` [PATCH 13/13] ANDROID: binder: Add tracing for binder priority inheritance Martijn Coenen

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.