2010-12-14 09:03:35

by Lai Jiangshan

[permalink] [raw]
Subject: [PATCH] rtmutex: multiple candidate owners without unrelated boosting


Current rtmutex just supports one pending owner only,
this patch make any rtmutex can have multiple candidate owners.

In this changelog I use "pending owner" and "candidate owner"
without distinguishing, they are the same meaning. But I
think the name "candidate owner" is more suitable, so I
only use "candidate owner" in the code.

In current rtmutex, the pending owner may be boosted by the tasks
in the rtmutex waitlist when the pending owner is deboosted
or a task in the waitlist is boosted. This boosting is unrelated,
because the pending owner does not really take the rtmutex.
It is not reasonable.

Example.

time1:
A(high prio) onwers the rtmutex.
B(mid prio) and C (low prio) in the waitlist.

time2
A release the lock, B becomes the pending owner
A(or other high prio task) continues to run. B's prio is lower
than A, so B is just queued at the runqueue.

time3
A or other high prio task sleeps, but we have passed some time
The B and C's prio are changed in the period (time2 ~ time3)
due to boosting or deboosting. Now C has the priority higher
than B. ***Is it reasonable that C has to boost B and help B to
get the rtmutex?

NO!! I think, it is unrelated/unneed boosting before B really
owns the rtmutex. We should give C a change to beat B and
win the rtmutex.

This is the motivation of this patch.

An approach(wrong): when C's priority become higher and B, we deprive
the pending ownership from B and give it to C and wakeup C.
But this approach may lead to livelock.

So my approach: just give pending ownership(candidate ownership)
to C and wakeup C. Thus we have multiple candidate owners(B and C).
Any candidate owner is not boosted until it really owns the rtmutex.

The candidate ownership is assigned to the top waiter always when
1) unlock time
2) the top waiter is changed

If any candidate owner is running and calls try_to_take_rt_mutex(),
it will win unconditionally and really own the lock.

How to indicate a candidate owner?
1) add a variable can_seq in the struct rtmutex, it is increased
when unlock (with waiters queued).
2) when a waiter is assigned candidate ownership:
waiter->cand_seq = rtmutex->cand_seq, waiter->cand_owner = 1;
So a waiter is candidate owner when if and only if
(waiter->cand_owner && waiter->cand_seq == lock->cand_seq)

Other advantage of this patch:
1) The states of a rtmutex are reduced a half, easier to read the code.
2) the codes become shorter.
3) pending owner is not dequeued: they will retain FIFO when it is stolen.
4) like normal mutex, unlock path just do very little work and wakeup candidate owner.
candidate owner dequeue its waiter when it wins the lock.

disadvantage
1) the size of struct rtmutex is slightly larger. (I can send another patch
to reduce it if anyone needs)

Not advantage nor disadvantage
1) Even we support multiple candidate owners, we hardly cause "thundering herd"
the number of candidate owners is likely 1.
2) two APIs are changed.
rt_mutex_owner() will not return pending owner
rt_mutex_next_owner() always return the top owner, it is a candidate owner.
will not return NULL if we only have a pending owner.
I have fixed the code that use these APIs.

need updated after this patch is accepted
1) Document/*
2) the testcase scripts/rt-tester/t4-l2-pi-deboost.tst

Signed-off-by: Lai Jiangshan <[email protected]>
---
include/linux/rtmutex.h | 2
kernel/futex.c | 25 +--
kernel/rtmutex.c | 339 +++++++++++++++++++-----------------------------
kernel/rtmutex_common.h | 20 --
4 files changed, 155 insertions(+), 231 deletions(-)
diff --git a/include/linux/rtmutex.h b/include/linux/rtmutex.h
index 8d522ff..155ddb3 100644
--- a/include/linux/rtmutex.h
+++ b/include/linux/rtmutex.h
@@ -24,11 +24,13 @@ extern int max_lock_depth; /* for sysctl */
* @wait_lock: spinlock to protect the structure
* @wait_list: pilist head to enqueue waiters in priority order
* @owner: the mutex owner
+ * @cand_seq: the sequence number for candidate owners
*/
struct rt_mutex {
raw_spinlock_t wait_lock;
struct plist_head wait_list;
struct task_struct *owner;
+ unsigned long cand_seq; /* don't need to init it! */
#ifdef CONFIG_DEBUG_RT_MUTEXES
int save_state;
const char *name, *file;
diff --git a/kernel/futex.c b/kernel/futex.c
index 6c683b3..6db2b92 100644
--- a/kernel/futex.c
+++ b/kernel/futex.c
@@ -778,15 +778,6 @@ static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_q *this)
new_owner = rt_mutex_next_owner(&pi_state->pi_mutex);

/*
- * This happens when we have stolen the lock and the original
- * pending owner did not enqueue itself back on the rt_mutex.
- * Thats not a tragedy. We know that way, that a lock waiter
- * is on the fly. We make the futex_q waiter the pending owner.
- */
- if (!new_owner)
- new_owner = this->task;
-
- /*
* We pass it to the next owner. (The WAITERS bit is always
* kept enabled while there is PI state around. We must also
* preserve the owner died bit.)
@@ -1508,8 +1499,8 @@ static int fixup_pi_state_owner(u32 __user *uaddr, struct futex_q *q,

/*
* We are here either because we stole the rtmutex from the
- * pending owner or we are the pending owner which failed to
- * get the rtmutex. We have to replace the pending owner TID
+ * candidate owner or we are the candidate owner which failed to
+ * get the rtmutex. We have to replace the candidate owner TID
* in the user space variable. This must be atomic as we have
* to preserve the owner died bit here.
*
@@ -1560,7 +1551,7 @@ retry:

/*
* To handle the page fault we need to drop the hash bucket
- * lock here. That gives the other task (either the pending
+ * lock here. That gives the other task (either the candidate
* owner itself or the task which stole the rtmutex) the
* chance to try the fixup of the pi_state. So once we are
* back from handling the fault we need to check the pi_state
@@ -1647,18 +1638,20 @@ static int fixup_owner(u32 __user *uaddr, int fshared, struct futex_q *q,
/*
* pi_state is incorrect, some other task did a lock steal and
* we returned due to timeout or signal without taking the
- * rt_mutex. Too late. We can access the rt_mutex_owner without
- * locking, as the other task is now blocked on the hash bucket
- * lock. Fix the state up.
+ * rt_mutex. Too late.
*/
+ raw_spin_lock(&q->pi_state->pi_mutex.wait_lock);
owner = rt_mutex_owner(&q->pi_state->pi_mutex);
+ if (!owner)
+ owner = rt_mutex_next_owner(&q->pi_state->pi_mutex);
+ raw_spin_unlock(&q->pi_state->pi_mutex.wait_lock);
ret = fixup_pi_state_owner(uaddr, q, owner, fshared);
goto out;
}

/*
* Paranoia check. If we did not take the lock, then we should not be
- * the owner, nor the pending owner, of the rt_mutex.
+ * the owner of the rt_mutex.
*/
if (rt_mutex_owner(&q->pi_state->pi_mutex) == current)
printk(KERN_ERR "fixup_owner: ret = %d pi-mutex: %p "
diff --git a/kernel/rtmutex.c b/kernel/rtmutex.c
index a960481..f5c8065 100644
--- a/kernel/rtmutex.c
+++ b/kernel/rtmutex.c
@@ -20,41 +20,32 @@
/*
* lock->owner state tracking:
*
- * lock->owner holds the task_struct pointer of the owner. Bit 0 and 1
- * are used to keep track of the "owner is pending" and "lock has
- * waiters" state.
+ * lock->owner holds the task_struct pointer of the owner. Bit 0
+ * are used to keep track of the "lock has waiters" state.
*
- * owner bit1 bit0
- * NULL 0 0 lock is free (fast acquire possible)
- * NULL 0 1 invalid state
- * NULL 1 0 Transitional State*
- * NULL 1 1 invalid state
- * taskpointer 0 0 lock is held (fast release possible)
- * taskpointer 0 1 task is pending owner
- * taskpointer 1 0 lock is held and has waiters
- * taskpointer 1 1 task is pending owner and lock has more waiters
+ * owner bit0
+ * NULL 0 lock is free (fast acquire possible)
+ * NULL 1 lock is free and has waiters and candidate owners*
+ * taskpointer 0 lock is held (fast release possible)
+ * taskpointer 1 lock is held and has waiters
*
- * Pending ownership is assigned to the top (highest priority)
- * waiter of the lock, when the lock is released. The thread is woken
- * up and can now take the lock. Until the lock is taken (bit 0
- * cleared) a competing higher priority thread can steal the lock
- * which puts the woken up thread back on the waiters list.
+ * Candidate ownership is assigned to the top (highest priority) waiter
+ * of the lock, when the lock is released or the top waiter of the lock
+ * is changed. The thread(s) is(are) woken up and can now take the lock.
*
* The fast atomic compare exchange based acquire and release is only
- * possible when bit 0 and 1 of lock->owner are 0.
+ * possible when bit 0 of lock->owner are 0.
*
- * (*) There's a small time where the owner can be NULL and the
- * "lock has waiters" bit is set. This can happen when grabbing the lock.
- * To prevent a cmpxchg of the owner releasing the lock, we need to set this
- * bit before looking at the lock, hence the reason this is a transitional
- * state.
+ * (*) It also can be a transitional state when grabbing the lock
+ * with ->wait_lock is held. To prevent any fast path cmpxchg to the lock,
+ * we need to set the bit0 before looking at the lock, and the owner may be
+ * NULL in this small time, hence this can be a transitional state.
*/

static void
-rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner,
- unsigned long mask)
+rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner)
{
- unsigned long val = (unsigned long)owner | mask;
+ unsigned long val = (unsigned long)owner;

if (rt_mutex_has_waiters(lock))
val |= RT_MUTEX_HAS_WAITERS;
@@ -203,15 +194,15 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
* reached or the state of the chain has changed while we
* dropped the locks.
*/
- if (!waiter || !waiter->task)
+ if (!waiter)
goto out_unlock_pi;

/*
* Check the orig_waiter state. After we dropped the locks,
* the previous owner of the lock might have released the lock
- * and made us the pending owner:
+ * and made us candidate owner:
*/
- if (orig_waiter && !orig_waiter->task)
+ if (orig_waiter && orig_waiter->cand_owner)
goto out_unlock_pi;

/*
@@ -254,6 +245,22 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,

/* Release the task */
raw_spin_unlock_irqrestore(&task->pi_lock, flags);
+ if (!rt_mutex_owner(lock)) {
+ /*
+ * the lock is free and has waiters, set the top waiter
+ * as a new candidate owner when it is not set.
+ */
+ if (top_waiter != rt_mutex_top_waiter(lock)) {
+ top_waiter = rt_mutex_top_waiter(lock);
+ top_waiter->cand_seq = lock->cand_seq;
+ if (!top_waiter->cand_owner) {
+ top_waiter->cand_owner = 1;
+ wake_up_process(top_waiter->task);
+ }
+ }
+ raw_spin_unlock(&lock->wait_lock);
+ goto out_put_task;
+ }
put_task_struct(task);

/* Grab the next task */
@@ -296,78 +303,16 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
}

/*
- * Optimization: check if we can steal the lock from the
- * assigned pending owner [which might not have taken the
- * lock yet]:
- */
-static inline int try_to_steal_lock(struct rt_mutex *lock,
- struct task_struct *task)
-{
- struct task_struct *pendowner = rt_mutex_owner(lock);
- struct rt_mutex_waiter *next;
- unsigned long flags;
-
- if (!rt_mutex_owner_pending(lock))
- return 0;
-
- if (pendowner == task)
- return 1;
-
- raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
- if (task->prio >= pendowner->prio) {
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
- return 0;
- }
-
- /*
- * Check if a waiter is enqueued on the pending owners
- * pi_waiters list. Remove it and readjust pending owners
- * priority.
- */
- if (likely(!rt_mutex_has_waiters(lock))) {
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
- return 1;
- }
-
- /* No chain handling, pending owner is not blocked on anything: */
- next = rt_mutex_top_waiter(lock);
- plist_del(&next->pi_list_entry, &pendowner->pi_waiters);
- __rt_mutex_adjust_prio(pendowner);
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
-
- /*
- * We are going to steal the lock and a waiter was
- * enqueued on the pending owners pi_waiters queue. So
- * we have to enqueue this waiter into
- * task->pi_waiters list. This covers the case,
- * where task is boosted because it holds another
- * lock and gets unboosted because the booster is
- * interrupted, so we would delay a waiter with higher
- * priority as task->normal_prio.
- *
- * Note: in the rare case of a SCHED_OTHER task changing
- * its priority and thus stealing the lock, next->task
- * might be task:
- */
- if (likely(next->task != task)) {
- raw_spin_lock_irqsave(&task->pi_lock, flags);
- plist_add(&next->pi_list_entry, &task->pi_waiters);
- __rt_mutex_adjust_prio(task);
- raw_spin_unlock_irqrestore(&task->pi_lock, flags);
- }
- return 1;
-}
-
-/*
* Try to take an rt-mutex
*
- * This fails
- * - when the lock has a real owner
- * - when a different pending owner exists and has higher priority than current
- *
* Must be called with lock->wait_lock held.
+ *
+ * @lock: the lock to be acquired.
+ * @task: the task which want to acquire the lock
+ * @waiter: the waiter queued to the lock's wait list. (could be NULL)
*/
-static int try_to_take_rt_mutex(struct rt_mutex *lock)
+static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
+ struct rt_mutex_waiter *waiter)
{
/*
* We have to be careful here if the atomic speedups are
@@ -390,15 +335,73 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
*/
mark_rt_mutex_waiters(lock);

- if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
+ if (rt_mutex_owner(lock))
return 0;

+ /*
+ * It is a queued waiter.
+ *
+ * Is it a candidate owner? If it is, it will win unconditionally.
+ * And it defeats the other candidate owner(s) (if any) and
+ * steal lock from them.
+ */
+ if (waiter) {
+ /* candidate owner? */
+ if (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)
+ goto get_lock;
+
+ /*
+ * top waiter must be a candidate owner.
+ * But checking it again is not a bad thing.
+ */
+ if (waiter == rt_mutex_top_waiter(lock))
+ goto get_lock;
+ }
+
+ /*
+ * Does it defeat the candidate owner(s) and steal lock from them?
+ *
+ * Note: in the rare case of a task is boosted but its waiter has not
+ * been requeued in the lock's wait list yet, thus it can also steal
+ * the lock.
+ */
+ if (rt_mutex_has_waiters(lock)) {
+ if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio)
+ return 0;
+ }
+
+get_lock:
+ if (waiter || rt_mutex_has_waiters(lock)) {
+ unsigned long flags;
+ struct rt_mutex_waiter *top;
+
+ raw_spin_lock_irqsave(&task->pi_lock, flags);
+
+ /* remove the queued waiter. */
+ if (waiter) {
+ plist_del(&waiter->list_entry, &lock->wait_list);
+ task->pi_blocked_on = NULL;
+ }
+
+ /*
+ * We have to enqueue the top waiter(if have) into
+ * task->pi_waiters list and would get boost from it.
+ */
+ if (rt_mutex_has_waiters(lock)) {
+ top = rt_mutex_top_waiter(lock);
+ top->pi_list_entry.prio = top->list_entry.prio;
+ plist_add(&top->pi_list_entry, &task->pi_waiters);
+ __rt_mutex_adjust_prio(task);
+ }
+ raw_spin_unlock_irqrestore(&task->pi_lock, flags);
+ }
+
/* We got the lock. */
debug_rt_mutex_lock(lock);

- rt_mutex_set_owner(lock, current, 0);
+ rt_mutex_set_owner(lock, task);

- rt_mutex_deadlock_account_lock(lock, current);
+ rt_mutex_deadlock_account_lock(lock, task);

return 1;
}
@@ -424,6 +427,7 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
__rt_mutex_adjust_prio(task);
waiter->task = task;
waiter->lock = lock;
+ waiter->cand_owner = 0;
plist_node_init(&waiter->list_entry, task->prio);
plist_node_init(&waiter->pi_list_entry, task->prio);

@@ -436,6 +440,9 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,

raw_spin_unlock_irqrestore(&task->pi_lock, flags);

+ if (!owner)
+ return 0;
+
if (waiter == rt_mutex_top_waiter(lock)) {
raw_spin_lock_irqsave(&owner->pi_lock, flags);
plist_del(&top_waiter->pi_list_entry, &owner->pi_waiters);
@@ -472,21 +479,19 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
/*
* Wake up the next waiter on the lock.
*
- * Remove the top waiter from the current tasks waiter list and from
- * the lock waiter list. Set it as pending owner. Then wake it up.
+ * Remove the top waiter from the current tasks waiter list.
+ * Set it as a candidate owner. Then wake it up.
*
* Called with lock->wait_lock held.
*/
static void wakeup_next_waiter(struct rt_mutex *lock)
{
struct rt_mutex_waiter *waiter;
- struct task_struct *pendowner;
unsigned long flags;

raw_spin_lock_irqsave(&current->pi_lock, flags);

waiter = rt_mutex_top_waiter(lock);
- plist_del(&waiter->list_entry, &lock->wait_list);

/*
* Remove it from current->pi_waiters. We do not adjust a
@@ -495,37 +500,14 @@ static void wakeup_next_waiter(struct rt_mutex *lock)
* lock->wait_lock.
*/
plist_del(&waiter->pi_list_entry, &current->pi_waiters);
- pendowner = waiter->task;
- waiter->task = NULL;
+ waiter->cand_owner = 1;
+ waiter->cand_seq = ++lock->cand_seq;

- rt_mutex_set_owner(lock, pendowner, RT_MUTEX_OWNER_PENDING);
+ rt_mutex_set_owner(lock, NULL);

raw_spin_unlock_irqrestore(&current->pi_lock, flags);

- /*
- * Clear the pi_blocked_on variable and enqueue a possible
- * waiter into the pi_waiters list of the pending owner. This
- * prevents that in case the pending owner gets unboosted a
- * waiter with higher priority than pending-owner->normal_prio
- * is blocked on the unboosted (pending) owner.
- */
- raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
-
- WARN_ON(!pendowner->pi_blocked_on);
- WARN_ON(pendowner->pi_blocked_on != waiter);
- WARN_ON(pendowner->pi_blocked_on->lock != lock);
-
- pendowner->pi_blocked_on = NULL;
-
- if (rt_mutex_has_waiters(lock)) {
- struct rt_mutex_waiter *next;
-
- next = rt_mutex_top_waiter(lock);
- plist_add(&next->pi_list_entry, &pendowner->pi_waiters);
- }
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
-
- wake_up_process(pendowner);
+ wake_up_process(waiter->task);
}

/*
@@ -543,11 +525,13 @@ static void remove_waiter(struct rt_mutex *lock,

raw_spin_lock_irqsave(&current->pi_lock, flags);
plist_del(&waiter->list_entry, &lock->wait_list);
- waiter->task = NULL;
current->pi_blocked_on = NULL;
raw_spin_unlock_irqrestore(&current->pi_lock, flags);

- if (first && owner != current) {
+ if (!owner)
+ return;
+
+ if (first) {

raw_spin_lock_irqsave(&owner->pi_lock, flags);

@@ -614,22 +598,21 @@ void rt_mutex_adjust_pi(struct task_struct *task)
* or TASK_UNINTERRUPTIBLE)
* @timeout: the pre-initialized and started timer, or NULL for none
* @waiter: the pre-initialized rt_mutex_waiter
- * @detect_deadlock: passed to task_blocks_on_rt_mutex
*
* lock->wait_lock must be held by the caller.
*/
static int __sched
__rt_mutex_slowlock(struct rt_mutex *lock, int state,
struct hrtimer_sleeper *timeout,
- struct rt_mutex_waiter *waiter,
- int detect_deadlock)
+ struct rt_mutex_waiter *waiter)
{
int ret = 0;

for (;;) {
/* Try to acquire the lock: */
- if (try_to_take_rt_mutex(lock))
+ if (try_to_take_rt_mutex(lock, current, waiter))
break;
+ waiter->cand_owner = 0;

/*
* TASK_INTERRUPTIBLE checks for signals and
@@ -645,39 +628,11 @@ __rt_mutex_slowlock(struct rt_mutex *lock, int state,
break;
}

- /*
- * waiter->task is NULL the first time we come here and
- * when we have been woken up by the previous owner
- * but the lock got stolen by a higher prio task.
- */
- if (!waiter->task) {
- ret = task_blocks_on_rt_mutex(lock, waiter, current,
- detect_deadlock);
- /*
- * If we got woken up by the owner then start loop
- * all over without going into schedule to try
- * to get the lock now:
- */
- if (unlikely(!waiter->task)) {
- /*
- * Reset the return value. We might
- * have returned with -EDEADLK and the
- * owner released the lock while we
- * were walking the pi chain.
- */
- ret = 0;
- continue;
- }
- if (unlikely(ret))
- break;
- }
-
raw_spin_unlock(&lock->wait_lock);

debug_rt_mutex_print_deadlock(waiter);

- if (waiter->task)
- schedule_rt_mutex(lock);
+ schedule_rt_mutex(lock);

raw_spin_lock(&lock->wait_lock);
set_current_state(state);
@@ -698,12 +653,11 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
int ret = 0;

debug_rt_mutex_init_waiter(&waiter);
- waiter.task = NULL;

raw_spin_lock(&lock->wait_lock);

/* Try to acquire the lock again: */
- if (try_to_take_rt_mutex(lock)) {
+ if (try_to_take_rt_mutex(lock, current, NULL)) {
raw_spin_unlock(&lock->wait_lock);
return 0;
}
@@ -717,12 +671,14 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
timeout->task = NULL;
}

- ret = __rt_mutex_slowlock(lock, state, timeout, &waiter,
- detect_deadlock);
+ ret = task_blocks_on_rt_mutex(lock, &waiter, current, detect_deadlock);
+
+ if (likely(!ret))
+ ret = __rt_mutex_slowlock(lock, state, timeout, &waiter);

set_current_state(TASK_RUNNING);

- if (unlikely(waiter.task))
+ if (unlikely(ret))
remove_waiter(lock, &waiter);

/*
@@ -737,14 +693,6 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
if (unlikely(timeout))
hrtimer_cancel(&timeout->timer);

- /*
- * Readjust priority, when we did not get the lock. We might
- * have been the pending owner and boosted. Since we did not
- * take the lock, the PI boost has to go.
- */
- if (unlikely(ret))
- rt_mutex_adjust_prio(current);
-
debug_rt_mutex_free_waiter(&waiter);

return ret;
@@ -762,7 +710,7 @@ rt_mutex_slowtrylock(struct rt_mutex *lock)

if (likely(rt_mutex_owner(lock) != current)) {

- ret = try_to_take_rt_mutex(lock);
+ ret = try_to_take_rt_mutex(lock, current, NULL);
/*
* try_to_take_rt_mutex() sets the lock waiters
* bit unconditionally. Clean this up.
@@ -992,7 +940,7 @@ void rt_mutex_init_proxy_locked(struct rt_mutex *lock,
{
__rt_mutex_init(lock, NULL);
debug_rt_mutex_proxy_lock(lock, proxy_owner);
- rt_mutex_set_owner(lock, proxy_owner, 0);
+ rt_mutex_set_owner(lock, proxy_owner);
rt_mutex_deadlock_account_lock(lock, proxy_owner);
}

@@ -1008,7 +956,7 @@ void rt_mutex_proxy_unlock(struct rt_mutex *lock,
struct task_struct *proxy_owner)
{
debug_rt_mutex_proxy_unlock(lock);
- rt_mutex_set_owner(lock, NULL, 0);
+ rt_mutex_set_owner(lock, NULL);
rt_mutex_deadlock_account_unlock(proxy_owner);
}

@@ -1034,20 +982,14 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,

raw_spin_lock(&lock->wait_lock);

- mark_rt_mutex_waiters(lock);
-
- if (!rt_mutex_owner(lock) || try_to_steal_lock(lock, task)) {
- /* We got the lock for task. */
- debug_rt_mutex_lock(lock);
- rt_mutex_set_owner(lock, task, 0);
+ if (try_to_take_rt_mutex(lock, task, NULL)) {
raw_spin_unlock(&lock->wait_lock);
- rt_mutex_deadlock_account_lock(lock, task);
return 1;
}

ret = task_blocks_on_rt_mutex(lock, waiter, task, detect_deadlock);

- if (ret && !waiter->task) {
+ if (ret && waiter->cand_owner) {
/*
* Reset the return value. We might have
* returned with -EDEADLK and the owner
@@ -1056,6 +998,10 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,
*/
ret = 0;
}
+
+ if (unlikely(ret))
+ remove_waiter(lock, waiter);
+
raw_spin_unlock(&lock->wait_lock);

debug_rt_mutex_print_deadlock(waiter);
@@ -1110,12 +1056,11 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,

set_current_state(TASK_INTERRUPTIBLE);

- ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter,
- detect_deadlock);
+ ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter);

set_current_state(TASK_RUNNING);

- if (unlikely(waiter->task))
+ if (unlikely(ret))
remove_waiter(lock, waiter);

/*
@@ -1126,13 +1071,5 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,

raw_spin_unlock(&lock->wait_lock);

- /*
- * Readjust priority, when we did not get the lock. We might have been
- * the pending owner and boosted. Since we did not take the lock, the
- * PI boost has to go.
- */
- if (unlikely(ret))
- rt_mutex_adjust_prio(current);
-
return ret;
}
diff --git a/kernel/rtmutex_common.h b/kernel/rtmutex_common.h
index 97a2f81..178d659 100644
--- a/kernel/rtmutex_common.h
+++ b/kernel/rtmutex_common.h
@@ -43,12 +43,16 @@ extern void schedule_rt_mutex_test(struct rt_mutex *lock);
* @list_entry: pi node to enqueue into the mutex waiters list
* @pi_list_entry: pi node to enqueue into the mutex owner waiters list
* @task: task reference to the blocked task
+ * @cand_owner: did it became candidate owner
+ * @cand_seq: sequence number for checking for candidate owner
*/
struct rt_mutex_waiter {
struct plist_node list_entry;
struct plist_node pi_list_entry;
struct task_struct *task;
struct rt_mutex *lock;
+ int cand_owner;
+ unsigned long cand_seq;
#ifdef CONFIG_DEBUG_RT_MUTEXES
unsigned long ip;
struct pid *deadlock_task_pid;
@@ -91,9 +95,8 @@ task_top_pi_waiter(struct task_struct *p)
/*
* lock->owner state tracking:
*/
-#define RT_MUTEX_OWNER_PENDING 1UL
-#define RT_MUTEX_HAS_WAITERS 2UL
-#define RT_MUTEX_OWNER_MASKALL 3UL
+#define RT_MUTEX_HAS_WAITERS 1UL
+#define RT_MUTEX_OWNER_MASKALL 1UL

static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
{
@@ -101,17 +104,6 @@ static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
((unsigned long)lock->owner & ~RT_MUTEX_OWNER_MASKALL);
}

-static inline struct task_struct *rt_mutex_real_owner(struct rt_mutex *lock)
-{
- return (struct task_struct *)
- ((unsigned long)lock->owner & ~RT_MUTEX_HAS_WAITERS);
-}
-
-static inline unsigned long rt_mutex_owner_pending(struct rt_mutex *lock)
-{
- return (unsigned long)lock->owner & RT_MUTEX_OWNER_PENDING;
-}
-
/*
* PI-futex support (proxy locking functions, etc.):
*/
diff --git a/scripts/rt-tester/check-all.sh b/scripts/rt-tester/check-all.sh
old mode 100644
new mode 100755
diff --git a/scripts/rt-tester/rt-tester.py b/scripts/rt-tester/rt-tester.py
old mode 100644
new mode 100755


2010-12-14 14:01:16

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Tue, 2010-12-14 at 17:04 +0800, Lai Jiangshan wrote:
> Current rtmutex just supports one pending owner only,
> this patch make any rtmutex can have multiple candidate owners.
>
> In this changelog I use "pending owner" and "candidate owner"
> without distinguishing, they are the same meaning. But I
> think the name "candidate owner" is more suitable, so I
> only use "candidate owner" in the code.
>
> In current rtmutex, the pending owner may be boosted by the tasks
> in the rtmutex waitlist when the pending owner is deboosted
> or a task in the waitlist is boosted. This boosting is unrelated,
> because the pending owner does not really take the rtmutex.
> It is not reasonable.
>
> Example.
>
> time1:
> A(high prio) onwers the rtmutex.
> B(mid prio) and C (low prio) in the waitlist.
>
> time2
> A release the lock, B becomes the pending owner
> A(or other high prio task) continues to run. B's prio is lower
> than A, so B is just queued at the runqueue.

>
> time3
> A or other high prio task sleeps, but we have passed some time
> The B and C's prio are changed in the period (time2 ~ time3)
> due to boosting or deboosting. Now C has the priority higher
> than B. ***Is it reasonable that C has to boost B and help B to
> get the rtmutex?

Interesting that we missed this point.

>
> NO!! I think, it is unrelated/unneed boosting before B really
> owns the rtmutex. We should give C a change to beat B and
> win the rtmutex.

Agreed.

>
> This is the motivation of this patch.
>
> An approach(wrong): when C's priority become higher and B, we deprive
> the pending ownership from B and give it to C and wakeup C.
> But this approach may lead to livelock.

I'm curious to how this can cause a livelock. I'm not doubting you, but
I just woke up, and I'm only half way through my first cup of coffee.

>
> So my approach: just give pending ownership(candidate ownership)
> to C and wakeup C. Thus we have multiple candidate owners(B and C).
> Any candidate owner is not boosted until it really owns the rtmutex.
>
> The candidate ownership is assigned to the top waiter always when
> 1) unlock time
> 2) the top waiter is changed

(not looking at the code itself yet) How is a candidate ownership
changed when the top waiter is changed? Or is that only when the lock
has been unlocked and the current candidate hasn't taken it yet?

>
> If any candidate owner is running and calls try_to_take_rt_mutex(),
> it will win unconditionally and really own the lock.

(still not looking at code :-) What happens if another high prio
process comes along and tries to take the lock before the candidate gets
there. That high prio task should still steal the lock. The candidate
will lose it then.

>
> How to indicate a candidate owner?
> 1) add a variable can_seq in the struct rtmutex, it is increased
> when unlock (with waiters queued).
> 2) when a waiter is assigned candidate ownership:
> waiter->cand_seq = rtmutex->cand_seq, waiter->cand_owner = 1;
> So a waiter is candidate owner when if and only if
> (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)

Again, what happens when another high prio task comes along and takes
the lock? Does cand_owner turn to zero? (Almost done with my coffee,
then I'll look at code. Or maybe after a second cup, rtmutex requires
two cups to review code).

>
> Other advantage of this patch:
> 1) The states of a rtmutex are reduced a half, easier to read the code.

This is good.

> 2) the codes become shorter.

This is also good.

> 3) pending owner is not dequeued: they will retain FIFO when it is stolen.

I don't quite understand this part.

> 4) like normal mutex, unlock path just do very little work and wakeup candidate owner.
> candidate owner dequeue its waiter when it wins the lock.

Hmm, this seems tricky. I kind of remember going this path and hitting
issues before.

>
> disadvantage
> 1) the size of struct rtmutex is slightly larger. (I can send another patch
> to reduce it if anyone needs)

If we do go this way, we would want to shrink this struct. The -rt patch
(which this came from) turns almost all spinlocks into rtmutexes, so any
small increase in this struct will result in a huge bloat of the kernel.

>
> Not advantage nor disadvantage
> 1) Even we support multiple candidate owners, we hardly cause "thundering herd"
> the number of candidate owners is likely 1.

This is because a candidate is only created when the lock is first
unlocked or it is unlocked and a lower prio waiter is boosted above the
current candidate, correct?

> 2) two APIs are changed.
> rt_mutex_owner() will not return pending owner
> rt_mutex_next_owner() always return the top owner, it is a candidate owner.
> will not return NULL if we only have a pending owner.
> I have fixed the code that use these APIs.
>
> need updated after this patch is accepted
> 1) Document/*
> 2) the testcase scripts/rt-tester/t4-l2-pi-deboost.tst
>
> Signed-off-by: Lai Jiangshan <[email protected]>

I'll put this on my todo list to review.

Thanks!

-- Steve

> ---
> include/linux/rtmutex.h | 2
> kernel/futex.c | 25 +--
> kernel/rtmutex.c | 339 +++++++++++++++++++-----------------------------
> kernel/rtmutex_common.h | 20 --
> 4 files changed, 155 insertions(+), 231 deletions(-)
> diff --git a/include/linux/rtmutex.h b/include/linux/rtmutex.h
> index 8d522ff..155ddb3 100644
> --- a/include/linux/rtmutex.h
> +++ b/include/linux/rtmutex.h
> @@ -24,11 +24,13 @@ extern int max_lock_depth; /* for sysctl */
> * @wait_lock: spinlock to protect the structure
> * @wait_list: pilist head to enqueue waiters in priority order
> * @owner: the mutex owner
> + * @cand_seq: the sequence number for candidate owners
> */
> struct rt_mutex {
> raw_spinlock_t wait_lock;
> struct plist_head wait_list;
> struct task_struct *owner;
> + unsigned long cand_seq; /* don't need to init it! */
> #ifdef CONFIG_DEBUG_RT_MUTEXES
> int save_state;
> const char *name, *file;
> diff --git a/kernel/futex.c b/kernel/futex.c
> index 6c683b3..6db2b92 100644
> --- a/kernel/futex.c
> +++ b/kernel/futex.c
> @@ -778,15 +778,6 @@ static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_q *this)
> new_owner = rt_mutex_next_owner(&pi_state->pi_mutex);
>
> /*
> - * This happens when we have stolen the lock and the original
> - * pending owner did not enqueue itself back on the rt_mutex.
> - * Thats not a tragedy. We know that way, that a lock waiter
> - * is on the fly. We make the futex_q waiter the pending owner.
> - */
> - if (!new_owner)
> - new_owner = this->task;
> -
> - /*
> * We pass it to the next owner. (The WAITERS bit is always
> * kept enabled while there is PI state around. We must also
> * preserve the owner died bit.)
> @@ -1508,8 +1499,8 @@ static int fixup_pi_state_owner(u32 __user *uaddr, struct futex_q *q,
>
> /*
> * We are here either because we stole the rtmutex from the
> - * pending owner or we are the pending owner which failed to
> - * get the rtmutex. We have to replace the pending owner TID
> + * candidate owner or we are the candidate owner which failed to
> + * get the rtmutex. We have to replace the candidate owner TID
> * in the user space variable. This must be atomic as we have
> * to preserve the owner died bit here.
> *
> @@ -1560,7 +1551,7 @@ retry:
>
> /*
> * To handle the page fault we need to drop the hash bucket
> - * lock here. That gives the other task (either the pending
> + * lock here. That gives the other task (either the candidate
> * owner itself or the task which stole the rtmutex) the
> * chance to try the fixup of the pi_state. So once we are
> * back from handling the fault we need to check the pi_state
> @@ -1647,18 +1638,20 @@ static int fixup_owner(u32 __user *uaddr, int fshared, struct futex_q *q,
> /*
> * pi_state is incorrect, some other task did a lock steal and
> * we returned due to timeout or signal without taking the
> - * rt_mutex. Too late. We can access the rt_mutex_owner without
> - * locking, as the other task is now blocked on the hash bucket
> - * lock. Fix the state up.
> + * rt_mutex. Too late.
> */
> + raw_spin_lock(&q->pi_state->pi_mutex.wait_lock);
> owner = rt_mutex_owner(&q->pi_state->pi_mutex);
> + if (!owner)
> + owner = rt_mutex_next_owner(&q->pi_state->pi_mutex);
> + raw_spin_unlock(&q->pi_state->pi_mutex.wait_lock);
> ret = fixup_pi_state_owner(uaddr, q, owner, fshared);
> goto out;
> }
>
> /*
> * Paranoia check. If we did not take the lock, then we should not be
> - * the owner, nor the pending owner, of the rt_mutex.
> + * the owner of the rt_mutex.
> */
> if (rt_mutex_owner(&q->pi_state->pi_mutex) == current)
> printk(KERN_ERR "fixup_owner: ret = %d pi-mutex: %p "
> diff --git a/kernel/rtmutex.c b/kernel/rtmutex.c
> index a960481..f5c8065 100644
> --- a/kernel/rtmutex.c
> +++ b/kernel/rtmutex.c
> @@ -20,41 +20,32 @@
> /*
> * lock->owner state tracking:
> *
> - * lock->owner holds the task_struct pointer of the owner. Bit 0 and 1
> - * are used to keep track of the "owner is pending" and "lock has
> - * waiters" state.
> + * lock->owner holds the task_struct pointer of the owner. Bit 0
> + * are used to keep track of the "lock has waiters" state.
> *
> - * owner bit1 bit0
> - * NULL 0 0 lock is free (fast acquire possible)
> - * NULL 0 1 invalid state
> - * NULL 1 0 Transitional State*
> - * NULL 1 1 invalid state
> - * taskpointer 0 0 lock is held (fast release possible)
> - * taskpointer 0 1 task is pending owner
> - * taskpointer 1 0 lock is held and has waiters
> - * taskpointer 1 1 task is pending owner and lock has more waiters
> + * owner bit0
> + * NULL 0 lock is free (fast acquire possible)
> + * NULL 1 lock is free and has waiters and candidate owners*
> + * taskpointer 0 lock is held (fast release possible)
> + * taskpointer 1 lock is held and has waiters
> *
> - * Pending ownership is assigned to the top (highest priority)
> - * waiter of the lock, when the lock is released. The thread is woken
> - * up and can now take the lock. Until the lock is taken (bit 0
> - * cleared) a competing higher priority thread can steal the lock
> - * which puts the woken up thread back on the waiters list.
> + * Candidate ownership is assigned to the top (highest priority) waiter
> + * of the lock, when the lock is released or the top waiter of the lock
> + * is changed. The thread(s) is(are) woken up and can now take the lock.
> *
> * The fast atomic compare exchange based acquire and release is only
> - * possible when bit 0 and 1 of lock->owner are 0.
> + * possible when bit 0 of lock->owner are 0.
> *
> - * (*) There's a small time where the owner can be NULL and the
> - * "lock has waiters" bit is set. This can happen when grabbing the lock.
> - * To prevent a cmpxchg of the owner releasing the lock, we need to set this
> - * bit before looking at the lock, hence the reason this is a transitional
> - * state.
> + * (*) It also can be a transitional state when grabbing the lock
> + * with ->wait_lock is held. To prevent any fast path cmpxchg to the lock,
> + * we need to set the bit0 before looking at the lock, and the owner may be
> + * NULL in this small time, hence this can be a transitional state.
> */
>
> static void
> -rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner,
> - unsigned long mask)
> +rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner)
> {
> - unsigned long val = (unsigned long)owner | mask;
> + unsigned long val = (unsigned long)owner;
>
> if (rt_mutex_has_waiters(lock))
> val |= RT_MUTEX_HAS_WAITERS;
> @@ -203,15 +194,15 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
> * reached or the state of the chain has changed while we
> * dropped the locks.
> */
> - if (!waiter || !waiter->task)
> + if (!waiter)
> goto out_unlock_pi;
>
> /*
> * Check the orig_waiter state. After we dropped the locks,
> * the previous owner of the lock might have released the lock
> - * and made us the pending owner:
> + * and made us candidate owner:
> */
> - if (orig_waiter && !orig_waiter->task)
> + if (orig_waiter && orig_waiter->cand_owner)
> goto out_unlock_pi;
>
> /*
> @@ -254,6 +245,22 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
>
> /* Release the task */
> raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> + if (!rt_mutex_owner(lock)) {
> + /*
> + * the lock is free and has waiters, set the top waiter
> + * as a new candidate owner when it is not set.
> + */
> + if (top_waiter != rt_mutex_top_waiter(lock)) {
> + top_waiter = rt_mutex_top_waiter(lock);
> + top_waiter->cand_seq = lock->cand_seq;
> + if (!top_waiter->cand_owner) {
> + top_waiter->cand_owner = 1;
> + wake_up_process(top_waiter->task);
> + }
> + }
> + raw_spin_unlock(&lock->wait_lock);
> + goto out_put_task;
> + }
> put_task_struct(task);
>
> /* Grab the next task */
> @@ -296,78 +303,16 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
> }
>
> /*
> - * Optimization: check if we can steal the lock from the
> - * assigned pending owner [which might not have taken the
> - * lock yet]:
> - */
> -static inline int try_to_steal_lock(struct rt_mutex *lock,
> - struct task_struct *task)
> -{
> - struct task_struct *pendowner = rt_mutex_owner(lock);
> - struct rt_mutex_waiter *next;
> - unsigned long flags;
> -
> - if (!rt_mutex_owner_pending(lock))
> - return 0;
> -
> - if (pendowner == task)
> - return 1;
> -
> - raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
> - if (task->prio >= pendowner->prio) {
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> - return 0;
> - }
> -
> - /*
> - * Check if a waiter is enqueued on the pending owners
> - * pi_waiters list. Remove it and readjust pending owners
> - * priority.
> - */
> - if (likely(!rt_mutex_has_waiters(lock))) {
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> - return 1;
> - }
> -
> - /* No chain handling, pending owner is not blocked on anything: */
> - next = rt_mutex_top_waiter(lock);
> - plist_del(&next->pi_list_entry, &pendowner->pi_waiters);
> - __rt_mutex_adjust_prio(pendowner);
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> -
> - /*
> - * We are going to steal the lock and a waiter was
> - * enqueued on the pending owners pi_waiters queue. So
> - * we have to enqueue this waiter into
> - * task->pi_waiters list. This covers the case,
> - * where task is boosted because it holds another
> - * lock and gets unboosted because the booster is
> - * interrupted, so we would delay a waiter with higher
> - * priority as task->normal_prio.
> - *
> - * Note: in the rare case of a SCHED_OTHER task changing
> - * its priority and thus stealing the lock, next->task
> - * might be task:
> - */
> - if (likely(next->task != task)) {
> - raw_spin_lock_irqsave(&task->pi_lock, flags);
> - plist_add(&next->pi_list_entry, &task->pi_waiters);
> - __rt_mutex_adjust_prio(task);
> - raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> - }
> - return 1;
> -}
> -
> -/*
> * Try to take an rt-mutex
> *
> - * This fails
> - * - when the lock has a real owner
> - * - when a different pending owner exists and has higher priority than current
> - *
> * Must be called with lock->wait_lock held.
> + *
> + * @lock: the lock to be acquired.
> + * @task: the task which want to acquire the lock
> + * @waiter: the waiter queued to the lock's wait list. (could be NULL)
> */
> -static int try_to_take_rt_mutex(struct rt_mutex *lock)
> +static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
> + struct rt_mutex_waiter *waiter)
> {
> /*
> * We have to be careful here if the atomic speedups are
> @@ -390,15 +335,73 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
> */
> mark_rt_mutex_waiters(lock);
>
> - if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
> + if (rt_mutex_owner(lock))
> return 0;
>
> + /*
> + * It is a queued waiter.
> + *
> + * Is it a candidate owner? If it is, it will win unconditionally.
> + * And it defeats the other candidate owner(s) (if any) and
> + * steal lock from them.
> + */
> + if (waiter) {
> + /* candidate owner? */
> + if (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)
> + goto get_lock;
> +
> + /*
> + * top waiter must be a candidate owner.
> + * But checking it again is not a bad thing.
> + */
> + if (waiter == rt_mutex_top_waiter(lock))
> + goto get_lock;
> + }
> +
> + /*
> + * Does it defeat the candidate owner(s) and steal lock from them?
> + *
> + * Note: in the rare case of a task is boosted but its waiter has not
> + * been requeued in the lock's wait list yet, thus it can also steal
> + * the lock.
> + */
> + if (rt_mutex_has_waiters(lock)) {
> + if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio)
> + return 0;
> + }
> +
> +get_lock:
> + if (waiter || rt_mutex_has_waiters(lock)) {
> + unsigned long flags;
> + struct rt_mutex_waiter *top;
> +
> + raw_spin_lock_irqsave(&task->pi_lock, flags);
> +
> + /* remove the queued waiter. */
> + if (waiter) {
> + plist_del(&waiter->list_entry, &lock->wait_list);
> + task->pi_blocked_on = NULL;
> + }
> +
> + /*
> + * We have to enqueue the top waiter(if have) into
> + * task->pi_waiters list and would get boost from it.
> + */
> + if (rt_mutex_has_waiters(lock)) {
> + top = rt_mutex_top_waiter(lock);
> + top->pi_list_entry.prio = top->list_entry.prio;
> + plist_add(&top->pi_list_entry, &task->pi_waiters);
> + __rt_mutex_adjust_prio(task);
> + }
> + raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> + }
> +
> /* We got the lock. */
> debug_rt_mutex_lock(lock);
>
> - rt_mutex_set_owner(lock, current, 0);
> + rt_mutex_set_owner(lock, task);
>
> - rt_mutex_deadlock_account_lock(lock, current);
> + rt_mutex_deadlock_account_lock(lock, task);
>
> return 1;
> }
> @@ -424,6 +427,7 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
> __rt_mutex_adjust_prio(task);
> waiter->task = task;
> waiter->lock = lock;
> + waiter->cand_owner = 0;
> plist_node_init(&waiter->list_entry, task->prio);
> plist_node_init(&waiter->pi_list_entry, task->prio);
>
> @@ -436,6 +440,9 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
>
> raw_spin_unlock_irqrestore(&task->pi_lock, flags);
>
> + if (!owner)
> + return 0;
> +
> if (waiter == rt_mutex_top_waiter(lock)) {
> raw_spin_lock_irqsave(&owner->pi_lock, flags);
> plist_del(&top_waiter->pi_list_entry, &owner->pi_waiters);
> @@ -472,21 +479,19 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
> /*
> * Wake up the next waiter on the lock.
> *
> - * Remove the top waiter from the current tasks waiter list and from
> - * the lock waiter list. Set it as pending owner. Then wake it up.
> + * Remove the top waiter from the current tasks waiter list.
> + * Set it as a candidate owner. Then wake it up.
> *
> * Called with lock->wait_lock held.
> */
> static void wakeup_next_waiter(struct rt_mutex *lock)
> {
> struct rt_mutex_waiter *waiter;
> - struct task_struct *pendowner;
> unsigned long flags;
>
> raw_spin_lock_irqsave(&current->pi_lock, flags);
>
> waiter = rt_mutex_top_waiter(lock);
> - plist_del(&waiter->list_entry, &lock->wait_list);
>
> /*
> * Remove it from current->pi_waiters. We do not adjust a
> @@ -495,37 +500,14 @@ static void wakeup_next_waiter(struct rt_mutex *lock)
> * lock->wait_lock.
> */
> plist_del(&waiter->pi_list_entry, &current->pi_waiters);
> - pendowner = waiter->task;
> - waiter->task = NULL;
> + waiter->cand_owner = 1;
> + waiter->cand_seq = ++lock->cand_seq;
>
> - rt_mutex_set_owner(lock, pendowner, RT_MUTEX_OWNER_PENDING);
> + rt_mutex_set_owner(lock, NULL);
>
> raw_spin_unlock_irqrestore(&current->pi_lock, flags);
>
> - /*
> - * Clear the pi_blocked_on variable and enqueue a possible
> - * waiter into the pi_waiters list of the pending owner. This
> - * prevents that in case the pending owner gets unboosted a
> - * waiter with higher priority than pending-owner->normal_prio
> - * is blocked on the unboosted (pending) owner.
> - */
> - raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
> -
> - WARN_ON(!pendowner->pi_blocked_on);
> - WARN_ON(pendowner->pi_blocked_on != waiter);
> - WARN_ON(pendowner->pi_blocked_on->lock != lock);
> -
> - pendowner->pi_blocked_on = NULL;
> -
> - if (rt_mutex_has_waiters(lock)) {
> - struct rt_mutex_waiter *next;
> -
> - next = rt_mutex_top_waiter(lock);
> - plist_add(&next->pi_list_entry, &pendowner->pi_waiters);
> - }
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> -
> - wake_up_process(pendowner);
> + wake_up_process(waiter->task);
> }
>
> /*
> @@ -543,11 +525,13 @@ static void remove_waiter(struct rt_mutex *lock,
>
> raw_spin_lock_irqsave(&current->pi_lock, flags);
> plist_del(&waiter->list_entry, &lock->wait_list);
> - waiter->task = NULL;
> current->pi_blocked_on = NULL;
> raw_spin_unlock_irqrestore(&current->pi_lock, flags);
>
> - if (first && owner != current) {
> + if (!owner)
> + return;
> +
> + if (first) {
>
> raw_spin_lock_irqsave(&owner->pi_lock, flags);
>
> @@ -614,22 +598,21 @@ void rt_mutex_adjust_pi(struct task_struct *task)
> * or TASK_UNINTERRUPTIBLE)
> * @timeout: the pre-initialized and started timer, or NULL for none
> * @waiter: the pre-initialized rt_mutex_waiter
> - * @detect_deadlock: passed to task_blocks_on_rt_mutex
> *
> * lock->wait_lock must be held by the caller.
> */
> static int __sched
> __rt_mutex_slowlock(struct rt_mutex *lock, int state,
> struct hrtimer_sleeper *timeout,
> - struct rt_mutex_waiter *waiter,
> - int detect_deadlock)
> + struct rt_mutex_waiter *waiter)
> {
> int ret = 0;
>
> for (;;) {
> /* Try to acquire the lock: */
> - if (try_to_take_rt_mutex(lock))
> + if (try_to_take_rt_mutex(lock, current, waiter))
> break;
> + waiter->cand_owner = 0;
>
> /*
> * TASK_INTERRUPTIBLE checks for signals and
> @@ -645,39 +628,11 @@ __rt_mutex_slowlock(struct rt_mutex *lock, int state,
> break;
> }
>
> - /*
> - * waiter->task is NULL the first time we come here and
> - * when we have been woken up by the previous owner
> - * but the lock got stolen by a higher prio task.
> - */
> - if (!waiter->task) {
> - ret = task_blocks_on_rt_mutex(lock, waiter, current,
> - detect_deadlock);
> - /*
> - * If we got woken up by the owner then start loop
> - * all over without going into schedule to try
> - * to get the lock now:
> - */
> - if (unlikely(!waiter->task)) {
> - /*
> - * Reset the return value. We might
> - * have returned with -EDEADLK and the
> - * owner released the lock while we
> - * were walking the pi chain.
> - */
> - ret = 0;
> - continue;
> - }
> - if (unlikely(ret))
> - break;
> - }
> -
> raw_spin_unlock(&lock->wait_lock);
>
> debug_rt_mutex_print_deadlock(waiter);
>
> - if (waiter->task)
> - schedule_rt_mutex(lock);
> + schedule_rt_mutex(lock);
>
> raw_spin_lock(&lock->wait_lock);
> set_current_state(state);
> @@ -698,12 +653,11 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
> int ret = 0;
>
> debug_rt_mutex_init_waiter(&waiter);
> - waiter.task = NULL;
>
> raw_spin_lock(&lock->wait_lock);
>
> /* Try to acquire the lock again: */
> - if (try_to_take_rt_mutex(lock)) {
> + if (try_to_take_rt_mutex(lock, current, NULL)) {
> raw_spin_unlock(&lock->wait_lock);
> return 0;
> }
> @@ -717,12 +671,14 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
> timeout->task = NULL;
> }
>
> - ret = __rt_mutex_slowlock(lock, state, timeout, &waiter,
> - detect_deadlock);
> + ret = task_blocks_on_rt_mutex(lock, &waiter, current, detect_deadlock);
> +
> + if (likely(!ret))
> + ret = __rt_mutex_slowlock(lock, state, timeout, &waiter);
>
> set_current_state(TASK_RUNNING);
>
> - if (unlikely(waiter.task))
> + if (unlikely(ret))
> remove_waiter(lock, &waiter);
>
> /*
> @@ -737,14 +693,6 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
> if (unlikely(timeout))
> hrtimer_cancel(&timeout->timer);
>
> - /*
> - * Readjust priority, when we did not get the lock. We might
> - * have been the pending owner and boosted. Since we did not
> - * take the lock, the PI boost has to go.
> - */
> - if (unlikely(ret))
> - rt_mutex_adjust_prio(current);
> -
> debug_rt_mutex_free_waiter(&waiter);
>
> return ret;
> @@ -762,7 +710,7 @@ rt_mutex_slowtrylock(struct rt_mutex *lock)
>
> if (likely(rt_mutex_owner(lock) != current)) {
>
> - ret = try_to_take_rt_mutex(lock);
> + ret = try_to_take_rt_mutex(lock, current, NULL);
> /*
> * try_to_take_rt_mutex() sets the lock waiters
> * bit unconditionally. Clean this up.
> @@ -992,7 +940,7 @@ void rt_mutex_init_proxy_locked(struct rt_mutex *lock,
> {
> __rt_mutex_init(lock, NULL);
> debug_rt_mutex_proxy_lock(lock, proxy_owner);
> - rt_mutex_set_owner(lock, proxy_owner, 0);
> + rt_mutex_set_owner(lock, proxy_owner);
> rt_mutex_deadlock_account_lock(lock, proxy_owner);
> }
>
> @@ -1008,7 +956,7 @@ void rt_mutex_proxy_unlock(struct rt_mutex *lock,
> struct task_struct *proxy_owner)
> {
> debug_rt_mutex_proxy_unlock(lock);
> - rt_mutex_set_owner(lock, NULL, 0);
> + rt_mutex_set_owner(lock, NULL);
> rt_mutex_deadlock_account_unlock(proxy_owner);
> }
>
> @@ -1034,20 +982,14 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,
>
> raw_spin_lock(&lock->wait_lock);
>
> - mark_rt_mutex_waiters(lock);
> -
> - if (!rt_mutex_owner(lock) || try_to_steal_lock(lock, task)) {
> - /* We got the lock for task. */
> - debug_rt_mutex_lock(lock);
> - rt_mutex_set_owner(lock, task, 0);
> + if (try_to_take_rt_mutex(lock, task, NULL)) {
> raw_spin_unlock(&lock->wait_lock);
> - rt_mutex_deadlock_account_lock(lock, task);
> return 1;
> }
>
> ret = task_blocks_on_rt_mutex(lock, waiter, task, detect_deadlock);
>
> - if (ret && !waiter->task) {
> + if (ret && waiter->cand_owner) {
> /*
> * Reset the return value. We might have
> * returned with -EDEADLK and the owner
> @@ -1056,6 +998,10 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,
> */
> ret = 0;
> }
> +
> + if (unlikely(ret))
> + remove_waiter(lock, waiter);
> +
> raw_spin_unlock(&lock->wait_lock);
>
> debug_rt_mutex_print_deadlock(waiter);
> @@ -1110,12 +1056,11 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,
>
> set_current_state(TASK_INTERRUPTIBLE);
>
> - ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter,
> - detect_deadlock);
> + ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter);
>
> set_current_state(TASK_RUNNING);
>
> - if (unlikely(waiter->task))
> + if (unlikely(ret))
> remove_waiter(lock, waiter);
>
> /*
> @@ -1126,13 +1071,5 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,
>
> raw_spin_unlock(&lock->wait_lock);
>
> - /*
> - * Readjust priority, when we did not get the lock. We might have been
> - * the pending owner and boosted. Since we did not take the lock, the
> - * PI boost has to go.
> - */
> - if (unlikely(ret))
> - rt_mutex_adjust_prio(current);
> -
> return ret;
> }
> diff --git a/kernel/rtmutex_common.h b/kernel/rtmutex_common.h
> index 97a2f81..178d659 100644
> --- a/kernel/rtmutex_common.h
> +++ b/kernel/rtmutex_common.h
> @@ -43,12 +43,16 @@ extern void schedule_rt_mutex_test(struct rt_mutex *lock);
> * @list_entry: pi node to enqueue into the mutex waiters list
> * @pi_list_entry: pi node to enqueue into the mutex owner waiters list
> * @task: task reference to the blocked task
> + * @cand_owner: did it became candidate owner
> + * @cand_seq: sequence number for checking for candidate owner
> */
> struct rt_mutex_waiter {
> struct plist_node list_entry;
> struct plist_node pi_list_entry;
> struct task_struct *task;
> struct rt_mutex *lock;
> + int cand_owner;
> + unsigned long cand_seq;
> #ifdef CONFIG_DEBUG_RT_MUTEXES
> unsigned long ip;
> struct pid *deadlock_task_pid;
> @@ -91,9 +95,8 @@ task_top_pi_waiter(struct task_struct *p)
> /*
> * lock->owner state tracking:
> */
> -#define RT_MUTEX_OWNER_PENDING 1UL
> -#define RT_MUTEX_HAS_WAITERS 2UL
> -#define RT_MUTEX_OWNER_MASKALL 3UL
> +#define RT_MUTEX_HAS_WAITERS 1UL
> +#define RT_MUTEX_OWNER_MASKALL 1UL
>
> static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
> {
> @@ -101,17 +104,6 @@ static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
> ((unsigned long)lock->owner & ~RT_MUTEX_OWNER_MASKALL);
> }
>
> -static inline struct task_struct *rt_mutex_real_owner(struct rt_mutex *lock)
> -{
> - return (struct task_struct *)
> - ((unsigned long)lock->owner & ~RT_MUTEX_HAS_WAITERS);
> -}
> -
> -static inline unsigned long rt_mutex_owner_pending(struct rt_mutex *lock)
> -{
> - return (unsigned long)lock->owner & RT_MUTEX_OWNER_PENDING;
> -}
> -
> /*
> * PI-futex support (proxy locking functions, etc.):
> */
> diff --git a/scripts/rt-tester/check-all.sh b/scripts/rt-tester/check-all.sh
> old mode 100644
> new mode 100755
> diff --git a/scripts/rt-tester/rt-tester.py b/scripts/rt-tester/rt-tester.py
> old mode 100644
> new mode 100755

2010-12-14 16:44:08

by Lai Jiangshan

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Tue, Dec 14, 2010 at 10:01 PM, Steven Rostedt <[email protected]> wrote:

>
>>
>> This is the motivation of this patch.
>>
>> An approach(wrong): when C's priority become higher and B, we deprive
>> the pending ownership from B and give it to C and wakeup C.
>> But this approach may lead to livelock.
>
> I'm curious to how this can cause a livelock. I'm not doubting you, but
> I just woke up, and I'm only half way through my first cup of coffee.
>

if B is deprived, B has go to sleep again. In rare condition,
B,C 's priority are changed frequent, the pending ownership is
given to B/ deprived from B and given to C/ deprived from C and given to B
......

No task can go forward, it is a kind of livelock.

>>
>> So my approach: just give pending ownership(candidate ownership)
>> to C and wakeup C. Thus we have multiple candidate owners(B and C).
>> Any candidate owner is not boosted until it really owns the rtmutex.
>>
>> The candidate ownership is assigned to the top waiter always when
>> 1) unlock time
>> 2) the top waiter is changed
>
> (not looking at the code itself yet) How is a candidate ownership
> changed when the top waiter is changed?

candidate ownerships will not changed until some task really owns it.
when the top waiter is changed, the top waiter is also assigned
candidate ownership. The candidate ownerships are not stored in
struct rtmutex, they are store in the corresponding waiters.

> Or is that only when the lock
> has been unlocked and the current candidate hasn't taken it yet?

Yes, the current candidate hasn't taken it yet, it is just woken up.

>
>>
>> If any candidate owner is running and calls try_to_take_rt_mutex(),
>> it will win unconditionally and really own the lock.
>
> (still not looking at code :-) ?What happens if another high prio
> process comes along and tries to take the lock before the candidate gets
> there. That high prio task should still steal the lock. The candidate
> will lose it then.

Another higher prio task will steal the lock before the candidate
gets there. The candidates will go to sleep again.

>
>>
>> How to indicate a candidate owner?
>> 1) add a variable can_seq in the struct rtmutex, it is increased
>> ? ?when unlock (with waiters queued).
>> 2) when a waiter is assigned candidate ownership:
>> ? ?waiter->cand_seq = rtmutex->cand_seq, waiter->cand_owner = 1;
>> So a waiter is candidate owner when if and only if
>> (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)
>
> Again, what happens when another high prio task comes along and takes
> the lock? Does cand_owner turn to zero? (Almost done with my coffee,
> then I'll look at code. Or maybe after a second cup, rtmutex requires
> two cups to review code).

After this lucky high prio task release the lock, the lock->cand_seq
is increased. so there is no candidate owner until we assign new
candidate owners. (we will assign a new one immediately)


>
>>
>> Other advantage of this patch:
>> 1) The states of a rtmutex are reduced a half, easier to read the code.
>
> This is good.
>
>> 2) the codes become shorter.
>
> This is also good.
>
>> 3) pending owner is not dequeued: they will retain FIFO when it is stolen.
>
> I don't quite understand this part.

if B C have the some priority and are queued in the waitlist of
a lock which is owned by A. A release it. B is assigned pending
ownership and dequeued. But D steal the lock and B is enqueued.
Now, the top waiter is C, not B. It breaks FIFO.

>
>> 4) like normal mutex, unlock path just do very little work and wakeup candidate owner.
>> ? ?candidate owner dequeue its waiter when it wins the lock.
>
> Hmm, this seems tricky. I kind of remember going this path and hitting
> issues before.

if we have only one pending owner, we must do something for
the pending owner and prepared it to be boosted when needed, so it's
not possible that the unlock path just do very little work.

multiple candidates make things simpler.


>
>>
>> disadvantage
>> 1) the size of struct rtmutex is slightly larger. (I can send another patch
>> ? ?to reduce it if anyone needs)
>
> If we do go this way, we would want to shrink this struct. The -rt patch
> (which this came from) turns almost all spinlocks into rtmutexes, so any
> small increase in this struct will result in a huge bloat of the kernel.
>
>>
>> Not advantage nor disadvantage
>> 1) Even we support multiple candidate owners, we hardly cause "thundering herd"
>> ? ?the number of candidate owners is likely 1.
>
> This is because a candidate is only created when the lock is first
> unlocked or it is unlocked and a lower prio waiter is boosted above the
> current candidate, correct?
>
>> 2) two APIs are changed.
>> ? ?rt_mutex_owner() will not return pending owner
>> ? ?rt_mutex_next_owner() always return the top owner, it is a candidate owner.
>> ? ? ? will not return NULL if we only have a pending owner.
>> ? ?I have fixed the code that use these APIs.
>>
>> need updated after this patch is accepted
>> 1) Document/*
>> 2) the testcase scripts/rt-tester/t4-l2-pi-deboost.tst
>>
>> Signed-off-by: ?Lai Jiangshan <[email protected]>
>
> I'll put this on my todo list to review.
>
> Thanks!
>

Thanks,
Lai

2010-12-14 17:01:00

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Wed, 2010-12-15 at 00:44 +0800, Lai Jiangshan wrote:
> On Tue, Dec 14, 2010 at 10:01 PM, Steven Rostedt <[email protected]> wrote:
>
> >
> >>
> >> This is the motivation of this patch.
> >>
> >> An approach(wrong): when C's priority become higher and B, we deprive
> >> the pending ownership from B and give it to C and wakeup C.
> >> But this approach may lead to livelock.
> >
> > I'm curious to how this can cause a livelock. I'm not doubting you, but
> > I just woke up, and I'm only half way through my first cup of coffee.
> >
>
> if B is deprived, B has go to sleep again. In rare condition,
> B,C 's priority are changed frequent, the pending ownership is
> given to B/ deprived from B and given to C/ deprived from C and given to B
> ......
>
> No task can go forward, it is a kind of livelock.

Hmm, to have C and B change prios frequently, they will need to either
have someone in some for loop doing chrt on them (which is just bad), or
have lots of RT tasks constantly blocking on tasks that they own, which
is also bad. I find this far from a livelock, and any case that does
this would have more issues than causing a livelock here.

>
> >>
> >> So my approach: just give pending ownership(candidate ownership)
> >> to C and wakeup C. Thus we have multiple candidate owners(B and C).
> >> Any candidate owner is not boosted until it really owns the rtmutex.
> >>
> >> The candidate ownership is assigned to the top waiter always when
> >> 1) unlock time
> >> 2) the top waiter is changed
> >
> > (not looking at the code itself yet) How is a candidate ownership
> > changed when the top waiter is changed?
>
> candidate ownerships will not changed until some task really owns it.
> when the top waiter is changed, the top waiter is also assigned
> candidate ownership. The candidate ownerships are not stored in
> struct rtmutex, they are store in the corresponding waiters.
>
> > Or is that only when the lock
> > has been unlocked and the current candidate hasn't taken it yet?
>
> Yes, the current candidate hasn't taken it yet, it is just woken up.
>
> >
> >>
> >> If any candidate owner is running and calls try_to_take_rt_mutex(),
> >> it will win unconditionally and really own the lock.
> >
> > (still not looking at code :-) What happens if another high prio
> > process comes along and tries to take the lock before the candidate gets
> > there. That high prio task should still steal the lock. The candidate
> > will lose it then.
>
> Another higher prio task will steal the lock before the candidate
> gets there. The candidates will go to sleep again.
>
> >
> >>
> >> How to indicate a candidate owner?
> >> 1) add a variable can_seq in the struct rtmutex, it is increased
> >> when unlock (with waiters queued).
> >> 2) when a waiter is assigned candidate ownership:
> >> waiter->cand_seq = rtmutex->cand_seq, waiter->cand_owner = 1;
> >> So a waiter is candidate owner when if and only if
> >> (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)
> >
> > Again, what happens when another high prio task comes along and takes
> > the lock? Does cand_owner turn to zero? (Almost done with my coffee,
> > then I'll look at code. Or maybe after a second cup, rtmutex requires
> > two cups to review code).
>
> After this lucky high prio task release the lock, the lock->cand_seq
> is increased. so there is no candidate owner until we assign new
> candidate owners. (we will assign a new one immediately)

When the lock was released the first time, we got a cand_owner, right?
In which case, lock->cand_seq is set.

But before the cand_owner gets the lock, a high prio task steals it. I
guess that sets lock->cand_seq back to zero. (I really need to go down
and look at the patch ;-)

>
>
> >
> >>
> >> Other advantage of this patch:
> >> 1) The states of a rtmutex are reduced a half, easier to read the code.
> >
> > This is good.
> >
> >> 2) the codes become shorter.
> >
> > This is also good.
> >
> >> 3) pending owner is not dequeued: they will retain FIFO when it is stolen.
> >
> > I don't quite understand this part.
>
> if B C have the some priority and are queued in the waitlist of
> a lock which is owned by A. A release it. B is assigned pending
> ownership and dequeued. But D steal the lock and B is enqueued.
> Now, the top waiter is C, not B. It breaks FIFO.

Ah, thanks for the explanation. I remember this issue.

>
> >
> >> 4) like normal mutex, unlock path just do very little work and wakeup candidate owner.
> >> candidate owner dequeue its waiter when it wins the lock.
> >
> > Hmm, this seems tricky. I kind of remember going this path and hitting
> > issues before.
>
> if we have only one pending owner, we must do something for
> the pending owner and prepared it to be boosted when needed, so it's
> not possible that the unlock path just do very little work.
>
> multiple candidates make things simpler.

I'm almost done with my third cup of coffee, I guess that may give me
the energy to review this patch.

-- Steve

2010-12-14 20:08:46

by Thomas Gleixner

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

B1;2401;0cOn Tue, 14 Dec 2010, Lai Jiangshan wrote:
> Not advantage nor disadvantage
> 1) Even we support multiple candidate owners, we hardly cause "thundering herd"
> the number of candidate owners is likely 1.

I'm not convinced about that function, see comments below

> 2) two APIs are changed.
> rt_mutex_owner() will not return pending owner
> rt_mutex_next_owner() always return the top owner, it is a candidate owner.
> will not return NULL if we only have a pending owner.
> I have fixed the code that use these APIs.

I think we want a separate function which can be used by futex code,
but that's a detail.

> --- a/include/linux/rtmutex.h
> +++ b/include/linux/rtmutex.h
> @@ -24,11 +24,13 @@ extern int max_lock_depth; /* for sysctl */
> * @wait_lock: spinlock to protect the structure
> * @wait_list: pilist head to enqueue waiters in priority order
> * @owner: the mutex owner
> + * @cand_seq: the sequence number for candidate owners
> */
> struct rt_mutex {
> raw_spinlock_t wait_lock;
> struct plist_head wait_list;
> struct task_struct *owner;
> + unsigned long cand_seq; /* don't need to init it! */

I really do not like unitialized stuff. Omitting this init saves ZERO
performance wise and it's going to confuse the hell out of someone
who tries to debug this code.

> /*
> @@ -254,6 +245,22 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
>
> /* Release the task */
> raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> + if (!rt_mutex_owner(lock)) {
> + /*
> + * the lock is free and has waiters, set the top waiter
> + * as a new candidate owner when it is not set.

It took me a while to grok this comment. It should read:

/*
* If the requeue above changed the top waiter, then we need to
* make it a candidate owner and possibly wake it up.
*/

Right ?

> + */
> + if (top_waiter != rt_mutex_top_waiter(lock)) {

Shouldn't this:

- clear cand_owner on the previous top_waiter ?
- increment the cand_seq ?

If we increment cand_seq, then we can get rid of cand_owner completely
as waiter->cand_seq == lock->cand_seq should be sufficient. And it
would automatically prefer the new top waiter and not give the lock to
some random on the fly task.

> + top_waiter = rt_mutex_top_waiter(lock);
> + top_waiter->cand_seq = lock->cand_seq;
> + if (!top_waiter->cand_owner) {
> + top_waiter->cand_owner = 1;
> + wake_up_process(top_waiter->task);
> + }
> + }
> + raw_spin_unlock(&lock->wait_lock);
> + goto out_put_task;

If I understand correctly, then this is the equivalent to the
original breakout based on !task->pi_blocked_on, right ?

> + }
> put_task_struct(task);

> */
> -static int try_to_take_rt_mutex(struct rt_mutex *lock)
> +static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
> + struct rt_mutex_waiter *waiter)
> {
> /*
> * We have to be careful here if the atomic speedups are
> @@ -390,15 +335,73 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
> */
> mark_rt_mutex_waiters(lock);
>
> - if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
> + if (rt_mutex_owner(lock))
> return 0;
>
> + /*
> + * It is a queued waiter.
> + *
> + * Is it a candidate owner? If it is, it will win unconditionally.
> + * And it defeats the other candidate owner(s) (if any) and
> + * steal lock from them.

Why ? If the boost code changes the top waiter and wakes up the new
candidate, then this new one really should get the lock and the old on
the fly candidate should queue itself back.

> + */
> + if (waiter) {
> + /* candidate owner? */
> + if (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)
> + goto get_lock;
> +
> + /*
> + * top waiter must be a candidate owner.
> + * But checking it again is not a bad thing.
> + */
> + if (waiter == rt_mutex_top_waiter(lock))
> + goto get_lock;

Huch ? This should never happen and therefor this should be a
WARN_ON_ONCE(). We really do not put "checking is not a bad thing"
code in such a fragile and complex construct. Something is very wrong
when you ever hit this.

> + }
> +
> + /*
> + * Does it defeat the candidate owner(s) and steal lock from them?
> + *
> + * Note: in the rare case of a task is boosted but its waiter has not
> + * been requeued in the lock's wait list yet, thus it can also steal
> + * the lock.

How should this happen with the new code ? The waiter remains in the
wait list and boosting does not change that. So that comment is
misleading.

> + */
> + if (rt_mutex_has_waiters(lock)) {
> + if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio)
> + return 0;
> + }
> +
> +get_lock:
> + if (waiter || rt_mutex_has_waiters(lock)) {
> + unsigned long flags;
> + struct rt_mutex_waiter *top;
> +
> + raw_spin_lock_irqsave(&task->pi_lock, flags);
> +
> + /* remove the queued waiter. */
> + if (waiter) {
> + plist_del(&waiter->list_entry, &lock->wait_list);
> + task->pi_blocked_on = NULL;
> + }
> +
> + /*
> + * We have to enqueue the top waiter(if have) into
> + * task->pi_waiters list and would get boost from it.
> + */
> + if (rt_mutex_has_waiters(lock)) {
> + top = rt_mutex_top_waiter(lock);
> + top->pi_list_entry.prio = top->list_entry.prio;
> + plist_add(&top->pi_list_entry, &task->pi_waiters);
> + __rt_mutex_adjust_prio(task);

Why should we boost ? Because we can have several candidates on the
fly and we don't care which one is the highest prio candidate? That
looks wrong.

> + }
> + raw_spin_unlock_irqrestore(&task->pi_lock, flags);

> @@ -424,6 +427,7 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
> __rt_mutex_adjust_prio(task);
> waiter->task = task;
> waiter->lock = lock;
> + waiter->cand_owner = 0;

This should init waiter->cand_seq to RTMUTEX_CAND_SEQ_MAX and get rid
of cand_owner.

> + waiter->cand_seq = ++lock->cand_seq;

If we make cand_seq an indicator then we need to do

lock->cand_seq = (lock->cand_seq + 1) & ~RTMUTEX_CAND_SEQ_MAX;

> @@ -1110,12 +1056,11 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,
>
> set_current_state(TASK_INTERRUPTIBLE);
>
> - ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter,
> - detect_deadlock);

Hmm, so we loose the deadlock detection in that case. Not sure whether
it matters though.

All in all I like the idea and I think we should give it a try, but I
definitely want to test a patch against -RT, as this will shake out
all problems in no time.

Thanks,

tglx

2010-12-14 23:16:51

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Tue, 2010-12-14 at 17:04 +0800, Lai Jiangshan wrote:

> /* Release the task */
> raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> + if (!rt_mutex_owner(lock)) {
> + /*
> + * the lock is free and has waiters, set the top waiter
> + * as a new candidate owner when it is not set.
> + */
> + if (top_waiter != rt_mutex_top_waiter(lock)) {
> + top_waiter = rt_mutex_top_waiter(lock);
> + top_waiter->cand_seq = lock->cand_seq;

Shouldn't this be:

top_waiter->cand_seq = ++lock->cand_seq;

?

-- Steve

> + if (!top_waiter->cand_owner) {
> + top_waiter->cand_owner = 1;
> + wake_up_process(top_waiter->task);
> + }
> + }
> + raw_spin_unlock(&lock->wait_lock);
> + goto out_put_task;
> + }
> put_task_struct(task);
>
> /* Grab the next task */
> @@ -296,78 +303,16 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
> }
>

2010-12-15 02:18:41

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Tue, 2010-12-14 at 17:04 +0800, Lai Jiangshan wrote:

OK, I was looking at this in a bit more detail (the coffee finally set
in) and I was at first looking to nuke the cand_owner since it is
redundant to cand_seq. But I think we can nuke the cand_seq instead and
use the top_waiter as the decider.

1) it is less racy, as I show below.
2) we don't need to add any new element to the rt_mutex struct. This is
a win, especially for -rt when that struct exists for every spinlock and
mutex in the kernel.

Maybe I'm missing something, but I wrote some comments below.

> Signed-off-by: Lai Jiangshan <[email protected]>
> ---
> include/linux/rtmutex.h | 2
> kernel/futex.c | 25 +--
> kernel/rtmutex.c | 339 +++++++++++++++++++-----------------------------
> kernel/rtmutex_common.h | 20 --
> 4 files changed, 155 insertions(+), 231 deletions(-)
> diff --git a/include/linux/rtmutex.h b/include/linux/rtmutex.h
> index 8d522ff..155ddb3 100644
> --- a/include/linux/rtmutex.h
> +++ b/include/linux/rtmutex.h
> @@ -24,11 +24,13 @@ extern int max_lock_depth; /* for sysctl */
> * @wait_lock: spinlock to protect the structure
> * @wait_list: pilist head to enqueue waiters in priority order
> * @owner: the mutex owner
> + * @cand_seq: the sequence number for candidate owners
> */
> struct rt_mutex {
> raw_spinlock_t wait_lock;
> struct plist_head wait_list;
> struct task_struct *owner;
> + unsigned long cand_seq; /* don't need to init it! */

And this.

> #ifdef CONFIG_DEBUG_RT_MUTEXES
> int save_state;
> const char *name, *file;
> diff --git a/kernel/futex.c b/kernel/futex.c
> index 6c683b3..6db2b92 100644
> --- a/kernel/futex.c
> +++ b/kernel/futex.c
> @@ -778,15 +778,6 @@ static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_q *this)
> new_owner = rt_mutex_next_owner(&pi_state->pi_mutex);
>
> /*
> - * This happens when we have stolen the lock and the original
> - * pending owner did not enqueue itself back on the rt_mutex.
> - * Thats not a tragedy. We know that way, that a lock waiter
> - * is on the fly. We make the futex_q waiter the pending owner.
> - */
> - if (!new_owner)
> - new_owner = this->task;
> -
> - /*
> * We pass it to the next owner. (The WAITERS bit is always
> * kept enabled while there is PI state around. We must also
> * preserve the owner died bit.)
> @@ -1508,8 +1499,8 @@ static int fixup_pi_state_owner(u32 __user *uaddr, struct futex_q *q,
>
> /*
> * We are here either because we stole the rtmutex from the
> - * pending owner or we are the pending owner which failed to
> - * get the rtmutex. We have to replace the pending owner TID
> + * candidate owner or we are the candidate owner which failed to
> + * get the rtmutex. We have to replace the candidate owner TID
> * in the user space variable. This must be atomic as we have
> * to preserve the owner died bit here.
> *
> @@ -1560,7 +1551,7 @@ retry:
>
> /*
> * To handle the page fault we need to drop the hash bucket
> - * lock here. That gives the other task (either the pending
> + * lock here. That gives the other task (either the candidate
> * owner itself or the task which stole the rtmutex) the
> * chance to try the fixup of the pi_state. So once we are
> * back from handling the fault we need to check the pi_state
> @@ -1647,18 +1638,20 @@ static int fixup_owner(u32 __user *uaddr, int fshared, struct futex_q *q,
> /*
> * pi_state is incorrect, some other task did a lock steal and
> * we returned due to timeout or signal without taking the
> - * rt_mutex. Too late. We can access the rt_mutex_owner without
> - * locking, as the other task is now blocked on the hash bucket
> - * lock. Fix the state up.
> + * rt_mutex. Too late.
> */
> + raw_spin_lock(&q->pi_state->pi_mutex.wait_lock);
> owner = rt_mutex_owner(&q->pi_state->pi_mutex);
> + if (!owner)
> + owner = rt_mutex_next_owner(&q->pi_state->pi_mutex);
> + raw_spin_unlock(&q->pi_state->pi_mutex.wait_lock);
> ret = fixup_pi_state_owner(uaddr, q, owner, fshared);
> goto out;
> }
>
> /*
> * Paranoia check. If we did not take the lock, then we should not be
> - * the owner, nor the pending owner, of the rt_mutex.
> + * the owner of the rt_mutex.
> */
> if (rt_mutex_owner(&q->pi_state->pi_mutex) == current)
> printk(KERN_ERR "fixup_owner: ret = %d pi-mutex: %p "
> diff --git a/kernel/rtmutex.c b/kernel/rtmutex.c
> index a960481..f5c8065 100644
> --- a/kernel/rtmutex.c
> +++ b/kernel/rtmutex.c
> @@ -20,41 +20,32 @@
> /*
> * lock->owner state tracking:
> *
> - * lock->owner holds the task_struct pointer of the owner. Bit 0 and 1
> - * are used to keep track of the "owner is pending" and "lock has
> - * waiters" state.
> + * lock->owner holds the task_struct pointer of the owner. Bit 0
> + * are used to keep track of the "lock has waiters" state.
> *
> - * owner bit1 bit0
> - * NULL 0 0 lock is free (fast acquire possible)
> - * NULL 0 1 invalid state
> - * NULL 1 0 Transitional State*
> - * NULL 1 1 invalid state
> - * taskpointer 0 0 lock is held (fast release possible)
> - * taskpointer 0 1 task is pending owner
> - * taskpointer 1 0 lock is held and has waiters
> - * taskpointer 1 1 task is pending owner and lock has more waiters
> + * owner bit0
> + * NULL 0 lock is free (fast acquire possible)
> + * NULL 1 lock is free and has waiters and candidate owners*
> + * taskpointer 0 lock is held (fast release possible)
> + * taskpointer 1 lock is held and has waiters
> *
> - * Pending ownership is assigned to the top (highest priority)
> - * waiter of the lock, when the lock is released. The thread is woken
> - * up and can now take the lock. Until the lock is taken (bit 0
> - * cleared) a competing higher priority thread can steal the lock
> - * which puts the woken up thread back on the waiters list.
> + * Candidate ownership is assigned to the top (highest priority) waiter
> + * of the lock, when the lock is released or the top waiter of the lock
> + * is changed. The thread(s) is(are) woken up and can now take the lock.
> *
> * The fast atomic compare exchange based acquire and release is only
> - * possible when bit 0 and 1 of lock->owner are 0.
> + * possible when bit 0 of lock->owner are 0.
> *
> - * (*) There's a small time where the owner can be NULL and the
> - * "lock has waiters" bit is set. This can happen when grabbing the lock.
> - * To prevent a cmpxchg of the owner releasing the lock, we need to set this
> - * bit before looking at the lock, hence the reason this is a transitional
> - * state.
> + * (*) It also can be a transitional state when grabbing the lock
> + * with ->wait_lock is held. To prevent any fast path cmpxchg to the lock,
> + * we need to set the bit0 before looking at the lock, and the owner may be
> + * NULL in this small time, hence this can be a transitional state.
> */
>
> static void
> -rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner,
> - unsigned long mask)
> +rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner)
> {
> - unsigned long val = (unsigned long)owner | mask;
> + unsigned long val = (unsigned long)owner;
>
> if (rt_mutex_has_waiters(lock))
> val |= RT_MUTEX_HAS_WAITERS;
> @@ -203,15 +194,15 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
> * reached or the state of the chain has changed while we
> * dropped the locks.
> */
> - if (!waiter || !waiter->task)
> + if (!waiter)
> goto out_unlock_pi;
>
> /*
> * Check the orig_waiter state. After we dropped the locks,
> * the previous owner of the lock might have released the lock
> - * and made us the pending owner:
> + * and made us candidate owner:
> */
> - if (orig_waiter && !orig_waiter->task)
> + if (orig_waiter && orig_waiter->cand_owner)
> goto out_unlock_pi;

This is OK.

>
> /*
> @@ -254,6 +245,22 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
>
> /* Release the task */
> raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> + if (!rt_mutex_owner(lock)) {
> + /*
> + * the lock is free and has waiters, set the top waiter
> + * as a new candidate owner when it is not set.
> + */
> + if (top_waiter != rt_mutex_top_waiter(lock)) {
> + top_waiter = rt_mutex_top_waiter(lock);
> + top_waiter->cand_seq = lock->cand_seq;

No need for the cand_seq.

> + if (!top_waiter->cand_owner) {
> + top_waiter->cand_owner = 1;
> + wake_up_process(top_waiter->task);
> + }
> + }
> + raw_spin_unlock(&lock->wait_lock);
> + goto out_put_task;
> + }
> put_task_struct(task);
>
> /* Grab the next task */
> @@ -296,78 +303,16 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
> }
>
> /*
> - * Optimization: check if we can steal the lock from the
> - * assigned pending owner [which might not have taken the
> - * lock yet]:
> - */
> -static inline int try_to_steal_lock(struct rt_mutex *lock,
> - struct task_struct *task)
> -{
> - struct task_struct *pendowner = rt_mutex_owner(lock);
> - struct rt_mutex_waiter *next;
> - unsigned long flags;
> -
> - if (!rt_mutex_owner_pending(lock))
> - return 0;
> -
> - if (pendowner == task)
> - return 1;
> -
> - raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
> - if (task->prio >= pendowner->prio) {
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> - return 0;
> - }
> -
> - /*
> - * Check if a waiter is enqueued on the pending owners
> - * pi_waiters list. Remove it and readjust pending owners
> - * priority.
> - */
> - if (likely(!rt_mutex_has_waiters(lock))) {
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> - return 1;
> - }
> -
> - /* No chain handling, pending owner is not blocked on anything: */
> - next = rt_mutex_top_waiter(lock);
> - plist_del(&next->pi_list_entry, &pendowner->pi_waiters);
> - __rt_mutex_adjust_prio(pendowner);
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> -
> - /*
> - * We are going to steal the lock and a waiter was
> - * enqueued on the pending owners pi_waiters queue. So
> - * we have to enqueue this waiter into
> - * task->pi_waiters list. This covers the case,
> - * where task is boosted because it holds another
> - * lock and gets unboosted because the booster is
> - * interrupted, so we would delay a waiter with higher
> - * priority as task->normal_prio.
> - *
> - * Note: in the rare case of a SCHED_OTHER task changing
> - * its priority and thus stealing the lock, next->task
> - * might be task:
> - */
> - if (likely(next->task != task)) {
> - raw_spin_lock_irqsave(&task->pi_lock, flags);
> - plist_add(&next->pi_list_entry, &task->pi_waiters);
> - __rt_mutex_adjust_prio(task);
> - raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> - }
> - return 1;
> -}
> -
> -/*
> * Try to take an rt-mutex
> *
> - * This fails
> - * - when the lock has a real owner
> - * - when a different pending owner exists and has higher priority than current
> - *
> * Must be called with lock->wait_lock held.
> + *
> + * @lock: the lock to be acquired.
> + * @task: the task which want to acquire the lock
> + * @waiter: the waiter queued to the lock's wait list. (could be NULL)
> */
> -static int try_to_take_rt_mutex(struct rt_mutex *lock)
> +static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
> + struct rt_mutex_waiter *waiter)
> {
> /*
> * We have to be careful here if the atomic speedups are
> @@ -390,15 +335,73 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
> */
> mark_rt_mutex_waiters(lock);
>
> - if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
> + if (rt_mutex_owner(lock))
> return 0;
>
> + /*
> + * It is a queued waiter.
> + *
> + * Is it a candidate owner? If it is, it will win unconditionally.
> + * And it defeats the other candidate owner(s) (if any) and
> + * steal lock from them.
> + */
> + if (waiter) {
> + /* candidate owner? */
> + if (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)
> + goto get_lock;

I think the below is a better check.

> +
> + /*
> + * top waiter must be a candidate owner.
> + * But checking it again is not a bad thing.
> + */
> + if (waiter == rt_mutex_top_waiter(lock))
> + goto get_lock;

The reason being, is that we could have the following scenario.

task A has lock, and B and C are waiting.

task A lets go of lock and gives it to B which is higher prio than C

task C is boosted and becomes the top_waiter.

before C gets the lock B get boosted too and it is now the top waiter.

Using the top_waiter to decide who gets the lock is better.


> + }
> +
> + /*
> + * Does it defeat the candidate owner(s) and steal lock from them?
> + *
> + * Note: in the rare case of a task is boosted but its waiter has not
> + * been requeued in the lock's wait list yet, thus it can also steal
> + * the lock.
> + */
> + if (rt_mutex_has_waiters(lock)) {
> + if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio)
> + return 0;
> + }
> +
> +get_lock:
> + if (waiter || rt_mutex_has_waiters(lock)) {
> + unsigned long flags;
> + struct rt_mutex_waiter *top;
> +
> + raw_spin_lock_irqsave(&task->pi_lock, flags);
> +
> + /* remove the queued waiter. */
> + if (waiter) {
> + plist_del(&waiter->list_entry, &lock->wait_list);
> + task->pi_blocked_on = NULL;
> + }
> +
> + /*
> + * We have to enqueue the top waiter(if have) into
> + * task->pi_waiters list and would get boost from it.
> + */
> + if (rt_mutex_has_waiters(lock)) {
> + top = rt_mutex_top_waiter(lock);
> + top->pi_list_entry.prio = top->list_entry.prio;
> + plist_add(&top->pi_list_entry, &task->pi_waiters);
> + __rt_mutex_adjust_prio(task);
> + }
> + raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> + }
> +
> /* We got the lock. */
> debug_rt_mutex_lock(lock);
>
> - rt_mutex_set_owner(lock, current, 0);
> + rt_mutex_set_owner(lock, task);
>
> - rt_mutex_deadlock_account_lock(lock, current);
> + rt_mutex_deadlock_account_lock(lock, task);
>
> return 1;
> }
> @@ -424,6 +427,7 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
> __rt_mutex_adjust_prio(task);
> waiter->task = task;
> waiter->lock = lock;
> + waiter->cand_owner = 0;
> plist_node_init(&waiter->list_entry, task->prio);
> plist_node_init(&waiter->pi_list_entry, task->prio);
>
> @@ -436,6 +440,9 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
>
> raw_spin_unlock_irqrestore(&task->pi_lock, flags);
>
> + if (!owner)
> + return 0;
> +
> if (waiter == rt_mutex_top_waiter(lock)) {
> raw_spin_lock_irqsave(&owner->pi_lock, flags);
> plist_del(&top_waiter->pi_list_entry, &owner->pi_waiters);
> @@ -472,21 +479,19 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
> /*
> * Wake up the next waiter on the lock.
> *
> - * Remove the top waiter from the current tasks waiter list and from
> - * the lock waiter list. Set it as pending owner. Then wake it up.
> + * Remove the top waiter from the current tasks waiter list.
> + * Set it as a candidate owner. Then wake it up.
> *
> * Called with lock->wait_lock held.
> */
> static void wakeup_next_waiter(struct rt_mutex *lock)
> {
> struct rt_mutex_waiter *waiter;
> - struct task_struct *pendowner;
> unsigned long flags;
>
> raw_spin_lock_irqsave(&current->pi_lock, flags);
>
> waiter = rt_mutex_top_waiter(lock);
> - plist_del(&waiter->list_entry, &lock->wait_list);
>
> /*
> * Remove it from current->pi_waiters. We do not adjust a
> @@ -495,37 +500,14 @@ static void wakeup_next_waiter(struct rt_mutex *lock)
> * lock->wait_lock.
> */
> plist_del(&waiter->pi_list_entry, &current->pi_waiters);
> - pendowner = waiter->task;
> - waiter->task = NULL;
> + waiter->cand_owner = 1;
> + waiter->cand_seq = ++lock->cand_seq;

I think we can keep the cand_owner, but get rid of the cand_seq. Because
I think the cand_seq is a bit racier. The top_waiter should be one to
decide which task is the best candidate.

-- Steve

>
> - rt_mutex_set_owner(lock, pendowner, RT_MUTEX_OWNER_PENDING);
> + rt_mutex_set_owner(lock, NULL);
>
> raw_spin_unlock_irqrestore(&current->pi_lock, flags);
>
> - /*
> - * Clear the pi_blocked_on variable and enqueue a possible
> - * waiter into the pi_waiters list of the pending owner. This
> - * prevents that in case the pending owner gets unboosted a
> - * waiter with higher priority than pending-owner->normal_prio
> - * is blocked on the unboosted (pending) owner.
> - */
> - raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
> -
> - WARN_ON(!pendowner->pi_blocked_on);
> - WARN_ON(pendowner->pi_blocked_on != waiter);
> - WARN_ON(pendowner->pi_blocked_on->lock != lock);
> -
> - pendowner->pi_blocked_on = NULL;
> -
> - if (rt_mutex_has_waiters(lock)) {
> - struct rt_mutex_waiter *next;
> -
> - next = rt_mutex_top_waiter(lock);
> - plist_add(&next->pi_list_entry, &pendowner->pi_waiters);
> - }
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> -
> - wake_up_process(pendowner);
> + wake_up_process(waiter->task);
> }
>
> /*
> @@ -543,11 +525,13 @@ static void remove_waiter(struct rt_mutex *lock,
>
> raw_spin_lock_irqsave(&current->pi_lock, flags);
> plist_del(&waiter->list_entry, &lock->wait_list);
> - waiter->task = NULL;
> current->pi_blocked_on = NULL;
> raw_spin_unlock_irqrestore(&current->pi_lock, flags);
>
> - if (first && owner != current) {
> + if (!owner)
> + return;
> +
> + if (first) {
>
> raw_spin_lock_irqsave(&owner->pi_lock, flags);
>
> @@ -614,22 +598,21 @@ void rt_mutex_adjust_pi(struct task_struct *task)
> * or TASK_UNINTERRUPTIBLE)
> * @timeout: the pre-initialized and started timer, or NULL for none
> * @waiter: the pre-initialized rt_mutex_waiter
> - * @detect_deadlock: passed to task_blocks_on_rt_mutex
> *
> * lock->wait_lock must be held by the caller.
> */
> static int __sched
> __rt_mutex_slowlock(struct rt_mutex *lock, int state,
> struct hrtimer_sleeper *timeout,
> - struct rt_mutex_waiter *waiter,
> - int detect_deadlock)
> + struct rt_mutex_waiter *waiter)
> {
> int ret = 0;
>
> for (;;) {
> /* Try to acquire the lock: */
> - if (try_to_take_rt_mutex(lock))
> + if (try_to_take_rt_mutex(lock, current, waiter))
> break;
> + waiter->cand_owner = 0;
>
> /*
> * TASK_INTERRUPTIBLE checks for signals and
> @@ -645,39 +628,11 @@ __rt_mutex_slowlock(struct rt_mutex *lock, int state,
> break;
> }
>
> - /*
> - * waiter->task is NULL the first time we come here and
> - * when we have been woken up by the previous owner
> - * but the lock got stolen by a higher prio task.
> - */
> - if (!waiter->task) {
> - ret = task_blocks_on_rt_mutex(lock, waiter, current,
> - detect_deadlock);
> - /*
> - * If we got woken up by the owner then start loop
> - * all over without going into schedule to try
> - * to get the lock now:
> - */
> - if (unlikely(!waiter->task)) {
> - /*
> - * Reset the return value. We might
> - * have returned with -EDEADLK and the
> - * owner released the lock while we
> - * were walking the pi chain.
> - */
> - ret = 0;
> - continue;
> - }
> - if (unlikely(ret))
> - break;
> - }
> -
> raw_spin_unlock(&lock->wait_lock);
>
> debug_rt_mutex_print_deadlock(waiter);
>
> - if (waiter->task)
> - schedule_rt_mutex(lock);
> + schedule_rt_mutex(lock);
>
> raw_spin_lock(&lock->wait_lock);
> set_current_state(state);
> @@ -698,12 +653,11 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
> int ret = 0;
>
> debug_rt_mutex_init_waiter(&waiter);
> - waiter.task = NULL;
>
> raw_spin_lock(&lock->wait_lock);
>
> /* Try to acquire the lock again: */
> - if (try_to_take_rt_mutex(lock)) {
> + if (try_to_take_rt_mutex(lock, current, NULL)) {
> raw_spin_unlock(&lock->wait_lock);
> return 0;
> }
> @@ -717,12 +671,14 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
> timeout->task = NULL;
> }
>
> - ret = __rt_mutex_slowlock(lock, state, timeout, &waiter,
> - detect_deadlock);
> + ret = task_blocks_on_rt_mutex(lock, &waiter, current, detect_deadlock);
> +
> + if (likely(!ret))
> + ret = __rt_mutex_slowlock(lock, state, timeout, &waiter);
>
> set_current_state(TASK_RUNNING);
>
> - if (unlikely(waiter.task))
> + if (unlikely(ret))
> remove_waiter(lock, &waiter);
>
> /*
> @@ -737,14 +693,6 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
> if (unlikely(timeout))
> hrtimer_cancel(&timeout->timer);
>
> - /*
> - * Readjust priority, when we did not get the lock. We might
> - * have been the pending owner and boosted. Since we did not
> - * take the lock, the PI boost has to go.
> - */
> - if (unlikely(ret))
> - rt_mutex_adjust_prio(current);
> -
> debug_rt_mutex_free_waiter(&waiter);
>
> return ret;
> @@ -762,7 +710,7 @@ rt_mutex_slowtrylock(struct rt_mutex *lock)
>
> if (likely(rt_mutex_owner(lock) != current)) {
>
> - ret = try_to_take_rt_mutex(lock);
> + ret = try_to_take_rt_mutex(lock, current, NULL);
> /*
> * try_to_take_rt_mutex() sets the lock waiters
> * bit unconditionally. Clean this up.
> @@ -992,7 +940,7 @@ void rt_mutex_init_proxy_locked(struct rt_mutex *lock,
> {
> __rt_mutex_init(lock, NULL);
> debug_rt_mutex_proxy_lock(lock, proxy_owner);
> - rt_mutex_set_owner(lock, proxy_owner, 0);
> + rt_mutex_set_owner(lock, proxy_owner);
> rt_mutex_deadlock_account_lock(lock, proxy_owner);
> }
>
> @@ -1008,7 +956,7 @@ void rt_mutex_proxy_unlock(struct rt_mutex *lock,
> struct task_struct *proxy_owner)
> {
> debug_rt_mutex_proxy_unlock(lock);
> - rt_mutex_set_owner(lock, NULL, 0);
> + rt_mutex_set_owner(lock, NULL);
> rt_mutex_deadlock_account_unlock(proxy_owner);
> }
>
> @@ -1034,20 +982,14 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,
>
> raw_spin_lock(&lock->wait_lock);
>
> - mark_rt_mutex_waiters(lock);
> -
> - if (!rt_mutex_owner(lock) || try_to_steal_lock(lock, task)) {
> - /* We got the lock for task. */
> - debug_rt_mutex_lock(lock);
> - rt_mutex_set_owner(lock, task, 0);
> + if (try_to_take_rt_mutex(lock, task, NULL)) {
> raw_spin_unlock(&lock->wait_lock);
> - rt_mutex_deadlock_account_lock(lock, task);
> return 1;
> }
>
> ret = task_blocks_on_rt_mutex(lock, waiter, task, detect_deadlock);
>
> - if (ret && !waiter->task) {
> + if (ret && waiter->cand_owner) {
> /*
> * Reset the return value. We might have
> * returned with -EDEADLK and the owner
> @@ -1056,6 +998,10 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,
> */
> ret = 0;
> }
> +
> + if (unlikely(ret))
> + remove_waiter(lock, waiter);
> +
> raw_spin_unlock(&lock->wait_lock);
>
> debug_rt_mutex_print_deadlock(waiter);
> @@ -1110,12 +1056,11 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,
>
> set_current_state(TASK_INTERRUPTIBLE);
>
> - ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter,
> - detect_deadlock);
> + ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter);
>
> set_current_state(TASK_RUNNING);
>
> - if (unlikely(waiter->task))
> + if (unlikely(ret))
> remove_waiter(lock, waiter);
>
> /*
> @@ -1126,13 +1071,5 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,
>
> raw_spin_unlock(&lock->wait_lock);
>
> - /*
> - * Readjust priority, when we did not get the lock. We might have been
> - * the pending owner and boosted. Since we did not take the lock, the
> - * PI boost has to go.
> - */
> - if (unlikely(ret))
> - rt_mutex_adjust_prio(current);
> -
> return ret;
> }
> diff --git a/kernel/rtmutex_common.h b/kernel/rtmutex_common.h
> index 97a2f81..178d659 100644
> --- a/kernel/rtmutex_common.h
> +++ b/kernel/rtmutex_common.h
> @@ -43,12 +43,16 @@ extern void schedule_rt_mutex_test(struct rt_mutex *lock);
> * @list_entry: pi node to enqueue into the mutex waiters list
> * @pi_list_entry: pi node to enqueue into the mutex owner waiters list
> * @task: task reference to the blocked task
> + * @cand_owner: did it became candidate owner
> + * @cand_seq: sequence number for checking for candidate owner
> */
> struct rt_mutex_waiter {
> struct plist_node list_entry;
> struct plist_node pi_list_entry;
> struct task_struct *task;
> struct rt_mutex *lock;
> + int cand_owner;
> + unsigned long cand_seq;
> #ifdef CONFIG_DEBUG_RT_MUTEXES
> unsigned long ip;
> struct pid *deadlock_task_pid;
> @@ -91,9 +95,8 @@ task_top_pi_waiter(struct task_struct *p)
> /*
> * lock->owner state tracking:
> */
> -#define RT_MUTEX_OWNER_PENDING 1UL
> -#define RT_MUTEX_HAS_WAITERS 2UL
> -#define RT_MUTEX_OWNER_MASKALL 3UL
> +#define RT_MUTEX_HAS_WAITERS 1UL
> +#define RT_MUTEX_OWNER_MASKALL 1UL
>
> static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
> {
> @@ -101,17 +104,6 @@ static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
> ((unsigned long)lock->owner & ~RT_MUTEX_OWNER_MASKALL);
> }
>
> -static inline struct task_struct *rt_mutex_real_owner(struct rt_mutex *lock)
> -{
> - return (struct task_struct *)
> - ((unsigned long)lock->owner & ~RT_MUTEX_HAS_WAITERS);
> -}
> -
> -static inline unsigned long rt_mutex_owner_pending(struct rt_mutex *lock)
> -{
> - return (unsigned long)lock->owner & RT_MUTEX_OWNER_PENDING;
> -}
> -
> /*
> * PI-futex support (proxy locking functions, etc.):
> */
> diff --git a/scripts/rt-tester/check-all.sh b/scripts/rt-tester/check-all.sh
> old mode 100644
> new mode 100755
> diff --git a/scripts/rt-tester/rt-tester.py b/scripts/rt-tester/rt-tester.py
> old mode 100644
> new mode 100755

2010-12-15 03:42:33

by Lai Jiangshan

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On 12/15/2010 04:07 AM, Thomas Gleixner wrote:
>> --- a/include/linux/rtmutex.h
>> +++ b/include/linux/rtmutex.h
>> @@ -24,11 +24,13 @@ extern int max_lock_depth; /* for sysctl */
>> * @wait_lock: spinlock to protect the structure
>> * @wait_list: pilist head to enqueue waiters in priority order
>> * @owner: the mutex owner
>> + * @cand_seq: the sequence number for candidate owners
>> */
>> struct rt_mutex {
>> raw_spinlock_t wait_lock;
>> struct plist_head wait_list;
>> struct task_struct *owner;
>> + unsigned long cand_seq; /* don't need to init it! */
>
> I really do not like unitialized stuff. Omitting this init saves ZERO
> performance wise and it's going to confuse the hell out of someone
> who tries to debug this code.

OK, I will init it.

>
>> /*
>> @@ -254,6 +245,22 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
>>
>> /* Release the task */
>> raw_spin_unlock_irqrestore(&task->pi_lock, flags);
>> + if (!rt_mutex_owner(lock)) {
>> + /*
>> + * the lock is free and has waiters, set the top waiter
>> + * as a new candidate owner when it is not set.
>
> It took me a while to grok this comment. It should read:
>
> /*
> * If the requeue above changed the top waiter, then we need to
> * make it a candidate owner and possibly wake it up.
> */
>
> Right ?

You are right.

>
>> + */
>> + if (top_waiter != rt_mutex_top_waiter(lock)) {
>
> Shouldn't this:
>
> - clear cand_owner on the previous top_waiter ?
> - increment the cand_seq ?
>
> If we increment cand_seq, then we can get rid of cand_owner completely
> as waiter->cand_seq == lock->cand_seq should be sufficient. And it
> would automatically prefer the new top waiter and not give the lock to
> some random on the fly task.


If we increment cand_seq, we do deprive candidate ownership from the
original one and we have only one candidate owner.

If we have just one candidate owner at most, we don't need
cand_seq either. we can save it in lock->owner, (like pending owner).

But this may lead to livelock if we do deprive candidate ownership
as I mentioned (the mail I reply to Steven)

Any random on the fly task is a reasonable task for the owner in this
patch.

A(assigned the first candidate ownership), B, C, D, E... are in the
waitlist. But before new owner really take the lock, B, D 's priority
are raised and they became the top waiter at least once,
(D is the top at the end, D, B, A, C, E....) B and D are also
candidate owners. In my patch, A, B, D have the chance to compete
the lock, they are all reasonable because they are candidate owners,
including A!

OK, let's consider the same case but without this patch.
A is assigned the pending ownership. B, C, D, E... are in the
waitlist, But before new owner really take the lock, B, D 's priority
are raised. In the end, only A can complete the lock, B and D
just do boost A and help A. only A!

Summary, the reason I will give the lock to some random on the fly task
which is one of the candidate owners:
1) avoid livelock
2) no worse than the code without this patch.

>
>> + top_waiter = rt_mutex_top_waiter(lock);
>> + top_waiter->cand_seq = lock->cand_seq;
>> + if (!top_waiter->cand_owner) {
>> + top_waiter->cand_owner = 1;
>> + wake_up_process(top_waiter->task);
>> + }
>> + }
>> + raw_spin_unlock(&lock->wait_lock);
>> + goto out_put_task;
>
> If I understand correctly, then this is the equivalent to the
> original breakout based on !task->pi_blocked_on, right ?

Correct, original breakout when !pending_owner_task->pi_bocked_on,
but candidate owners are still in the waitlist and ->pi_bocked_on
still pointers to its waiter. Now we breakout when the lock
has no owner(but has candidate owner(s) on the fly, we don't boost it).

>
>> + }
>> put_task_struct(task);
>
>> */
>> -static int try_to_take_rt_mutex(struct rt_mutex *lock)
>> +static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
>> + struct rt_mutex_waiter *waiter)
>> {
>> /*
>> * We have to be careful here if the atomic speedups are
>> @@ -390,15 +335,73 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
>> */
>> mark_rt_mutex_waiters(lock);
>>
>> - if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
>> + if (rt_mutex_owner(lock))
>> return 0;
>>
>> + /*
>> + * It is a queued waiter.
>> + *
>> + * Is it a candidate owner? If it is, it will win unconditionally.
>> + * And it defeats the other candidate owner(s) (if any) and
>> + * steal lock from them.
>
> Why ? If the boost code changes the top waiter and wakes up the new
> candidate, then this new one really should get the lock and the old on
> the fly candidate should queue itself back.

See above, all candidate owner have the chance to compete the lock.
It is required for avoiding the livelock.

If we have other way to avoiding the livelock, we will use only
one candidate owner, you, Steven and I will very happy.

>
>> + */
>> + if (waiter) {
>> + /* candidate owner? */
>> + if (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)
>> + goto get_lock;
>> +
>> + /*
>> + * top waiter must be a candidate owner.
>> + * But checking it again is not a bad thing.
>> + */
>> + if (waiter == rt_mutex_top_waiter(lock))
>> + goto get_lock;
>
> Huch ? This should never happen and therefor this should be a
> WARN_ON_ONCE(). We really do not put "checking is not a bad thing"
> code in such a fragile and complex construct. Something is very wrong
> when you ever hit this.
>
>> + }
>> +
>> + /*
>> + * Does it defeat the candidate owner(s) and steal lock from them?
>> + *
>> + * Note: in the rare case of a task is boosted but its waiter has not
>> + * been requeued in the lock's wait list yet, thus it can also steal
>> + * the lock.
>
> How should this happen with the new code ? The waiter remains in the
> wait list and boosting does not change that. So that comment is
> misleading.

When someone boosted current task p, but it fail to take the p->pi_block_on->lock->wait_lock,
and current task p success take it and call try_to_take_rt_mutex()
before p->pi_block_on is requeued in the waitlist.
current task should also have chance to win the lock even it is
not candidate owner/top waiter. It will win when it has the higher
priority than the top waiter.


>
>> + */
>> + if (rt_mutex_has_waiters(lock)) {
>> + if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio)
>> + return 0;
>> + }
>> +
>> +get_lock:
>> + if (waiter || rt_mutex_has_waiters(lock)) {
>> + unsigned long flags;
>> + struct rt_mutex_waiter *top;
>> +
>> + raw_spin_lock_irqsave(&task->pi_lock, flags);
>> +
>> + /* remove the queued waiter. */
>> + if (waiter) {
>> + plist_del(&waiter->list_entry, &lock->wait_list);
>> + task->pi_blocked_on = NULL;
>> + }
>> +
>> + /*
>> + * We have to enqueue the top waiter(if have) into
>> + * task->pi_waiters list and would get boost from it.
>> + */
>> + if (rt_mutex_has_waiters(lock)) {
>> + top = rt_mutex_top_waiter(lock);
>> + top->pi_list_entry.prio = top->list_entry.prio;
>> + plist_add(&top->pi_list_entry, &task->pi_waiters);
>> + __rt_mutex_adjust_prio(task);
>
> Why should we boost ? Because we can have several candidates on the
> fly and we don't care which one is the highest prio candidate? That
> looks wrong.

A really owner should (be boosted)/(prepare to be booted) by the waiters.
such code is also needed in original code.

>
>> + }
>> + raw_spin_unlock_irqrestore(&task->pi_lock, flags);
>
>> @@ -424,6 +427,7 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
>> __rt_mutex_adjust_prio(task);
>> waiter->task = task;
>> waiter->lock = lock;
>> + waiter->cand_owner = 0;
>
> This should init waiter->cand_seq to RTMUTEX_CAND_SEQ_MAX and get rid
> of cand_owner.

good! thanks.

>
>> + waiter->cand_seq = ++lock->cand_seq;
>
> If we make cand_seq an indicator then we need to do
>
> lock->cand_seq = (lock->cand_seq + 1) & ~RTMUTEX_CAND_SEQ_MAX;
>
>> @@ -1110,12 +1056,11 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,
>>
>> set_current_state(TASK_INTERRUPTIBLE);
>>
>> - ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter,
>> - detect_deadlock);
>
> Hmm, so we loose the deadlock detection in that case. Not sure whether
> it matters though.
>
> All in all I like the idea and I think we should give it a try, but I
> definitely want to test a patch against -RT, as this will shake out
> all problems in no time.
>

Thanks,
Lai

2010-12-15 04:16:50

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Wed, 2010-12-15 at 11:41 +0800, Lai Jiangshan wrote:

> But this may lead to livelock if we do deprive candidate ownership
> as I mentioned (the mail I reply to Steven)

But I don't really agree with this livelock situation. It seems very
contrived, and as I said in my reply. If something is causing all these
tasks to have their processes boosted (and probably unboosted) there's
much bigger issues at stake here.

>
> Any random on the fly task is a reasonable task for the owner in this
> patch.

Which I disagree with. The highest priority process should be the only
one that gets it.

>
> A(assigned the first candidate ownership), B, C, D, E... are in the
> waitlist. But before new owner really take the lock, B, D 's priority
> are raised and they became the top waiter at least once,
> (D is the top at the end, D, B, A, C, E....) B and D are also
> candidate owners. In my patch, A, B, D have the chance to compete
> the lock, they are all reasonable because they are candidate owners,
> including A!

Only the highest prio process in that list should be a candidate.

>
> OK, let's consider the same case but without this patch.
> A is assigned the pending ownership. B, C, D, E... are in the
> waitlist, But before new owner really take the lock, B, D 's priority
> are raised. In the end, only A can complete the lock, B and D
> just do boost A and help A. only A!

I agree this is wrong, and we welcome the change. But I disagree that if
we wake up both B and D that either one has the chance to get that lock.
If we wake up B and then D gets boosted to a higher priority than B,
then D needs to get that lock. Period! I think you pointed out correctly
that the current code does not do that, but lets not make a "race to the
finish" to get that lock.


>
> Summary, the reason I will give the lock to some random on the fly task
> which is one of the candidate owners:
> 1) avoid livelock

Which I do not see exist.

> 2) no worse than the code without this patch.

Not an excuse. I think you brought up a good point that the current code
has a poor design in the way it handles a waiting process getting
boosted, it should have fair game to the new task. But I also believe
that if we woke this process up, and its higher prio than the current
pending owner, than it should get the lock. It only makes sense in a RT
environment where fair is not fair, or better said... fair is fair
otherwise (FIFO ;-)


>
> >
> >> + top_waiter = rt_mutex_top_waiter(lock);
> >> + top_waiter->cand_seq = lock->cand_seq;
> >> + if (!top_waiter->cand_owner) {
> >> + top_waiter->cand_owner = 1;
> >> + wake_up_process(top_waiter->task);
> >> + }
> >> + }
> >> + raw_spin_unlock(&lock->wait_lock);
> >> + goto out_put_task;
> >
> > If I understand correctly, then this is the equivalent to the
> > original breakout based on !task->pi_blocked_on, right ?
>
> Correct, original breakout when !pending_owner_task->pi_bocked_on,
> but candidate owners are still in the waitlist and ->pi_bocked_on
> still pointers to its waiter. Now we breakout when the lock
> has no owner(but has candidate owner(s) on the fly, we don't boost it).
>
> >
> >> + }
> >> put_task_struct(task);
> >
> >> */
> >> -static int try_to_take_rt_mutex(struct rt_mutex *lock)
> >> +static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
> >> + struct rt_mutex_waiter *waiter)
> >> {
> >> /*
> >> * We have to be careful here if the atomic speedups are
> >> @@ -390,15 +335,73 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
> >> */
> >> mark_rt_mutex_waiters(lock);
> >>
> >> - if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
> >> + if (rt_mutex_owner(lock))
> >> return 0;
> >>
> >> + /*
> >> + * It is a queued waiter.
> >> + *
> >> + * Is it a candidate owner? If it is, it will win unconditionally.
> >> + * And it defeats the other candidate owner(s) (if any) and
> >> + * steal lock from them.
> >
> > Why ? If the boost code changes the top waiter and wakes up the new
> > candidate, then this new one really should get the lock and the old on
> > the fly candidate should queue itself back.
>
> See above, all candidate owner have the chance to compete the lock.
> It is required for avoiding the livelock.

There is no livelock!

>
> If we have other way to avoiding the livelock, we will use only
> one candidate owner, you, Steven and I will very happy.

Explain it again. What you said was:

"In rare condition, B,C 's priority are changed frequent", which is not
something that happens at all. If it does, we have much bigger problems
at stake.

I think your "livelock" scenario is contrived. And its not a livelock
because you will eventual run out of prios to keep boosting B and C.


>
> >
> >> + */
> >> + if (waiter) {
> >> + /* candidate owner? */
> >> + if (waiter->cand_owner && waiter->cand_seq == lock->cand_seq)
> >> + goto get_lock;
> >> +
> >> + /*
> >> + * top waiter must be a candidate owner.
> >> + * But checking it again is not a bad thing.
> >> + */
> >> + if (waiter == rt_mutex_top_waiter(lock))
> >> + goto get_lock;
> >
> > Huch ? This should never happen and therefor this should be a
> > WARN_ON_ONCE(). We really do not put "checking is not a bad thing"
> > code in such a fragile and complex construct. Something is very wrong
> > when you ever hit this.
> >
> >> + }
> >> +
> >> + /*
> >> + * Does it defeat the candidate owner(s) and steal lock from them?
> >> + *
> >> + * Note: in the rare case of a task is boosted but its waiter has not
> >> + * been requeued in the lock's wait list yet, thus it can also steal
> >> + * the lock.
> >
> > How should this happen with the new code ? The waiter remains in the
> > wait list and boosting does not change that. So that comment is
> > misleading.
>
> When someone boosted current task p, but it fail to take the p->pi_block_on->lock->wait_lock,

Nothing boosts p until it has both p->pi_lock and
p->pi_blocked_on->lock->wait_lock, so this scenario does not exist.

> and current task p success take it and call try_to_take_rt_mutex()
> before p->pi_block_on is requeued in the waitlist.
> current task should also have chance to win the lock even it is
> not candidate owner/top waiter. It will win when it has the higher
> priority than the top waiter.

I don't see how this can happen with the way the locks are taken.


>
>
> >
> >> + */
> >> + if (rt_mutex_has_waiters(lock)) {
> >> + if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio)
> >> + return 0;
> >> + }
> >> +
> >> +get_lock:
> >> + if (waiter || rt_mutex_has_waiters(lock)) {
> >> + unsigned long flags;
> >> + struct rt_mutex_waiter *top;
> >> +
> >> + raw_spin_lock_irqsave(&task->pi_lock, flags);
> >> +
> >> + /* remove the queued waiter. */
> >> + if (waiter) {
> >> + plist_del(&waiter->list_entry, &lock->wait_list);
> >> + task->pi_blocked_on = NULL;
> >> + }
> >> +
> >> + /*
> >> + * We have to enqueue the top waiter(if have) into
> >> + * task->pi_waiters list and would get boost from it.
> >> + */
> >> + if (rt_mutex_has_waiters(lock)) {
> >> + top = rt_mutex_top_waiter(lock);
> >> + top->pi_list_entry.prio = top->list_entry.prio;
> >> + plist_add(&top->pi_list_entry, &task->pi_waiters);
> >> + __rt_mutex_adjust_prio(task);
> >
> > Why should we boost ? Because we can have several candidates on the
> > fly and we don't care which one is the highest prio candidate? That
> > looks wrong.
>
> A really owner should (be boosted)/(prepare to be booted) by the waiters.
> such code is also needed in original code.

No one could have boosted any of the waiters because we have all the
waiters blocked_on->lock->wait_list lock.

Thomas is correct.


>
> >
> >> + }
> >> + raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> >
> >> @@ -424,6 +427,7 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
> >> __rt_mutex_adjust_prio(task);
> >> waiter->task = task;
> >> waiter->lock = lock;
> >> + waiter->cand_owner = 0;
> >
> > This should init waiter->cand_seq to RTMUTEX_CAND_SEQ_MAX and get rid
> > of cand_owner.
>
> good! thanks.

As I said in another email, I think we should keep cand_owner (rename it
though) and get rid of cand_seq. Looks to me that we can simply use the
top_waiter to decide who gets the lock.

-- Steve

>
> >
> >> + waiter->cand_seq = ++lock->cand_seq;
> >
> > If we make cand_seq an indicator then we need to do
> >
> > lock->cand_seq = (lock->cand_seq + 1) & ~RTMUTEX_CAND_SEQ_MAX;
> >
> >> @@ -1110,12 +1056,11 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,
> >>
> >> set_current_state(TASK_INTERRUPTIBLE);
> >>
> >> - ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter,
> >> - detect_deadlock);
> >
> > Hmm, so we loose the deadlock detection in that case. Not sure whether
> > it matters though.
> >
> > All in all I like the idea and I think we should give it a try, but I
> > definitely want to test a patch against -RT, as this will shake out
> > all problems in no time.
> >
>
> Thanks,
> Lai

2010-12-15 04:24:32

by Lai Jiangshan

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On 12/15/2010 01:00 AM, Steven Rostedt wrote:
> On Wed, 2010-12-15 at 00:44 +0800, Lai Jiangshan wrote:
>> On Tue, Dec 14, 2010 at 10:01 PM, Steven Rostedt <[email protected]> wrote:
>>
>>>
>>>>
>>>> This is the motivation of this patch.
>>>>
>>>> An approach(wrong): when C's priority become higher and B, we deprive
>>>> the pending ownership from B and give it to C and wakeup C.
>>>> But this approach may lead to livelock.
>>>
>>> I'm curious to how this can cause a livelock. I'm not doubting you, but
>>> I just woke up, and I'm only half way through my first cup of coffee.
>>>
>>
>> if B is deprived, B has go to sleep again. In rare condition,
>> B,C 's priority are changed frequent, the pending ownership is
>> given to B/ deprived from B and given to C/ deprived from C and given to B
>> ......
>>
>> No task can go forward, it is a kind of livelock.
>
> Hmm, to have C and B change prios frequently, they will need to either
> have someone in some for loop doing chrt on them (which is just bad), or
> have lots of RT tasks constantly blocking on tasks that they own, which
> is also bad. I find this far from a livelock, and any case that does
> this would have more issues than causing a livelock here.
>

Hi, Thomas,

Do you think this is a kind of livelock.
If it is not, I will send a much simpler patch.

Thanks,
Lai.

2010-12-15 07:49:02

by Thomas Gleixner

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Wed, 15 Dec 2010, Lai Jiangshan wrote:
> On 12/15/2010 04:07 AM, Thomas Gleixner wrote:
> >> + */
> >> + if (top_waiter != rt_mutex_top_waiter(lock)) {
> >
> > Shouldn't this:
> >
> > - clear cand_owner on the previous top_waiter ?
> > - increment the cand_seq ?
> >
> > If we increment cand_seq, then we can get rid of cand_owner completely
> > as waiter->cand_seq == lock->cand_seq should be sufficient. And it
> > would automatically prefer the new top waiter and not give the lock to
> > some random on the fly task.
>
>
> If we increment cand_seq, we do deprive candidate ownership from the
> original one and we have only one candidate owner.

Which is good.

> If we have just one candidate owner at most, we don't need
> cand_seq either. we can save it in lock->owner, (like pending owner).

Even better.

> But this may lead to livelock if we do deprive candidate ownership
> as I mentioned (the mail I reply to Steven)

I think that's a non realistic scenario. If you have something which
is changing the priority of multiple waiters with high frequency, then
the rtmutex code is the least of your worries.

> Any random on the fly task is a reasonable task for the owner in this
> patch.
>
> A(assigned the first candidate ownership), B, C, D, E... are in the
> waitlist. But before new owner really take the lock, B, D 's priority
> are raised and they became the top waiter at least once,
> (D is the top at the end, D, B, A, C, E....) B and D are also
> candidate owners. In my patch, A, B, D have the chance to compete
> the lock, they are all reasonable because they are candidate owners,
> including A!
>
> OK, let's consider the same case but without this patch.
> A is assigned the pending ownership. B, C, D, E... are in the
> waitlist, But before new owner really take the lock, B, D 's priority
> are raised. In the end, only A can complete the lock, B and D
> just do boost A and help A. only A!

You mean with the current code. Right, that's not pretty but not
really an issue either. I inustrumented that on RT and the situation
happens twice out of 12 Mio.

I'm way more interested in simplifying the code than in handling this
corner case.

> Summary, the reason I will give the lock to some random on the fly task
> which is one of the candidate owners:
> 1) avoid livelock

There is none.

> 2) no worse than the code without this patch.

That's not an argument at all.

> >> -static int try_to_take_rt_mutex(struct rt_mutex *lock)
> >> +static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
> >> + struct rt_mutex_waiter *waiter)
> >> {
> >> /*
> >> * We have to be careful here if the atomic speedups are
> >> @@ -390,15 +335,73 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
> >> */
> >> mark_rt_mutex_waiters(lock);
> >>
> >> - if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
> >> + if (rt_mutex_owner(lock))
> >> return 0;
> >>
> >> + /*
> >> + * It is a queued waiter.
> >> + *
> >> + * Is it a candidate owner? If it is, it will win unconditionally.
> >> + * And it defeats the other candidate owner(s) (if any) and
> >> + * steal lock from them.
> >
> > Why ? If the boost code changes the top waiter and wakes up the new
> > candidate, then this new one really should get the lock and the old on
> > the fly candidate should queue itself back.
>
> See above, all candidate owner have the chance to compete the lock.
> It is required for avoiding the livelock.
>
> If we have other way to avoiding the livelock, we will use only
> one candidate owner, you, Steven and I will very happy.

Have you ever been able to observe that live lock ? If yes, what did
it take to produce it ?

> >> + /*
> >> + * Does it defeat the candidate owner(s) and steal lock from them?
> >> + *
> >> + * Note: in the rare case of a task is boosted but its waiter has not
> >> + * been requeued in the lock's wait list yet, thus it can also steal
> >> + * the lock.
> >
> > How should this happen with the new code ? The waiter remains in the
> > wait list and boosting does not change that. So that comment is
> > misleading.
>
> When someone boosted current task p, but it fail to take the p->pi_block_on->lock->wait_lock,
> and current task p success take it and call try_to_take_rt_mutex()
> before p->pi_block_on is requeued in the waitlist.
> current task should also have chance to win the lock even it is
> not candidate owner/top waiter. It will win when it has the higher
> priority than the top waiter.

Err? If it's not getting the lock then P is hardly the top waiter
simply because the requeue in lock->wait_list did not happen
yet. That's fully serialized.

> >> + /*
> >> + * We have to enqueue the top waiter(if have) into
> >> + * task->pi_waiters list and would get boost from it.
> >> + */
> >> + if (rt_mutex_has_waiters(lock)) {
> >> + top = rt_mutex_top_waiter(lock);
> >> + top->pi_list_entry.prio = top->list_entry.prio;
> >> + plist_add(&top->pi_list_entry, &task->pi_waiters);
> >> + __rt_mutex_adjust_prio(task);
> >
> > Why should we boost ? Because we can have several candidates on the
> > fly and we don't care which one is the highest prio candidate? That
> > looks wrong.
>
> A really owner should (be boosted)/(prepare to be booted) by the waiters.
> such code is also needed in original code.

No, that's only necessary because you allow multiple possible owners
and you can end up with a lock owner which has less priority than the
highest priority waiter. If you restrict that to the highest prio
waiter then no boosting is necessary at all.

Thanks,

tglx

2010-12-15 08:03:00

by Thomas Gleixner

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Tue, 14 Dec 2010, Steven Rostedt wrote:

> On Tue, 2010-12-14 at 17:04 +0800, Lai Jiangshan wrote:
>
> OK, I was looking at this in a bit more detail (the coffee finally set
> in) and I was at first looking to nuke the cand_owner since it is
> redundant to cand_seq. But I think we can nuke the cand_seq instead and
> use the top_waiter as the decider.

So you just use cand_owner (the name sucks) to flag that the waiter
has been woken up either by the boost code or by an unlock. The waiter
clears that flag with waiter->lock->wait_lock held before calling
schedule().

Though I think we do need it at all. wakeup of an already running task
is almost a nop, so having one less state to worry about is good.

Thanks,

tglx

2010-12-15 08:08:58

by Lai Jiangshan

[permalink] [raw]
Subject: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting


In current rtmutex, the pending owner may be boosted by the tasks
in the rtmutex waitlist when the pending owner is deboosted
or a task in the waitlist is boosted. This boosting is unrelated,
because the pending owner does not really take the rtmutex.
It is not reasonable.

Example.

time1:
A(high prio) onwers the rtmutex.
B(mid prio) and C (low prio) in the waitlist.

time2
A release the lock, B becomes the pending owner
A(or other high prio task) continues to run. B's prio is lower
than A, so B is just queued at the runqueue.

time3
A or other high prio task sleeps, but we have passed some time
The B and C's prio are changed in the period (time2 ~ time3)
due to boosting or deboosting. Now C has the priority higher
than B. ***Is it reasonable that C has to boost B and help B to
get the rtmutex?

NO!! I think, it is unrelated/unneed boosting before B really
owns the rtmutex. We should give C a change to beat B and
win the rtmutex.

This is the motivation of this patch. This patch ensures
only the top waiter or higher priority task can take the lock.

How?
1) we don't dequeue the top waiter when unlock, if the top waiter
is changed, the old top waiter will fail and go to sleep.
2) when requiring lock, it will get the lock when it is not locked and:
there is no waiter OR higher priority than waiters OR it is top waiter.
3) In any time, the top waiter is changed, the top waiter will be wokenup.

The algorithm is much simpler than before, no pending owner, no
boosting for pending owner.

Other advantage of this patch:
1) The states of a rtmutex are reduced a half, easier to read the code.
2) the codes become shorter.
3) top waiter is not dequeued until it really take the lock:
they will retain FIFO when it is stolen.

Not advantage nor disadvantage
1) Even we may wakeup multiple waiters(any time when top waiter changed),
we hardly cause "thundering herd",
the number of wokenup task is likely 1 or very little.
2) two APIs are changed.
rt_mutex_owner() will not return pending owner, it will return NULL when
the top waiter is going to take the lock.
rt_mutex_next_owner() always return the top waiter.
will not return NULL if we have waiters
because the top waiter is not dequeued.

I have fixed the code that use these APIs.

need updated after this patch is accepted
1) Document/*
2) the testcase scripts/rt-tester/t4-l2-pi-deboost.tst

Signed-off-by: Lai Jiangshan <[email protected]>
---
kernel/futex.c | 26 +---
kernel/rtmutex.c | 306 ++++++++++++++++--------------------------------
kernel/rtmutex_common.h | 16 --
3 files changed, 116 insertions(+), 232 deletions(-)

diff --git a/kernel/futex.c b/kernel/futex.c
index 6c683b3..5f4ea5f 100644
--- a/kernel/futex.c
+++ b/kernel/futex.c
@@ -775,18 +775,10 @@ static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_q *this)
return -EINVAL;

raw_spin_lock(&pi_state->pi_mutex.wait_lock);
+ /* set new owner to the most possible owner(top waiter). */
new_owner = rt_mutex_next_owner(&pi_state->pi_mutex);

/*
- * This happens when we have stolen the lock and the original
- * pending owner did not enqueue itself back on the rt_mutex.
- * Thats not a tragedy. We know that way, that a lock waiter
- * is on the fly. We make the futex_q waiter the pending owner.
- */
- if (!new_owner)
- new_owner = this->task;
-
- /*
* We pass it to the next owner. (The WAITERS bit is always
* kept enabled while there is PI state around. We must also
* preserve the owner died bit.)
@@ -1508,8 +1500,8 @@ static int fixup_pi_state_owner(u32 __user *uaddr, struct futex_q *q,

/*
* We are here either because we stole the rtmutex from the
- * pending owner or we are the pending owner which failed to
- * get the rtmutex. We have to replace the pending owner TID
+ * most possible owner or we are the most possible owner which
+ * failed to get the rtmutex. We have to replace the newowner TID
* in the user space variable. This must be atomic as we have
* to preserve the owner died bit here.
*
@@ -1560,7 +1552,7 @@ retry:

/*
* To handle the page fault we need to drop the hash bucket
- * lock here. That gives the other task (either the pending
+ * lock here. That gives the other task (either the most possible
* owner itself or the task which stole the rtmutex) the
* chance to try the fixup of the pi_state. So once we are
* back from handling the fault we need to check the pi_state
@@ -1647,18 +1639,20 @@ static int fixup_owner(u32 __user *uaddr, int fshared, struct futex_q *q,
/*
* pi_state is incorrect, some other task did a lock steal and
* we returned due to timeout or signal without taking the
- * rt_mutex. Too late. We can access the rt_mutex_owner without
- * locking, as the other task is now blocked on the hash bucket
- * lock. Fix the state up.
+ * rt_mutex. Too late.
*/
+ raw_spin_lock(&q->pi_state->pi_mutex.wait_lock);
owner = rt_mutex_owner(&q->pi_state->pi_mutex);
+ if (!owner)
+ owner = rt_mutex_next_owner(&q->pi_state->pi_mutex);
+ raw_spin_unlock(&q->pi_state->pi_mutex.wait_lock);
ret = fixup_pi_state_owner(uaddr, q, owner, fshared);
goto out;
}

/*
* Paranoia check. If we did not take the lock, then we should not be
- * the owner, nor the pending owner, of the rt_mutex.
+ * the owner of the rt_mutex.
*/
if (rt_mutex_owner(&q->pi_state->pi_mutex) == current)
printk(KERN_ERR "fixup_owner: ret = %d pi-mutex: %p "
diff --git a/kernel/rtmutex.c b/kernel/rtmutex.c
index a960481..53b9b49 100644
--- a/kernel/rtmutex.c
+++ b/kernel/rtmutex.c
@@ -20,41 +20,29 @@
/*
* lock->owner state tracking:
*
- * lock->owner holds the task_struct pointer of the owner. Bit 0 and 1
- * are used to keep track of the "owner is pending" and "lock has
- * waiters" state.
+ * lock->owner holds the task_struct pointer of the owner. Bit 0
+ * are used to keep track of the "lock has waiters" state.
*
- * owner bit1 bit0
- * NULL 0 0 lock is free (fast acquire possible)
- * NULL 0 1 invalid state
- * NULL 1 0 Transitional State*
- * NULL 1 1 invalid state
- * taskpointer 0 0 lock is held (fast release possible)
- * taskpointer 0 1 task is pending owner
- * taskpointer 1 0 lock is held and has waiters
- * taskpointer 1 1 task is pending owner and lock has more waiters
- *
- * Pending ownership is assigned to the top (highest priority)
- * waiter of the lock, when the lock is released. The thread is woken
- * up and can now take the lock. Until the lock is taken (bit 0
- * cleared) a competing higher priority thread can steal the lock
- * which puts the woken up thread back on the waiters list.
+ * owner bit0
+ * NULL 0 lock is free (fast acquire possible)
+ * NULL 1 lock is free and has waiters and the top waiter
+ * is going to take the lock*
+ * taskpointer 0 lock is held (fast release possible)
+ * taskpointer 1 lock is held and has waiters
*
* The fast atomic compare exchange based acquire and release is only
- * possible when bit 0 and 1 of lock->owner are 0.
+ * possible when bit 0 of lock->owner are 0.
*
- * (*) There's a small time where the owner can be NULL and the
- * "lock has waiters" bit is set. This can happen when grabbing the lock.
- * To prevent a cmpxchg of the owner releasing the lock, we need to set this
- * bit before looking at the lock, hence the reason this is a transitional
- * state.
+ * (*) It also can be a transitional state when grabbing the lock
+ * with ->wait_lock is held. To prevent any fast path cmpxchg to the lock,
+ * we need to set the bit0 before looking at the lock, and the owner may be
+ * NULL in this small time, hence this can be a transitional state.
*/

static void
-rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner,
- unsigned long mask)
+rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner)
{
- unsigned long val = (unsigned long)owner | mask;
+ unsigned long val = (unsigned long)owner;

if (rt_mutex_has_waiters(lock))
val |= RT_MUTEX_HAS_WAITERS;
@@ -203,15 +191,14 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
* reached or the state of the chain has changed while we
* dropped the locks.
*/
- if (!waiter || !waiter->task)
+ if (!waiter)
goto out_unlock_pi;

/*
* Check the orig_waiter state. After we dropped the locks,
* the previous owner of the lock might have released the lock
- * and made us the pending owner:
*/
- if (orig_waiter && !orig_waiter->task)
+ if (orig_waiter && !rt_mutex_owner(orig_lock))
goto out_unlock_pi;

/*
@@ -254,6 +241,17 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,

/* Release the task */
raw_spin_unlock_irqrestore(&task->pi_lock, flags);
+ if (!rt_mutex_owner(lock)) {
+ /*
+ * If the requeue above changed the top waiter, then we need
+ * to wake the new top waiter up to try to get the lock.
+ */
+
+ if (top_waiter != rt_mutex_top_waiter(lock))
+ wake_up_process(rt_mutex_top_waiter(lock)->task);
+ raw_spin_unlock(&lock->wait_lock);
+ goto out_put_task;
+ }
put_task_struct(task);

/* Grab the next task */
@@ -296,78 +294,16 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
}

/*
- * Optimization: check if we can steal the lock from the
- * assigned pending owner [which might not have taken the
- * lock yet]:
- */
-static inline int try_to_steal_lock(struct rt_mutex *lock,
- struct task_struct *task)
-{
- struct task_struct *pendowner = rt_mutex_owner(lock);
- struct rt_mutex_waiter *next;
- unsigned long flags;
-
- if (!rt_mutex_owner_pending(lock))
- return 0;
-
- if (pendowner == task)
- return 1;
-
- raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
- if (task->prio >= pendowner->prio) {
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
- return 0;
- }
-
- /*
- * Check if a waiter is enqueued on the pending owners
- * pi_waiters list. Remove it and readjust pending owners
- * priority.
- */
- if (likely(!rt_mutex_has_waiters(lock))) {
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
- return 1;
- }
-
- /* No chain handling, pending owner is not blocked on anything: */
- next = rt_mutex_top_waiter(lock);
- plist_del(&next->pi_list_entry, &pendowner->pi_waiters);
- __rt_mutex_adjust_prio(pendowner);
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
-
- /*
- * We are going to steal the lock and a waiter was
- * enqueued on the pending owners pi_waiters queue. So
- * we have to enqueue this waiter into
- * task->pi_waiters list. This covers the case,
- * where task is boosted because it holds another
- * lock and gets unboosted because the booster is
- * interrupted, so we would delay a waiter with higher
- * priority as task->normal_prio.
- *
- * Note: in the rare case of a SCHED_OTHER task changing
- * its priority and thus stealing the lock, next->task
- * might be task:
- */
- if (likely(next->task != task)) {
- raw_spin_lock_irqsave(&task->pi_lock, flags);
- plist_add(&next->pi_list_entry, &task->pi_waiters);
- __rt_mutex_adjust_prio(task);
- raw_spin_unlock_irqrestore(&task->pi_lock, flags);
- }
- return 1;
-}
-
-/*
* Try to take an rt-mutex
*
- * This fails
- * - when the lock has a real owner
- * - when a different pending owner exists and has higher priority than current
- *
* Must be called with lock->wait_lock held.
+ *
+ * @lock: the lock to be acquired.
+ * @task: the task which want to acquire the lock
+ * @waiter: the waiter queued to the lock's wait list. (could be NULL)
*/
-static int try_to_take_rt_mutex(struct rt_mutex *lock)
+static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
+ struct rt_mutex_waiter *waiter)
{
/*
* We have to be careful here if the atomic speedups are
@@ -390,15 +326,52 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
*/
mark_rt_mutex_waiters(lock);

- if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
+ if (rt_mutex_owner(lock))
return 0;

+ /*
+ * It will get the lock because at least one of these conditions:
+ * 1) there is no waiter
+ * 2) higher priority than waiters
+ * 3) it is top waiter
+ */
+ if (rt_mutex_has_waiters(lock)) {
+ if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio) {
+ if (!waiter || waiter != rt_mutex_top_waiter(lock))
+ return 0;
+ }
+ }
+
+ if (waiter || rt_mutex_has_waiters(lock)) {
+ unsigned long flags;
+ struct rt_mutex_waiter *top;
+
+ raw_spin_lock_irqsave(&task->pi_lock, flags);
+
+ /* remove the queued waiter. */
+ if (waiter) {
+ plist_del(&waiter->list_entry, &lock->wait_list);
+ task->pi_blocked_on = NULL;
+ }
+
+ /*
+ * We have to enqueue the top waiter(if have) into
+ * task->pi_waiters list and would get boost from it.
+ */
+ if (rt_mutex_has_waiters(lock)) {
+ top = rt_mutex_top_waiter(lock);
+ top->pi_list_entry.prio = top->list_entry.prio;
+ plist_add(&top->pi_list_entry, &task->pi_waiters);
+ }
+ raw_spin_unlock_irqrestore(&task->pi_lock, flags);
+ }
+
/* We got the lock. */
debug_rt_mutex_lock(lock);

- rt_mutex_set_owner(lock, current, 0);
+ rt_mutex_set_owner(lock, task);

- rt_mutex_deadlock_account_lock(lock, current);
+ rt_mutex_deadlock_account_lock(lock, task);

return 1;
}
@@ -436,6 +409,9 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,

raw_spin_unlock_irqrestore(&task->pi_lock, flags);

+ if (!owner)
+ return 0;
+
if (waiter == rt_mutex_top_waiter(lock)) {
raw_spin_lock_irqsave(&owner->pi_lock, flags);
plist_del(&top_waiter->pi_list_entry, &owner->pi_waiters);
@@ -472,21 +448,18 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
/*
* Wake up the next waiter on the lock.
*
- * Remove the top waiter from the current tasks waiter list and from
- * the lock waiter list. Set it as pending owner. Then wake it up.
+ * Remove the top waiter from the current tasks waiter list and wake it up.
*
* Called with lock->wait_lock held.
*/
static void wakeup_next_waiter(struct rt_mutex *lock)
{
struct rt_mutex_waiter *waiter;
- struct task_struct *pendowner;
unsigned long flags;

raw_spin_lock_irqsave(&current->pi_lock, flags);

waiter = rt_mutex_top_waiter(lock);
- plist_del(&waiter->list_entry, &lock->wait_list);

/*
* Remove it from current->pi_waiters. We do not adjust a
@@ -495,37 +468,12 @@ static void wakeup_next_waiter(struct rt_mutex *lock)
* lock->wait_lock.
*/
plist_del(&waiter->pi_list_entry, &current->pi_waiters);
- pendowner = waiter->task;
- waiter->task = NULL;

- rt_mutex_set_owner(lock, pendowner, RT_MUTEX_OWNER_PENDING);
+ rt_mutex_set_owner(lock, NULL);

raw_spin_unlock_irqrestore(&current->pi_lock, flags);

- /*
- * Clear the pi_blocked_on variable and enqueue a possible
- * waiter into the pi_waiters list of the pending owner. This
- * prevents that in case the pending owner gets unboosted a
- * waiter with higher priority than pending-owner->normal_prio
- * is blocked on the unboosted (pending) owner.
- */
- raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
-
- WARN_ON(!pendowner->pi_blocked_on);
- WARN_ON(pendowner->pi_blocked_on != waiter);
- WARN_ON(pendowner->pi_blocked_on->lock != lock);
-
- pendowner->pi_blocked_on = NULL;
-
- if (rt_mutex_has_waiters(lock)) {
- struct rt_mutex_waiter *next;
-
- next = rt_mutex_top_waiter(lock);
- plist_add(&next->pi_list_entry, &pendowner->pi_waiters);
- }
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
-
- wake_up_process(pendowner);
+ wake_up_process(waiter->task);
}

/*
@@ -543,11 +491,13 @@ static void remove_waiter(struct rt_mutex *lock,

raw_spin_lock_irqsave(&current->pi_lock, flags);
plist_del(&waiter->list_entry, &lock->wait_list);
- waiter->task = NULL;
current->pi_blocked_on = NULL;
raw_spin_unlock_irqrestore(&current->pi_lock, flags);

- if (first && owner != current) {
+ if (!owner)
+ return;
+
+ if (first) {

raw_spin_lock_irqsave(&owner->pi_lock, flags);

@@ -614,21 +564,19 @@ void rt_mutex_adjust_pi(struct task_struct *task)
* or TASK_UNINTERRUPTIBLE)
* @timeout: the pre-initialized and started timer, or NULL for none
* @waiter: the pre-initialized rt_mutex_waiter
- * @detect_deadlock: passed to task_blocks_on_rt_mutex
*
* lock->wait_lock must be held by the caller.
*/
static int __sched
__rt_mutex_slowlock(struct rt_mutex *lock, int state,
struct hrtimer_sleeper *timeout,
- struct rt_mutex_waiter *waiter,
- int detect_deadlock)
+ struct rt_mutex_waiter *waiter)
{
int ret = 0;

for (;;) {
/* Try to acquire the lock: */
- if (try_to_take_rt_mutex(lock))
+ if (try_to_take_rt_mutex(lock, current, waiter))
break;

/*
@@ -645,39 +593,11 @@ __rt_mutex_slowlock(struct rt_mutex *lock, int state,
break;
}

- /*
- * waiter->task is NULL the first time we come here and
- * when we have been woken up by the previous owner
- * but the lock got stolen by a higher prio task.
- */
- if (!waiter->task) {
- ret = task_blocks_on_rt_mutex(lock, waiter, current,
- detect_deadlock);
- /*
- * If we got woken up by the owner then start loop
- * all over without going into schedule to try
- * to get the lock now:
- */
- if (unlikely(!waiter->task)) {
- /*
- * Reset the return value. We might
- * have returned with -EDEADLK and the
- * owner released the lock while we
- * were walking the pi chain.
- */
- ret = 0;
- continue;
- }
- if (unlikely(ret))
- break;
- }
-
raw_spin_unlock(&lock->wait_lock);

debug_rt_mutex_print_deadlock(waiter);

- if (waiter->task)
- schedule_rt_mutex(lock);
+ schedule_rt_mutex(lock);

raw_spin_lock(&lock->wait_lock);
set_current_state(state);
@@ -698,12 +618,11 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
int ret = 0;

debug_rt_mutex_init_waiter(&waiter);
- waiter.task = NULL;

raw_spin_lock(&lock->wait_lock);

/* Try to acquire the lock again: */
- if (try_to_take_rt_mutex(lock)) {
+ if (try_to_take_rt_mutex(lock, current, NULL)) {
raw_spin_unlock(&lock->wait_lock);
return 0;
}
@@ -717,12 +636,14 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
timeout->task = NULL;
}

- ret = __rt_mutex_slowlock(lock, state, timeout, &waiter,
- detect_deadlock);
+ ret = task_blocks_on_rt_mutex(lock, &waiter, current, detect_deadlock);
+
+ if (likely(!ret))
+ ret = __rt_mutex_slowlock(lock, state, timeout, &waiter);

set_current_state(TASK_RUNNING);

- if (unlikely(waiter.task))
+ if (unlikely(ret))
remove_waiter(lock, &waiter);

/*
@@ -737,14 +658,6 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
if (unlikely(timeout))
hrtimer_cancel(&timeout->timer);

- /*
- * Readjust priority, when we did not get the lock. We might
- * have been the pending owner and boosted. Since we did not
- * take the lock, the PI boost has to go.
- */
- if (unlikely(ret))
- rt_mutex_adjust_prio(current);
-
debug_rt_mutex_free_waiter(&waiter);

return ret;
@@ -762,7 +675,7 @@ rt_mutex_slowtrylock(struct rt_mutex *lock)

if (likely(rt_mutex_owner(lock) != current)) {

- ret = try_to_take_rt_mutex(lock);
+ ret = try_to_take_rt_mutex(lock, current, NULL);
/*
* try_to_take_rt_mutex() sets the lock waiters
* bit unconditionally. Clean this up.
@@ -992,7 +905,7 @@ void rt_mutex_init_proxy_locked(struct rt_mutex *lock,
{
__rt_mutex_init(lock, NULL);
debug_rt_mutex_proxy_lock(lock, proxy_owner);
- rt_mutex_set_owner(lock, proxy_owner, 0);
+ rt_mutex_set_owner(lock, proxy_owner);
rt_mutex_deadlock_account_lock(lock, proxy_owner);
}

@@ -1008,7 +921,7 @@ void rt_mutex_proxy_unlock(struct rt_mutex *lock,
struct task_struct *proxy_owner)
{
debug_rt_mutex_proxy_unlock(lock);
- rt_mutex_set_owner(lock, NULL, 0);
+ rt_mutex_set_owner(lock, NULL);
rt_mutex_deadlock_account_unlock(proxy_owner);
}

@@ -1034,20 +947,14 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,

raw_spin_lock(&lock->wait_lock);

- mark_rt_mutex_waiters(lock);
-
- if (!rt_mutex_owner(lock) || try_to_steal_lock(lock, task)) {
- /* We got the lock for task. */
- debug_rt_mutex_lock(lock);
- rt_mutex_set_owner(lock, task, 0);
+ if (try_to_take_rt_mutex(lock, task, NULL)) {
raw_spin_unlock(&lock->wait_lock);
- rt_mutex_deadlock_account_lock(lock, task);
return 1;
}

ret = task_blocks_on_rt_mutex(lock, waiter, task, detect_deadlock);

- if (ret && !waiter->task) {
+ if (ret && !rt_mutex_owner(lock)) {
/*
* Reset the return value. We might have
* returned with -EDEADLK and the owner
@@ -1056,6 +963,10 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,
*/
ret = 0;
}
+
+ if (unlikely(ret))
+ remove_waiter(lock, waiter);
+
raw_spin_unlock(&lock->wait_lock);

debug_rt_mutex_print_deadlock(waiter);
@@ -1110,12 +1021,11 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,

set_current_state(TASK_INTERRUPTIBLE);

- ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter,
- detect_deadlock);
+ ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter);

set_current_state(TASK_RUNNING);

- if (unlikely(waiter->task))
+ if (unlikely(ret))
remove_waiter(lock, waiter);

/*
@@ -1126,13 +1036,5 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,

raw_spin_unlock(&lock->wait_lock);

- /*
- * Readjust priority, when we did not get the lock. We might have been
- * the pending owner and boosted. Since we did not take the lock, the
- * PI boost has to go.
- */
- if (unlikely(ret))
- rt_mutex_adjust_prio(current);
-
return ret;
}
diff --git a/kernel/rtmutex_common.h b/kernel/rtmutex_common.h
index 97a2f81..53a66c8 100644
--- a/kernel/rtmutex_common.h
+++ b/kernel/rtmutex_common.h
@@ -91,9 +91,8 @@ task_top_pi_waiter(struct task_struct *p)
/*
* lock->owner state tracking:
*/
-#define RT_MUTEX_OWNER_PENDING 1UL
-#define RT_MUTEX_HAS_WAITERS 2UL
-#define RT_MUTEX_OWNER_MASKALL 3UL
+#define RT_MUTEX_HAS_WAITERS 1UL
+#define RT_MUTEX_OWNER_MASKALL 1UL

static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
{
@@ -101,17 +100,6 @@ static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
((unsigned long)lock->owner & ~RT_MUTEX_OWNER_MASKALL);
}

-static inline struct task_struct *rt_mutex_real_owner(struct rt_mutex *lock)
-{
- return (struct task_struct *)
- ((unsigned long)lock->owner & ~RT_MUTEX_HAS_WAITERS);
-}
-
-static inline unsigned long rt_mutex_owner_pending(struct rt_mutex *lock)
-{
- return (unsigned long)lock->owner & RT_MUTEX_OWNER_PENDING;
-}
-
/*
* PI-futex support (proxy locking functions, etc.):
*/
diff --git a/scripts/rt-tester/check-all.sh b/scripts/rt-tester/check-all.sh
old mode 100644
new mode 100755
diff --git a/scripts/rt-tester/rt-tester.py b/scripts/rt-tester/rt-tester.py
old mode 100644
new mode 100755

2010-12-15 12:07:54

by Thomas Gleixner

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Wed, 15 Dec 2010, Lai Jiangshan wrote:
> + if (waiter || rt_mutex_has_waiters(lock)) {
> + unsigned long flags;
> + struct rt_mutex_waiter *top;
> +
> + raw_spin_lock_irqsave(&task->pi_lock, flags);
> +
> + /* remove the queued waiter. */
> + if (waiter) {
> + plist_del(&waiter->list_entry, &lock->wait_list);
> + task->pi_blocked_on = NULL;
> + }
> +
> + /*
> + * We have to enqueue the top waiter(if have) into
> + * task->pi_waiters list and would get boost from it.

No, we don't get boosted from it. We just have to enqueue it into
pi_waiters list. There is no boosting happening at this point. Please
be very careful with the comments in this code.

Otherwise this looks really interesting. Still this wants to be ported
to -rt and stress tested there.

Thanks,

tglx

2010-12-15 14:02:50

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Wed, 2010-12-15 at 09:02 +0100, Thomas Gleixner wrote:
> On Tue, 14 Dec 2010, Steven Rostedt wrote:
>
> > On Tue, 2010-12-14 at 17:04 +0800, Lai Jiangshan wrote:
> >
> > OK, I was looking at this in a bit more detail (the coffee finally set
> > in) and I was at first looking to nuke the cand_owner since it is
> > redundant to cand_seq. But I think we can nuke the cand_seq instead and
> > use the top_waiter as the decider.
>
> So you just use cand_owner (the name sucks) to flag that the waiter
> has been woken up either by the boost code or by an unlock. The waiter
> clears that flag with waiter->lock->wait_lock held before calling
> schedule().
>
> Though I think we do need it at all. wakeup of an already running task
> is almost a nop, so having one less state to worry about is good.

I was hoping to remove it completely, and yes I was hoping we could
because a wakeup of a woken task is almost a nop. But then I saw this in
rt_mutex_adjust_prio_chain():


> /*
> * Check the orig_waiter state. After we dropped the locks,
> * the previous owner of the lock might have released the lock
> - * and made us the pending owner:
> + * and made us candidate owner:
> */
> - if (orig_waiter && !orig_waiter->task)
> + if (orig_waiter && orig_waiter->cand_owner)
> goto out_unlock_pi;
>

I'm not sure what else we could use to check if the original waiter has
been given the lock.

-- Steve

2010-12-15 14:16:54

by Thomas Gleixner

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Wed, 15 Dec 2010, Steven Rostedt wrote:

> On Wed, 2010-12-15 at 09:02 +0100, Thomas Gleixner wrote:
> > On Tue, 14 Dec 2010, Steven Rostedt wrote:
> >
> > > On Tue, 2010-12-14 at 17:04 +0800, Lai Jiangshan wrote:
> > >
> > > OK, I was looking at this in a bit more detail (the coffee finally set
> > > in) and I was at first looking to nuke the cand_owner since it is
> > > redundant to cand_seq. But I think we can nuke the cand_seq instead and
> > > use the top_waiter as the decider.
> >
> > So you just use cand_owner (the name sucks) to flag that the waiter
> > has been woken up either by the boost code or by an unlock. The waiter
> > clears that flag with waiter->lock->wait_lock held before calling
> > schedule().
> >
> > Though I think we do need it at all. wakeup of an already running task
> > is almost a nop, so having one less state to worry about is good.
>
> I was hoping to remove it completely, and yes I was hoping we could
> because a wakeup of a woken task is almost a nop. But then I saw this in
> rt_mutex_adjust_prio_chain():
>
>
> > /*
> > * Check the orig_waiter state. After we dropped the locks,
> > * the previous owner of the lock might have released the lock
> > - * and made us the pending owner:
> > + * and made us candidate owner:
> > */
> > - if (orig_waiter && !orig_waiter->task)
> > + if (orig_waiter && orig_waiter->cand_owner)
> > goto out_unlock_pi;
> >
>
> I'm not sure what else we could use to check if the original waiter has
> been given the lock.

That does not matter. The interesting part is whether the lock on
which orig_waiter is blocked on was unlocked. Lai's follow up patch does:

+ if (orig_waiter && !rt_mutex_owner(orig_lock))
goto out_unlock_pi;

Thanks,

tglx

2010-12-15 14:30:15

by Lai Jiangshan

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Wed, Dec 15, 2010 at 8:07 PM, Thomas Gleixner <[email protected]> wrote:
> On Wed, 15 Dec 2010, Lai Jiangshan wrote:
>> + ? ? if (waiter || rt_mutex_has_waiters(lock)) {
>> + ? ? ? ? ? ? unsigned long flags;
>> + ? ? ? ? ? ? struct rt_mutex_waiter *top;
>> +
>> + ? ? ? ? ? ? raw_spin_lock_irqsave(&task->pi_lock, flags);
>> +
>> + ? ? ? ? ? ? /* remove the queued waiter. */
>> + ? ? ? ? ? ? if (waiter) {
>> + ? ? ? ? ? ? ? ? ? ? plist_del(&waiter->list_entry, &lock->wait_list);
>> + ? ? ? ? ? ? ? ? ? ? task->pi_blocked_on = NULL;
>> + ? ? ? ? ? ? }
>> +
>> + ? ? ? ? ? ? /*
>> + ? ? ? ? ? ? ?* We have to enqueue the top waiter(if have) into
>> + ? ? ? ? ? ? ?* task->pi_waiters list and would get boost from it.
>
> No, we don't get boosted from it. We just have to enqueue it into
> pi_waiters list. There is no boosting happening at this point. Please
> be very careful with the comments in this code.
>
> Otherwise this looks really interesting. Still this wants to be ported
> to -rt and stress tested there.
>

Hi, Thomas,

Where can I find the code of stress testcases?

Thanks,
Lai

2010-12-15 14:32:41

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Wed, 2010-12-15 at 15:16 +0100, Thomas Gleixner wrote:
> On Wed, 15 Dec 2010, Steven Rostedt wrote:
>

> That does not matter. The interesting part is whether the lock on
> which orig_waiter is blocked on was unlocked. Lai's follow up patch does:
>
> + if (orig_waiter && !rt_mutex_owner(orig_lock))
> goto out_unlock_pi;
>

I haven't looked at his new patch yet. But if this is the case, then we
can nuke all cand_* fields. This was the only reason I kept the
cand_owner in the first place.

Thanks,

-- Steve

2010-12-15 14:51:11

by Thomas Gleixner

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On Wed, 15 Dec 2010, Steven Rostedt wrote:

> On Wed, 2010-12-15 at 15:16 +0100, Thomas Gleixner wrote:
> > On Wed, 15 Dec 2010, Steven Rostedt wrote:
> >
>
> > That does not matter. The interesting part is whether the lock on
> > which orig_waiter is blocked on was unlocked. Lai's follow up patch does:
> >
> > + if (orig_waiter && !rt_mutex_owner(orig_lock))
> > goto out_unlock_pi;
> >
>
> I haven't looked at his new patch yet. But if this is the case, then we
> can nuke all cand_* fields. This was the only reason I kept the
> cand_owner in the first place.

Lai nuked it completely.

2010-12-15 14:53:25

by Thomas Gleixner

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Wed, 15 Dec 2010, Lai Jiangshan wrote:
> On Wed, Dec 15, 2010 at 8:07 PM, Thomas Gleixner <[email protected]> wrote:
> > On Wed, 15 Dec 2010, Lai Jiangshan wrote:
> >> + ? ? if (waiter || rt_mutex_has_waiters(lock)) {
> >> + ? ? ? ? ? ? unsigned long flags;
> >> + ? ? ? ? ? ? struct rt_mutex_waiter *top;
> >> +
> >> + ? ? ? ? ? ? raw_spin_lock_irqsave(&task->pi_lock, flags);
> >> +
> >> + ? ? ? ? ? ? /* remove the queued waiter. */
> >> + ? ? ? ? ? ? if (waiter) {
> >> + ? ? ? ? ? ? ? ? ? ? plist_del(&waiter->list_entry, &lock->wait_list);
> >> + ? ? ? ? ? ? ? ? ? ? task->pi_blocked_on = NULL;
> >> + ? ? ? ? ? ? }
> >> +
> >> + ? ? ? ? ? ? /*
> >> + ? ? ? ? ? ? ?* We have to enqueue the top waiter(if have) into
> >> + ? ? ? ? ? ? ?* task->pi_waiters list and would get boost from it.
> >
> > No, we don't get boosted from it. We just have to enqueue it into
> > pi_waiters list. There is no boosting happening at this point. Please
> > be very careful with the comments in this code.
> >
> > Otherwise this looks really interesting. Still this wants to be ported
> > to -rt and stress tested there.
> >
>
> Hi, Thomas,
>
> Where can I find the code of stress testcases?

RT itself will stress test the rtmutex code as much as it goes as we
convert almost every lock into a rtmutex there.

Steven, could you have a look at that ?

Thanks,

tglx

2010-12-15 15:02:03

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Wed, 2010-12-15 at 15:52 +0100, Thomas Gleixner wrote:

> RT itself will stress test the rtmutex code as much as it goes as we
> convert almost every lock into a rtmutex there.
>
> Steven, could you have a look at that ?
>

Yep, I'll look to porting it today.

-- Steve

2010-12-15 15:04:27

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Wed, 2010-12-15 at 16:09 +0800, Lai Jiangshan wrote:

Some English updates.

>
> Signed-off-by: Lai Jiangshan <[email protected]>
> ---
> kernel/futex.c | 26 +---
> kernel/rtmutex.c | 306 ++++++++++++++++--------------------------------
> kernel/rtmutex_common.h | 16 --
> 3 files changed, 116 insertions(+), 232 deletions(-)
>
> diff --git a/kernel/futex.c b/kernel/futex.c
> index 6c683b3..5f4ea5f 100644
> --- a/kernel/futex.c
> +++ b/kernel/futex.c
> @@ -775,18 +775,10 @@ static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_q *this)
> return -EINVAL;
>
> raw_spin_lock(&pi_state->pi_mutex.wait_lock);
> + /* set new owner to the most possible owner(top waiter). */

"most possible owner" sounds very strange. Lets say what it actually is.

/* set new owner to the highest prio waiter (top waiter) */


> new_owner = rt_mutex_next_owner(&pi_state->pi_mutex);
>
> /*
> - * This happens when we have stolen the lock and the original
> - * pending owner did not enqueue itself back on the rt_mutex.
> - * Thats not a tragedy. We know that way, that a lock waiter
> - * is on the fly. We make the futex_q waiter the pending owner.
> - */
> - if (!new_owner)
> - new_owner = this->task;
> -
> - /*
> * We pass it to the next owner. (The WAITERS bit is always
> * kept enabled while there is PI state around. We must also
> * preserve the owner died bit.)
> @@ -1508,8 +1500,8 @@ static int fixup_pi_state_owner(u32 __user *uaddr, struct futex_q *q,
>
> /*
> * We are here either because we stole the rtmutex from the
> - * pending owner or we are the pending owner which failed to
> - * get the rtmutex. We have to replace the pending owner TID
> + * most possible owner or we are the most possible owner which
> + * failed to get the rtmutex. We have to replace the newowner TID
> * in the user space variable. This must be atomic as we have
> * to preserve the owner died bit here.


/*
* We are here either because we stole the rtmutex from the
* previous highest prio waiter or we are the highest prio
* waiter but failed to get the rtmutex the first time.
* We have to replace ...


> *
> @@ -1560,7 +1552,7 @@ retry:
>
> /*
> * To handle the page fault we need to drop the hash bucket
> - * lock here. That gives the other task (either the pending
> + * lock here. That gives the other task (either the most possible
> * owner itself or the task which stole the rtmutex) the

* ... That gives the other task (either the highest prio waiter
* itself or the ...

> * chance to try the fixup of the pi_state. So once we are
> * back from handling the fault we need to check the pi_state
> @@ -1647,18 +1639,20 @@ static int fixup_owner(u32 __user *uaddr, int fshared, struct futex_q *q,
> /*
> * pi_state is incorrect, some other task did a lock steal and
> * we returned due to timeout or signal without taking the
> - * rt_mutex. Too late. We can access the rt_mutex_owner without
> - * locking, as the other task is now blocked on the hash bucket
> - * lock. Fix the state up.
> + * rt_mutex. Too late.

You cut off the entire "We can access the rt_mutex_owner..." but I don't
see how this change is in the code.


> */
> + raw_spin_lock(&q->pi_state->pi_mutex.wait_lock);
> owner = rt_mutex_owner(&q->pi_state->pi_mutex);
> + if (!owner)
> + owner = rt_mutex_next_owner(&q->pi_state->pi_mutex);
> + raw_spin_unlock(&q->pi_state->pi_mutex.wait_lock);
> ret = fixup_pi_state_owner(uaddr, q, owner, fshared);
> goto out;
> }
>
> /*
> * Paranoia check. If we did not take the lock, then we should not be
> - * the owner, nor the pending owner, of the rt_mutex.
> + * the owner of the rt_mutex.
> */
> if (rt_mutex_owner(&q->pi_state->pi_mutex) == current)
> printk(KERN_ERR "fixup_owner: ret = %d pi-mutex: %p "
> diff --git a/kernel/rtmutex.c b/kernel/rtmutex.c
> index a960481..53b9b49 100644
> --- a/kernel/rtmutex.c
> +++ b/kernel/rtmutex.c
> @@ -20,41 +20,29 @@
> /*
> * lock->owner state tracking:
> *
> - * lock->owner holds the task_struct pointer of the owner. Bit 0 and 1
> - * are used to keep track of the "owner is pending" and "lock has
> - * waiters" state.
> + * lock->owner holds the task_struct pointer of the owner. Bit 0
> + * are used to keep track of the "lock has waiters" state.

s/are/is/

"Bit 0 is used to ..."

> *
> - * owner bit1 bit0
> - * NULL 0 0 lock is free (fast acquire possible)
> - * NULL 0 1 invalid state
> - * NULL 1 0 Transitional State*
> - * NULL 1 1 invalid state
> - * taskpointer 0 0 lock is held (fast release possible)
> - * taskpointer 0 1 task is pending owner
> - * taskpointer 1 0 lock is held and has waiters
> - * taskpointer 1 1 task is pending owner and lock has more waiters
> - *
> - * Pending ownership is assigned to the top (highest priority)
> - * waiter of the lock, when the lock is released. The thread is woken
> - * up and can now take the lock. Until the lock is taken (bit 0
> - * cleared) a competing higher priority thread can steal the lock
> - * which puts the woken up thread back on the waiters list.
> + * owner bit0
> + * NULL 0 lock is free (fast acquire possible)
> + * NULL 1 lock is free and has waiters and the top waiter
> + * is going to take the lock*
> + * taskpointer 0 lock is held (fast release possible)
> + * taskpointer 1 lock is held and has waiters

* taskpointer 1 lock is held and has waiters*


> *
> * The fast atomic compare exchange based acquire and release is only
> - * possible when bit 0 and 1 of lock->owner are 0.
> + * possible when bit 0 of lock->owner are 0.

s/are/is/

> *
> - * (*) There's a small time where the owner can be NULL and the
> - * "lock has waiters" bit is set. This can happen when grabbing the lock.
> - * To prevent a cmpxchg of the owner releasing the lock, we need to set this
> - * bit before looking at the lock, hence the reason this is a transitional
> - * state.
> + * (*) It also can be a transitional state when grabbing the lock
> + * with ->wait_lock is held. To prevent any fast path cmpxchg to the lock,
> + * we need to set the bit0 before looking at the lock, and the owner may be
> + * NULL in this small time, hence this can be a transitional state.


* (*) There is a small time when bit 0 is set but there are no
* waiters. This can happen when grabbing the lock in the slow path.
* To prevent a cmpxchg of the owner releasing the lock, we need to
* set this bit before looking at the lock.


> */
>
> static void
> -rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner,
> - unsigned long mask)
> +rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner)
> {
> - unsigned long val = (unsigned long)owner | mask;
> + unsigned long val = (unsigned long)owner;
>
> if (rt_mutex_has_waiters(lock))
> val |= RT_MUTEX_HAS_WAITERS;
> @@ -203,15 +191,14 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
> * reached or the state of the chain has changed while we
> * dropped the locks.
> */
> - if (!waiter || !waiter->task)
> + if (!waiter)
> goto out_unlock_pi;
>
> /*
> * Check the orig_waiter state. After we dropped the locks,
> * the previous owner of the lock might have released the lock

s/lock/lock./

> - * and made us the pending owner:
> */
> - if (orig_waiter && !orig_waiter->task)
> + if (orig_waiter && !rt_mutex_owner(orig_lock))
> goto out_unlock_pi;
>
> /*
> @@ -254,6 +241,17 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
>
> /* Release the task */
> raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> + if (!rt_mutex_owner(lock)) {
> + /*
> + * If the requeue above changed the top waiter, then we need
> + * to wake the new top waiter up to try to get the lock.
> + */
> +
> + if (top_waiter != rt_mutex_top_waiter(lock))
> + wake_up_process(rt_mutex_top_waiter(lock)->task);
> + raw_spin_unlock(&lock->wait_lock);
> + goto out_put_task;
> + }
> put_task_struct(task);
>
> /* Grab the next task */
> @@ -296,78 +294,16 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
> }
>
> /*
> - * Optimization: check if we can steal the lock from the
> - * assigned pending owner [which might not have taken the
> - * lock yet]:
> - */
> -static inline int try_to_steal_lock(struct rt_mutex *lock,
> - struct task_struct *task)
> -{
> - struct task_struct *pendowner = rt_mutex_owner(lock);
> - struct rt_mutex_waiter *next;
> - unsigned long flags;
> -
> - if (!rt_mutex_owner_pending(lock))
> - return 0;
> -
> - if (pendowner == task)
> - return 1;
> -
> - raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
> - if (task->prio >= pendowner->prio) {
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> - return 0;
> - }
> -
> - /*
> - * Check if a waiter is enqueued on the pending owners
> - * pi_waiters list. Remove it and readjust pending owners
> - * priority.
> - */
> - if (likely(!rt_mutex_has_waiters(lock))) {
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> - return 1;
> - }
> -
> - /* No chain handling, pending owner is not blocked on anything: */
> - next = rt_mutex_top_waiter(lock);
> - plist_del(&next->pi_list_entry, &pendowner->pi_waiters);
> - __rt_mutex_adjust_prio(pendowner);
> - raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
> -
> - /*
> - * We are going to steal the lock and a waiter was
> - * enqueued on the pending owners pi_waiters queue. So
> - * we have to enqueue this waiter into
> - * task->pi_waiters list. This covers the case,
> - * where task is boosted because it holds another
> - * lock and gets unboosted because the booster is
> - * interrupted, so we would delay a waiter with higher
> - * priority as task->normal_prio.
> - *
> - * Note: in the rare case of a SCHED_OTHER task changing
> - * its priority and thus stealing the lock, next->task
> - * might be task:
> - */
> - if (likely(next->task != task)) {
> - raw_spin_lock_irqsave(&task->pi_lock, flags);
> - plist_add(&next->pi_list_entry, &task->pi_waiters);
> - __rt_mutex_adjust_prio(task);
> - raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> - }
> - return 1;
> -}
> -
> -/*
> * Try to take an rt-mutex
> *
> - * This fails
> - * - when the lock has a real owner
> - * - when a different pending owner exists and has higher priority than current
> - *
> * Must be called with lock->wait_lock held.
> + *
> + * @lock: the lock to be acquired.
> + * @task: the task which want to acquire the lock

"the task which wants to acquire the lock"

> + * @waiter: the waiter queued to the lock's wait list. (could be NULL)

"the waiter that is queued to the ..."

Is this always current's waiter?

> */
> -static int try_to_take_rt_mutex(struct rt_mutex *lock)
> +static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
> + struct rt_mutex_waiter *waiter)
> {
> /*
> * We have to be careful here if the atomic speedups are
> @@ -390,15 +326,52 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
> */
> mark_rt_mutex_waiters(lock);
>
> - if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
> + if (rt_mutex_owner(lock))
> return 0;
>
> + /*
> + * It will get the lock because at least one of these conditions:

* It will get the lock because of one of these conditions:

> + * 1) there is no waiter
> + * 2) higher priority than waiters
> + * 3) it is top waiter
> + */
> + if (rt_mutex_has_waiters(lock)) {
> + if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio) {
> + if (!waiter || waiter != rt_mutex_top_waiter(lock))
> + return 0;
> + }
> + }
> +
> + if (waiter || rt_mutex_has_waiters(lock)) {
> + unsigned long flags;
> + struct rt_mutex_waiter *top;
> +
> + raw_spin_lock_irqsave(&task->pi_lock, flags);
> +
> + /* remove the queued waiter. */
> + if (waiter) {
> + plist_del(&waiter->list_entry, &lock->wait_list);
> + task->pi_blocked_on = NULL;
> + }
> +
> + /*
> + * We have to enqueue the top waiter(if have) into

... top waiter (if it exists) ...


> + * task->pi_waiters list and would get boost from it.

s/ and would get boost from it//

-- Steve

> + */
> + if (rt_mutex_has_waiters(lock)) {
> + top = rt_mutex_top_waiter(lock);
> + top->pi_list_entry.prio = top->list_entry.prio;
> + plist_add(&top->pi_list_entry, &task->pi_waiters);
> + }
> + raw_spin_unlock_irqrestore(&task->pi_lock, flags);
> + }
> +
> /* We got the lock. */
> debug_rt_mutex_lock(lock);

2010-12-15 21:53:28

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Wed, 2010-12-15 at 16:09 +0800, Lai Jiangshan wrote:


> /*
> @@ -543,11 +491,13 @@ static void remove_waiter(struct rt_mutex *lock,
>
> raw_spin_lock_irqsave(&current->pi_lock, flags);
> plist_del(&waiter->list_entry, &lock->wait_list);
> - waiter->task = NULL;
> current->pi_blocked_on = NULL;
> raw_spin_unlock_irqrestore(&current->pi_lock, flags);
>
> - if (first && owner != current) {
> + if (!owner)
> + return;
> +
> + if (first) {

This is a bug. There's a small chance that the mutex timed out, and at
that same time, the owner gave up the lock and woke this task up. Which
means this task is the new owner of the lock iff it was the
rt_mutex_top_waiter().

The fix is to do this:


if (!owner) {
if (first) {
ret = try_to_take_rt_mutex();
BUG_ON(!ret);
}
return first;
}

if (first) {

We need to make remove_waiter return 1 if it took the lock and 0 if it
did not, so it can pass this information back to the caller.

if (unlikely(ret)) {
if (remove_waiter(...))
ret = 0;
}

-- Steve

2010-12-16 01:13:52

by Lai Jiangshan

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On 12/16/2010 05:53 AM, Steven Rostedt wrote:
> On Wed, 2010-12-15 at 16:09 +0800, Lai Jiangshan wrote:
>
>
>> /*
>> @@ -543,11 +491,13 @@ static void remove_waiter(struct rt_mutex *lock,
>>
>> raw_spin_lock_irqsave(&current->pi_lock, flags);
>> plist_del(&waiter->list_entry, &lock->wait_list);
>> - waiter->task = NULL;
>> current->pi_blocked_on = NULL;
>> raw_spin_unlock_irqrestore(&current->pi_lock, flags);
>>
>> - if (first && owner != current) {
>> + if (!owner)
>> + return;
>> +
>> + if (first) {
>
> This is a bug. There's a small chance that the mutex timed out, and at
> that same time, the owner gave up the lock and woke this task up. Which
> means this task is the new owner of the lock iff it was the
> rt_mutex_top_waiter().
>
> The fix is to do this:
>
>
> if (!owner) {
> if (first) {
> ret = try_to_take_rt_mutex();
> BUG_ON(!ret);
> }
> return first;
> }
>
> if (first) {
>
> We need to make remove_waiter return 1 if it took the lock and 0 if it
> did not, so it can pass this information back to the caller.
>
> if (unlikely(ret)) {
> if (remove_waiter(...))
> ret = 0;
> }
>

It has called try_to_take_rt_mutex() in __rt_mutex_slowlock(),
when timeout or got signal, it returns from __rt_mutex_slowlock()
with lock->wait_lock still held, and then calls remove_waiter(),

so we don't need to call try_to_take_rt_mutex() in remove_waiter().
It is strange that remove_waiter() do some "require lock" work.

Thanks,
Lai

2010-12-16 13:56:37

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Thu, 2010-12-16 at 09:14 +0800, Lai Jiangshan wrote:

> It has called try_to_take_rt_mutex() in __rt_mutex_slowlock(),
> when timeout or got signal, it returns from __rt_mutex_slowlock()
> with lock->wait_lock still held, and then calls remove_waiter(),
>
> so we don't need to call try_to_take_rt_mutex() in remove_waiter().
> It is strange that remove_waiter() do some "require lock" work.

Hmm yeah, I need to remember all the details here. There were so many
corner cases that had to be dealt with, but we have also
modified/simplified the code over time (hopefully your patch will add to
the simplification) that my worries may no longer exist, and the code in
-rt (needed for spinlock -> mutex) may have legacy code there too.

I'll spend more time analyzing this and make sure we in deed do not need
to worry about races between wakeups and lock releasing.

Thanks,

-- Steve

2010-12-16 14:07:54

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Thu, 2010-12-16 at 09:14 +0800, Lai Jiangshan wrote:

> It has called try_to_take_rt_mutex() in __rt_mutex_slowlock(),
> when timeout or got signal, it returns from __rt_mutex_slowlock()
> with lock->wait_lock still held, and then calls remove_waiter(),
>
> so we don't need to call try_to_take_rt_mutex() in remove_waiter().
> It is strange that remove_waiter() do some "require lock" work.

I think you are correct here. It should never get to this path where
!owner && first is true. If we timed out then we either got the lock or
we could not. If we could not than owner had to be set or we are not the
top waiter. Thus we need:

if (!owner) {
BUG_ON(first);
return;
}

-- Steve

2010-12-16 20:33:53

by Darren Hart

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On 12/15/2010 06:24 AM, Lai Jiangshan wrote:
> On Wed, Dec 15, 2010 at 8:07 PM, Thomas Gleixner<[email protected]> wrote:
>> On Wed, 15 Dec 2010, Lai Jiangshan wrote:
>>> + if (waiter || rt_mutex_has_waiters(lock)) {
>>> + unsigned long flags;
>>> + struct rt_mutex_waiter *top;
>>> +
>>> + raw_spin_lock_irqsave(&task->pi_lock, flags);
>>> +
>>> + /* remove the queued waiter. */
>>> + if (waiter) {
>>> + plist_del(&waiter->list_entry,&lock->wait_list);
>>> + task->pi_blocked_on = NULL;
>>> + }
>>> +
>>> + /*
>>> + * We have to enqueue the top waiter(if have) into
>>> + * task->pi_waiters list and would get boost from it.
>>
>> No, we don't get boosted from it. We just have to enqueue it into
>> pi_waiters list. There is no boosting happening at this point. Please
>> be very careful with the comments in this code.
>>
>> Otherwise this looks really interesting. Still this wants to be ported
>> to -rt and stress tested there.
>>
>
> Hi, Thomas,
>
> Where can I find the code of stress testcases?

See http://rt.wiki.kernel.org, There are several benchmarks available
for RT there.

I believe there are some built-int rt-mutex testers as well:\
CONFIG_RT_MUTEX_TESTER

--
Darren Hart
Yocto Linux Kernel

2010-12-16 20:55:16

by Darren Hart

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On 12/14/2010 12:07 PM, Thomas Gleixner wrote:
> B1;2401;0cOn Tue, 14 Dec 2010, Lai Jiangshan wrote:
>> Not advantage nor disadvantage
>> 1) Even we support multiple candidate owners, we hardly cause "thundering herd"
>> the number of candidate owners is likely 1.
>
> I'm not convinced about that function, see comments below
>
>> 2) two APIs are changed.
>> rt_mutex_owner() will not return pending owner
>> rt_mutex_next_owner() always return the top owner, it is a candidate owner.
>> will not return NULL if we only have a pending owner.
>> I have fixed the code that use these APIs.
>
> I think we want a separate function which can be used by futex code,
> but that's a detail.


We do use both functions in the futex code.


> @@ -778,15 +778,6 @@ static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_q *this)
> new_owner = rt_mutex_next_owner(&pi_state->pi_mutex);
>
> /*
> - * This happens when we have stolen the lock and the original
> - * pending owner did not enqueue itself back on the rt_mutex.
> - * Thats not a tragedy. We know that way, that a lock waiter
> - * is on the fly. We make the futex_q waiter the pending owner.
> - */
> - if (!new_owner)
> - new_owner = this->task;
> -


If I'm reading Lai's statement above correctly, the change actually just
eliminates the need for the !new_owner check, as in that case it will
return current. Is this correct? And indeed, Lai's patch removes it.
This looks correct to me.


> @@ -1647,18 +1638,20 @@ static int fixup_owner(u32 __user *uaddr, int fshared, struct futex_q *q,
> /*
> * pi_state is incorrect, some other task did a lock steal and
> * we returned due to timeout or signal without taking the
> - * rt_mutex. Too late. We can access the rt_mutex_owner without
> - * locking, as the other task is now blocked on the hash bucket
> - * lock. Fix the state up.


How does this patch change this behavior? Removing the comment and
adding the lock says that "the other task is now able to contend for the
pi_mutex".


> + * rt_mutex. Too late.
> */
> + raw_spin_lock(&q->pi_state->pi_mutex.wait_lock);
> owner = rt_mutex_owner(&q->pi_state->pi_mutex);
> + if (!owner)
> + owner = rt_mutex_next_owner(&q->pi_state->pi_mutex);
> + raw_spin_unlock(&q->pi_state->pi_mutex.wait_lock);
> ret = fixup_pi_state_owner(uaddr, q, owner, fshared);
> goto out;
> }
>
> /*
> * Paranoia check. If we did not take the lock, then we should not be
> - * the owner, nor the pending owner, of the rt_mutex.
> + * the owner of the rt_mutex.


Is this changed because we could now be a "candidate owner" ?


> */
> if (rt_mutex_owner(&q->pi_state->pi_mutex) == current)
> printk(KERN_ERR "fixup_owner: ret = %d pi-mutex: %p "


The other uses of rt_mutex_owner in futex.c don't appear to be impacted
by the change in API described above as they just compare the result to
current (futex_lock_pi and futex_wait_requeue_pi).


--
Darren Hart
Yocto Linux Kernel

2010-12-17 03:09:29

by Lai Jiangshan

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On 12/17/2010 04:33 AM, Darren Hart wrote:
>> Hi, Thomas,
>>
>> Where can I find the code of stress testcases?
>
> See http://rt.wiki.kernel.org, There are several benchmarks available
> for RT there.
>
> I believe there are some built-int rt-mutex testers as well:\
> CONFIG_RT_MUTEX_TESTER
>

Thank you very much,

I have tested the testcases in scripts/rt-tester/ with CONFIG_RT_MUTEX_TESTER=y

I also try to port it to -rt. Steven, have you ported it to -rt?

Thanks,
Lai.

2010-12-17 03:17:47

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Fri, 2010-12-17 at 11:10 +0800, Lai Jiangshan wrote:
> On 12/17/2010 04:33 AM, Darren Hart wrote:

> I also try to port it to -rt. Steven, have you ported it to -rt?

I'm porting it now. -rt does a a bit to the rtmutex.c code, and I spent
a bit of time cleaning it up. I finally got your patch to compile, and
now I'm debugging it ;-)

I'm hoping to have it ready to post tomorrow.

-- Steve

2010-12-17 03:30:37

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Thu, 2010-12-16 at 22:17 -0500, Steven Rostedt wrote:
> On Fri, 2010-12-17 at 11:10 +0800, Lai Jiangshan wrote:
> > On 12/17/2010 04:33 AM, Darren Hart wrote:
>
> > I also try to port it to -rt. Steven, have you ported it to -rt?
>
> I'm porting it now. -rt does a a bit to the rtmutex.c code, and I spent
> a bit of time cleaning it up. I finally got your patch to compile, and
> now I'm debugging it ;-)
>
> I'm hoping to have it ready to post tomorrow.

Update: I just successfully booted a port of your patch. I'll keep it
running over night and post it tomorrow morning (if it doesn't crash).

/me kicks off a bunch of dbenches

-- Steve

2010-12-23 07:23:24

by Lai Jiangshan

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: multiple candidate owners without unrelated boosting

On 12/17/2010 04:55 AM, Darren Hart wrote:
> On 12/14/2010 12:07 PM, Thomas Gleixner wrote:
>> B1;2401;0cOn Tue, 14 Dec 2010, Lai Jiangshan wrote:
>>> Not advantage nor disadvantage
>>> 1) Even we support multiple candidate owners, we hardly cause "thundering herd"
>>> the number of candidate owners is likely 1.
>>
>> I'm not convinced about that function, see comments below
>>
>>> 2) two APIs are changed.
>>> rt_mutex_owner() will not return pending owner
>>> rt_mutex_next_owner() always return the top owner, it is a candidate owner.
>>> will not return NULL if we only have a pending owner.
>>> I have fixed the code that use these APIs.
>>
>> I think we want a separate function which can be used by futex code,
>> but that's a detail.
>
>
> We do use both functions in the futex code.
>
>
>> @@ -778,15 +778,6 @@ static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_q *this)
>> new_owner = rt_mutex_next_owner(&pi_state->pi_mutex);
>>
>> /*
>> - * This happens when we have stolen the lock and the original
>> - * pending owner did not enqueue itself back on the rt_mutex.
>> - * Thats not a tragedy. We know that way, that a lock waiter
>> - * is on the fly. We make the futex_q waiter the pending owner.
>> - */
>> - if (!new_owner)
>> - new_owner = this->task;
>> -
>
>
> If I'm reading Lai's statement above correctly, the change actually just
> eliminates the need for the !new_owner check, as in that case it will
> return current. Is this correct? And indeed, Lai's patch removes it.
> This looks correct to me.
>


After this patch applied, the topwaiter will not be deququed when the lock
is released(any waiter is dequeued only when it really get the lock or give up).
So the wait list will not be empty if someone is still waiting on.
In this code, this->task is waiting, so rt_mutex_next_owner() will not return NULL.

>
>> @@ -1647,18 +1638,20 @@ static int fixup_owner(u32 __user *uaddr, int fshared, struct futex_q *q,
>> /*
>> * pi_state is incorrect, some other task did a lock steal and
>> * we returned due to timeout or signal without taking the
>> - * rt_mutex. Too late. We can access the rt_mutex_owner without
>> - * locking, as the other task is now blocked on the hash bucket
>> - * lock. Fix the state up.
>
>
> How does this patch change this behavior? Removing the comment and
> adding the lock says that "the other task is now able to contend for the
> pi_mutex".

After this patch applied, it is possible that:
current task calls rt_mutex_trylock() and returns faill
&& the rt_mutex_owner() return NULL.

This happen when the lock is just released and the highest prio waiter
is wokenup, but it has not taken the lock yet. So we have to
call rt_mutex_next_owner() to get the highest prio waiter and set it to
q->pi_state->owner.

The top waiter is possible changed by priority boosting. so we need
to hold the wait_lock when access to it.

>
>
>> + * rt_mutex. Too late.
>> */
>> + raw_spin_lock(&q->pi_state->pi_mutex.wait_lock);
>> owner = rt_mutex_owner(&q->pi_state->pi_mutex);
>> + if (!owner)
>> + owner = rt_mutex_next_owner(&q->pi_state->pi_mutex);
>> + raw_spin_unlock(&q->pi_state->pi_mutex.wait_lock);
>> ret = fixup_pi_state_owner(uaddr, q, owner, fshared);
>> goto out;
>> }
>>
>> /*
>> * Paranoia check. If we did not take the lock, then we should not be
>> - * the owner, nor the pending owner, of the rt_mutex.
>> + * the owner of the rt_mutex.
>
>
> Is this changed because we could now be a "candidate owner" ?

It now is not possible "candidate owner" either.
but since it is a "Paranoia check", we don't need to check all things and
add unneed overhead.

>
>
>> */
>> if (rt_mutex_owner(&q->pi_state->pi_mutex) == current)
>> printk(KERN_ERR "fixup_owner: ret = %d pi-mutex: %p "
>
>
> The other uses of rt_mutex_owner in futex.c don't appear to be impacted
> by the change in API described above as they just compare the result to
> current (futex_lock_pi and futex_wait_requeue_pi).
>
>

Rignt, the other uses of rt_mutex_owner() do not be impacted.

Thanks a lot.
Lai.

2010-12-23 08:47:59

by Lai Jiangshan

[permalink] [raw]
Subject: [PATCH V3] rtmutex: ensure only the top waiter or higher priority task can take the lock and remove unrelated boosting


In current rtmutex, the pending owner may be boosted by the tasks
in the rtmutex's waitlist when the pending owner is deboosted
or a task in the waitlist is boosted. This boosting is unrelated,
because the pending owner does not really take the rtmutex.
It is not reasonable.

Example.

time1:
A(high prio) onwers the rtmutex.
B(mid prio) and C (low prio) in the waitlist.

time2
A release the lock, B becomes the pending owner
A(or other high prio task) continues to run. B's prio is lower
than A, so B is just queued at the runqueue.

time3
A or other high prio task sleeps, but we have passed some time
The B and C's prio are changed in the period (time2 ~ time3)
due to boosting or deboosting. Now C has the priority higher
than B. ***Is it reasonable that C has to boost B and help B to
get the rtmutex?

NO!! I think, it is unrelated/unneed boosting before B really
owns the rtmutex. We should give C a chance to beat B and
win the rtmutex.

This is the motivation of this patch. This patch *ensures*
only the top waiter or higher priority task can take the lock.

How?
1) we don't dequeue the top waiter when unlock, if the top waiter
is changed, the old top waiter will fail and go to sleep again.
2) when requiring lock, it will get the lock when the lock is not taken and:
there is no waiter OR higher priority than waiters OR it is top waiter.
3) In any time, the top waiter is changed, the top waiter will be woken up.

The algorithm is much simpler than before, no pending owner, no
boosting for pending owner.

Other advantage of this patch:
1) The states of a rtmutex are reduced a half, easier to read the code.
2) the codes become shorter.
3) top waiter is not dequeued until it really take the lock:
they will retain FIFO when it is stolen.

Not advantage nor disadvantage
1) Even we may wakeup multiple waiters(any time when top waiter changed),
we hardly cause "thundering herd",
the number of wokenup task is likely 1 or very little.
2) two APIs are changed.
rt_mutex_owner() will not return pending owner, it will return NULL when
the top waiter is going to take the lock.
rt_mutex_next_owner() always return the top waiter.
will not return NULL if we have waiters
because the top waiter is not dequeued.

I have fixed the code that use these APIs.

need updated after this patch is accepted
1) Document/*
2) the testcase scripts/rt-tester/t4-l2-pi-deboost.tst

Signed-off-by: Lai Jiangshan <[email protected]>
---
futex.c | 32 ++---
rtmutex.c | 318 +++++++++++++++++++------------------------------------
rtmutex_common.h | 16 --
3 files changed, 128 insertions(+), 238 deletions(-)

diff --git a/kernel/futex.c b/kernel/futex.c
index 3019b92..b0c9d23 100644
--- a/kernel/futex.c
+++ b/kernel/futex.c
@@ -788,18 +788,10 @@ static int wake_futex_pi(u32 __user *uaddr, u32 uval, struct futex_q *this)
return -EINVAL;

raw_spin_lock(&pi_state->pi_mutex.wait_lock);
+ /* set new owner to the highest priority waiter (top waiter). */
new_owner = rt_mutex_next_owner(&pi_state->pi_mutex);

/*
- * This happens when we have stolen the lock and the original
- * pending owner did not enqueue itself back on the rt_mutex.
- * Thats not a tragedy. We know that way, that a lock waiter
- * is on the fly. We make the futex_q waiter the pending owner.
- */
- if (!new_owner)
- new_owner = this->task;
-
- /*
* We pass it to the next owner. (The WAITERS bit is always
* kept enabled while there is PI state around. We must also
* preserve the owner died bit.)
@@ -1522,10 +1514,10 @@ static int fixup_pi_state_owner(u32 __user *uaddr, struct futex_q *q,

/*
* We are here either because we stole the rtmutex from the
- * pending owner or we are the pending owner which failed to
- * get the rtmutex. We have to replace the pending owner TID
- * in the user space variable. This must be atomic as we have
- * to preserve the owner died bit here.
+ * previous highest priority waiter or we are the highest priority
+ * waiter but failed to get the rtmutex the first time.
+ * We have to replace the newowner TID in the user space variable.
+ * This must be atomic as we have to preserve the owner died bit here.
*
* Note: We write the user space value _before_ changing the pi_state
* because we can fault here. Imagine swapped out pages or a fork
@@ -1574,8 +1566,8 @@ retry:

/*
* To handle the page fault we need to drop the hash bucket
- * lock here. That gives the other task (either the pending
- * owner itself or the task which stole the rtmutex) the
+ * lock here. That gives the other task (either the highest priority
+ * waiter itself or the task which stole the rtmutex) the
* chance to try the fixup of the pi_state. So once we are
* back from handling the fault we need to check the pi_state
* after reacquiring the hash bucket lock and before trying to
@@ -1651,18 +1643,20 @@ static int fixup_owner(u32 __user *uaddr, struct futex_q *q, int locked)
/*
* pi_state is incorrect, some other task did a lock steal and
* we returned due to timeout or signal without taking the
- * rt_mutex. Too late. We can access the rt_mutex_owner without
- * locking, as the other task is now blocked on the hash bucket
- * lock. Fix the state up.
+ * rt_mutex. Too late.
*/
+ raw_spin_lock(&q->pi_state->pi_mutex.wait_lock);
owner = rt_mutex_owner(&q->pi_state->pi_mutex);
+ if (!owner)
+ owner = rt_mutex_next_owner(&q->pi_state->pi_mutex);
+ raw_spin_unlock(&q->pi_state->pi_mutex.wait_lock);
ret = fixup_pi_state_owner(uaddr, q, owner);
goto out;
}

/*
* Paranoia check. If we did not take the lock, then we should not be
- * the owner, nor the pending owner, of the rt_mutex.
+ * the owner of the rt_mutex.
*/
if (rt_mutex_owner(&q->pi_state->pi_mutex) == current)
printk(KERN_ERR "fixup_owner: ret = %d pi-mutex: %p "
diff --git a/kernel/rtmutex.c b/kernel/rtmutex.c
index a960481..ab44911 100644
--- a/kernel/rtmutex.c
+++ b/kernel/rtmutex.c
@@ -20,41 +20,34 @@
/*
* lock->owner state tracking:
*
- * lock->owner holds the task_struct pointer of the owner. Bit 0 and 1
- * are used to keep track of the "owner is pending" and "lock has
- * waiters" state.
+ * lock->owner holds the task_struct pointer of the owner. Bit 0
+ * is used to keep track of the "lock has waiters" state.
*
- * owner bit1 bit0
- * NULL 0 0 lock is free (fast acquire possible)
- * NULL 0 1 invalid state
- * NULL 1 0 Transitional State*
- * NULL 1 1 invalid state
- * taskpointer 0 0 lock is held (fast release possible)
- * taskpointer 0 1 task is pending owner
- * taskpointer 1 0 lock is held and has waiters
- * taskpointer 1 1 task is pending owner and lock has more waiters
- *
- * Pending ownership is assigned to the top (highest priority)
- * waiter of the lock, when the lock is released. The thread is woken
- * up and can now take the lock. Until the lock is taken (bit 0
- * cleared) a competing higher priority thread can steal the lock
- * which puts the woken up thread back on the waiters list.
+ * owner bit0
+ * NULL 0 lock is free (fast acquire possible)
+ * NULL 1 lock is free and has waiters and the top waiter
+ * is going to take the lock*
+ * taskpointer 0 lock is held (fast release possible)
+ * taskpointer 1 lock is held and has waiters**
*
* The fast atomic compare exchange based acquire and release is only
- * possible when bit 0 and 1 of lock->owner are 0.
+ * possible when bit 0 of lock->owner is 0.
+ *
+ * (*) It also can be a transitional state when grabbing the lock
+ * with ->wait_lock is held. To prevent any fast path cmpxchg to the lock,
+ * we need to set the bit0 before looking at the lock, and the owner may be
+ * NULL in this small time, hence this can be a transitional state.
*
- * (*) There's a small time where the owner can be NULL and the
- * "lock has waiters" bit is set. This can happen when grabbing the lock.
- * To prevent a cmpxchg of the owner releasing the lock, we need to set this
- * bit before looking at the lock, hence the reason this is a transitional
- * state.
+ * (**) There is a small time when bit 0 is set but there are no
+ * waiters. This can happen when grabbing the lock in the slow path.
+ * To prevent a cmpxchg of the owner releasing the lock, we need to
+ * set this bit before looking at the lock.
*/

static void
-rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner,
- unsigned long mask)
+rt_mutex_set_owner(struct rt_mutex *lock, struct task_struct *owner)
{
- unsigned long val = (unsigned long)owner | mask;
+ unsigned long val = (unsigned long)owner;

if (rt_mutex_has_waiters(lock))
val |= RT_MUTEX_HAS_WAITERS;
@@ -203,15 +196,14 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
* reached or the state of the chain has changed while we
* dropped the locks.
*/
- if (!waiter || !waiter->task)
+ if (!waiter)
goto out_unlock_pi;

/*
* Check the orig_waiter state. After we dropped the locks,
- * the previous owner of the lock might have released the lock
- * and made us the pending owner:
+ * the previous owner of the lock might have released the lock.
*/
- if (orig_waiter && !orig_waiter->task)
+ if (orig_waiter && !rt_mutex_owner(orig_lock))
goto out_unlock_pi;

/*
@@ -254,6 +246,17 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,

/* Release the task */
raw_spin_unlock_irqrestore(&task->pi_lock, flags);
+ if (!rt_mutex_owner(lock)) {
+ /*
+ * If the requeue above changed the top waiter, then we need
+ * to wake the new top waiter up to try to get the lock.
+ */
+
+ if (top_waiter != rt_mutex_top_waiter(lock))
+ wake_up_process(rt_mutex_top_waiter(lock)->task);
+ raw_spin_unlock(&lock->wait_lock);
+ goto out_put_task;
+ }
put_task_struct(task);

/* Grab the next task */
@@ -296,78 +299,16 @@ static int rt_mutex_adjust_prio_chain(struct task_struct *task,
}

/*
- * Optimization: check if we can steal the lock from the
- * assigned pending owner [which might not have taken the
- * lock yet]:
- */
-static inline int try_to_steal_lock(struct rt_mutex *lock,
- struct task_struct *task)
-{
- struct task_struct *pendowner = rt_mutex_owner(lock);
- struct rt_mutex_waiter *next;
- unsigned long flags;
-
- if (!rt_mutex_owner_pending(lock))
- return 0;
-
- if (pendowner == task)
- return 1;
-
- raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
- if (task->prio >= pendowner->prio) {
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
- return 0;
- }
-
- /*
- * Check if a waiter is enqueued on the pending owners
- * pi_waiters list. Remove it and readjust pending owners
- * priority.
- */
- if (likely(!rt_mutex_has_waiters(lock))) {
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
- return 1;
- }
-
- /* No chain handling, pending owner is not blocked on anything: */
- next = rt_mutex_top_waiter(lock);
- plist_del(&next->pi_list_entry, &pendowner->pi_waiters);
- __rt_mutex_adjust_prio(pendowner);
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
-
- /*
- * We are going to steal the lock and a waiter was
- * enqueued on the pending owners pi_waiters queue. So
- * we have to enqueue this waiter into
- * task->pi_waiters list. This covers the case,
- * where task is boosted because it holds another
- * lock and gets unboosted because the booster is
- * interrupted, so we would delay a waiter with higher
- * priority as task->normal_prio.
- *
- * Note: in the rare case of a SCHED_OTHER task changing
- * its priority and thus stealing the lock, next->task
- * might be task:
- */
- if (likely(next->task != task)) {
- raw_spin_lock_irqsave(&task->pi_lock, flags);
- plist_add(&next->pi_list_entry, &task->pi_waiters);
- __rt_mutex_adjust_prio(task);
- raw_spin_unlock_irqrestore(&task->pi_lock, flags);
- }
- return 1;
-}
-
-/*
* Try to take an rt-mutex
*
- * This fails
- * - when the lock has a real owner
- * - when a different pending owner exists and has higher priority than current
- *
* Must be called with lock->wait_lock held.
+ *
+ * @lock: the lock to be acquired.
+ * @task: the task which wants to acquire the lock
+ * @waiter: the waiter that is queued to the lock's wait list. (could be NULL)
*/
-static int try_to_take_rt_mutex(struct rt_mutex *lock)
+static int try_to_take_rt_mutex(struct rt_mutex *lock, struct task_struct *task,
+ struct rt_mutex_waiter *waiter)
{
/*
* We have to be careful here if the atomic speedups are
@@ -390,15 +331,52 @@ static int try_to_take_rt_mutex(struct rt_mutex *lock)
*/
mark_rt_mutex_waiters(lock);

- if (rt_mutex_owner(lock) && !try_to_steal_lock(lock, current))
+ if (rt_mutex_owner(lock))
return 0;

+ /*
+ * It will get the lock because of one of these conditions:
+ * 1) there is no waiter
+ * 2) higher priority than waiters
+ * 3) it is top waiter
+ */
+ if (rt_mutex_has_waiters(lock)) {
+ if (task->prio >= rt_mutex_top_waiter(lock)->list_entry.prio) {
+ if (!waiter || waiter != rt_mutex_top_waiter(lock))
+ return 0;
+ }
+ }
+
+ if (waiter || rt_mutex_has_waiters(lock)) {
+ unsigned long flags;
+ struct rt_mutex_waiter *top;
+
+ raw_spin_lock_irqsave(&task->pi_lock, flags);
+
+ /* remove the queued waiter. */
+ if (waiter) {
+ plist_del(&waiter->list_entry, &lock->wait_list);
+ task->pi_blocked_on = NULL;
+ }
+
+ /*
+ * We have to enqueue the top waiter(if it exists) into
+ * task->pi_waiters list.
+ */
+ if (rt_mutex_has_waiters(lock)) {
+ top = rt_mutex_top_waiter(lock);
+ top->pi_list_entry.prio = top->list_entry.prio;
+ plist_add(&top->pi_list_entry, &task->pi_waiters);
+ }
+ raw_spin_unlock_irqrestore(&task->pi_lock, flags);
+ }
+
/* We got the lock. */
debug_rt_mutex_lock(lock);

- rt_mutex_set_owner(lock, current, 0);
+ rt_mutex_set_owner(lock, task);

- rt_mutex_deadlock_account_lock(lock, current);
+ rt_mutex_deadlock_account_lock(lock, task);

return 1;
}
@@ -436,6 +414,9 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,

raw_spin_unlock_irqrestore(&task->pi_lock, flags);

+ if (!owner)
+ return 0;
+
if (waiter == rt_mutex_top_waiter(lock)) {
raw_spin_lock_irqsave(&owner->pi_lock, flags);
plist_del(&top_waiter->pi_list_entry, &owner->pi_waiters);
@@ -472,21 +453,18 @@ static int task_blocks_on_rt_mutex(struct rt_mutex *lock,
/*
* Wake up the next waiter on the lock.
*
- * Remove the top waiter from the current tasks waiter list and from
- * the lock waiter list. Set it as pending owner. Then wake it up.
+ * Remove the top waiter from the current tasks waiter list and wake it up.
*
* Called with lock->wait_lock held.
*/
static void wakeup_next_waiter(struct rt_mutex *lock)
{
struct rt_mutex_waiter *waiter;
- struct task_struct *pendowner;
unsigned long flags;

raw_spin_lock_irqsave(&current->pi_lock, flags);

waiter = rt_mutex_top_waiter(lock);
- plist_del(&waiter->list_entry, &lock->wait_list);

/*
* Remove it from current->pi_waiters. We do not adjust a
@@ -495,43 +473,19 @@ static void wakeup_next_waiter(struct rt_mutex *lock)
* lock->wait_lock.
*/
plist_del(&waiter->pi_list_entry, &current->pi_waiters);
- pendowner = waiter->task;
- waiter->task = NULL;

- rt_mutex_set_owner(lock, pendowner, RT_MUTEX_OWNER_PENDING);
+ rt_mutex_set_owner(lock, NULL);

raw_spin_unlock_irqrestore(&current->pi_lock, flags);

- /*
- * Clear the pi_blocked_on variable and enqueue a possible
- * waiter into the pi_waiters list of the pending owner. This
- * prevents that in case the pending owner gets unboosted a
- * waiter with higher priority than pending-owner->normal_prio
- * is blocked on the unboosted (pending) owner.
- */
- raw_spin_lock_irqsave(&pendowner->pi_lock, flags);
-
- WARN_ON(!pendowner->pi_blocked_on);
- WARN_ON(pendowner->pi_blocked_on != waiter);
- WARN_ON(pendowner->pi_blocked_on->lock != lock);
-
- pendowner->pi_blocked_on = NULL;
-
- if (rt_mutex_has_waiters(lock)) {
- struct rt_mutex_waiter *next;
-
- next = rt_mutex_top_waiter(lock);
- plist_add(&next->pi_list_entry, &pendowner->pi_waiters);
- }
- raw_spin_unlock_irqrestore(&pendowner->pi_lock, flags);
-
- wake_up_process(pendowner);
+ wake_up_process(waiter->task);
}

/*
- * Remove a waiter from a lock
+ * Remove a waiter from a lock and give up
*
- * Must be called with lock->wait_lock held
+ * Must be called with lock->wait_lock held and
+ * have just failed to try_to_take_rt_mutex().
*/
static void remove_waiter(struct rt_mutex *lock,
struct rt_mutex_waiter *waiter)
@@ -543,11 +497,13 @@ static void remove_waiter(struct rt_mutex *lock,

raw_spin_lock_irqsave(&current->pi_lock, flags);
plist_del(&waiter->list_entry, &lock->wait_list);
- waiter->task = NULL;
current->pi_blocked_on = NULL;
raw_spin_unlock_irqrestore(&current->pi_lock, flags);

- if (first && owner != current) {
+ if (!owner)
+ return;
+
+ if (first) {

raw_spin_lock_irqsave(&owner->pi_lock, flags);

@@ -614,21 +570,19 @@ void rt_mutex_adjust_pi(struct task_struct *task)
* or TASK_UNINTERRUPTIBLE)
* @timeout: the pre-initialized and started timer, or NULL for none
* @waiter: the pre-initialized rt_mutex_waiter
- * @detect_deadlock: passed to task_blocks_on_rt_mutex
*
* lock->wait_lock must be held by the caller.
*/
static int __sched
__rt_mutex_slowlock(struct rt_mutex *lock, int state,
struct hrtimer_sleeper *timeout,
- struct rt_mutex_waiter *waiter,
- int detect_deadlock)
+ struct rt_mutex_waiter *waiter)
{
int ret = 0;

for (;;) {
/* Try to acquire the lock: */
- if (try_to_take_rt_mutex(lock))
+ if (try_to_take_rt_mutex(lock, current, waiter))
break;

/*
@@ -645,39 +599,11 @@ __rt_mutex_slowlock(struct rt_mutex *lock, int state,
break;
}

- /*
- * waiter->task is NULL the first time we come here and
- * when we have been woken up by the previous owner
- * but the lock got stolen by a higher prio task.
- */
- if (!waiter->task) {
- ret = task_blocks_on_rt_mutex(lock, waiter, current,
- detect_deadlock);
- /*
- * If we got woken up by the owner then start loop
- * all over without going into schedule to try
- * to get the lock now:
- */
- if (unlikely(!waiter->task)) {
- /*
- * Reset the return value. We might
- * have returned with -EDEADLK and the
- * owner released the lock while we
- * were walking the pi chain.
- */
- ret = 0;
- continue;
- }
- if (unlikely(ret))
- break;
- }
-
raw_spin_unlock(&lock->wait_lock);

debug_rt_mutex_print_deadlock(waiter);

- if (waiter->task)
- schedule_rt_mutex(lock);
+ schedule_rt_mutex(lock);

raw_spin_lock(&lock->wait_lock);
set_current_state(state);
@@ -698,12 +624,11 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
int ret = 0;

debug_rt_mutex_init_waiter(&waiter);
- waiter.task = NULL;

raw_spin_lock(&lock->wait_lock);

/* Try to acquire the lock again: */
- if (try_to_take_rt_mutex(lock)) {
+ if (try_to_take_rt_mutex(lock, current, NULL)) {
raw_spin_unlock(&lock->wait_lock);
return 0;
}
@@ -717,12 +642,14 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
timeout->task = NULL;
}

- ret = __rt_mutex_slowlock(lock, state, timeout, &waiter,
- detect_deadlock);
+ ret = task_blocks_on_rt_mutex(lock, &waiter, current, detect_deadlock);
+
+ if (likely(!ret))
+ ret = __rt_mutex_slowlock(lock, state, timeout, &waiter);

set_current_state(TASK_RUNNING);

- if (unlikely(waiter.task))
+ if (unlikely(ret))
remove_waiter(lock, &waiter);

/*
@@ -737,14 +664,6 @@ rt_mutex_slowlock(struct rt_mutex *lock, int state,
if (unlikely(timeout))
hrtimer_cancel(&timeout->timer);

- /*
- * Readjust priority, when we did not get the lock. We might
- * have been the pending owner and boosted. Since we did not
- * take the lock, the PI boost has to go.
- */
- if (unlikely(ret))
- rt_mutex_adjust_prio(current);
-
debug_rt_mutex_free_waiter(&waiter);

return ret;
@@ -762,7 +681,7 @@ rt_mutex_slowtrylock(struct rt_mutex *lock)

if (likely(rt_mutex_owner(lock) != current)) {

- ret = try_to_take_rt_mutex(lock);
+ ret = try_to_take_rt_mutex(lock, current, NULL);
/*
* try_to_take_rt_mutex() sets the lock waiters
* bit unconditionally. Clean this up.
@@ -992,7 +911,7 @@ void rt_mutex_init_proxy_locked(struct rt_mutex *lock,
{
__rt_mutex_init(lock, NULL);
debug_rt_mutex_proxy_lock(lock, proxy_owner);
- rt_mutex_set_owner(lock, proxy_owner, 0);
+ rt_mutex_set_owner(lock, proxy_owner);
rt_mutex_deadlock_account_lock(lock, proxy_owner);
}

@@ -1008,7 +927,7 @@ void rt_mutex_proxy_unlock(struct rt_mutex *lock,
struct task_struct *proxy_owner)
{
debug_rt_mutex_proxy_unlock(lock);
- rt_mutex_set_owner(lock, NULL, 0);
+ rt_mutex_set_owner(lock, NULL);
rt_mutex_deadlock_account_unlock(proxy_owner);
}

@@ -1034,20 +953,14 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,

raw_spin_lock(&lock->wait_lock);

- mark_rt_mutex_waiters(lock);
-
- if (!rt_mutex_owner(lock) || try_to_steal_lock(lock, task)) {
- /* We got the lock for task. */
- debug_rt_mutex_lock(lock);
- rt_mutex_set_owner(lock, task, 0);
+ if (try_to_take_rt_mutex(lock, task, NULL)) {
raw_spin_unlock(&lock->wait_lock);
- rt_mutex_deadlock_account_lock(lock, task);
return 1;
}

ret = task_blocks_on_rt_mutex(lock, waiter, task, detect_deadlock);

- if (ret && !waiter->task) {
+ if (ret && !rt_mutex_owner(lock)) {
/*
* Reset the return value. We might have
* returned with -EDEADLK and the owner
@@ -1056,6 +969,10 @@ int rt_mutex_start_proxy_lock(struct rt_mutex *lock,
*/
ret = 0;
}
+
+ if (unlikely(ret))
+ remove_waiter(lock, waiter);
+
raw_spin_unlock(&lock->wait_lock);

debug_rt_mutex_print_deadlock(waiter);
@@ -1110,12 +1027,11 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,

set_current_state(TASK_INTERRUPTIBLE);

- ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter,
- detect_deadlock);
+ ret = __rt_mutex_slowlock(lock, TASK_INTERRUPTIBLE, to, waiter);

set_current_state(TASK_RUNNING);

- if (unlikely(waiter->task))
+ if (unlikely(ret))
remove_waiter(lock, waiter);

/*
@@ -1126,13 +1042,5 @@ int rt_mutex_finish_proxy_lock(struct rt_mutex *lock,

raw_spin_unlock(&lock->wait_lock);

- /*
- * Readjust priority, when we did not get the lock. We might have been
- * the pending owner and boosted. Since we did not take the lock, the
- * PI boost has to go.
- */
- if (unlikely(ret))
- rt_mutex_adjust_prio(current);
-
return ret;
}
diff --git a/kernel/rtmutex_common.h b/kernel/rtmutex_common.h
index 97a2f81..53a66c8 100644
--- a/kernel/rtmutex_common.h
+++ b/kernel/rtmutex_common.h
@@ -91,9 +91,8 @@ task_top_pi_waiter(struct task_struct *p)
/*
* lock->owner state tracking:
*/
-#define RT_MUTEX_OWNER_PENDING 1UL
-#define RT_MUTEX_HAS_WAITERS 2UL
-#define RT_MUTEX_OWNER_MASKALL 3UL
+#define RT_MUTEX_HAS_WAITERS 1UL
+#define RT_MUTEX_OWNER_MASKALL 1UL

static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
{
@@ -101,17 +100,6 @@ static inline struct task_struct *rt_mutex_owner(struct rt_mutex *lock)
((unsigned long)lock->owner & ~RT_MUTEX_OWNER_MASKALL);
}

-static inline struct task_struct *rt_mutex_real_owner(struct rt_mutex *lock)
-{
- return (struct task_struct *)
- ((unsigned long)lock->owner & ~RT_MUTEX_HAS_WAITERS);
-}
-
-static inline unsigned long rt_mutex_owner_pending(struct rt_mutex *lock)
-{
- return (unsigned long)lock->owner & RT_MUTEX_OWNER_PENDING;
-}
-
/*
* PI-futex support (proxy locking functions, etc.):
*/

2010-12-23 09:05:50

by Lai Jiangshan

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On 12/15/2010 11:04 PM, Steven Rostedt wrote:
> On Wed, 2010-12-15 at 16:09 +0800, Lai Jiangshan wrote:
>
> Some English updates.

Updated, Thank you very much.

>
>>

[...]

>> + * owner bit0
>> + * NULL 0 lock is free (fast acquire possible)
>> + * NULL 1 lock is free and has waiters and the top waiter
>> + * is going to take the lock*
>> + * taskpointer 0 lock is held (fast release possible)
>> + * taskpointer 1 lock is held and has waiters
>
> * taskpointer 1 lock is held and has waiters*
>
>
>> *
>> * The fast atomic compare exchange based acquire and release is only
>> - * possible when bit 0 and 1 of lock->owner are 0.
>> + * possible when bit 0 of lock->owner are 0.
>
> s/are/is/
>
>> *
>> - * (*) There's a small time where the owner can be NULL and the
>> - * "lock has waiters" bit is set. This can happen when grabbing the lock.
>> - * To prevent a cmpxchg of the owner releasing the lock, we need to set this
>> - * bit before looking at the lock, hence the reason this is a transitional
>> - * state.
>> + * (*) It also can be a transitional state when grabbing the lock
>> + * with ->wait_lock is held. To prevent any fast path cmpxchg to the lock,
>> + * we need to set the bit0 before looking at the lock, and the owner may be
>> + * NULL in this small time, hence this can be a transitional state.
>
>
> * (*) There is a small time when bit 0 is set but there are no
> * waiters. This can happen when grabbing the lock in the slow path.
> * To prevent a cmpxchg of the owner releasing the lock, we need to
> * set this bit before looking at the lock.
>
>

Very good! Added.


>
>> It has called try_to_take_rt_mutex() in __rt_mutex_slowlock(),
>> when timeout or got signal, it returns from __rt_mutex_slowlock()
>> with lock->wait_lock still held, and then calls remove_waiter(),
>>
>> so we don't need to call try_to_take_rt_mutex() in remove_waiter().
>> It is strange that remove_waiter() do some "require lock" work.
>
> I think you are correct here. It should never get to this path where
> !owner && first is true. If we timed out then we either got the lock or
> we could not. If we could not than owner had to be set or we are not the
> top waiter. Thus we need:
>
> if (!owner) {
> BUG_ON(first);
> return;
> }
>

The "BUG_ON" is not added, but new comments for this are added.
Because we have just tested it in try_to_take_rt_mutex(), don't need
to check it again.

If prerequisites are passed by complex paths or are
provided by different threads, I will happy to add more checks.

Thanks,
Lai

2010-12-23 12:56:15

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH] rtmutex: ensure only the top waiter or higher priority task can take the lock and reduce unrelated boosting

On Thu, 2010-12-23 at 17:07 +0800, Lai Jiangshan wrote:
> On 12/15/2010 11:04 PM, Steven Rostedt wrote:
> > On Wed, 2010-12-15 at 16:09 +0800, Lai Jiangshan wrote:
> >
> > Some English updates.
>
> Updated, Thank you very much.
>

Note, I've been working all week on a port for -rt. It required some
clean ups, but as I finished (last week), I keep hitting a "hung task"
when running dbench in a loop over night. I spent most the week trying
to debug it. But I'm starting to think this is something to do with -rt
and not your patch. So I'll be posting my patches later today.

-- Steve